Skip to content

Commit f9bb2a8

Browse files
committed
HDFS-15200. Delete Corrupt Replica Immediately Irrespective of Replicas On Stale Storage. Contributed by Ayush Saxena.
1 parent bb41dda commit f9bb2a8

File tree

6 files changed

+71
-2
lines changed

6 files changed

+71
-2
lines changed

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -304,6 +304,13 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
304304
= "dfs.namenode.blockreport.max.lock.hold.time";
305305
public static final long
306306
DFS_NAMENODE_BLOCKREPORT_MAX_LOCK_HOLD_TIME_DEFAULT = 4;
307+
308+
public static final String
309+
DFS_NAMENODE_CORRUPT_BLOCK_DELETE_IMMEDIATELY_ENABLED =
310+
"dfs.namenode.corrupt.block.delete.immediately.enabled";
311+
public static final boolean
312+
DFS_NAMENODE_CORRUPT_BLOCK_DELETE_IMMEDIATELY_ENABLED_DEFAULT = true;
313+
307314
@Deprecated
308315
public static final String DFS_WEBHDFS_USER_PATTERN_KEY =
309316
HdfsClientConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY;

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

Lines changed: 10 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -459,6 +459,11 @@ public long getTotalECBlockGroups() {
459459
* from ENTERING_MAINTENANCE to IN_MAINTENANCE.
460460
*/
461461
private final short minReplicationToBeInMaintenance;
462+
/**
463+
* Whether to delete corrupt replica immediately irrespective of other
464+
* replicas available on stale storages.
465+
*/
466+
private final boolean deleteCorruptReplicaImmediately;
462467

463468
/** Storages accessible from multiple DNs. */
464469
private final ProvidedStorageMap providedStorageMap;
@@ -615,6 +620,10 @@ public BlockManager(final Namesystem namesystem, boolean haEnabled,
615620
DFSConfigKeys.DFS_NAMENODE_BLOCKREPORT_QUEUE_SIZE_DEFAULT);
616621
blockReportThread = new BlockReportProcessingThread(queueSize);
617622

623+
this.deleteCorruptReplicaImmediately =
624+
conf.getBoolean(DFS_NAMENODE_CORRUPT_BLOCK_DELETE_IMMEDIATELY_ENABLED,
625+
DFS_NAMENODE_CORRUPT_BLOCK_DELETE_IMMEDIATELY_ENABLED_DEFAULT);
626+
618627
LOG.info("defaultReplication = {}", defaultReplication);
619628
LOG.info("maxReplication = {}", maxReplication);
620629
LOG.info("minReplication = {}", minReplication);
@@ -1870,7 +1879,7 @@ private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn,
18701879
}
18711880

18721881
// Check how many copies we have of the block
1873-
if (nr.replicasOnStaleNodes() > 0) {
1882+
if (nr.replicasOnStaleNodes() > 0 && !deleteCorruptReplicaImmediately) {
18741883
blockLog.debug("BLOCK* invalidateBlocks: postponing " +
18751884
"invalidation of {} on {} because {} replica(s) are located on " +
18761885
"nodes with potentially out-of-date block reports", b, dn,

hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -5777,6 +5777,16 @@
57775777
</description>
57785778
</property>
57795779

5780+
5781+
<property>
5782+
<name>dfs.namenode.corrupt.block.delete.immediately.enabled</name>
5783+
<value>true</value>
5784+
<description>
5785+
Whether the corrupt replicas should be deleted immediately, irrespective
5786+
of other replicas on stale storages..
5787+
</description>
5788+
</property>
5789+
57805790
<property>
57815791
<name>dfs.journalnode.edits.dir.perm</name>
57825792
<value>700</value>

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

Lines changed: 37 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,9 @@
2121
import com.google.common.collect.ImmutableList;
2222
import com.google.common.collect.LinkedListMultimap;
2323
import com.google.common.collect.Lists;
24+
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
2425
import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
26+
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
2527
import org.apache.hadoop.hdfs.server.namenode.NameNode;
2628
import org.slf4j.LoggerFactory;
2729
import org.apache.hadoop.conf.Configuration;
@@ -502,7 +504,41 @@ public void testNeededReconstructionWhileAppending() throws IOException {
502504
}
503505
}
504506
}
505-
507+
508+
@Test(timeout = 60000)
509+
public void testDeleteCorruptReplicaWithStatleStorages() throws Exception {
510+
Configuration conf = new HdfsConfiguration();
511+
conf.setInt(HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.
512+
MIN_REPLICATION, 2);
513+
Path file = new Path("/test-file");
514+
MiniDFSCluster cluster =
515+
new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
516+
try {
517+
cluster.waitActive();
518+
BlockManager blockManager = cluster.getNamesystem().getBlockManager();
519+
blockManager.getDatanodeManager().markAllDatanodesStale();
520+
FileSystem fs = cluster.getFileSystem();
521+
FSDataOutputStream out = fs.create(file);
522+
for (int i = 0; i < 1024 * 1024 * 1; i++) {
523+
out.write(i);
524+
}
525+
out.hflush();
526+
MiniDFSCluster.DataNodeProperties datanode = cluster.stopDataNode(0);
527+
for (int i = 0; i < 1024 * 1024 * 1; i++) {
528+
out.write(i);
529+
}
530+
out.close();
531+
cluster.restartDataNode(datanode);
532+
cluster.triggerBlockReports();
533+
DataNodeTestUtils.triggerBlockReport(datanode.getDatanode());
534+
assertEquals(0, blockManager.getCorruptBlocks());
535+
} finally {
536+
if (cluster != null) {
537+
cluster.shutdown();
538+
}
539+
}
540+
}
541+
506542
/**
507543
* Tell the block manager that replication is completed for the given
508544
* pipeline.

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

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,8 @@
2727
import org.apache.hadoop.test.GenericTestUtils;
2828
import org.junit.Test;
2929

30+
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CORRUPT_BLOCK_DELETE_IMMEDIATELY_ENABLED;
31+
3032
/**
3133
* Tests corruption of replicas in case of failover.
3234
*/
@@ -35,6 +37,8 @@ public class TestCorruptionWithFailover {
3537
@Test
3638
public void testCorruptReplicaAfterFailover() throws Exception {
3739
Configuration conf = new Configuration();
40+
conf.setBoolean(DFS_NAMENODE_CORRUPT_BLOCK_DELETE_IMMEDIATELY_ENABLED,
41+
false);
3842
// Enable data to be written, to less replicas in case of pipeline failure.
3943
conf.setInt(HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.
4044
MIN_REPLICATION, 2);

hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818

1919
package org.apache.hadoop.hdfs.server.namenode;
2020

21+
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CORRUPT_BLOCK_DELETE_IMMEDIATELY_ENABLED;
2122
import static org.apache.hadoop.hdfs.MiniDFSCluster.HDFS_MINIDFS_BASEDIR;
2223
import static org.junit.Assert.assertEquals;
2324
import static org.junit.Assert.assertFalse;
@@ -187,6 +188,8 @@ public static String runFsck(Configuration conf, int expectedErrCode,
187188
@Before
188189
public void setUp() throws Exception {
189190
conf = new Configuration();
191+
conf.setBoolean(DFS_NAMENODE_CORRUPT_BLOCK_DELETE_IMMEDIATELY_ENABLED,
192+
false);
190193
}
191194

192195
@After

0 commit comments

Comments
 (0)