Revert "HDFS-14849. Erasure Coding: the internal block is replicated many times when datanode is decommissioning. Contributed by HuangTao."
This reverts commit ce58c05f1d
.
This commit is contained in:
parent
14b4fbc019
commit
0d5d0b914a
|
@ -2359,15 +2359,13 @@ public class BlockManager implements BlockStatsMXBean {
|
||||||
if (isStriped) {
|
if (isStriped) {
|
||||||
blockIndex = ((BlockInfoStriped) block)
|
blockIndex = ((BlockInfoStriped) block)
|
||||||
.getStorageBlockIndex(storage);
|
.getStorageBlockIndex(storage);
|
||||||
if (state == StoredReplicaState.LIVE) {
|
|
||||||
if (!bitSet.get(blockIndex)) {
|
if (!bitSet.get(blockIndex)) {
|
||||||
bitSet.set(blockIndex);
|
bitSet.set(blockIndex);
|
||||||
} else {
|
} else if (state == StoredReplicaState.LIVE) {
|
||||||
numReplicas.subtract(StoredReplicaState.LIVE, 1);
|
numReplicas.subtract(StoredReplicaState.LIVE, 1);
|
||||||
numReplicas.add(StoredReplicaState.REDUNDANT, 1);
|
numReplicas.add(StoredReplicaState.REDUNDANT, 1);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
if (node.getNumberOfBlocksToBeReplicated() >= replicationStreamsHardLimit) {
|
if (node.getNumberOfBlocksToBeReplicated() >= replicationStreamsHardLimit) {
|
||||||
continue;
|
continue;
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs;
|
||||||
|
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
@ -30,6 +31,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,6 @@ import com.google.common.base.Joiner;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.LinkedListMultimap;
|
import com.google.common.collect.LinkedListMultimap;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
|
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.CreateFlag;
|
import org.apache.hadoop.fs.CreateFlag;
|
||||||
|
@ -749,63 +748,6 @@ public class TestBlockManager {
|
||||||
numReplicas.redundantInternalBlocks());
|
numReplicas.redundantInternalBlocks());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testChooseSrcDNWithDupECInDecommissioningNode() throws Exception {
|
|
||||||
long blockId = -9223372036854775776L; // real ec block id
|
|
||||||
Block aBlock = new Block(blockId, 0, 0);
|
|
||||||
// RS-3-2 EC policy
|
|
||||||
ErasureCodingPolicy ecPolicy =
|
|
||||||
SystemErasureCodingPolicies.getPolicies().get(1);
|
|
||||||
// striped blockInfo
|
|
||||||
BlockInfoStriped aBlockInfoStriped = new BlockInfoStriped(aBlock, ecPolicy);
|
|
||||||
// ec storageInfo
|
|
||||||
DatanodeStorageInfo ds1 = DFSTestUtil.createDatanodeStorageInfo(
|
|
||||||
"storage1", "1.1.1.1", "rack1", "host1");
|
|
||||||
DatanodeStorageInfo ds2 = DFSTestUtil.createDatanodeStorageInfo(
|
|
||||||
"storage2", "2.2.2.2", "rack2", "host2");
|
|
||||||
DatanodeStorageInfo ds3 = DFSTestUtil.createDatanodeStorageInfo(
|
|
||||||
"storage3", "3.3.3.3", "rack3", "host3");
|
|
||||||
DatanodeStorageInfo ds4 = DFSTestUtil.createDatanodeStorageInfo(
|
|
||||||
"storage4", "4.4.4.4", "rack4", "host4");
|
|
||||||
DatanodeStorageInfo ds5 = DFSTestUtil.createDatanodeStorageInfo(
|
|
||||||
"storage5", "5.5.5.5", "rack5", "host5");
|
|
||||||
DatanodeStorageInfo ds6 = DFSTestUtil.createDatanodeStorageInfo(
|
|
||||||
"storage6", "6.6.6.6", "rack6", "host6");
|
|
||||||
|
|
||||||
// link block with storage
|
|
||||||
aBlockInfoStriped.addStorage(ds1, aBlock);
|
|
||||||
aBlockInfoStriped.addStorage(ds2, new Block(blockId + 1, 0, 0));
|
|
||||||
aBlockInfoStriped.addStorage(ds3, new Block(blockId + 2, 0, 0));
|
|
||||||
aBlockInfoStriped.addStorage(ds4, new Block(blockId + 3, 0, 0));
|
|
||||||
aBlockInfoStriped.addStorage(ds5, new Block(blockId + 4, 0, 0));
|
|
||||||
// NOTE: duplicate block 0,this DN will replace the decommission ds1 DN
|
|
||||||
aBlockInfoStriped.addStorage(ds6, aBlock);
|
|
||||||
|
|
||||||
addEcBlockToBM(blockId, ecPolicy);
|
|
||||||
// decommission datanode where store block 0
|
|
||||||
ds1.getDatanodeDescriptor().startDecommission();
|
|
||||||
|
|
||||||
List<DatanodeDescriptor> containingNodes =
|
|
||||||
new LinkedList<DatanodeDescriptor>();
|
|
||||||
List<DatanodeStorageInfo> nodesContainingLiveReplicas =
|
|
||||||
new LinkedList<DatanodeStorageInfo>();
|
|
||||||
NumberReplicas numReplicas = new NumberReplicas();
|
|
||||||
List<Byte> liveBlockIndices = new ArrayList<>();
|
|
||||||
|
|
||||||
bm.chooseSourceDatanodes(
|
|
||||||
aBlockInfoStriped,
|
|
||||||
containingNodes,
|
|
||||||
nodesContainingLiveReplicas,
|
|
||||||
numReplicas, liveBlockIndices,
|
|
||||||
LowRedundancyBlocks.QUEUE_HIGHEST_PRIORITY);
|
|
||||||
assertEquals("There are 5 live replicas in " +
|
|
||||||
"[ds2, ds3, ds4, ds5, ds6] datanodes ",
|
|
||||||
5, numReplicas.liveReplicas());
|
|
||||||
assertEquals("The ds1 datanode is in decommissioning, " +
|
|
||||||
"so there is no redundant replica",
|
|
||||||
0, numReplicas.redundantInternalBlocks());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testFavorDecomUntilHardLimit() throws Exception {
|
public void testFavorDecomUntilHardLimit() throws Exception {
|
||||||
bm.maxReplicationStreams = 0;
|
bm.maxReplicationStreams = 0;
|
||||||
|
|
Loading…
Reference in New Issue