HDFS-3119. Overreplicated block is not deleted even after the replication factor is reduced after sync follwed by closing that file. Contributed by Ashish Singhi.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1311380 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
909176c1fc
commit
3ced5ea066
@ -117,6 +117,10 @@ Trunk (unreleased changes)
|
||||
|
||||
HDFS-3121. Add HDFS tests for HADOOP-8014 change. (John George via
|
||||
suresh)
|
||||
|
||||
HDFS-3119. Overreplicated block is not deleted even after the replication
|
||||
factor is reduced after sync follwed by closing that file. (Ashish Singhi
|
||||
via umamahesh)
|
||||
|
||||
Release 2.0.0 - UNRELEASED
|
||||
|
||||
|
@ -2767,7 +2767,7 @@ private void updateNeededReplications(final Block block,
|
||||
}
|
||||
}
|
||||
|
||||
public void checkReplication(Block block, int numExpectedReplicas) {
|
||||
public void checkReplication(Block block, short numExpectedReplicas) {
|
||||
// filter out containingNodes that are marked for decommission.
|
||||
NumberReplicas number = countNodes(block);
|
||||
if (isNeededReplication(block, numExpectedReplicas, number.liveReplicas())) {
|
||||
@ -2775,6 +2775,10 @@ public void checkReplication(Block block, int numExpectedReplicas) {
|
||||
number.liveReplicas(),
|
||||
number.decommissionedReplicas(),
|
||||
numExpectedReplicas);
|
||||
return;
|
||||
}
|
||||
if (number.liveReplicas() > numExpectedReplicas) {
|
||||
processOverReplicatedBlock(block, numExpectedReplicas, null, null);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2120,10 +2120,12 @@ private boolean completeFileInternal(String src,
|
||||
|
||||
/**
|
||||
* Check all blocks of a file. If any blocks are lower than their intended
|
||||
* replication factor, then insert them into neededReplication
|
||||
* replication factor, then insert them into neededReplication and if
|
||||
* the blocks are more than the intended replication factor then insert
|
||||
* them into invalidateBlocks.
|
||||
*/
|
||||
private void checkReplicationFactor(INodeFile file) {
|
||||
int numExpectedReplicas = file.getReplication();
|
||||
short numExpectedReplicas = file.getReplication();
|
||||
Block[] pendingBlocks = file.getBlocks();
|
||||
int nrBlocks = pendingBlocks.length;
|
||||
for (int i = 0; i < nrBlocks; i++) {
|
||||
|
@ -17,12 +17,13 @@
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.server.blockmanagement;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
@ -36,13 +37,15 @@
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestOverReplicatedBlocks extends TestCase {
|
||||
public class TestOverReplicatedBlocks {
|
||||
/** Test processOverReplicatedBlock can handle corrupt replicas fine.
|
||||
* It make sure that it won't treat corrupt replicas as valid ones
|
||||
* thus prevents NN deleting valid replicas but keeping
|
||||
* corrupt ones.
|
||||
*/
|
||||
@Test
|
||||
public void testProcesOverReplicateBlock() throws IOException {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L);
|
||||
@ -113,4 +116,30 @@ public void testProcesOverReplicateBlock() throws IOException {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
/**
|
||||
* Test over replicated block should get invalidated when decreasing the
|
||||
* replication for a partial block.
|
||||
*/
|
||||
@Test
|
||||
public void testInvalidateOverReplicatedBlock() throws Exception {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
|
||||
.build();
|
||||
try {
|
||||
final FSNamesystem namesystem = cluster.getNamesystem();
|
||||
final BlockManager bm = namesystem.getBlockManager();
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
Path p = new Path(MiniDFSCluster.getBaseDirectory(), "/foo1");
|
||||
FSDataOutputStream out = fs.create(p, (short) 2);
|
||||
out.writeBytes("HDFS-3119: " + p);
|
||||
out.hsync();
|
||||
fs.setReplication(p, (short) 1);
|
||||
out.close();
|
||||
ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, p);
|
||||
assertEquals("Expected only one live replica for the block", 1, bm
|
||||
.countNodes(block.getLocalBlock()).liveReplicas());
|
||||
} finally {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user