Skip to content

Commit 75fc1f8

Browse files
surendralilhoresmengcl
authored andcommitted
CDH-81547. HDFS-14687. Standby Namenode never come out of safemode when EC files are being written. Contributed by Surendra Singh Lilhore.
Reviewed-by: Siyao Meng <[email protected]> Reviewed-by: Wei-Chiu CHuang <[email protected]> Signed-off-by: Wei-Chiu Chuang <[email protected]> (cherry picked from commit b8db5b9) Change-Id: I57856ad89adea25ed7d5527f155ebb80f17bdffd
1 parent 32d396f commit 75fc1f8

File tree

2 files changed

+56
-3
lines changed

2 files changed

+56
-3
lines changed

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -95,9 +95,16 @@ void removeAllMessagesForDatanode(DatanodeDescriptor dn) {
9595

9696
void enqueueReportedBlock(DatanodeStorageInfo storageInfo, Block block,
9797
ReplicaState reportedState) {
98-
block = new Block(block);
99-
getBlockQueue(block).add(
100-
new ReportedBlockInfo(storageInfo, block, reportedState));
98+
if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
99+
Block blkId = new Block(BlockIdManager.convertToStripedID(block
100+
.getBlockId()));
101+
getBlockQueue(blkId).add(
102+
new ReportedBlockInfo(storageInfo, new Block(block), reportedState));
103+
} else {
104+
block = new Block(block);
105+
getBlockQueue(block).add(
106+
new ReportedBlockInfo(storageInfo, block, reportedState));
107+
}
101108
count++;
102109
}
103110

hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingDataNodeMessages.java

Lines changed: 46 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,10 +22,19 @@
2222

2323
import java.util.Queue;
2424

25+
import org.apache.hadoop.conf.Configuration;
26+
import org.apache.hadoop.fs.Path;
27+
import org.apache.hadoop.hdfs.DFSConfigKeys;
2528
import org.apache.hadoop.hdfs.DFSTestUtil;
29+
import org.apache.hadoop.hdfs.DistributedFileSystem;
30+
import org.apache.hadoop.hdfs.MiniDFSCluster;
31+
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
2632
import org.apache.hadoop.hdfs.protocol.Block;
33+
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
34+
import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
2735
import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
2836
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
37+
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
2938
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
3039
import org.junit.Test;
3140

@@ -67,4 +76,41 @@ public void testQueues() {
6776
assertNull(msgs.takeBlockQueue(block1Gs1));
6877
assertEquals(0, msgs.count());
6978
}
79+
80+
@Test
81+
public void testPendingDataNodeMessagesWithEC() throws Exception {
82+
ErasureCodingPolicy ecPolicy = SystemErasureCodingPolicies.getPolicies()
83+
.get(3);
84+
Path dirPath = new Path("/testPendingDataNodeMessagesWithEC");
85+
Configuration conf = new Configuration();
86+
conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 20 * 60000);
87+
88+
int numDn = ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits();
89+
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
90+
.numDataNodes(numDn).nnTopology(MiniDFSNNTopology.simpleHATopology())
91+
.build();
92+
try {
93+
cluster.transitionToActive(0);
94+
95+
DistributedFileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
96+
fs.enableErasureCodingPolicy(ecPolicy.getName());
97+
fs.mkdirs(dirPath);
98+
fs.setErasureCodingPolicy(dirPath, ecPolicy.getName());
99+
100+
DFSTestUtil.createFile(fs, new Path(dirPath, "file"),
101+
ecPolicy.getCellSize() * ecPolicy.getNumDataUnits(), (short) 1, 0);
102+
103+
cluster.getNameNode(0).getRpcServer().rollEditLog();
104+
cluster.getNameNode(1).getNamesystem().getEditLogTailer().doTailEdits();
105+
106+
// PendingDataNodeMessages datanode message queue should be empty after
107+
// processing IBR
108+
int pendingIBRMsg = cluster.getNameNode(1).getNamesystem()
109+
.getBlockManager().getPendingDataNodeMessageCount();
110+
assertEquals("All DN message should processed after tail edits", 0,
111+
pendingIBRMsg);
112+
} finally {
113+
cluster.shutdown();
114+
}
115+
}
70116
}

0 commit comments

Comments
 (0)