|
@@ -21,7 +21,8 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
|
|
import org.apache.hadoop.hdds.protocol.proto
|
|
|
.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
|
|
|
-import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
|
|
+import org.apache.hadoop.hdds.scm.container.common.helpers
|
|
|
+ .StorageContainerException;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.hdds.protocol.proto
|
|
|
.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
|
|
@@ -54,7 +55,8 @@ import org.slf4j.LoggerFactory;
|
|
|
import java.io.IOException;
|
|
|
import java.util.List;
|
|
|
|
|
|
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_NOT_FOUND;
|
|
|
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
|
|
+ .Result.CONTAINER_NOT_FOUND;
|
|
|
|
|
|
/**
|
|
|
* Handle block deletion commands.
|
|
@@ -68,6 +70,7 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
|
|
|
private final Configuration conf;
|
|
|
private int invocationCount;
|
|
|
private long totalTime;
|
|
|
+ private boolean cmdExecuted;
|
|
|
|
|
|
public DeleteBlocksCommandHandler(ContainerSet cset,
|
|
|
Configuration conf) {
|
|
@@ -78,93 +81,98 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
|
|
|
@Override
|
|
|
public void handle(SCMCommand command, OzoneContainer container,
|
|
|
StateContext context, SCMConnectionManager connectionManager) {
|
|
|
- if (command.getType() != SCMCommandProto.Type.deleteBlocksCommand) {
|
|
|
- LOG.warn("Skipping handling command, expected command "
|
|
|
- + "type {} but found {}",
|
|
|
- SCMCommandProto.Type.deleteBlocksCommand, command.getType());
|
|
|
- return;
|
|
|
- }
|
|
|
- LOG.debug("Processing block deletion command.");
|
|
|
- invocationCount++;
|
|
|
+ cmdExecuted = false;
|
|
|
long startTime = Time.monotonicNow();
|
|
|
-
|
|
|
- // move blocks to deleting state.
|
|
|
- // this is a metadata update, the actual deletion happens in another
|
|
|
- // recycling thread.
|
|
|
- DeleteBlocksCommand cmd = (DeleteBlocksCommand) command;
|
|
|
- List<DeletedBlocksTransaction> containerBlocks = cmd.blocksTobeDeleted();
|
|
|
-
|
|
|
-
|
|
|
- DeletedContainerBlocksSummary summary =
|
|
|
- DeletedContainerBlocksSummary.getFrom(containerBlocks);
|
|
|
- LOG.info("Start to delete container blocks, TXIDs={}, "
|
|
|
- + "numOfContainers={}, numOfBlocks={}",
|
|
|
- summary.getTxIDSummary(),
|
|
|
- summary.getNumOfContainers(),
|
|
|
- summary.getNumOfBlocks());
|
|
|
-
|
|
|
- ContainerBlocksDeletionACKProto.Builder resultBuilder =
|
|
|
- ContainerBlocksDeletionACKProto.newBuilder();
|
|
|
- containerBlocks.forEach(entry -> {
|
|
|
- DeleteBlockTransactionResult.Builder txResultBuilder =
|
|
|
- DeleteBlockTransactionResult.newBuilder();
|
|
|
- txResultBuilder.setTxID(entry.getTxID());
|
|
|
- try {
|
|
|
- long containerId = entry.getContainerID();
|
|
|
- Container cont = containerSet.getContainer(containerId);
|
|
|
- if(cont == null) {
|
|
|
- throw new StorageContainerException("Unable to find the container "
|
|
|
- + containerId, CONTAINER_NOT_FOUND);
|
|
|
- }
|
|
|
- ContainerProtos.ContainerType containerType = cont.getContainerType();
|
|
|
- switch (containerType) {
|
|
|
- case KeyValueContainer:
|
|
|
- KeyValueContainerData containerData = (KeyValueContainerData)
|
|
|
- cont.getContainerData();
|
|
|
- deleteKeyValueContainerBlocks(containerData, entry);
|
|
|
- txResultBuilder.setSuccess(true);
|
|
|
- break;
|
|
|
- default:
|
|
|
- LOG.error("Delete Blocks Command Handler is not implemented for " +
|
|
|
- "containerType {}", containerType);
|
|
|
- }
|
|
|
- } catch (IOException e) {
|
|
|
- LOG.warn("Failed to delete blocks for container={}, TXID={}",
|
|
|
- entry.getContainerID(), entry.getTxID(), e);
|
|
|
- txResultBuilder.setSuccess(false);
|
|
|
+ try {
|
|
|
+ if (command.getType() != SCMCommandProto.Type.deleteBlocksCommand) {
|
|
|
+ LOG.warn("Skipping handling command, expected command "
|
|
|
+ + "type {} but found {}",
|
|
|
+ SCMCommandProto.Type.deleteBlocksCommand, command.getType());
|
|
|
+ return;
|
|
|
}
|
|
|
- resultBuilder.addResults(txResultBuilder.build());
|
|
|
- });
|
|
|
- ContainerBlocksDeletionACKProto blockDeletionACK = resultBuilder.build();
|
|
|
-
|
|
|
- // Send ACK back to SCM as long as meta updated
|
|
|
- // TODO Or we should wait until the blocks are actually deleted?
|
|
|
- if (!containerBlocks.isEmpty()) {
|
|
|
- for (EndpointStateMachine endPoint : connectionManager.getValues()) {
|
|
|
+ LOG.debug("Processing block deletion command.");
|
|
|
+ invocationCount++;
|
|
|
+
|
|
|
+ // move blocks to deleting state.
|
|
|
+ // this is a metadata update, the actual deletion happens in another
|
|
|
+ // recycling thread.
|
|
|
+ DeleteBlocksCommand cmd = (DeleteBlocksCommand) command;
|
|
|
+ List<DeletedBlocksTransaction> containerBlocks = cmd.blocksTobeDeleted();
|
|
|
+
|
|
|
+ DeletedContainerBlocksSummary summary =
|
|
|
+ DeletedContainerBlocksSummary.getFrom(containerBlocks);
|
|
|
+ LOG.info("Start to delete container blocks, TXIDs={}, "
|
|
|
+ + "numOfContainers={}, numOfBlocks={}",
|
|
|
+ summary.getTxIDSummary(),
|
|
|
+ summary.getNumOfContainers(),
|
|
|
+ summary.getNumOfBlocks());
|
|
|
+
|
|
|
+ ContainerBlocksDeletionACKProto.Builder resultBuilder =
|
|
|
+ ContainerBlocksDeletionACKProto.newBuilder();
|
|
|
+ containerBlocks.forEach(entry -> {
|
|
|
+ DeleteBlockTransactionResult.Builder txResultBuilder =
|
|
|
+ DeleteBlockTransactionResult.newBuilder();
|
|
|
+ txResultBuilder.setTxID(entry.getTxID());
|
|
|
try {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Sending following block deletion ACK to SCM");
|
|
|
- for (DeleteBlockTransactionResult result :
|
|
|
- blockDeletionACK.getResultsList()) {
|
|
|
- LOG.debug(result.getTxID() + " : " + result.getSuccess());
|
|
|
- }
|
|
|
+ long containerId = entry.getContainerID();
|
|
|
+ Container cont = containerSet.getContainer(containerId);
|
|
|
+ if (cont == null) {
|
|
|
+ throw new StorageContainerException("Unable to find the container "
|
|
|
+ + containerId, CONTAINER_NOT_FOUND);
|
|
|
+ }
|
|
|
+ ContainerProtos.ContainerType containerType = cont.getContainerType();
|
|
|
+ switch (containerType) {
|
|
|
+ case KeyValueContainer:
|
|
|
+ KeyValueContainerData containerData = (KeyValueContainerData)
|
|
|
+ cont.getContainerData();
|
|
|
+ deleteKeyValueContainerBlocks(containerData, entry);
|
|
|
+ txResultBuilder.setSuccess(true);
|
|
|
+ break;
|
|
|
+ default:
|
|
|
+ LOG.error(
|
|
|
+ "Delete Blocks Command Handler is not implemented for " +
|
|
|
+ "containerType {}", containerType);
|
|
|
}
|
|
|
- endPoint.getEndPoint()
|
|
|
- .sendContainerBlocksDeletionACK(blockDeletionACK);
|
|
|
} catch (IOException e) {
|
|
|
- LOG.error("Unable to send block deletion ACK to SCM {}",
|
|
|
- endPoint.getAddress().toString(), e);
|
|
|
+ LOG.warn("Failed to delete blocks for container={}, TXID={}",
|
|
|
+ entry.getContainerID(), entry.getTxID(), e);
|
|
|
+ txResultBuilder.setSuccess(false);
|
|
|
+ }
|
|
|
+ resultBuilder.addResults(txResultBuilder.build());
|
|
|
+ });
|
|
|
+ ContainerBlocksDeletionACKProto blockDeletionACK = resultBuilder.build();
|
|
|
+
|
|
|
+ // Send ACK back to SCM as long as meta updated
|
|
|
+ // TODO Or we should wait until the blocks are actually deleted?
|
|
|
+ if (!containerBlocks.isEmpty()) {
|
|
|
+ for (EndpointStateMachine endPoint : connectionManager.getValues()) {
|
|
|
+ try {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Sending following block deletion ACK to SCM");
|
|
|
+ for (DeleteBlockTransactionResult result :
|
|
|
+ blockDeletionACK.getResultsList()) {
|
|
|
+ LOG.debug(result.getTxID() + " : " + result.getSuccess());
|
|
|
+ }
|
|
|
+ }
|
|
|
+ endPoint.getEndPoint()
|
|
|
+ .sendContainerBlocksDeletionACK(blockDeletionACK);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.error("Unable to send block deletion ACK to SCM {}",
|
|
|
+ endPoint.getAddress().toString(), e);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
+ cmdExecuted = true;
|
|
|
+ } finally {
|
|
|
+ updateCommandStatus(context, command, cmdExecuted, LOG);
|
|
|
+ long endTime = Time.monotonicNow();
|
|
|
+ totalTime += endTime - startTime;
|
|
|
}
|
|
|
-
|
|
|
- long endTime = Time.monotonicNow();
|
|
|
- totalTime += endTime - startTime;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Move a bunch of blocks from a container to deleting state.
|
|
|
- * This is a meta update, the actual deletes happen in async mode.
|
|
|
+ * Move a bunch of blocks from a container to deleting state. This is a meta
|
|
|
+ * update, the actual deletes happen in async mode.
|
|
|
*
|
|
|
* @param containerData - KeyValueContainerData
|
|
|
* @param delTX a block deletion transaction.
|
|
@@ -222,7 +230,7 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
|
|
|
}
|
|
|
} else {
|
|
|
LOG.debug("Block {} not found or already under deletion in"
|
|
|
- + " container {}, skip deleting it.", blk, containerId);
|
|
|
+ + " container {}, skip deleting it.", blk, containerId);
|
|
|
}
|
|
|
}
|
|
|
|