From 0cb3738567d0a9ffa2dd1402a402f1e7d9afbb6e Mon Sep 17 00:00:00 2001 From: Surendra Singh Lilhore Date: Sun, 18 Aug 2019 18:12:19 -0700 Subject: [PATCH] HDFS-14687. Standby Namenode never come out of safemode when EC files are being written. Contributed by Surendra Singh Lilhore. Reviewed-by: Siyao Meng Reviewed-by: Wei-Chiu CHuang Signed-off-by: Wei-Chiu Chuang (cherry picked from commit b8db5b9a9812023754ed1b3e5b428e161f0add50) (cherry picked from commit 6b01effd011ab1f88bdbb9f2dbf9bbb0765984e6) --- .../PendingDataNodeMessages.java | 13 ++++-- .../TestPendingDataNodeMessages.java | 46 +++++++++++++++++++ 2 files changed, 56 insertions(+), 3 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java index 133a28826b0..6e9dfa2c008 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java @@ -95,9 +95,16 @@ void removeAllMessagesForDatanode(DatanodeDescriptor dn) { void enqueueReportedBlock(DatanodeStorageInfo storageInfo, Block block, ReplicaState reportedState) { - block = new Block(block); - getBlockQueue(block).add( - new ReportedBlockInfo(storageInfo, block, reportedState)); + if (BlockIdManager.isStripedBlockID(block.getBlockId())) { + Block blkId = new Block(BlockIdManager.convertToStripedID(block + .getBlockId())); + getBlockQueue(blkId).add( + new ReportedBlockInfo(storageInfo, new Block(block), reportedState)); + } else { + block = new Block(block); + getBlockQueue(block).add( + new ReportedBlockInfo(storageInfo, block, reportedState)); + } count++; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingDataNodeMessages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingDataNodeMessages.java index dac89af1e91..f1ba8022642 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingDataNodeMessages.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingDataNodeMessages.java @@ -22,10 +22,19 @@ import java.util.Queue; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.MiniDFSNNTopology; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies; import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; +import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.junit.Test; @@ -67,4 +76,41 @@ public void testQueues() { assertNull(msgs.takeBlockQueue(block1Gs1)); assertEquals(0, msgs.count()); } + + @Test + public void testPendingDataNodeMessagesWithEC() throws Exception { + ErasureCodingPolicy ecPolicy = SystemErasureCodingPolicies.getPolicies() + .get(3); + Path dirPath = new Path("/testPendingDataNodeMessagesWithEC"); + Configuration conf = new Configuration(); + conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 20 * 60000); + + int numDn = ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits(); + final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf) + .numDataNodes(numDn).nnTopology(MiniDFSNNTopology.simpleHATopology()) + .build(); + try { + cluster.transitionToActive(0); + + DistributedFileSystem fs = HATestUtil.configureFailoverFs(cluster, conf); + fs.enableErasureCodingPolicy(ecPolicy.getName()); + fs.mkdirs(dirPath); + fs.setErasureCodingPolicy(dirPath, ecPolicy.getName()); + + DFSTestUtil.createFile(fs, new Path(dirPath, "file"), + ecPolicy.getCellSize() * ecPolicy.getNumDataUnits(), (short) 1, 0); + + cluster.getNameNode(0).getRpcServer().rollEditLog(); + cluster.getNameNode(1).getNamesystem().getEditLogTailer().doTailEdits(); + + // PendingDataNodeMessages datanode message queue should be empty after + // processing IBR + int pendingIBRMsg = cluster.getNameNode(1).getNamesystem() + .getBlockManager().getPendingDataNodeMessageCount(); + assertEquals("All DN message should processed after tail edits", 0, + pendingIBRMsg); + } finally { + cluster.shutdown(); + } + } }