|
@@ -46,6 +46,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DAT
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_UPGRADE_PERMISSION_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_UPGRADE_PERMISSION_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_DEFAULT;
|
|
@@ -108,6 +109,7 @@ import org.apache.hadoop.fs.permission.FsAction;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
+import org.apache.hadoop.hdfs.HAUtil;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
@@ -144,6 +146,11 @@ import org.apache.hadoop.hdfs.server.common.Storage;
|
|
|
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
|
|
|
import org.apache.hadoop.hdfs.server.common.Util;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.PendingDataNodeMessages.BlockReceivedDeleteMessage;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.PendingDataNodeMessages.BlockReportMessage;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.PendingDataNodeMessages.CommitBlockSynchronizationMessage;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.PendingDataNodeMessages.DataNodeMessage;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
|
@@ -170,6 +177,7 @@ import org.apache.hadoop.util.Daemon;
|
|
|
import org.apache.hadoop.util.VersionInfo;
|
|
|
import org.mortbay.util.ajax.JSON;
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
|
|
|
/***************************************************
|
|
@@ -293,6 +301,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
// lock to protect FSNamesystem.
|
|
|
private ReentrantReadWriteLock fsLock;
|
|
|
|
|
|
+ private PendingDataNodeMessages pendingDatanodeMessages = new PendingDataNodeMessages();
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Used when this NN is in standby state to read from the shared edit log.
|
|
|
+ */
|
|
|
+ private EditLogTailer editLogTailer = null;
|
|
|
+
|
|
|
+ PendingDataNodeMessages getPendingDataNodeMessages() {
|
|
|
+ return pendingDatanodeMessages;
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* Instantiates an FSNamesystem loaded from the image and edits
|
|
@@ -303,7 +321,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
* @return an FSNamesystem which contains the loaded namespace
|
|
|
* @throws IOException if loading fails
|
|
|
*/
|
|
|
- public static FSNamesystem loadFromDisk(Configuration conf) throws IOException {
|
|
|
+ public static FSNamesystem loadFromDisk(Configuration conf)
|
|
|
+ throws IOException {
|
|
|
Collection<URI> namespaceDirs = FSNamesystem.getNamespaceDirs(conf);
|
|
|
Collection<URI> namespaceEditsDirs =
|
|
|
FSNamesystem.getNamespaceEditsDirs(conf);
|
|
@@ -322,7 +341,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
long loadStart = now();
|
|
|
StartupOption startOpt = NameNode.getStartupOption(conf);
|
|
|
- namesystem.loadFSImage(startOpt, fsImage);
|
|
|
+ String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
|
|
|
+ namesystem.loadFSImage(startOpt, fsImage,
|
|
|
+ HAUtil.isHAEnabled(conf, nameserviceId));
|
|
|
long timeTakenToLoadFSImage = now() - loadStart;
|
|
|
LOG.info("Finished loading FSImage in " + timeTakenToLoadFSImage + " msecs");
|
|
|
NameNode.getNameNodeMetrics().setFsImageLoadTime(
|
|
@@ -368,7 +389,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
this.safeMode = new SafeModeInfo(conf);
|
|
|
}
|
|
|
|
|
|
- void loadFSImage(StartupOption startOpt, FSImage fsImage)
|
|
|
+ void loadFSImage(StartupOption startOpt, FSImage fsImage, boolean haEnabled)
|
|
|
throws IOException {
|
|
|
// format before starting up if requested
|
|
|
if (startOpt == StartupOption.FORMAT) {
|
|
@@ -379,10 +400,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
boolean success = false;
|
|
|
try {
|
|
|
- if (fsImage.recoverTransitionRead(startOpt, this)) {
|
|
|
+ // We shouldn't be calling saveNamespace if we've come up in standby state.
|
|
|
+ if (fsImage.recoverTransitionRead(startOpt, this) && !haEnabled) {
|
|
|
fsImage.saveNamespace(this);
|
|
|
}
|
|
|
- fsImage.openEditLog();
|
|
|
+ // This will start a new log segment and write to the seen_txid file, so
|
|
|
+ // we shouldn't do it when coming up in standby state
|
|
|
+ if (!haEnabled) {
|
|
|
+ fsImage.openEditLogForWrite();
|
|
|
+ }
|
|
|
|
|
|
success = true;
|
|
|
} finally {
|
|
@@ -449,6 +475,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
LOG.info("Starting services required for active state");
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ if (!dir.fsImage.editLog.isOpenForWrite()) {
|
|
|
+ // During startup, we're already open for write during initialization.
|
|
|
+ // TODO(HA): consider adding a startup state?
|
|
|
+ dir.fsImage.editLog.initJournalsForWrite();
|
|
|
+ // May need to recover
|
|
|
+ dir.fsImage.editLog.recoverUnclosedStreams();
|
|
|
+ dir.fsImage.editLog.openForWrite();
|
|
|
+ }
|
|
|
if (UserGroupInformation.isSecurityEnabled()) {
|
|
|
startSecretManager();
|
|
|
}
|
|
@@ -459,7 +493,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Start services required in active state
|
|
|
+ * Stop services required in active state
|
|
|
* @throws InterruptedException
|
|
|
*/
|
|
|
void stopActiveServices() {
|
|
@@ -470,6 +504,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
if (leaseManager != null) {
|
|
|
leaseManager.stopMonitor();
|
|
|
}
|
|
|
+ dir.fsImage.editLog.close();
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
@@ -478,11 +513,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
/** Start services required in standby state */
|
|
|
void startStandbyServices() {
|
|
|
LOG.info("Starting services required for standby state");
|
|
|
+ if (!dir.fsImage.editLog.isOpenForRead()) {
|
|
|
+ // During startup, we're already open for read.
|
|
|
+ dir.fsImage.editLog.initSharedJournalsForRead();
|
|
|
+ }
|
|
|
+ editLogTailer = new EditLogTailer(this);
|
|
|
+ editLogTailer.start();
|
|
|
}
|
|
|
|
|
|
/** Stop services required in standby state */
|
|
|
- void stopStandbyServices() {
|
|
|
+ void stopStandbyServices() throws IOException {
|
|
|
LOG.info("Stopping services started for standby state");
|
|
|
+ if (editLogTailer != null) {
|
|
|
+ editLogTailer.stop();
|
|
|
+ }
|
|
|
+ dir.fsImage.editLog.close();
|
|
|
}
|
|
|
|
|
|
public static Collection<URI> getNamespaceDirs(Configuration conf) {
|
|
@@ -520,7 +565,22 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
public static Collection<URI> getNamespaceEditsDirs(Configuration conf) {
|
|
|
- return getStorageDirs(conf, DFS_NAMENODE_EDITS_DIR_KEY);
|
|
|
+ Collection<URI> editsDirs = getStorageDirs(conf, DFS_NAMENODE_EDITS_DIR_KEY);
|
|
|
+ editsDirs.addAll(getSharedEditsDirs(conf));
|
|
|
+ return editsDirs;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns edit directories that are shared between primary and secondary.
|
|
|
+ * @param conf
|
|
|
+ * @return Collection of edit directories.
|
|
|
+ */
|
|
|
+ public static Collection<URI> getSharedEditsDirs(Configuration conf) {
|
|
|
+ // don't use getStorageDirs here, because we want an empty default
|
|
|
+ // rather than the dir in /tmp
|
|
|
+ Collection<String> dirNames = conf.getTrimmedStringCollection(
|
|
|
+ DFS_NAMENODE_SHARED_EDITS_DIR_KEY);
|
|
|
+ return Util.stringCollectionAsURIs(dirNames);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -634,6 +694,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
} finally {
|
|
|
// using finally to ensure we also wait for lease daemon
|
|
|
try {
|
|
|
+ // TODO: these lines spew lots of warnings about "already stopped" logs, etc
|
|
|
stopActiveServices();
|
|
|
stopStandbyServices();
|
|
|
if (dir != null) {
|
|
@@ -1796,12 +1857,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
* @throws QuotaExceededException If addition of block exceeds space quota
|
|
|
*/
|
|
|
private Block allocateBlock(String src, INode[] inodes,
|
|
|
- DatanodeDescriptor targets[]) throws QuotaExceededException {
|
|
|
+ DatanodeDescriptor targets[]) throws QuotaExceededException,
|
|
|
+ SafeModeException {
|
|
|
assert hasWriteLock();
|
|
|
Block b = new Block(DFSUtil.getRandom().nextLong(), 0, 0);
|
|
|
while(isValidBlock(b)) {
|
|
|
b.setBlockId(DFSUtil.getRandom().nextLong());
|
|
|
}
|
|
|
+ // Increment the generation stamp for every new block.
|
|
|
+ nextGenerationStamp();
|
|
|
b.setGenerationStamp(getGenerationStamp());
|
|
|
b = dir.addBlock(src, inodes, b, targets);
|
|
|
NameNode.stateChangeLog.info("BLOCK* NameSystem.allocateBlock: "
|
|
@@ -2703,11 +2767,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- FSImage getFSImage() {
|
|
|
+ public FSImage getFSImage() {
|
|
|
return dir.fsImage;
|
|
|
}
|
|
|
|
|
|
- FSEditLog getEditLog() {
|
|
|
+ public FSEditLog getEditLog() {
|
|
|
return getFSImage().getEditLog();
|
|
|
}
|
|
|
|
|
@@ -3726,6 +3790,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
private ObjectName mbeanName;
|
|
|
+
|
|
|
/**
|
|
|
* Register the FSNamesystem MBean using the name
|
|
|
* "hadoop:service=NameNode,name=FSNamesystemState"
|
|
@@ -3766,6 +3831,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
*/
|
|
|
void setGenerationStamp(long stamp) {
|
|
|
generationStamp.setStamp(stamp);
|
|
|
+ notifyGenStampUpdate(stamp);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -4523,4 +4589,55 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
byte[] password) throws InvalidToken {
|
|
|
getDelegationTokenSecretManager().verifyToken(identifier, password);
|
|
|
}
|
|
|
+
|
|
|
+ public boolean isGenStampInFuture(long genStamp) {
|
|
|
+ return (genStamp > getGenerationStamp());
|
|
|
+ }
|
|
|
+
|
|
|
+ public void notifyGenStampUpdate(long gs) {
|
|
|
+ LOG.info("=> notified of genstamp update for: " + gs);
|
|
|
+ DataNodeMessage msg = pendingDatanodeMessages.take(gs);
|
|
|
+ while (msg != null) {
|
|
|
+ LOG.info("processing message: " + msg);
|
|
|
+ try {
|
|
|
+ switch (msg.getType()) {
|
|
|
+ case BLOCK_RECEIVED_DELETE:
|
|
|
+ BlockReceivedDeleteMessage m = (BlockReceivedDeleteMessage) msg;
|
|
|
+ if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
+ NameNode.stateChangeLog
|
|
|
+ .debug("*BLOCK* NameNode.blockReceivedAndDeleted: " + "from "
|
|
|
+ + m.getNodeReg().getName() + " "
|
|
|
+ + m.getReceivedAndDeletedBlocks().length + " blocks.");
|
|
|
+ }
|
|
|
+ this.getBlockManager().blockReceivedAndDeleted(m.getNodeReg(),
|
|
|
+ m.getPoolId(), m.getReceivedAndDeletedBlocks());
|
|
|
+ break;
|
|
|
+ case BLOCK_REPORT:
|
|
|
+ BlockReportMessage mbr = (BlockReportMessage) msg;
|
|
|
+ if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
+ NameNode.stateChangeLog.debug("*BLOCK* NameNode.blockReport: "
|
|
|
+ + "from " + mbr.getNodeReg().getName() + " "
|
|
|
+ + mbr.getBlockList().getNumberOfBlocks() + " blocks");
|
|
|
+ }
|
|
|
+ this.getBlockManager().processReport(mbr.getNodeReg(),
|
|
|
+ mbr.getPoolId(), mbr.getBlockList());
|
|
|
+ break;
|
|
|
+ case COMMIT_BLOCK_SYNCHRONIZATION:
|
|
|
+ CommitBlockSynchronizationMessage mcbm = (CommitBlockSynchronizationMessage) msg;
|
|
|
+ this.commitBlockSynchronization(mcbm.getBlock(),
|
|
|
+ mcbm.getNewgenerationstamp(), mcbm.getNewlength(),
|
|
|
+ mcbm.isCloseFile(), mcbm.isDeleteblock(), mcbm.getNewtargets());
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ } catch (IOException ex) {
|
|
|
+ LOG.warn("Could not process the message " + msg.getType(), ex);
|
|
|
+ }
|
|
|
+ msg = pendingDatanodeMessages.take(gs);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ public EditLogTailer getEditLogTailer() {
|
|
|
+ return editLogTailer;
|
|
|
+ }
|
|
|
}
|