HDFS-11164: Mover should avoid unnecessary retries if the block is pinned. Contributed by Rakesh R

This commit is contained in:
Uma Maheswara Rao G 2016-12-13 17:09:58 -08:00
parent 9947aeb60c
commit e24a923db5
9 changed files with 395 additions and 13 deletions

View File

@ -0,0 +1,33 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocol.datatransfer;
import java.io.IOException;
/**
* Indicates a failure due to block pinning.
*/
public class BlockPinningException extends IOException {
// Required by {@link java.io.Serializable}.
private static final long serialVersionUID = 1L;
public BlockPinningException(String errMsg) {
super(errMsg);
}
}

View File

@ -24,11 +24,11 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferTraceInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ChecksumTypeProto;
import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@ -107,6 +107,11 @@ public abstract class DataTransferProtoUtil {
public static void checkBlockOpStatus(
BlockOpResponseProto response,
String logInfo) throws IOException {
checkBlockOpStatus(response, logInfo, false);
}
public static void checkBlockOpStatus(BlockOpResponseProto response,
String logInfo, boolean checkBlockPinningErr) throws IOException {
if (response.getStatus() != Status.SUCCESS) {
if (response.getStatus() == Status.ERROR_ACCESS_TOKEN) {
throw new InvalidBlockTokenException(
@ -114,6 +119,14 @@ public abstract class DataTransferProtoUtil {
+ ", status message " + response.getMessage()
+ ", " + logInfo
);
} else if (checkBlockPinningErr
&& response.getStatus() == Status.ERROR_BLOCK_PINNED) {
throw new BlockPinningException(
"Got error"
+ ", status=" + response.getStatus().name()
+ ", status message " + response.getMessage()
+ ", " + logInfo
);
} else {
throw new IOException(
"Got error"

View File

@ -243,6 +243,7 @@ enum Status {
OOB_RESERVED2 = 10; // Reserved
OOB_RESERVED3 = 11; // Reserved
IN_PROGRESS = 12;
ERROR_BLOCK_PINNED = 13;
}
enum ShortCircuitFdResponse {

View File

@ -36,6 +36,7 @@ import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
@ -55,6 +56,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.HdfsConstants;
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;
@ -224,6 +226,10 @@ public class Dispatcher {
this.target = target;
}
public DatanodeInfo getSource() {
return source.getDatanodeInfo();
}
@Override
public String toString() {
final Block b = reportedBlock != null ? reportedBlock.getBlock() : null;
@ -367,6 +373,15 @@ public class Dispatcher {
} catch (IOException e) {
LOG.warn("Failed to move " + this, e);
target.getDDatanode().setHasFailure();
// Check that the failure is due to block pinning errors.
if (e instanceof BlockPinningException) {
// Pinned block can't be moved. Add this block into failure list.
// Later in the next iteration mover will exclude these blocks from
// pending moves.
target.getDDatanode().addBlockPinningFailures(this);
return;
}
// Proxy or target may have some issues, delay before using these nodes
// further in order to avoid a potential storm of "threads quota
// exceeded" warnings when the dispatcher gets out of sync with work
@ -419,7 +434,7 @@ public class Dispatcher {
}
}
String logInfo = "reportedBlock move is failed";
DataTransferProtoUtil.checkBlockOpStatus(response, logInfo);
DataTransferProtoUtil.checkBlockOpStatus(response, logInfo, true);
}
/** reset the object */
@ -600,6 +615,7 @@ public class Dispatcher {
/** blocks being moved but not confirmed yet */
private final List<PendingMove> pendings;
private volatile boolean hasFailure = false;
private Map<Long, Set<DatanodeInfo>> blockPinningFailures = new HashMap<>();
private volatile boolean hasSuccess = false;
private ExecutorService moveExecutor;
@ -685,6 +701,22 @@ public class Dispatcher {
this.hasFailure = true;
}
void addBlockPinningFailures(PendingMove pendingBlock) {
synchronized (blockPinningFailures) {
long blockId = pendingBlock.reportedBlock.getBlock().getBlockId();
Set<DatanodeInfo> pinnedLocations = blockPinningFailures.get(blockId);
if (pinnedLocations == null) {
pinnedLocations = new HashSet<>();
blockPinningFailures.put(blockId, pinnedLocations);
}
pinnedLocations.add(pendingBlock.getSource());
}
}
Map<Long, Set<DatanodeInfo>> getBlockPinningFailureList() {
return blockPinningFailures;
}
void setHasSuccess() {
this.hasSuccess = true;
}
@ -1154,6 +1186,34 @@ public class Dispatcher {
}
}
/**
* Check any of the block movements are failed due to block pinning errors. If
* yes, add the failed blockId and its respective source node location to the
* excluded list.
*/
public static void checkForBlockPinningFailures(
Map<Long, Set<DatanodeInfo>> excludedPinnedBlocks,
Iterable<? extends StorageGroup> targets) {
for (StorageGroup t : targets) {
Map<Long, Set<DatanodeInfo>> blockPinningFailureList = t.getDDatanode()
.getBlockPinningFailureList();
Set<Entry<Long, Set<DatanodeInfo>>> entrySet = blockPinningFailureList
.entrySet();
for (Entry<Long, Set<DatanodeInfo>> entry : entrySet) {
Long blockId = entry.getKey();
Set<DatanodeInfo> locs = excludedPinnedBlocks.get(blockId);
if (locs == null) {
// blockId doesn't exists in the excluded list.
locs = entry.getValue();
excludedPinnedBlocks.put(blockId, locs);
} else {
// blockId already exists in the excluded list, add the pinned node.
locs.addAll(entry.getValue());
}
}
}
}
/**
* @return true if some moves are success.
*/

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.StripedBlockInfo;
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
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.Op;
@ -1022,7 +1023,7 @@ class DataXceiver extends Receiver implements Runnable {
String msg = "Not able to copy block " + block.getBlockId() + " " +
"to " + peer.getRemoteAddressString() + " because it's pinned ";
LOG.info(msg);
sendResponse(ERROR, msg);
sendResponse(Status.ERROR_BLOCK_PINNED, msg);
return;
}
@ -1156,7 +1157,7 @@ class DataXceiver extends Receiver implements Runnable {
String logInfo = "copy block " + block + " from "
+ proxySock.getRemoteSocketAddress();
DataTransferProtoUtil.checkBlockOpStatus(copyResponse, logInfo);
DataTransferProtoUtil.checkBlockOpStatus(copyResponse, logInfo, true);
// get checksum info about the block we're copying
ReadOpChecksumInfoProto checksumInfo = copyResponse.getReadOpChecksumInfo();
@ -1183,6 +1184,9 @@ class DataXceiver extends Receiver implements Runnable {
}
} catch (IOException ioe) {
opStatus = ERROR;
if (ioe instanceof BlockPinningException) {
opStatus = Status.ERROR_BLOCK_PINNED;
}
errMsg = "opReplaceBlock " + block + " received exception " + ioe;
LOG.info(errMsg);
if (!IoeDuringCopyBlockOperation) {

View File

@ -46,7 +46,6 @@ import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.security.SecurityUtil;
@ -117,10 +116,12 @@ public class Mover {
private final List<Path> targetPaths;
private final int retryMaxAttempts;
private final AtomicInteger retryCount;
private final Map<Long, Set<DatanodeInfo>> excludedPinnedBlocks;
private final BlockStoragePolicy[] blockStoragePolicies;
Mover(NameNodeConnector nnc, Configuration conf, AtomicInteger retryCount) {
Mover(NameNodeConnector nnc, Configuration conf, AtomicInteger retryCount,
Map<Long, Set<DatanodeInfo>> excludedPinnedBlocks) {
final long movedWinWidth = conf.getLong(
DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_KEY,
DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_DEFAULT);
@ -144,6 +145,7 @@ public class Mover {
this.targetPaths = nnc.getTargetPaths();
this.blockStoragePolicies = new BlockStoragePolicy[1 <<
BlockStoragePolicySuite.ID_BIT_LENGTH];
this.excludedPinnedBlocks = excludedPinnedBlocks;
}
void init() throws IOException {
@ -292,6 +294,8 @@ public class Mover {
// wait for pending move to finish and retry the failed migration
boolean hasFailed = Dispatcher.waitForMoveCompletion(storages.targets
.values());
Dispatcher.checkForBlockPinningFailures(excludedPinnedBlocks,
storages.targets.values());
boolean hasSuccess = Dispatcher.checkForSuccess(storages.targets
.values());
if (hasFailed && !hasSuccess) {
@ -461,6 +465,19 @@ public class Mover {
return true;
}
// Check the given block is pinned in the source datanode. A pinned block
// can't be moved to a different datanode. So we can skip adding these
// blocks to different nodes.
long blockId = db.getBlock().getBlockId();
if (excludedPinnedBlocks.containsKey(blockId)) {
Set<DatanodeInfo> locs = excludedPinnedBlocks.get(blockId);
for (DatanodeInfo dn : locs) {
if (source.getDatanodeInfo().equals(dn)) {
return false;
}
}
}
if (dispatcher.getCluster().isNodeGroupAware()) {
if (chooseTarget(db, source, targetTypes, Matcher.SAME_NODE_GROUP)) {
return true;
@ -614,6 +631,8 @@ public class Mover {
DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT,
TimeUnit.SECONDS) * 1000;
AtomicInteger retryCount = new AtomicInteger(0);
// TODO: Need to limit the size of the pinned blocks to limit memory usage
Map<Long, Set<DatanodeInfo>> excludedPinnedBlocks = new HashMap<>();
LOG.info("namenodes = " + namenodes);
checkKeytabAndInit(conf);
@ -628,7 +647,8 @@ public class Mover {
Iterator<NameNodeConnector> iter = connectors.iterator();
while (iter.hasNext()) {
NameNodeConnector nnc = iter.next();
final Mover m = new Mover(nnc, conf, retryCount);
final Mover m = new Mover(nnc, conf, retryCount,
excludedPinnedBlocks);
final ExitStatus r = m.run();
if (r == ExitStatus.SUCCESS) {

View File

@ -25,10 +25,17 @@ import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doAnswer;
/**
* Utility class for accessing package-private DataNode information during tests.
@ -175,4 +182,25 @@ public class DataNodeTestUtils {
dn.getDirectoryScanner().reconcile();
}
}
/**
* This method is used to mock the data node block pinning API.
*
* @param dn datanode
* @param pinned true if the block is pinned, false otherwise
* @throws IOException
*/
public static void mockDatanodeBlkPinning(final DataNode dn,
final boolean pinned) throws IOException {
final FsDatasetSpi<? extends FsVolumeSpi> data = dn.data;
dn.data = Mockito.spy(data);
doAnswer(new Answer<Object>() {
public Object answer(InvocationOnMock invocation) throws IOException {
// Bypass the argument to FsDatasetImpl#getPinning to show that
// the block is pinned.
return pinned;
}
}).when(dn.data).getPinning(any(ExtendedBlock.class));
}
}

View File

@ -208,6 +208,67 @@ public class TestBlockReplacement {
}
}
/**
* Test to verify that the copying of pinned block to a different destination
* datanode will throw IOException with error code Status.ERROR_BLOCK_PINNED.
*
*/
@Test(timeout = 90000)
public void testBlockReplacementWithPinnedBlocks() throws Exception {
final Configuration conf = new HdfsConfiguration();
// create only one datanode in the cluster with DISK and ARCHIVE storage
// types.
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
.storageTypes(
new StorageType[] {StorageType.DISK, StorageType.ARCHIVE})
.build();
try {
cluster.waitActive();
final DistributedFileSystem dfs = cluster.getFileSystem();
String fileName = "/testBlockReplacementWithPinnedBlocks/file";
final Path file = new Path(fileName);
DFSTestUtil.createFile(dfs, file, 1024, (short) 1, 1024);
LocatedBlock lb = dfs.getClient().getLocatedBlocks(fileName, 0).get(0);
DatanodeInfo[] oldNodes = lb.getLocations();
assertEquals("Wrong block locations", oldNodes.length, 1);
DatanodeInfo source = oldNodes[0];
ExtendedBlock b = lb.getBlock();
DatanodeInfo[] datanodes = dfs.getDataNodeStats();
DatanodeInfo destin = null;
for (DatanodeInfo datanodeInfo : datanodes) {
// choose different destination node
if (!oldNodes[0].equals(datanodeInfo)) {
destin = datanodeInfo;
break;
}
}
assertNotNull("Failed to choose destination datanode!", destin);
assertFalse("Source and destin datanode should be different",
source.equals(destin));
// Mock FsDatasetSpi#getPinning to show that the block is pinned.
for (int i = 0; i < cluster.getDataNodes().size(); i++) {
DataNode dn = cluster.getDataNodes().get(i);
LOG.info("Simulate block pinning in datanode " + dn);
DataNodeTestUtils.mockDatanodeBlkPinning(dn, true);
}
// Block movement to a different datanode should fail as the block is
// pinned.
assertTrue("Status code mismatches!", replaceBlock(b, source, source,
destin, StorageType.ARCHIVE, Status.ERROR_BLOCK_PINNED));
} finally {
cluster.shutdown();
}
}
@Test
public void testBlockMoveAcrossStorageInSameNode() throws Exception {
final Configuration conf = new HdfsConfiguration();
@ -236,7 +297,7 @@ public class TestBlockReplacement {
// move block to ARCHIVE by using same DataNodeInfo for source, proxy and
// destination so that movement happens within datanode
assertTrue(replaceBlock(block, source, source, source,
StorageType.ARCHIVE));
StorageType.ARCHIVE, Status.SUCCESS));
// wait till namenode notified
Thread.sleep(3000);
@ -311,7 +372,7 @@ public class TestBlockReplacement {
private boolean replaceBlock( ExtendedBlock block, DatanodeInfo source,
DatanodeInfo sourceProxy, DatanodeInfo destination) throws IOException {
return replaceBlock(block, source, sourceProxy, destination,
StorageType.DEFAULT);
StorageType.DEFAULT, Status.SUCCESS);
}
/*
@ -322,7 +383,8 @@ public class TestBlockReplacement {
DatanodeInfo source,
DatanodeInfo sourceProxy,
DatanodeInfo destination,
StorageType targetStorageType) throws IOException, SocketException {
StorageType targetStorageType,
Status opStatus) throws IOException, SocketException {
Socket sock = new Socket();
try {
sock.connect(NetUtils.createSocketAddr(destination.getXferAddr()),
@ -342,7 +404,7 @@ public class TestBlockReplacement {
while (proto.getStatus() == Status.IN_PROGRESS) {
proto = BlockOpResponseProto.parseDelimitedFrom(reply);
}
return proto.getStatus() == Status.SUCCESS;
return proto.getStatus() == opStatus;
} finally {
sock.close();
}

View File

@ -37,11 +37,13 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSF
import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.URI;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -64,6 +66,7 @@ import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.StripedFileTestUtil;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@ -72,6 +75,8 @@ import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DBlock;
import org.apache.hadoop.hdfs.server.balancer.ExitStatus;
import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.mover.Mover.MLocation;
import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
@ -121,7 +126,7 @@ public class TestMover {
final List<NameNodeConnector> nncs = NameNodeConnector.newNameNodeConnectors(
nnMap, Mover.class.getSimpleName(), Mover.MOVER_ID_PATH, conf,
NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS);
return new Mover(nncs.get(0), conf, new AtomicInteger(0));
return new Mover(nncs.get(0), conf, new AtomicInteger(0), new HashMap<>());
}
@Test
@ -705,4 +710,160 @@ public class TestMover {
UserGroupInformation.setConfiguration(new Configuration());
}
}
/**
* Test to verify that mover can't move pinned blocks.
*/
@Test(timeout = 90000)
public void testMoverWithPinnedBlocks() throws Exception {
final Configuration conf = new HdfsConfiguration();
initConf(conf);
// Sets bigger retry max attempts value so that test case will timed out if
// block pinning errors are not handled properly during block movement.
conf.setInt(DFSConfigKeys.DFS_MOVER_RETRY_MAX_ATTEMPTS_KEY, 10000);
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(3)
.build();
try {
cluster.waitActive();
final DistributedFileSystem dfs = cluster.getFileSystem();
final String file = "/testMoverWithPinnedBlocks/file";
Path dir = new Path("/testMoverWithPinnedBlocks");
dfs.mkdirs(dir);
// write to DISK
dfs.setStoragePolicy(dir, "HOT");
final FSDataOutputStream out = dfs.create(new Path(file));
byte[] fileData = StripedFileTestUtil
.generateBytes(DEFAULT_BLOCK_SIZE * 3);
out.write(fileData);
out.close();
// verify before movement
LocatedBlock lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
StorageType[] storageTypes = lb.getStorageTypes();
for (StorageType storageType : storageTypes) {
Assert.assertTrue(StorageType.DISK == storageType);
}
// Adding one SSD based data node to the cluster.
StorageType[][] newtypes = new StorageType[][] {{StorageType.SSD}};
startAdditionalDNs(conf, 1, newtypes, cluster);
// Mock FsDatasetSpi#getPinning to show that the block is pinned.
for (int i = 0; i < cluster.getDataNodes().size(); i++) {
DataNode dn = cluster.getDataNodes().get(i);
LOG.info("Simulate block pinning in datanode {}", dn);
DataNodeTestUtils.mockDatanodeBlkPinning(dn, true);
}
// move file blocks to ONE_SSD policy
dfs.setStoragePolicy(dir, "ONE_SSD");
int rc = ToolRunner.run(conf, new Mover.Cli(),
new String[] {"-p", dir.toString()});
int exitcode = ExitStatus.NO_MOVE_BLOCK.getExitCode();
Assert.assertEquals("Movement should fail", exitcode, rc);
} finally {
cluster.shutdown();
}
}
/**
* Test to verify that mover should work well with pinned blocks as well as
* failed blocks. Mover should continue retrying the failed blocks only.
*/
@Test(timeout = 90000)
public void testMoverFailedRetryWithPinnedBlocks() throws Exception {
final Configuration conf = new HdfsConfiguration();
initConf(conf);
conf.set(DFSConfigKeys.DFS_MOVER_RETRY_MAX_ATTEMPTS_KEY, "2");
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(2)
.storageTypes(
new StorageType[][] {{StorageType.DISK, StorageType.ARCHIVE},
{StorageType.DISK, StorageType.ARCHIVE}}).build();
try {
cluster.waitActive();
final DistributedFileSystem dfs = cluster.getFileSystem();
final String parenDir = "/parent";
dfs.mkdirs(new Path(parenDir));
final String file1 = "/parent/testMoverFailedRetryWithPinnedBlocks1";
// write to DISK
final FSDataOutputStream out = dfs.create(new Path(file1), (short) 2);
byte[] fileData = StripedFileTestUtil
.generateBytes(DEFAULT_BLOCK_SIZE * 2);
out.write(fileData);
out.close();
// Adding pinned blocks.
createFileWithFavoredDatanodes(conf, cluster, dfs);
// Delete block file so, block move will fail with FileNotFoundException
LocatedBlocks locatedBlocks = dfs.getClient().getLocatedBlocks(file1, 0);
Assert.assertEquals("Wrong block count", 2,
locatedBlocks.locatedBlockCount());
LocatedBlock lb = locatedBlocks.get(0);
cluster.corruptBlockOnDataNodesByDeletingBlockFile(lb.getBlock());
// move to ARCHIVE
dfs.setStoragePolicy(new Path(parenDir), "COLD");
int rc = ToolRunner.run(conf, new Mover.Cli(),
new String[] {"-p", parenDir.toString()});
Assert.assertEquals("Movement should fail after some retry",
ExitStatus.NO_MOVE_PROGRESS.getExitCode(), rc);
} finally {
cluster.shutdown();
}
}
private void createFileWithFavoredDatanodes(final Configuration conf,
final MiniDFSCluster cluster, final DistributedFileSystem dfs)
throws IOException {
// Adding two DISK based data node to the cluster.
// Also, ensure that blocks are pinned in these new data nodes.
StorageType[][] newtypes =
new StorageType[][] {{StorageType.DISK}, {StorageType.DISK}};
startAdditionalDNs(conf, 2, newtypes, cluster);
ArrayList<DataNode> dataNodes = cluster.getDataNodes();
InetSocketAddress[] favoredNodes = new InetSocketAddress[2];
int j = 0;
for (int i = dataNodes.size() - 1; i >= 2; i--) {
favoredNodes[j++] = dataNodes.get(i).getXferAddress();
}
final String file = "/parent/testMoverFailedRetryWithPinnedBlocks2";
final FSDataOutputStream out = dfs.create(new Path(file),
FsPermission.getDefault(), true, DEFAULT_BLOCK_SIZE, (short) 2,
DEFAULT_BLOCK_SIZE, null, favoredNodes);
byte[] fileData = StripedFileTestUtil.generateBytes(DEFAULT_BLOCK_SIZE * 2);
out.write(fileData);
out.close();
// Mock FsDatasetSpi#getPinning to show that the block is pinned.
LocatedBlocks locatedBlocks = dfs.getClient().getLocatedBlocks(file, 0);
Assert.assertEquals("Wrong block count", 2,
locatedBlocks.locatedBlockCount());
LocatedBlock lb = locatedBlocks.get(0);
DatanodeInfo datanodeInfo = lb.getLocations()[0];
for (DataNode dn : cluster.getDataNodes()) {
if (dn.getDatanodeId().getDatanodeUuid()
.equals(datanodeInfo.getDatanodeUuid())) {
LOG.info("Simulate block pinning in datanode {}", datanodeInfo);
DataNodeTestUtils.mockDatanodeBlkPinning(dn, true);
break;
}
}
}
private void startAdditionalDNs(final Configuration conf,
int newNodesRequired, StorageType[][] newTypes,
final MiniDFSCluster cluster) throws IOException {
cluster.startDataNodes(conf, newNodesRequired, newTypes, true, null, null,
null, null, null, false, false, false, null);
cluster.triggerHeartbeats();
}
}