Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HDDS-11640. Deletion services in SCM, OM and DN should have consistent metrics #7426

Open
wants to merge 7 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 6 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,29 @@ public final class BlockDeletingServiceMetrics {
" to container lock wait timeout.")
private MutableGaugeLong totalLockTimeoutTransactionCount;

// Iteration specific metrics.
@Metric("Start time of last iteration of BlockDeletingService")
private MutableGaugeLong startTimeOfLastIteration;

@Metric("Total time taken by the last iteration of BlockDeletingService")
private MutableGaugeLong durationOfLastIteration;

@Metric(about = "Total number of blocks chosen to be deleted in latest iteration.")
private MutableGaugeLong blockChosenCountInLastIteration;

@Metric(about = "Total number of containers chosen to be deleted in the latest iteration.")
private MutableGaugeLong containerChosenCountInLastIteration;

@Metric(about = "Total number of successful delete blocks in latest iteration.")
private MutableCounterLong successCountInLastIteration;

@Metric(about = "The total bytes for blocks successfully deleted in the latest iteration.")
private MutableCounterLong successBytesInLastIteration;

@Metric(about = "The number of failed delete blocks in the latest iteration.")
private MutableCounterLong failureCountInLastIteration;


private BlockDeletingServiceMetrics() {
}

Expand Down Expand Up @@ -184,6 +207,36 @@ public long getTotalLockTimeoutTransactionCount() {
return totalLockTimeoutTransactionCount.value();
}

public void setStartTimeOfLastIteration(long startTimeOfLastIteration) {
this.startTimeOfLastIteration.set(startTimeOfLastIteration);
}

public void setDurationOfLastIteration(long durationOfLastIteration) {
this.durationOfLastIteration.set(durationOfLastIteration);
}

public void setBlockChosenCountInLastIteration(
long blockChosenCountInLastIteration) {
this.blockChosenCountInLastIteration.set(blockChosenCountInLastIteration);
}

public void setContainerChosenCountInLastIteration(
long containerChosenCountInLastIteration) {
this.containerChosenCountInLastIteration.set(containerChosenCountInLastIteration);
}

public void incrSuccessCountInLastIteration(long delta) {
this.successCountInLastIteration.incr(delta);
}

public void incrSuccessBytesInLastIteration(long delta) {
this.successBytesInLastIteration.incr(delta);
}

public void incrFailureCountInLastIteration(long delta) {
this.failureCountInLastIteration.incr(delta);
}

@Override
public String toString() {
StringBuffer buffer = new StringBuffer();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis;
import org.apache.hadoop.ozone.container.keyvalue.statemachine.background.BlockDeletingTask;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -127,12 +128,14 @@ public Long getNumBlocksToDelete() {
public BackgroundTaskQueue getTasks() {
BackgroundTaskQueue queue = new BackgroundTaskQueue();

BlockDeletingTask.resetLastIterationCounts();
try {
// We at most list a number of containers a time,
// in case there are too many containers and start too many workers.
// We must ensure there is no empty container in this result.
// The chosen result depends on what container deletion policy is
// configured.
long startTime = Time.monotonicNow();
List<ContainerBlockInfo> containers =
chooseContainerForBlockDeletion(getBlockLimitPerInterval(),
containerDeletionPolicy);
Expand All @@ -150,8 +153,15 @@ public BackgroundTaskQueue getTasks() {
queue.add(containerBlockInfos);
totalBlocks += containerBlockInfo.getNumBlocksToDelete();
}
metrics.setDurationOfLastIteration(Time.monotonicNow() - startTime);
metrics.setStartTimeOfLastIteration(startTime);
metrics.setBlockChosenCountInLastIteration(totalBlocks);
metrics.setContainerChosenCountInLastIteration(containers.size());
metrics.incrTotalBlockChosenCount(totalBlocks);
metrics.incrTotalContainerChosenCount(containers.size());
metrics.incrSuccessCountInLastIteration(BlockDeletingTask.getSuccessBlockDeleteInLastIteration());
metrics.incrSuccessBytesInLastIteration(BlockDeletingTask.getSuccessBytesDeletedInLastIteration());
metrics.incrFailureCountInLastIteration(BlockDeletingTask.getFailBlockDeleteInLastIteration());
Tejaskriya marked this conversation as resolved.
Show resolved Hide resolved
if (containers.size() > 0) {
LOG.debug("Queued {} blocks from {} containers for deletion",
totalBlocks, containers.size());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,9 @@ public class BlockDeletingTask implements BackgroundTask {
private final OzoneContainer ozoneContainer;
private final ConfigurationSource conf;
private Duration blockDeletingMaxLockHoldingTime;
private static long successBlockDeleteInLastIteration;
private static long successBytesDeletedInLastIteration;
private static long failBlockDeleteInLastIteration;

public BlockDeletingTask(
BlockDeletingService blockDeletingService,
Expand Down Expand Up @@ -257,6 +260,8 @@ public ContainerBackgroundTaskResult deleteViaSchema1(
containerData.getVolume().decrementUsedSpace(releasedBytes);
metrics.incrSuccessCount(deletedBlocksCount);
metrics.incrSuccessBytes(releasedBytes);
incrSuccessBlockDeleteInLastIteration(deletedBlocksCount);
incrSuccessBytesDeletedInLastIteration(releasedBytes);
}

if (!succeedBlocks.isEmpty()) {
Expand All @@ -271,6 +276,7 @@ public ContainerBackgroundTaskResult deleteViaSchema1(
LOG.warn("Deletion operation was not successful for container: " +
container.getContainerData().getContainerID(), exception);
metrics.incrFailureCount();
incrFailBlockDeleteInLastIteration(1);
throw exception;
}
}
Expand Down Expand Up @@ -408,6 +414,8 @@ private ContainerBackgroundTaskResult deleteViaTransactionStore(
containerData.getVolume().decrementUsedSpace(releasedBytes);
metrics.incrSuccessCount(deletedBlocksCount);
metrics.incrSuccessBytes(releasedBytes);
incrSuccessBlockDeleteInLastIteration(deletedBlocksCount);
incrSuccessBytesDeletedInLastIteration(releasedBytes);
}

LOG.debug("Container: {}, deleted blocks: {}, space reclaimed: {}, " +
Expand All @@ -419,6 +427,7 @@ private ContainerBackgroundTaskResult deleteViaTransactionStore(
LOG.warn("Deletion operation was not successful for container: " +
container.getContainerData().getContainerID(), exception);
metrics.incrFailureCount();
incrFailBlockDeleteInLastIteration(1);
Tejaskriya marked this conversation as resolved.
Show resolved Hide resolved
throw exception;
}
}
Expand Down Expand Up @@ -517,6 +526,35 @@ public int getPriority() {
public Duration getBlockDeletingMaxLockHoldingTime() {
return blockDeletingMaxLockHoldingTime;
}
public static void resetLastIterationCounts() {
successBlockDeleteInLastIteration = 0;
successBytesDeletedInLastIteration = 0;
failBlockDeleteInLastIteration = 0;
}

public static long getSuccessBlockDeleteInLastIteration() {
return successBlockDeleteInLastIteration;
}

public static long getSuccessBytesDeletedInLastIteration() {
return successBytesDeletedInLastIteration;
}

public static long getFailBlockDeleteInLastIteration() {
return failBlockDeleteInLastIteration;
}

public static void incrSuccessBlockDeleteInLastIteration(long delta) {
BlockDeletingTask.successBlockDeleteInLastIteration += delta;
}

public static void incrSuccessBytesDeletedInLastIteration(long delta) {
BlockDeletingTask.successBytesDeletedInLastIteration += delta;
}

public static void incrFailBlockDeleteInLastIteration(long delta) {
BlockDeletingTask.failBlockDeleteInLastIteration += delta;
}

private interface Deleter {
void apply(Table<?, DeletedBlocksTransaction> deleteTxnsTable,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -175,6 +175,8 @@ public EmptyTaskResult call() throws Exception {
}

Set<Long> processedTxIDs = new HashSet<>();
long commandsSent = 0;
long totalTransactionsSent = 0;
for (Map.Entry<UUID, List<DeletedBlocksTransaction>> entry :
transactions.getDatanodeTransactionMap().entrySet()) {
UUID dnId = entry.getKey();
Expand All @@ -192,6 +194,8 @@ public EmptyTaskResult call() throws Exception {
new CommandForDatanode<>(dnId, command));
metrics.incrBlockDeletionCommandSent();
metrics.incrBlockDeletionTransactionSent(dnTXs.size());
commandsSent++;
totalTransactionsSent = totalTransactionsSent + dnTXs.size();
if (LOG.isDebugEnabled()) {
LOG.debug(
"Added delete block command for datanode {} in the queue,"
Expand All @@ -202,12 +206,17 @@ public EmptyTaskResult call() throws Exception {
}
}
}
long totalTime = Time.monotonicNow() - startTime;
LOG.info("Totally added {} blocks to be deleted for"
+ " {} datanodes / {} totalnodes, task elapsed time: {}ms",
transactions.getBlocksDeleted(),
transactions.getDatanodeTransactionMap().size(),
included.size(),
Time.monotonicNow() - startTime);
totalTime);
metrics.setStartTimeOfLastIteration(startTime);
metrics.setDurationOfLastIteration(totalTime);
metrics.setNumBlockDeletionCommandSentInLastIteration(commandsSent);
metrics.setNumBlockDeletionTransactionSentInLastIteration(totalTransactionsSent);
deletedBlockLog.incrementCount(new ArrayList<>(processedTxIDs));
} catch (NotLeaderException nle) {
LOG.warn("Skip current run, since not leader any more.", nle);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -466,14 +466,17 @@ public void commitTransactions(

ArrayList<Long> txIDsToBeDeleted = new ArrayList<>();
Set<UUID> dnsWithCommittedTxn;
int numTransactionsSuccess = 0, numTransactionsFail = 0;
for (DeleteBlockTransactionResult transactionResult :
transactionResults) {
if (isTransactionFailed(transactionResult)) {
metrics.incrBlockDeletionTransactionFailure();
numTransactionsFail++;
continue;
}
try {
metrics.incrBlockDeletionTransactionSuccess();
numTransactionsSuccess++;
long txID = transactionResult.getTxID();
// set of dns which have successfully committed transaction txId.
dnsWithCommittedTxn = transactionToDNsCommitMap.get(txID);
Expand Down Expand Up @@ -524,7 +527,12 @@ public void commitTransactions(
}
try {
deletedBlockLogStateManager.removeTransactionsFromDB(txIDsToBeDeleted);
metrics.incrBlockDeletionTransactionCompleted(txIDsToBeDeleted.size());
int numTransactionsCompleted = txIDsToBeDeleted.size();
metrics.incrBlockDeletionTransactionCompleted(numTransactionsCompleted);
metrics.setNumBlockDeletionTransactionCompletedInLastIteration(numTransactionsCompleted);
metrics.setNumBlockDeletionTransactionSuccessInLastIteration(numTransactionsSuccess);
metrics.setNumBlockDeletionTransactionFailureInLastIteration(numTransactionsFail);

} catch (IOException e) {
LOG.warn("Could not commit delete block transactions: "
+ txIDsToBeDeleted, e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -196,6 +196,62 @@ public long getNumBlockDeletionTransactionDataNodes() {
return numBlockDeletionTransactionDataNodes.value();
}

@Metric(about = "Start time taken of last iteration of ScmBlockDeletingService")
private MutableGaugeLong startTimeOfLastIteration;

@Metric(about = "Total time taken by the last iteration of ScmBlockDeletingService.")
private MutableGaugeLong durationOfLastIteration;

@Metric(about = "Total number of individual delete transaction commands sent " +
"to all DN in last iteration.")
private MutableGaugeLong numBlockDeletionCommandSentInLastIteration;

@Metric(about = "Total number of individual delete transactions sent to " +
"all DN in last iteration.")
private MutableGaugeLong numBlockDeletionTransactionSentInLastIteration;

@Metric(about = "Total number of success execution of delete transactions in last iteration.")
private MutableGaugeLong numBlockDeletionTransactionSuccessInLastIteration;

@Metric(about = "Total number of failure execution of delete transactions in last iteration.")
private MutableGaugeLong numBlockDeletionTransactionFailureInLastIteration;

@Metric(about = "Total number of completed txs which are removed from DB in last iteration.")
private MutableGaugeLong numBlockDeletionTransactionCompletedInLastIteration;

public void setStartTimeOfLastIteration(long startTimeOfLastIteration) {
this.startTimeOfLastIteration.set(startTimeOfLastIteration);
}

public void setDurationOfLastIteration(long durationOfLastIteration) {
this.durationOfLastIteration.set(durationOfLastIteration);
}

public void setNumBlockDeletionCommandSentInLastIteration(
long numBlockDeletionCommandSentInLastIteration) {
this.numBlockDeletionCommandSentInLastIteration.set(numBlockDeletionCommandSentInLastIteration);
}

public void setNumBlockDeletionTransactionSentInLastIteration(
long numBlockDeletionTransactionSentInLastIteration) {
this.numBlockDeletionTransactionSentInLastIteration.set(numBlockDeletionTransactionSentInLastIteration);
}

public void setNumBlockDeletionTransactionSuccessInLastIteration(
long numBlockDeletionTransactionSuccessInLastIteration) {
this.numBlockDeletionTransactionSuccessInLastIteration.set(numBlockDeletionTransactionSuccessInLastIteration);
}

public void setNumBlockDeletionTransactionFailureInLastIteration(
long numBlockDeletionTransactionFailureInLastIteration) {
this.numBlockDeletionTransactionFailureInLastIteration.set(numBlockDeletionTransactionFailureInLastIteration);
}

public void setNumBlockDeletionTransactionCompletedInLastIteration(
long numBlockDeletionTransactionCompletedInLastIteration) {
this.numBlockDeletionTransactionCompletedInLastIteration.set(numBlockDeletionTransactionCompletedInLastIteration);
}

@Override
public String toString() {
StringBuffer buffer = new StringBuffer();
Expand Down