Return-Path: X-Original-To: apmail-hadoop-common-commits-archive@www.apache.org Delivered-To: apmail-hadoop-common-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 7FA4118DDF for ; Wed, 6 Jan 2016 18:47:26 +0000 (UTC) Received: (qmail 96242 invoked by uid 500); 6 Jan 2016 18:47:26 -0000 Delivered-To: apmail-hadoop-common-commits-archive@hadoop.apache.org Received: (qmail 96169 invoked by uid 500); 6 Jan 2016 18:47:26 -0000 Mailing-List: contact common-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: common-dev@hadoop.apache.org Delivered-To: mailing list common-commits@hadoop.apache.org Received: (qmail 96159 invoked by uid 99); 6 Jan 2016 18:47:26 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 06 Jan 2016 18:47:26 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 135C4DFDC7; Wed, 6 Jan 2016 18:47:26 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: arp@apache.org To: common-commits@hadoop.apache.org Date: Wed, 06 Jan 2016 18:47:26 -0000 Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: [1/2] hadoop git commit: HDFS-9498. Move code that tracks blocks with future generation stamps to BlockManagerSafeMode. (Contributed by Mingliang Liu) Repository: hadoop Updated Branches: refs/heads/branch-2 1cc001db4 -> 2e6990cdf refs/heads/trunk b9936689c -> 67c978060 HDFS-9498. Move code that tracks blocks with future generation stamps to BlockManagerSafeMode. (Contributed by Mingliang Liu) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/67c97806 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/67c97806 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/67c97806 Branch: refs/heads/trunk Commit: 67c9780609f707c11626f05028ddfd28f1b878f1 Parents: b993668 Author: Arpit Agarwal Authored: Wed Jan 6 10:30:59 2016 -0800 Committer: Arpit Agarwal Committed: Wed Jan 6 10:30:59 2016 -0800 ---------------------------------------------------------------------- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 5 +- .../server/blockmanagement/BlockManager.java | 59 ++--------- .../blockmanagement/BlockManagerSafeMode.java | 99 ++++++++++++++---- .../hdfs/server/namenode/FSNamesystem.java | 30 ++---- .../TestBlockManagerSafeMode.java | 102 +++++++++++++++---- .../hdfs/server/namenode/NameNodeAdapter.java | 2 +- .../hdfs/server/namenode/TestCheckpoint.java | 2 +- .../hdfs/server/namenode/TestEditLogRace.java | 2 +- .../server/namenode/TestFSEditLogLoader.java | 4 +- .../hdfs/server/namenode/TestFSNamesystem.java | 4 +- .../hdfs/server/namenode/TestINodeFile.java | 2 +- .../TestNameNodeMetadataConsistency.java | 58 ++++------- 12 files changed, 215 insertions(+), 154 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hadoop/blob/67c97806/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 50c940c..1b2ff92 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -1,4 +1,4 @@ - Hadoop HDFS Change Log +Hadoop HDFS Change Log Trunk (Unreleased) @@ -1800,6 +1800,9 @@ Release 2.8.0 - UNRELEASED HDFS-7779. Support changing ownership, group and replication in HDFS Web UI. (Ravi Prakash via wheat9) + HDFS-9498. Move code that tracks blocks with future generation stamps + to BlockManagerSafeMode. (Mingliang Liu via Arpit Agarwal) + OPTIMIZATIONS HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than http://git-wip-us.apache.org/repos/asf/hadoop/blob/67c97806/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index c77b38e..0f27240 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -73,7 +73,6 @@ import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys; import org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult; import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo; -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; import org.apache.hadoop.hdfs.server.namenode.CachedBlock; @@ -309,12 +308,6 @@ public class BlockManager implements BlockStatsMXBean { /** Check whether there are any non-EC blocks using StripedID */ private boolean hasNonEcBlockUsingStripedID = false; - /** Keeps track of how many bytes are in Future Generation blocks. */ - private AtomicLong numberOfBytesInFutureBlocks; - - /** Reports if Name node was started with Rollback option. */ - private boolean inRollBack = false; - public BlockManager(final Namesystem namesystem, final Configuration conf) throws IOException { this.namesystem = namesystem; @@ -393,8 +386,6 @@ public class BlockManager implements BlockStatsMXBean { DFSConfigKeys.DFS_BLOCK_MISREPLICATION_PROCESSING_LIMIT, DFSConfigKeys.DFS_BLOCK_MISREPLICATION_PROCESSING_LIMIT_DEFAULT); this.blockReportLeaseManager = new BlockReportLeaseManager(conf); - this.numberOfBytesInFutureBlocks = new AtomicLong(); - this.inRollBack = isInRollBackMode(NameNode.getStartupOption(conf)); bmSafeMode = new BlockManagerSafeMode(this, namesystem, conf); @@ -1964,14 +1955,18 @@ public class BlockManager implements BlockStatsMXBean { return bmSafeMode.getSafeModeTip(); } - public void leaveSafeMode(boolean force) { - bmSafeMode.leaveSafeMode(force); + public boolean leaveSafeMode(boolean force) { + return bmSafeMode.leaveSafeMode(force); } void checkSafeMode() { bmSafeMode.checkSafeMode(); } + public long getBytesInFuture() { + return bmSafeMode.getBytesInFuture(); + } + /** * Removes the blocks from blocksmap and updates the safemode blocks total. * @param blocks An instance of {@link BlocksMapUpdateInfo} which contains a @@ -2370,12 +2365,7 @@ public class BlockManager implements BlockStatsMXBean { // If block does not belong to any file, we check if it violates // an integrity assumption of Name node if (storedBlock == null) { - if (namesystem.isInStartupSafeMode() - && !shouldPostponeBlocksFromFuture - && !inRollBack - && namesystem.isGenStampInFuture(iblk)) { - numberOfBytesInFutureBlocks.addAndGet(iblk.getBytesOnDisk()); - } + bmSafeMode.checkBlocksWithFutureGS(iblk); continue; } @@ -4254,39 +4244,8 @@ public class BlockManager implements BlockStatsMXBean { return haContext.getState().shouldPopulateReplQueues(); } - /** - * Returns the number of bytes that reside in blocks with Generation Stamps - * greater than generation stamp known to Namenode. - * - * @return Bytes in future - */ - public long getBytesInFuture() { - return numberOfBytesInFutureBlocks.get(); - } - - /** - * Clears the bytes in future counter. - */ - public void clearBytesInFuture() { - numberOfBytesInFutureBlocks.set(0); - } - - /** - * Returns true if Namenode was started with a RollBack option. - * - * @param option - StartupOption - * @return boolean - */ - private boolean isInRollBackMode(HdfsServerConstants.StartupOption option) { - if (option == HdfsServerConstants.StartupOption.ROLLBACK) { - return true; - } - if ((option == HdfsServerConstants.StartupOption.ROLLINGUPGRADE) && - (option.getRollingUpgradeStartupOption() == - HdfsServerConstants.RollingUpgradeStartupOption.ROLLBACK)) { - return true; - } - return false; + boolean getShouldPostponeBlocksFromFuture() { + return shouldPostponeBlocksFromFuture; } // async processing of an action, used for IBRs. http://git-wip-us.apache.org/repos/asf/hadoop/blob/67c97806/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerSafeMode.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerSafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerSafeMode.java index 297532e..aba3c85 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerSafeMode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerSafeMode.java @@ -21,6 +21,9 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.Namesystem; import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase; @@ -106,6 +109,11 @@ class BlockManagerSafeMode { /** Counter for tracking startup progress of reported blocks. */ private Counter awaitingReportedBlocksCounter; + /** Keeps track of how many bytes are in Future Generation blocks. */ + private final AtomicLong numberOfBytesInFutureBlocks = new AtomicLong(); + /** Reports if Name node was started with Rollback option. */ + private final boolean inRollBack; + BlockManagerSafeMode(BlockManager blockManager, Namesystem namesystem, Configuration conf) { this.blockManager = blockManager; @@ -135,9 +143,10 @@ class BlockManagerSafeMode { this.replQueueThreshold = conf.getFloat(DFS_NAMENODE_REPL_QUEUE_THRESHOLD_PCT_KEY, (float) threshold); - this.extension = conf.getInt(DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 0); + this.inRollBack = isInRollBackMode(NameNode.getStartupOption(conf)); + LOG.info("{} = {}", DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, threshold); LOG.info("{} = {}", DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY, datanodeThreshold); @@ -300,15 +309,15 @@ class BlockManagerSafeMode { numLive, datanodeThreshold); } - if (blockManager.getBytesInFuture() > 0) { + if (getBytesInFuture() > 0) { msg += "Name node detected blocks with generation stamps " + - "in future. This means that Name node metadata is inconsistent." + + "in future. This means that Name node metadata is inconsistent. " + "This can happen if Name node metadata files have been manually " + - "replaced. Exiting safe mode will cause loss of " + blockManager - .getBytesInFuture() + " byte(s). Please restart name node with " + - "right metadata or use \"hdfs dfsadmin -safemode forceExit" + - "if you are certain that the NameNode was started with the" + - "correct FsImage and edit logs. If you encountered this during" + + "replaced. Exiting safe mode will cause loss of " + + getBytesInFuture() + " byte(s). Please restart name node with " + + "right metadata or use \"hdfs dfsadmin -safemode forceExit\" " + + "if you are certain that the NameNode was started with the " + + "correct FsImage and edit logs. If you encountered this during " + "a rollback, it is safe to exit with -safemode forceExit."; return msg; } @@ -333,11 +342,31 @@ class BlockManagerSafeMode { /** * Leave start up safe mode. + * * @param force - true to force exit + * @return true if it leaves safe mode successfully else false */ - void leaveSafeMode(boolean force) { + boolean leaveSafeMode(boolean force) { assert namesystem.hasWriteLock() : "Leaving safe mode needs write lock!"; + final long bytesInFuture = numberOfBytesInFutureBlocks.get(); + if (bytesInFuture > 0) { + if (force) { + LOG.warn("Leaving safe mode due to forceExit. This will cause a data " + + "loss of {} byte(s).", bytesInFuture); + numberOfBytesInFutureBlocks.set(0); + } else { + LOG.error("Refusing to leave safe mode without a force flag. " + + "Exiting safe mode will cause a deletion of {} byte(s). Please " + + "use -forceExit flag to exit safe mode forcefully if data loss is" + + " acceptable.", bytesInFuture); + return false; + } + } else if (force) { + LOG.warn("forceExit used when normal exist would suffice. Treating " + + "force exit as normal safe mode exit."); + } + // if not done yet, initialize replication queues. // In the standby, do not populate repl queues if (!blockManager.isPopulatingReplQueues() && @@ -345,14 +374,6 @@ class BlockManagerSafeMode { blockManager.initializeReplQueues(); } - if (!force && blockManager.getBytesInFuture() > 0) { - LOG.error("Refusing to leave safe mode without a force flag. " + - "Exiting safe mode will cause a deletion of {} byte(s). Please use " + - "-forceExit flag to exit safe mode forcefully if data loss is " + - "acceptable.", blockManager.getBytesInFuture()); - return; - } - if (status != BMSafeModeStatus.OFF) { NameNode.stateChangeLog.info("STATE* Safe mode is OFF"); } @@ -379,6 +400,8 @@ class BlockManagerSafeMode { BlockManagerSafeMode.STEP_AWAITING_REPORTED_BLOCKS); prog.endPhase(Phase.SAFEMODE); } + + return true; } /** @@ -436,6 +459,35 @@ class BlockManagerSafeMode { } } + /** + * Check if the block report replica has a generation stamp (GS) in future. + * If safe mode is not currently on, this is a no-op. + * + * @param brr block report replica which belongs to no file in BlockManager + */ + void checkBlocksWithFutureGS(BlockReportReplica brr) { + assert namesystem.hasWriteLock(); + if (status == BMSafeModeStatus.OFF) { + return; + } + + if (!blockManager.getShouldPostponeBlocksFromFuture() && + !inRollBack && + namesystem.isGenStampInFuture(brr)) { + numberOfBytesInFutureBlocks.addAndGet(brr.getBytesOnDisk()); + } + } + + /** + * Returns the number of bytes that reside in blocks with Generation Stamps + * greater than generation stamp known to Namenode. + * + * @return Bytes in future + */ + long getBytesInFuture() { + return numberOfBytesInFutureBlocks.get(); + } + void close() { assert namesystem.hasWriteLock() : "Closing bmSafeMode needs write lock!"; try { @@ -454,6 +506,19 @@ class BlockManagerSafeMode { return reachedTime.get() + extension - monotonicNow(); } + /** + * Returns true if Namenode was started with a RollBack option. + * + * @param option - StartupOption + * @return boolean + */ + private static boolean isInRollBackMode(StartupOption option) { + return (option == StartupOption.ROLLBACK) || + (option == StartupOption.ROLLINGUPGRADE && + option.getRollingUpgradeStartupOption() == + RollingUpgradeStartupOption.ROLLBACK); + } + /** Check if we are ready to initialize replication queues. */ private void initializeReplQueuesIfNecessary() { assert namesystem.hasWriteLock(); http://git-wip-us.apache.org/repos/asf/hadoop/blob/67c97806/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index 97cb6fb..fa110e5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -986,6 +986,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, @Override public void startSecretManagerIfNecessary() { + assert hasWriteLock() : "Starting secret manager needs write lock"; boolean shouldRun = shouldUseDelegationTokens() && !isInSafeMode() && getEditLog().isOpenForWrite(); boolean running = dtSecretManager.isRunning(); @@ -4006,29 +4007,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, checkSuperuserPrivilege(); switch(action) { case SAFEMODE_LEAVE: // leave safe mode - if (blockManager.getBytesInFuture() > 0) { - LOG.error("Refusing to leave safe mode without a force flag. " + - "Exiting safe mode will cause a deletion of " + blockManager - .getBytesInFuture() + " byte(s). Please use " + - "-forceExit flag to exit safe mode forcefully and data loss is " + - "acceptable."); - } else { - leaveSafeMode(); - } + leaveSafeMode(false); break; case SAFEMODE_ENTER: // enter safe mode enterSafeMode(false); break; case SAFEMODE_FORCE_EXIT: - if (blockManager.getBytesInFuture() > 0) { - LOG.warn("Leaving safe mode due to forceExit. This will cause a data " - + "loss of " + blockManager.getBytesInFuture() + " byte(s)."); - blockManager.clearBytesInFuture(); - } else { - LOG.warn("forceExit used when normal exist would suffice. Treating " + - "force exit as normal safe mode exit."); - } - leaveSafeMode(); + leaveSafeMode(true); break; default: LOG.error("Unexpected safe mode action"); @@ -4125,16 +4110,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, /** * Leave safe mode. + * @param force true if to leave safe mode forcefully with -forceExit option */ - void leaveSafeMode() { + void leaveSafeMode(boolean force) { writeLock(); try { if (!isInSafeMode()) { NameNode.stateChangeLog.info("STATE* Safe mode is already OFF"); return; } - setManualAndResourceLowSafeMode(false, false); - blockManager.leaveSafeMode(true); + if (blockManager.leaveSafeMode(force)) { + setManualAndResourceLowSafeMode(false, false); + startSecretManagerIfNecessary(); + } } finally { writeUnlock(); } http://git-wip-us.apache.org/repos/asf/hadoop/blob/67c97806/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManagerSafeMode.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManagerSafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManagerSafeMode.java index 606b282..47d4a43 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManagerSafeMode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManagerSafeMode.java @@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerSafeMode.BMSafeModeStatus; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; @@ -31,7 +32,6 @@ import org.apache.hadoop.test.GenericTestUtils; import org.junit.Before; import org.junit.Test; -import org.mockito.Mockito; import org.mockito.internal.util.reflection.Whitebox; import java.io.IOException; @@ -42,7 +42,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -92,9 +91,10 @@ public class TestBlockManagerSafeMode { DATANODE_NUM); FSNamesystem fsn = mock(FSNamesystem.class); - Mockito.doReturn(true).when(fsn).hasWriteLock(); - Mockito.doReturn(true).when(fsn).hasReadLock(); - Mockito.doReturn(true).when(fsn).isRunning(); + doReturn(true).when(fsn).hasWriteLock(); + doReturn(true).when(fsn).hasReadLock(); + doReturn(true).when(fsn).isRunning(); + doReturn(true).when(fsn).isGenStampInFuture(any(Block.class)); NameNode.initMetrics(conf, NamenodeRole.NAMENODE); bm = spy(new BlockManager(fsn, conf)); @@ -110,7 +110,7 @@ public class TestBlockManagerSafeMode { * Test set block total. * * The block total is set which will call checkSafeMode for the first time - * and bmSafeMode transfers from INITIALIZED to PENDING_THRESHOLD status + * and bmSafeMode transfers from OFF to PENDING_THRESHOLD status */ @Test(timeout = 30000) public void testInitialize() { @@ -170,7 +170,7 @@ public class TestBlockManagerSafeMode { * * Once the block threshold is reached, the block manger leaves safe mode and * increment will be a no-op. - * The safe mode status lifecycle: INITIALIZED -> PENDING_THRESHOLD -> OFF + * The safe mode status lifecycle: OFF -> PENDING_THRESHOLD -> OFF */ @Test(timeout = 30000) public void testIncrementSafeBlockCount() { @@ -198,7 +198,7 @@ public class TestBlockManagerSafeMode { * * Once the block threshold is reached, the block manger leaves safe mode and * increment will be a no-op. - * The safe mode status lifecycle: INITIALIZED -> PENDING_THRESHOLD -> EXTENSION-> OFF + * The safe mode status lifecycle: OFF -> PENDING_THRESHOLD -> EXTENSION-> OFF */ @Test(timeout = 30000) public void testIncrementSafeBlockCountWithExtension() throws Exception { @@ -220,7 +220,7 @@ public class TestBlockManagerSafeMode { * Test that the block safe decreases the block safe. * * The block manager stays in safe mode. - * The safe mode status lifecycle: INITIALIZED -> PENDING_THRESHOLD + * The safe mode status lifecycle: OFF -> PENDING_THRESHOLD */ @Test(timeout = 30000) public void testDecrementSafeBlockCount() { @@ -242,7 +242,7 @@ public class TestBlockManagerSafeMode { * Test when the block safe increment and decrement interleave. * * Both the increment and decrement will be a no-op if the safe mode is OFF. - * The safe mode status lifecycle: INITIALIZED -> PENDING_THRESHOLD -> OFF + * The safe mode status lifecycle: OFF -> PENDING_THRESHOLD -> OFF */ @Test(timeout = 30000) public void testIncrementAndDecrementSafeBlockCount() { @@ -309,24 +309,31 @@ public class TestBlockManagerSafeMode { } /** - * Test block manager won't leave safe mode if there are orphan blocks. + * Test block manager won't leave safe mode if there are blocks with + * generation stamp (GS) in future. */ @Test(timeout = 30000) public void testStayInSafeModeWhenBytesInFuture() throws Exception { bmSafeMode.activate(BLOCK_TOTAL); - when(bm.getBytesInFuture()).thenReturn(1L); + // Inject blocks with future GS + injectBlocksWithFugureGS(100L); + assertEquals(100L, bmSafeMode.getBytesInFuture()); + // safe blocks are enough setBlockSafe(BLOCK_THRESHOLD); // PENDING_THRESHOLD -> EXTENSION bmSafeMode.checkSafeMode(); - try { - waitForExtensionPeriod(); - fail("Safe mode should not leave extension period with orphan blocks!"); - } catch (TimeoutException e) { - assertEquals(BMSafeModeStatus.EXTENSION, getSafeModeStatus()); - } + + assertFalse("Shouldn't leave safe mode in case of blocks with future GS! ", + bmSafeMode.leaveSafeMode(false)); + assertTrue("Leaving safe mode forcefully should succeed regardless of " + + "blocks with future GS.", bmSafeMode.leaveSafeMode(true)); + assertEquals("Number of blocks with future GS should have been cleared " + + "after leaving safe mode", 0L, bmSafeMode.getBytesInFuture()); + assertTrue("Leaving safe mode should succeed after blocks with future GS " + + "are cleared.", bmSafeMode.leaveSafeMode(false)); } /** @@ -353,7 +360,7 @@ public class TestBlockManagerSafeMode { tip = bmSafeMode.getSafeModeTip(); assertTrue(tip.contains( String.format("The reported blocks %d has reached the threshold" - + " %.4f of total blocks %d. ", + + " %.4f of total blocks %d. ", getblockSafe(), THRESHOLD, BLOCK_TOTAL))); assertTrue(tip.contains( String.format("The number of live datanodes %d has reached the " + @@ -363,7 +370,6 @@ public class TestBlockManagerSafeMode { waitForExtensionPeriod(); tip = bmSafeMode.getSafeModeTip(); - System.out.println(tip); assertTrue(tip.contains( String.format("The reported blocks %d has reached the threshold" + " %.4f of total blocks %d. ", @@ -375,7 +381,55 @@ public class TestBlockManagerSafeMode { } /** - * Mock block manager internal state for decrement safe block + * Test get safe mode tip in case of blocks with future GS. + */ + @Test(timeout = 30000) + public void testGetSafeModeTipForBlocksWithFutureGS() throws Exception { + bmSafeMode.activate(BLOCK_TOTAL); + + injectBlocksWithFugureGS(40L); + String tip = bmSafeMode.getSafeModeTip(); + assertTrue(tip.contains( + String.format( + "The reported blocks %d needs additional %d blocks to reach the " + + "threshold %.4f of total blocks %d.%n", + 0, BLOCK_THRESHOLD, THRESHOLD, BLOCK_TOTAL))); + assertTrue(tip.contains( + "Name node detected blocks with generation stamps " + + "in future. This means that Name node metadata is inconsistent. " + + "This can happen if Name node metadata files have been manually " + + "replaced. Exiting safe mode will cause loss of " + + 40 + " byte(s). Please restart name node with " + + "right metadata or use \"hdfs dfsadmin -safemode forceExit\" " + + "if you are certain that the NameNode was started with the " + + "correct FsImage and edit logs. If you encountered this during " + + "a rollback, it is safe to exit with -safemode forceExit." + )); + assertFalse(tip.contains("Safe mode will be turned off")); + + // blocks with future GS were already injected before. + setBlockSafe(BLOCK_THRESHOLD); + tip = bmSafeMode.getSafeModeTip(); + assertTrue(tip.contains( + String.format("The reported blocks %d has reached the threshold" + + " %.4f of total blocks %d. ", + getblockSafe(), THRESHOLD, BLOCK_TOTAL))); + assertTrue(tip.contains( + "Name node detected blocks with generation stamps " + + "in future. This means that Name node metadata is inconsistent. " + + "This can happen if Name node metadata files have been manually " + + "replaced. Exiting safe mode will cause loss of " + + 40 + " byte(s). Please restart name node with " + + "right metadata or use \"hdfs dfsadmin -safemode forceExit\" " + + "if you are certain that the NameNode was started with the " + + "correct FsImage and edit logs. If you encountered this during " + + "a rollback, it is safe to exit with -safemode forceExit." + )); + assertFalse(tip.contains("Safe mode will be turned off")); + } + + /** + * Mock block manager internal state for decrement safe block. */ private void mockBlockManagerForBlockSafeDecrement() { BlockInfo storedBlock = mock(BlockInfo.class); @@ -402,6 +456,12 @@ public class TestBlockManagerSafeMode { }, EXTENSION / 10, EXTENSION * 2); } + private void injectBlocksWithFugureGS(long numBytesInFuture) { + BlockReportReplica brr = mock(BlockReportReplica.class); + when(brr.getBytesOnDisk()).thenReturn(numBytesInFuture); + bmSafeMode.checkBlocksWithFutureGS(brr); + } + private void setSafeModeStatus(BMSafeModeStatus status) { Whitebox.setInternalState(bmSafeMode, "status", status); } http://git-wip-us.apache.org/repos/asf/hadoop/blob/67c97806/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java index 69980db..1ae9fb2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java @@ -96,7 +96,7 @@ public class NameNodeAdapter { } public static void leaveSafeMode(NameNode namenode) { - namenode.getNamesystem().leaveSafeMode(); + namenode.getNamesystem().leaveSafeMode(false); } public static void abortEditLogs(NameNode nn) { http://git-wip-us.apache.org/repos/asf/hadoop/blob/67c97806/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java index b697af3..7c9df29 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java @@ -1608,7 +1608,7 @@ public class TestCheckpoint { FSNamesystem fsns = cluster.getNamesystem(); fsns.enterSafeMode(false); fsns.saveNamespace(0, 0); - fsns.leaveSafeMode(); + fsns.leaveSafeMode(false); secondary = startSecondaryNameNode(conf); http://git-wip-us.apache.org/repos/asf/hadoop/blob/67c97806/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogRace.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogRace.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogRace.java index bb7dcdb..fcffbc3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogRace.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogRace.java @@ -306,7 +306,7 @@ public class TestEditLogRace { assertEquals(fsimage.getStorage().getMostRecentCheckpointTxId(), editLog.getLastWrittenTxId() - 1); - namesystem.leaveSafeMode(); + namesystem.leaveSafeMode(false); LOG.info("Save " + i + ": complete"); } } finally { http://git-wip-us.apache.org/repos/asf/hadoop/blob/67c97806/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java index 2bb3d5f..4152712 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java @@ -464,7 +464,7 @@ public class TestFSEditLogLoader { fns.enterSafeMode(false); fns.saveNamespace(0, 0); - fns.leaveSafeMode(); + fns.leaveSafeMode(false); // Add a striped block to the file BlockInfoStriped stripedBlk = new BlockInfoStriped( @@ -542,7 +542,7 @@ public class TestFSEditLogLoader { file.toCompleteFile(System.currentTimeMillis()); fns.enterSafeMode(false); fns.saveNamespace(0, 0); - fns.leaveSafeMode(); + fns.leaveSafeMode(false); //update the last block long newBlkNumBytes = 1024*8; http://git-wip-us.apache.org/repos/asf/hadoop/blob/67c97806/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java index be72192..6308179 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java @@ -105,7 +105,7 @@ public class TestFSNamesystem { Mockito.when(fsImage.getEditLog()).thenReturn(fsEditLog); FSNamesystem fsn = new FSNamesystem(conf, fsImage); - fsn.leaveSafeMode(); + fsn.leaveSafeMode(false); assertTrue("After leaving safemode FSNamesystem.isInStartupSafeMode still " + "returned true", !fsn.isInStartupSafeMode()); assertTrue("After leaving safemode FSNamesystem.isInSafeMode still returned" @@ -145,7 +145,7 @@ public class TestFSNamesystem { assertTrue("FSNamesystem didn't enter safemode", fsn.isInSafeMode()); assertTrue("Replication queues were being populated during very first " + "safemode", !bm.isPopulatingReplQueues()); - fsn.leaveSafeMode(); + fsn.leaveSafeMode(false); assertTrue("FSNamesystem didn't leave safemode", !fsn.isInSafeMode()); assertTrue("Replication queues weren't being populated even after leaving " + "safemode", bm.isPopulatingReplQueues()); http://git-wip-us.apache.org/repos/asf/hadoop/blob/67c97806/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java index 89b2854..98e8426 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java @@ -490,7 +490,7 @@ public class TestINodeFile { // Apply editlogs to fsimage, ensure inodeUnderConstruction is handled fsn.enterSafeMode(false); fsn.saveNamespace(0, 0); - fsn.leaveSafeMode(); + fsn.leaveSafeMode(false); outStream.close(); http://git-wip-us.apache.org/repos/asf/hadoop/blob/67c97806/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetadataConsistency.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetadataConsistency.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetadataConsistency.java index d4fb0d1..367e3fa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetadataConsistency.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetadataConsistency.java @@ -23,26 +23,26 @@ import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; -import org.hamcrest.CoreMatchers; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; + import org.junit.After; import org.junit.Before; import org.junit.Test; -import org.mockito.Mockito; -import org.mockito.internal.util.reflection.Whitebox; import java.io.IOException; import java.io.OutputStream; import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThat; -import static org.mockito.Mockito.*; +import static org.junit.Assert.assertTrue; public class TestNameNodeMetadataConsistency { private static final Path filePath1 = new Path("/testdata1.txt"); private static final Path filePath2 = new Path("/testdata2.txt"); + private static final String TEST_DATA_IN_FUTURE = "This is test data"; private static final int SCAN_INTERVAL = 1; private static final int SCAN_WAIT = 3; @@ -75,59 +75,45 @@ public class TestNameNodeMetadataConsistency { @Test public void testGenerationStampInFuture() throws IOException, InterruptedException { - - String testData = " This is test data"; - int datalen = testData.length(); - cluster.waitActive(); + FileSystem fs = cluster.getFileSystem(); OutputStream ostream = fs.create(filePath1); - ostream.write(testData.getBytes()); + ostream.write(TEST_DATA_IN_FUTURE.getBytes()); ostream.close(); - ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, filePath1); - long genStamp = block.getGenerationStamp(); - // Re-write the Generation Stamp to a Generation Stamp in future. - cluster.changeGenStampOfBlock(0, block, genStamp + 1); - MiniDFSCluster.DataNodeProperties dnProps = cluster.stopDataNode(0); - + ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, filePath1); + final long genStamp = block.getGenerationStamp(); + final int datanodeIndex = 0; + cluster.changeGenStampOfBlock(datanodeIndex, block, genStamp + 1); + // stop the data node so that it won't remove block + final DataNodeProperties dnProps = cluster.stopDataNode(datanodeIndex); - // Simulate Namenode forgetting a Block + // Simulate Namenode forgetting a Block cluster.restartNameNode(true); - BlockInfo bInfo = cluster.getNameNode().getNamesystem().getBlockManager - ().getStoredBlock(block.getLocalBlock()); cluster.getNameNode().getNamesystem().writeLock(); + BlockInfo bInfo = cluster.getNameNode().getNamesystem().getBlockManager() + .getStoredBlock(block.getLocalBlock()); cluster.getNameNode().getNamesystem().getBlockManager() .removeBlock(bInfo); cluster.getNameNode().getNamesystem().writeUnlock(); // we also need to tell block manager that we are in the startup path - FSNamesystem spyNameSystem = spy(cluster.getNameNode().getNamesystem()); - Whitebox.setInternalState(cluster.getNameNode() - .getNamesystem().getBlockManager(), - "namesystem", spyNameSystem); - Whitebox.setInternalState(cluster.getNameNode(), - "namesystem", spyNameSystem); - Mockito.doReturn(true).when(spyNameSystem).isInStartupSafeMode(); - + BlockManagerTestUtil.setStartupSafeModeForTest( + cluster.getNameNode().getNamesystem().getBlockManager()); - // Since Data Node is already shutdown we didn't remove blocks cluster.restartDataNode(dnProps); waitTil(TimeUnit.SECONDS.toMillis(SCAN_WAIT)); cluster.triggerBlockReports(); - - // Give some buffer to process the block reports waitTil(TimeUnit.SECONDS.toMillis(SCAN_WAIT)); // Make sure that we find all written bytes in future block - assertEquals(datalen, cluster.getNameNode().getBytesWithFutureGenerationStamps()); - + assertEquals(TEST_DATA_IN_FUTURE.length(), + cluster.getNameNode().getBytesWithFutureGenerationStamps()); // Assert safemode reason - String safeModeMessage = cluster.getNameNode().getNamesystem() - .getSafeModeTip(); - assertThat(safeModeMessage, CoreMatchers.containsString("Name node " + - "detected blocks with generation stamps in future")); + assertTrue(cluster.getNameNode().getNamesystem().getSafeModeTip().contains( + "Name node detected blocks with generation stamps in future")); } /**