You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by we...@apache.org on 2019/08/19 01:22:48 UTC
[hadoop] branch branch-3.0 updated: HDFS-14687. Standby Namenode
never come out of safemode when EC files are being written. Contributed by
Surendra Singh Lilhore.
This is an automated email from the ASF dual-hosted git repository.
weichiu pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/hadoop.git
The following commit(s) were added to refs/heads/branch-3.0 by this push:
new 0cb3738 HDFS-14687. Standby Namenode never come out of safemode when EC files are being written. Contributed by Surendra Singh Lilhore.
0cb3738 is described below
commit 0cb3738567d0a9ffa2dd1402a402f1e7d9afbb6e
Author: Surendra Singh Lilhore <su...@apache.org>
AuthorDate: Sun Aug 18 18:12:19 2019 -0700
HDFS-14687. Standby Namenode never come out of safemode when EC files are being written. Contributed by Surendra Singh Lilhore.
Reviewed-by: Siyao Meng <sm...@cloudera.com>
Reviewed-by: Wei-Chiu CHuang <we...@apache.org>
Signed-off-by: Wei-Chiu Chuang <we...@apache.org>
(cherry picked from commit b8db5b9a9812023754ed1b3e5b428e161f0add50)
(cherry picked from commit 6b01effd011ab1f88bdbb9f2dbf9bbb0765984e6)
---
.../blockmanagement/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 133a288..6e9dfa2 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 @@ class PendingDataNodeMessages {
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 dac89af..f1ba802 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 static org.junit.Assert.assertNull;
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 class TestPendingDataNodeMessages {
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();
+ }
+ }
}
---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org