HDFS-11032: [SPS]: Handling of block movement failure at the coordinator datanode. Contributed by Rakesh R

This commit is contained in:
Uma Maheswara Rao G 2016-12-22 17:07:49 -08:00 committed by Uma Maheswara Rao Gangumalla
parent 5179d99b7e
commit d81611fe55
2 changed files with 143 additions and 34 deletions

View File

@ -50,6 +50,7 @@
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockPinningException;
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
@ -251,6 +252,12 @@ private BlockMovementStatus moveBlock() {
+ " satisfying storageType:{}",
block, source, target, targetStorageType);
return BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_SUCCESS;
} catch (BlockPinningException e) {
// Pinned block won't be able to move to a different node. So, its not
// required to do retries, just marked as SUCCESS.
LOG.debug("Pinned block can't be moved, so skipping block:{}", block,
e);
return BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_SUCCESS;
} catch (IOException e) {
// TODO: handle failure retries
LOG.warn(
@ -282,7 +289,7 @@ private void receiveResponse(DataInputStream in) throws IOException {
response = BlockOpResponseProto.parseFrom(vintPrefixed(in));
}
String logInfo = "reportedBlock move is failed";
DataTransferProtoUtil.checkBlockOpStatus(response, logInfo);
DataTransferProtoUtil.checkBlockOpStatus(response, logInfo, true);
}
}

View File

@ -21,6 +21,7 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.TimeoutException;
@ -36,12 +37,15 @@
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.client.HdfsAdmin;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -66,10 +70,16 @@ public class TestStoragePolicySatisfier {
final private long capacity = 2 * 256 * 1024 * 1024;
final private String file = "/testMoveWhenStoragePolicyNotSatisfying";
private DistributedFileSystem dfs = null;
private static final int DEFAULT_BLOCK_SIZE = 1024;
@Before
public void setUp() throws IOException {
config.setLong("dfs.block.size", 1024);
private void shutdownCluster() {
if (hdfsCluster != null) {
hdfsCluster.shutdown();
}
}
private void createCluster() throws IOException {
config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
hdfsCluster = startCluster(config, allDiskTypes, numOfDatanodes,
storagesPerDatanode, capacity);
dfs = hdfsCluster.getFileSystem();
@ -81,6 +91,7 @@ public void testWhenStoragePolicySetToCOLD()
throws Exception {
try {
createCluster();
// Change policy to COLD
dfs.setStoragePolicy(new Path(file), "COLD");
FSNamesystem namesystem = hdfsCluster.getNamesystem();
@ -99,7 +110,7 @@ public void testWhenStoragePolicySetToCOLD()
// Wait till namenode notified about the block location details
waitExpectedStorageType(file, StorageType.ARCHIVE, 3, 30000);
} finally {
hdfsCluster.shutdown();
shutdownCluster();
}
}
@ -107,6 +118,7 @@ public void testWhenStoragePolicySetToCOLD()
public void testWhenStoragePolicySetToALLSSD()
throws Exception {
try {
createCluster();
// Change policy to ALL_SSD
dfs.setStoragePolicy(new Path(file), "ALL_SSD");
FSNamesystem namesystem = hdfsCluster.getNamesystem();
@ -127,7 +139,7 @@ public void testWhenStoragePolicySetToALLSSD()
// areas
waitExpectedStorageType(file, StorageType.SSD, 3, 30000);
} finally {
hdfsCluster.shutdown();
shutdownCluster();
}
}
@ -135,6 +147,7 @@ public void testWhenStoragePolicySetToALLSSD()
public void testWhenStoragePolicySetToONESSD()
throws Exception {
try {
createCluster();
// Change policy to ONE_SSD
dfs.setStoragePolicy(new Path(file), "ONE_SSD");
FSNamesystem namesystem = hdfsCluster.getNamesystem();
@ -154,7 +167,7 @@ public void testWhenStoragePolicySetToONESSD()
waitExpectedStorageType(file, StorageType.SSD, 1, 30000);
waitExpectedStorageType(file, StorageType.DISK, 2, 30000);
} finally {
hdfsCluster.shutdown();
shutdownCluster();
}
}
@ -165,6 +178,7 @@ public void testWhenStoragePolicySetToONESSD()
@Test(timeout = 300000)
public void testPerTrackIdBlocksStorageMovementResults() throws Exception {
try {
createCluster();
// Change policy to ONE_SSD
dfs.setStoragePolicy(new Path(file), "ONE_SSD");
FSNamesystem namesystem = hdfsCluster.getNamesystem();
@ -186,7 +200,7 @@ public void testPerTrackIdBlocksStorageMovementResults() throws Exception {
waitForBlocksMovementResult(1, 30000);
} finally {
hdfsCluster.shutdown();
shutdownCluster();
}
}
@ -196,18 +210,18 @@ public void testPerTrackIdBlocksStorageMovementResults() throws Exception {
*/
@Test(timeout = 300000)
public void testMultipleFilesForSatisfyStoragePolicy() throws Exception {
List<String> files = new ArrayList<>();
files.add(file);
// Creates 4 more files. Send all of them for satisfying the storage policy
// together.
for (int i = 0; i < 4; i++) {
String file1 = "/testMoveWhenStoragePolicyNotSatisfying_" + i;
files.add(file1);
writeContent(file1);
}
try {
createCluster();
List<String> files = new ArrayList<>();
files.add(file);
// Creates 4 more files. Send all of them for satisfying the storage
// policy together.
for (int i = 0; i < 4; i++) {
String file1 = "/testMoveWhenStoragePolicyNotSatisfying_" + i;
files.add(file1);
writeContent(file1);
}
FSNamesystem namesystem = hdfsCluster.getNamesystem();
List<Long> blockCollectionIds = new ArrayList<>();
// Change policy to ONE_SSD
@ -237,7 +251,7 @@ public void testMultipleFilesForSatisfyStoragePolicy() throws Exception {
waitForBlocksMovementResult(blockCollectionIds.size(), 30000);
} finally {
hdfsCluster.shutdown();
shutdownCluster();
}
}
@ -247,10 +261,10 @@ public void testMultipleFilesForSatisfyStoragePolicy() throws Exception {
*/
@Test(timeout = 300000)
public void testSatisfyFileWithHdfsAdmin() throws Exception {
HdfsAdmin hdfsAdmin =
new HdfsAdmin(FileSystem.getDefaultUri(config), config);
try {
createCluster();
HdfsAdmin hdfsAdmin =
new HdfsAdmin(FileSystem.getDefaultUri(config), config);
// Change policy to COLD
dfs.setStoragePolicy(new Path(file), "COLD");
@ -267,7 +281,7 @@ public void testSatisfyFileWithHdfsAdmin() throws Exception {
// Wait till namenode notified about the block location details
waitExpectedStorageType(file, StorageType.ARCHIVE, 3, 30000);
} finally {
hdfsCluster.shutdown();
shutdownCluster();
}
}
@ -277,11 +291,10 @@ public void testSatisfyFileWithHdfsAdmin() throws Exception {
*/
@Test(timeout = 300000)
public void testSatisfyDirWithHdfsAdmin() throws Exception {
HdfsAdmin hdfsAdmin =
new HdfsAdmin(FileSystem.getDefaultUri(config), config);
try {
createCluster();
HdfsAdmin hdfsAdmin =
new HdfsAdmin(FileSystem.getDefaultUri(config), config);
final String subDir = "/subDir";
final String subFile1 = subDir + "/subFile1";
final String subDir2 = subDir + "/subDir2";
@ -310,7 +323,7 @@ public void testSatisfyDirWithHdfsAdmin() throws Exception {
// take no effect for the sub-dir's file in the directory.
waitExpectedStorageType(subFile2, StorageType.DEFAULT, 3, 30000);
} finally {
hdfsCluster.shutdown();
shutdownCluster();
}
}
@ -321,6 +334,7 @@ public void testSatisfyDirWithHdfsAdmin() throws Exception {
@Test(timeout = 300000)
public void testSatisfyWithExceptions() throws Exception {
try {
createCluster();
final String nonExistingFile = "/noneExistingFile";
hdfsCluster.getConfiguration(0).
setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, false);
@ -354,7 +368,7 @@ public void testSatisfyWithExceptions() throws Exception {
}
} finally {
hdfsCluster.shutdown();
shutdownCluster();
}
}
@ -376,6 +390,7 @@ public void testSatisfyWithExceptions() throws Exception {
public void testWhenOnlyFewTargetDatanodeAreAvailableToSatisfyStoragePolicy()
throws Exception {
try {
createCluster();
// Change policy to COLD
dfs.setStoragePolicy(new Path(file), "COLD");
FSNamesystem namesystem = hdfsCluster.getNamesystem();
@ -397,7 +412,7 @@ public void testWhenOnlyFewTargetDatanodeAreAvailableToSatisfyStoragePolicy()
waitForBlocksMovementResult(1, 30000);
} finally {
hdfsCluster.shutdown();
shutdownCluster();
}
}
@ -418,6 +433,7 @@ public void testWhenOnlyFewTargetDatanodeAreAvailableToSatisfyStoragePolicy()
public void testWhenNoTargetDatanodeToSatisfyStoragePolicy()
throws Exception {
try {
createCluster();
// Change policy to COLD
dfs.setStoragePolicy(new Path(file), "COLD");
FSNamesystem namesystem = hdfsCluster.getNamesystem();
@ -440,7 +456,7 @@ public void testWhenNoTargetDatanodeToSatisfyStoragePolicy()
// re-attempted.
waitForAttemptedItems(1, 30000);
} finally {
hdfsCluster.shutdown();
shutdownCluster();
}
}
@ -452,6 +468,7 @@ public void testWhenNoTargetDatanodeToSatisfyStoragePolicy()
public void testWhenMoverIsAlreadyRunningBeforeStoragePolicySatisfier()
throws IOException {
try {
createCluster();
// Simulate Mover by creating MOVER_ID file
DFSTestUtil.createFile(hdfsCluster.getFileSystem(),
HdfsServerConstants.MOVER_ID_PATH, 0, (short) 1, 0);
@ -461,10 +478,95 @@ public void testWhenMoverIsAlreadyRunningBeforeStoragePolicySatisfier()
Assert.assertFalse("SPS should not start "
+ "when a Mover instance is running", running);
} finally {
hdfsCluster.shutdown();
shutdownCluster();
}
}
/**
* Test to verify that satisfy worker can't move blocks. If the given block is
* pinned it shouldn't be considered for retries.
*/
@Test(timeout = 120000)
public void testMoveWithBlockPinning() throws Exception {
config.setBoolean(DFSConfigKeys.DFS_DATANODE_BLOCK_PINNING_ENABLED, true);
hdfsCluster = new MiniDFSCluster.Builder(config).numDataNodes(3)
.storageTypes(
new StorageType[][] {{StorageType.DISK, StorageType.DISK},
{StorageType.DISK, StorageType.DISK},
{StorageType.DISK, StorageType.DISK}})
.build();
hdfsCluster.waitActive();
dfs = hdfsCluster.getFileSystem();
// create a file with replication factor 3 and mark 2 pinned block
// locations.
final String file1 = createFileAndSimulateFavoredNodes(2);
// Change policy to COLD
dfs.setStoragePolicy(new Path(file1), "COLD");
FSNamesystem namesystem = hdfsCluster.getNamesystem();
INode inode = namesystem.getFSDirectory().getINode(file1);
StorageType[][] newtypes =
new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE},
{StorageType.ARCHIVE, StorageType.ARCHIVE},
{StorageType.ARCHIVE, StorageType.ARCHIVE}};
// Adding DISK based datanodes
startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
storagesPerDatanode, capacity, hdfsCluster);
namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
hdfsCluster.triggerHeartbeats();
// No block movement will be scheduled as there is no target node available
// with the required storage type.
waitForAttemptedItems(1, 30000);
waitForBlocksMovementResult(1, 30000);
waitExpectedStorageType(file1, StorageType.ARCHIVE, 1, 30000);
waitExpectedStorageType(file1, StorageType.DISK, 2, 30000);
}
private String createFileAndSimulateFavoredNodes(int favoredNodesCount)
throws IOException {
ArrayList<DataNode> dns = hdfsCluster.getDataNodes();
final String file1 = "/testMoveWithBlockPinning";
// replication factor 3
InetSocketAddress[] favoredNodes = new InetSocketAddress[favoredNodesCount];
for (int i = 0; i < favoredNodesCount; i++) {
favoredNodes[i] = dns.get(i).getXferAddress();
}
DFSTestUtil.createFile(dfs, new Path(file1), false, 1024, 100,
DEFAULT_BLOCK_SIZE, (short) 3, 0, false, favoredNodes);
LocatedBlocks locatedBlocks = dfs.getClient().getLocatedBlocks(file1, 0);
Assert.assertEquals("Wrong block count", 1,
locatedBlocks.locatedBlockCount());
// verify storage type before movement
LocatedBlock lb = locatedBlocks.get(0);
StorageType[] storageTypes = lb.getStorageTypes();
for (StorageType storageType : storageTypes) {
Assert.assertTrue(StorageType.DISK == storageType);
}
// Mock FsDatasetSpi#getPinning to show that the block is pinned.
DatanodeInfo[] locations = lb.getLocations();
Assert.assertEquals(3, locations.length);
Assert.assertTrue(favoredNodesCount < locations.length);
for(DatanodeInfo dnInfo: locations){
LOG.info("Simulate block pinning in datanode {}",
locations[favoredNodesCount]);
DataNode dn = hdfsCluster.getDataNode(dnInfo.getIpcPort());
DataNodeTestUtils.mockDatanodeBlkPinning(dn, true);
favoredNodesCount--;
if (favoredNodesCount <= 0) {
break;// marked favoredNodesCount number of pinned block location
}
}
return file1;
}
private void waitForAttemptedItems(long expectedBlkMovAttemptedCount,
int timeout) throws TimeoutException, InterruptedException {
BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();