|
@@ -17,21 +17,9 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
-import static org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed;
|
|
|
-
|
|
|
-import java.io.BufferedInputStream;
|
|
|
-import java.io.BufferedOutputStream;
|
|
|
-import java.io.DataInputStream;
|
|
|
-import java.io.DataOutputStream;
|
|
|
import java.io.IOException;
|
|
|
-import java.io.InputStream;
|
|
|
-import java.io.OutputStream;
|
|
|
-import java.net.Socket;
|
|
|
-import java.util.ArrayList;
|
|
|
import java.util.Collection;
|
|
|
-import java.util.Collections;
|
|
|
import java.util.EnumSet;
|
|
|
-import java.util.List;
|
|
|
import java.util.concurrent.Callable;
|
|
|
import java.util.concurrent.CompletionService;
|
|
|
import java.util.concurrent.ExecutorCompletionService;
|
|
@@ -47,20 +35,15 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.StorageType;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSUtilClient;
|
|
|
-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;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
|
|
|
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
|
|
|
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
|
+import org.apache.hadoop.hdfs.server.common.sps.BlockMovementAttemptFinished;
|
|
|
+import org.apache.hadoop.hdfs.server.common.sps.BlockMovementStatus;
|
|
|
+import org.apache.hadoop.hdfs.server.common.sps.BlockStorageMovementTracker;
|
|
|
+import org.apache.hadoop.hdfs.server.common.sps.BlocksMovementsStatusHandler;
|
|
|
+import org.apache.hadoop.hdfs.server.common.sps.BlockDispatcher;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
|
|
|
-import org.apache.hadoop.io.IOUtils;
|
|
|
-import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.util.Daemon;
|
|
|
import org.slf4j.Logger;
|
|
@@ -81,7 +64,6 @@ public class StoragePolicySatisfyWorker {
|
|
|
.getLogger(StoragePolicySatisfyWorker.class);
|
|
|
|
|
|
private final DataNode datanode;
|
|
|
- private final int ioFileBufferSize;
|
|
|
|
|
|
private final int moverThreads;
|
|
|
private final ExecutorService moveExecutor;
|
|
@@ -89,10 +71,10 @@ public class StoragePolicySatisfyWorker {
|
|
|
private final BlocksMovementsStatusHandler handler;
|
|
|
private final BlockStorageMovementTracker movementTracker;
|
|
|
private Daemon movementTrackerThread;
|
|
|
+ private final BlockDispatcher blkDispatcher;
|
|
|
|
|
|
public StoragePolicySatisfyWorker(Configuration conf, DataNode datanode) {
|
|
|
this.datanode = datanode;
|
|
|
- this.ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(conf);
|
|
|
|
|
|
moverThreads = conf.getInt(DFSConfigKeys.DFS_MOVER_MOVERTHREADS_KEY,
|
|
|
DFSConfigKeys.DFS_MOVER_MOVERTHREADS_DEFAULT);
|
|
@@ -103,7 +85,10 @@ public class StoragePolicySatisfyWorker {
|
|
|
handler);
|
|
|
movementTrackerThread = new Daemon(movementTracker);
|
|
|
movementTrackerThread.setName("BlockStorageMovementTracker");
|
|
|
-
|
|
|
+ DNConf dnConf = datanode.getDnConf();
|
|
|
+ int ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(conf);
|
|
|
+ blkDispatcher = new BlockDispatcher(dnConf.getSocketTimeout(),
|
|
|
+ ioFileBufferSize, dnConf.getConnectToDnViaHostname());
|
|
|
// TODO: Needs to manage the number of concurrent moves per DataNode.
|
|
|
}
|
|
|
|
|
@@ -183,8 +168,7 @@ public class StoragePolicySatisfyWorker {
|
|
|
assert sourceStorageType != targetStorageType
|
|
|
: "Source and Target storage type shouldn't be same!";
|
|
|
BlockMovingTask blockMovingTask = new BlockMovingTask(blockPoolID,
|
|
|
- blkMovingInfo.getBlock(), blkMovingInfo.getSource(),
|
|
|
- blkMovingInfo.getTarget(), sourceStorageType, targetStorageType);
|
|
|
+ blkMovingInfo);
|
|
|
Future<BlockMovementAttemptFinished> moveCallable = moverCompletionService
|
|
|
.submit(blockMovingTask);
|
|
|
movementTracker.addBlock(blkMovingInfo.getBlock(),
|
|
@@ -199,244 +183,45 @@ public class StoragePolicySatisfyWorker {
|
|
|
private class BlockMovingTask implements
|
|
|
Callable<BlockMovementAttemptFinished> {
|
|
|
private final String blockPoolID;
|
|
|
- private final Block block;
|
|
|
- private final DatanodeInfo source;
|
|
|
- private final DatanodeInfo target;
|
|
|
- private final StorageType srcStorageType;
|
|
|
- private final StorageType targetStorageType;
|
|
|
+ private final BlockMovingInfo blkMovingInfo;
|
|
|
|
|
|
- BlockMovingTask(String blockPoolID, Block block,
|
|
|
- DatanodeInfo source, DatanodeInfo target,
|
|
|
- StorageType srcStorageType, StorageType targetStorageType) {
|
|
|
+ BlockMovingTask(String blockPoolID, BlockMovingInfo blkMovInfo) {
|
|
|
this.blockPoolID = blockPoolID;
|
|
|
- this.block = block;
|
|
|
- this.source = source;
|
|
|
- this.target = target;
|
|
|
- this.srcStorageType = srcStorageType;
|
|
|
- this.targetStorageType = targetStorageType;
|
|
|
+ this.blkMovingInfo = blkMovInfo;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public BlockMovementAttemptFinished call() {
|
|
|
BlockMovementStatus status = moveBlock();
|
|
|
- return new BlockMovementAttemptFinished(block, source, target, status);
|
|
|
+ return new BlockMovementAttemptFinished(blkMovingInfo.getBlock(),
|
|
|
+ blkMovingInfo.getSource(), blkMovingInfo.getTarget(), status);
|
|
|
}
|
|
|
|
|
|
private BlockMovementStatus moveBlock() {
|
|
|
- LOG.info("Start moving block:{} from src:{} to destin:{} to satisfy "
|
|
|
- + "storageType, sourceStoragetype:{} and destinStoragetype:{}",
|
|
|
- block, source, target, srcStorageType, targetStorageType);
|
|
|
- Socket sock = null;
|
|
|
- DataOutputStream out = null;
|
|
|
- DataInputStream in = null;
|
|
|
+ datanode.incrementXmitsInProgress();
|
|
|
+ ExtendedBlock eb = new ExtendedBlock(blockPoolID,
|
|
|
+ blkMovingInfo.getBlock());
|
|
|
try {
|
|
|
- datanode.incrementXmitsInProgress();
|
|
|
-
|
|
|
- ExtendedBlock extendedBlock = new ExtendedBlock(blockPoolID, block);
|
|
|
- DNConf dnConf = datanode.getDnConf();
|
|
|
-
|
|
|
- String dnAddr = datanode.getDatanodeId()
|
|
|
- .getXferAddr(dnConf.getConnectToDnViaHostname());
|
|
|
- sock = datanode.newSocket();
|
|
|
- NetUtils.connect(sock, NetUtils.createSocketAddr(dnAddr),
|
|
|
- dnConf.getSocketTimeout());
|
|
|
- sock.setSoTimeout(2 * dnConf.getSocketTimeout());
|
|
|
- LOG.debug("Connecting to datanode {}", dnAddr);
|
|
|
-
|
|
|
- OutputStream unbufOut = sock.getOutputStream();
|
|
|
- InputStream unbufIn = sock.getInputStream();
|
|
|
Token<BlockTokenIdentifier> accessToken = datanode.getBlockAccessToken(
|
|
|
- extendedBlock, EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE),
|
|
|
- new StorageType[]{targetStorageType}, new String[0]);
|
|
|
-
|
|
|
+ eb, EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE),
|
|
|
+ new StorageType[]{blkMovingInfo.getTargetStorageType()},
|
|
|
+ new String[0]);
|
|
|
DataEncryptionKeyFactory keyFactory = datanode
|
|
|
- .getDataEncryptionKeyFactoryForBlock(extendedBlock);
|
|
|
- IOStreamPair saslStreams = datanode.getSaslClient().socketSend(sock,
|
|
|
- unbufOut, unbufIn, keyFactory, accessToken, target);
|
|
|
- unbufOut = saslStreams.out;
|
|
|
- unbufIn = saslStreams.in;
|
|
|
- out = new DataOutputStream(
|
|
|
- new BufferedOutputStream(unbufOut, ioFileBufferSize));
|
|
|
- in = new DataInputStream(
|
|
|
- new BufferedInputStream(unbufIn, ioFileBufferSize));
|
|
|
- sendRequest(out, extendedBlock, accessToken, source, targetStorageType);
|
|
|
- receiveResponse(in);
|
|
|
+ .getDataEncryptionKeyFactoryForBlock(eb);
|
|
|
|
|
|
- LOG.info(
|
|
|
- "Successfully moved block:{} from src:{} to destin:{} for"
|
|
|
- + " 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;
|
|
|
+ return blkDispatcher.moveBlock(blkMovingInfo,
|
|
|
+ datanode.getSaslClient(), eb, datanode.newSocket(),
|
|
|
+ keyFactory, accessToken);
|
|
|
} catch (IOException e) {
|
|
|
// TODO: handle failure retries
|
|
|
LOG.warn(
|
|
|
"Failed to move block:{} from src:{} to destin:{} to satisfy "
|
|
|
+ "storageType:{}",
|
|
|
- block, source, target, targetStorageType, e);
|
|
|
+ blkMovingInfo.getBlock(), blkMovingInfo.getSource(),
|
|
|
+ blkMovingInfo.getTarget(), blkMovingInfo.getTargetStorageType(), e);
|
|
|
return BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_FAILURE;
|
|
|
} finally {
|
|
|
datanode.decrementXmitsInProgress();
|
|
|
- IOUtils.closeStream(out);
|
|
|
- IOUtils.closeStream(in);
|
|
|
- IOUtils.closeSocket(sock);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /** Send a reportedBlock replace request to the output stream. */
|
|
|
- private void sendRequest(DataOutputStream out, ExtendedBlock eb,
|
|
|
- Token<BlockTokenIdentifier> accessToken, DatanodeInfo srcDn,
|
|
|
- StorageType destinStorageType) throws IOException {
|
|
|
- new Sender(out).replaceBlock(eb, destinStorageType, accessToken,
|
|
|
- srcDn.getDatanodeUuid(), srcDn, null);
|
|
|
- }
|
|
|
-
|
|
|
- /** Receive a reportedBlock copy response from the input stream. */
|
|
|
- private void receiveResponse(DataInputStream in) throws IOException {
|
|
|
- BlockOpResponseProto response = BlockOpResponseProto
|
|
|
- .parseFrom(vintPrefixed(in));
|
|
|
- while (response.getStatus() == Status.IN_PROGRESS) {
|
|
|
- // read intermediate responses
|
|
|
- response = BlockOpResponseProto.parseFrom(vintPrefixed(in));
|
|
|
- }
|
|
|
- String logInfo = "reportedBlock move is failed";
|
|
|
- DataTransferProtoUtil.checkBlockOpStatus(response, logInfo, true);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Block movement status code.
|
|
|
- */
|
|
|
- public enum BlockMovementStatus {
|
|
|
- /** Success. */
|
|
|
- DN_BLK_STORAGE_MOVEMENT_SUCCESS(0),
|
|
|
- /**
|
|
|
- * Failure due to generation time stamp mismatches or network errors
|
|
|
- * or no available space.
|
|
|
- */
|
|
|
- DN_BLK_STORAGE_MOVEMENT_FAILURE(-1);
|
|
|
-
|
|
|
- // TODO: need to support different type of failures. Failure due to network
|
|
|
- // errors, block pinned, no space available etc.
|
|
|
-
|
|
|
- private final int code;
|
|
|
-
|
|
|
- BlockMovementStatus(int code) {
|
|
|
- this.code = code;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * @return the status code.
|
|
|
- */
|
|
|
- int getStatusCode() {
|
|
|
- return code;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * This class represents status from a block movement task. This will have the
|
|
|
- * information of the task which was successful or failed due to errors.
|
|
|
- */
|
|
|
- static class BlockMovementAttemptFinished {
|
|
|
- private final Block block;
|
|
|
- private final DatanodeInfo src;
|
|
|
- private final DatanodeInfo target;
|
|
|
- private final BlockMovementStatus status;
|
|
|
-
|
|
|
- BlockMovementAttemptFinished(Block block, DatanodeInfo src,
|
|
|
- DatanodeInfo target, BlockMovementStatus status) {
|
|
|
- this.block = block;
|
|
|
- this.src = src;
|
|
|
- this.target = target;
|
|
|
- this.status = status;
|
|
|
- }
|
|
|
-
|
|
|
- Block getBlock() {
|
|
|
- return block;
|
|
|
- }
|
|
|
-
|
|
|
- BlockMovementStatus getStatus() {
|
|
|
- return status;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public String toString() {
|
|
|
- return new StringBuilder().append("Block movement attempt finished(\n ")
|
|
|
- .append(" block : ")
|
|
|
- .append(block).append(" src node: ").append(src)
|
|
|
- .append(" target node: ").append(target)
|
|
|
- .append(" movement status: ").append(status).append(")").toString();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Blocks movements status handler, which is used to collect details of the
|
|
|
- * completed block movements and it will send these attempted finished(with
|
|
|
- * success or failure) blocks to the namenode via heartbeat.
|
|
|
- */
|
|
|
- public static class BlocksMovementsStatusHandler {
|
|
|
- private final List<Block> blockIdVsMovementStatus =
|
|
|
- new ArrayList<>();
|
|
|
-
|
|
|
- /**
|
|
|
- * Collect all the storage movement attempt finished blocks. Later this will
|
|
|
- * be send to namenode via heart beat.
|
|
|
- *
|
|
|
- * @param moveAttemptFinishedBlks
|
|
|
- * set of storage movement attempt finished blocks
|
|
|
- */
|
|
|
- void handle(List<BlockMovementAttemptFinished> moveAttemptFinishedBlks) {
|
|
|
- List<Block> blocks = new ArrayList<>();
|
|
|
-
|
|
|
- for (BlockMovementAttemptFinished item : moveAttemptFinishedBlks) {
|
|
|
- blocks.add(item.getBlock());
|
|
|
- }
|
|
|
- // Adding to the tracking report list. Later this will be send to
|
|
|
- // namenode via datanode heartbeat.
|
|
|
- synchronized (blockIdVsMovementStatus) {
|
|
|
- blockIdVsMovementStatus.addAll(blocks);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * @return unmodifiable list of storage movement attempt finished blocks.
|
|
|
- */
|
|
|
- List<Block> getMoveAttemptFinishedBlocks() {
|
|
|
- List<Block> moveAttemptFinishedBlks = new ArrayList<>();
|
|
|
- // 1. Adding all the completed block ids.
|
|
|
- synchronized (blockIdVsMovementStatus) {
|
|
|
- if (blockIdVsMovementStatus.size() > 0) {
|
|
|
- moveAttemptFinishedBlks = Collections
|
|
|
- .unmodifiableList(blockIdVsMovementStatus);
|
|
|
- }
|
|
|
- }
|
|
|
- return moveAttemptFinishedBlks;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Remove the storage movement attempt finished blocks from the tracking
|
|
|
- * list.
|
|
|
- *
|
|
|
- * @param moveAttemptFinishedBlks
|
|
|
- * set of storage movement attempt finished blocks
|
|
|
- */
|
|
|
- void remove(List<Block> moveAttemptFinishedBlks) {
|
|
|
- if (moveAttemptFinishedBlks != null) {
|
|
|
- blockIdVsMovementStatus.removeAll(moveAttemptFinishedBlks);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Clear the blockID vs movement status tracking map.
|
|
|
- */
|
|
|
- void removeAll() {
|
|
|
- synchronized (blockIdVsMovementStatus) {
|
|
|
- blockIdVsMovementStatus.clear();
|
|
|
}
|
|
|
}
|
|
|
}
|