|
@@ -18,18 +18,20 @@
|
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.server.common.Util.now;
|
|
|
-import java.net.URI;
|
|
|
+
|
|
|
import java.io.IOException;
|
|
|
+import java.lang.reflect.Constructor;
|
|
|
+import java.net.URI;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collection;
|
|
|
import java.util.List;
|
|
|
-import java.lang.reflect.Constructor;
|
|
|
|
|
|
import org.apache.commons.lang.StringUtils;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.Options;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
@@ -37,14 +39,34 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
|
|
|
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CloseOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ConcatDeleteOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.LogSegmentOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.OpInstanceCache;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenewDelegationTokenOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetOwnerOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.security.token.delegation.DelegationKey;
|
|
|
-import org.apache.hadoop.conf.Configuration;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
@@ -269,7 +291,7 @@ public class FSEditLog {
|
|
|
IOUtils.closeStream(s);
|
|
|
}
|
|
|
|
|
|
- startLogSegment(segmentTxId, true);
|
|
|
+ startLogSegmentAndWriteHeaderTxn(segmentTxId);
|
|
|
assert state == State.IN_SEGMENT : "Bad state: " + state;
|
|
|
}
|
|
|
|
|
@@ -864,18 +886,48 @@ public class FSEditLog {
|
|
|
endCurrentLogSegment(true);
|
|
|
|
|
|
long nextTxId = getLastWrittenTxId() + 1;
|
|
|
- startLogSegment(nextTxId, true);
|
|
|
+ startLogSegmentAndWriteHeaderTxn(nextTxId);
|
|
|
|
|
|
assert curSegmentTxId == nextTxId;
|
|
|
return nextTxId;
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Remote namenode just has started a log segment, start log segment locally.
|
|
|
+ */
|
|
|
+ public synchronized void startLogSegment(long txid,
|
|
|
+ boolean abortCurrentLogSegment) throws IOException {
|
|
|
+ LOG.info("Namenode started a new log segment at txid " + txid);
|
|
|
+ if (isSegmentOpen()) {
|
|
|
+ if (getLastWrittenTxId() == txid - 1) {
|
|
|
+ //In sync with the NN, so end and finalize the current segment`
|
|
|
+ endCurrentLogSegment(false);
|
|
|
+ } else {
|
|
|
+ //Missed some transactions: probably lost contact with NN temporarily.
|
|
|
+ final String mess = "Cannot start a new log segment at txid " + txid
|
|
|
+ + " since only up to txid " + getLastWrittenTxId()
|
|
|
+ + " have been written in the log segment starting at "
|
|
|
+ + getCurSegmentTxId() + ".";
|
|
|
+ if (abortCurrentLogSegment) {
|
|
|
+ //Mark the current segment as aborted.
|
|
|
+ LOG.warn(mess);
|
|
|
+ abortCurrentLogSegment();
|
|
|
+ } else {
|
|
|
+ throw new IOException(mess);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ setNextTxId(txid);
|
|
|
+ startLogSegment(txid);
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* Start writing to the log segment with the given txid.
|
|
|
* Transitions from BETWEEN_LOG_SEGMENTS state to IN_LOG_SEGMENT state.
|
|
|
*/
|
|
|
- synchronized void startLogSegment(final long segmentTxId,
|
|
|
- boolean writeHeaderTxn) throws IOException {
|
|
|
+ private void startLogSegment(final long segmentTxId) throws IOException {
|
|
|
+ assert Thread.holdsLock(this);
|
|
|
+
|
|
|
LOG.info("Starting log segment at " + segmentTxId);
|
|
|
Preconditions.checkArgument(segmentTxId > 0,
|
|
|
"Bad txid: %s", segmentTxId);
|
|
@@ -903,12 +955,15 @@ public class FSEditLog {
|
|
|
|
|
|
curSegmentTxId = segmentTxId;
|
|
|
state = State.IN_SEGMENT;
|
|
|
+ }
|
|
|
|
|
|
- if (writeHeaderTxn) {
|
|
|
- logEdit(LogSegmentOp.getInstance(cache.get(),
|
|
|
- FSEditLogOpCodes.OP_START_LOG_SEGMENT));
|
|
|
- logSync();
|
|
|
- }
|
|
|
+ synchronized void startLogSegmentAndWriteHeaderTxn(final long segmentTxId
|
|
|
+ ) throws IOException {
|
|
|
+ startLogSegment(segmentTxId);
|
|
|
+
|
|
|
+ logEdit(LogSegmentOp.getInstance(cache.get(),
|
|
|
+ FSEditLogOpCodes.OP_START_LOG_SEGMENT));
|
|
|
+ logSync();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1057,6 +1112,17 @@ public class FSEditLog {
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
+ /** Write the batch of edits to edit log. */
|
|
|
+ public synchronized void journal(long firstTxId, int numTxns, byte[] data) {
|
|
|
+ final long expectedTxId = getLastWrittenTxId() + 1;
|
|
|
+ Preconditions.checkState(firstTxId == expectedTxId,
|
|
|
+ "received txid batch starting at %s but expected txid %s",
|
|
|
+ firstTxId, expectedTxId);
|
|
|
+ setNextTxId(firstTxId + numTxns - 1);
|
|
|
+ logEdit(data.length, data);
|
|
|
+ logSync();
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Write an operation to the edit log. Do not sync to persistent
|
|
|
* store yet.
|