Skip to content

Conversation

@sodonnel
Copy link
Contributor

@sodonnel sodonnel commented Nov 11, 2019

What changes were proposed in this pull request?

The datanode receives commands over the heartbeat and queues all commands on a single queue in StateContext.commandQueue. Inside DatanodeStateMachine a single thread is used to process this queue (started by initCommandHander thread) and it passes each command to a ‘handler’. Each command type has its own handler.

The delete container command immediately executes the command on the thread used to process the command queue. Therefore if the delete is slow for some reason (it must access disk, so this is possible) it could cause other commands to backup.

This should be changed to use a threadpool to queue the deleteContainer command, in a similar way to ReplicateContainerCommand.

What is the link to the Apache JIRA

https://issues.apache.org/jira/browse/HDDS-2448

@sodonnel
Copy link
Contributor Author

This patch is not ready to commit yet, as it needs HDDS-2450 committed first and then the threadpool size made configurable.

Copy link
Contributor

@anuengineer anuengineer left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1. LGTM. If you can push the change for 2 -> a name , then I will commit it immediately. Thanks for getting this done.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you define this as delete thread pool Size somewhere, otherwise 2 is kind of confusing to people.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, that "2" needs to be configurable just like for the "replicate container thread pool size" in HDDS-2450. Now HDDS-2450 has been committed, I will rebase this patch and make this thread pool configurable in the same way.

@sodonnel sodonnel force-pushed the HDDS-2448-del-container branch 2 times, most recently from 8a994d4 to 7796c58 Compare November 15, 2019 11:15
@sodonnel sodonnel force-pushed the HDDS-2448-del-container branch from 7796c58 to 471a648 Compare November 19, 2019 12:14
Copy link
Contributor

@adoroszlai adoroszlai left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @sodonnel for the improvement.

I have a few minor nits about code and config naming. Please consider them for this PR, should another round of rebase be necessary. Otherwise, please consider them for your next change related to this area. Thanks.

I would also love a brief summary about how this was tested. ;)

private long totalTime;
private final AtomicInteger invocationCount = new AtomicInteger(0);
private final AtomicLong totalTime = new AtomicLong(0);
private final ThreadPoolExecutor executor;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Prefer declaring it as the more generic ExecutorService.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have fixed this.

Comment on lines 96 to 97
return invocationCount.get() == 0 ?
0 : totalTime.get() / invocationCount.get();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The result of invocationCount.get() could be saved in a local variable for consistency.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Well spotted. I have fixed this.

return replicationMaxStreams;
}

@Config(key = "delete.container.threads",
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: I think container.delete.threads.max would better reflect both the hierarchical nature of config item naming, and the fact that it's an upper limit, not a fixed thread count.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You are correct. It makes more sense to have "container.delete". I have changed this and also the related method names so they match up.

Comment on lines 74 to 77
if (val < 1) {
LOG.warn("hdds.datanode.delete.container.threads must be greater than" +
"zero and was set to {}. Defaulting to {}",
val, deleteContainerThreads);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is great only for the case when config is set during load from XML. Later, if the config is being set programmatically, we have two (minor) issues:

  1. the default value is not the real default, rather the previously set valid config
  2. warning is human-friendly, but irrelevant to programmatic callers

Eg.

setDeleteContainerThreads(10);
setDeleteContainerThreads(-1);

would log ... Defaulting to 10.

So I think this could be improved by:

  1. creating a constant for the default value
  2. using an Integer to distinguish between unset and set states
  3. logging the warning only if previously unset
  4. consider throwing an exception (or silently ignoring invalid values) if previously already set

(I wanted to mention this for the previous, replication-related PR, but was late to the party. ;) )

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These are good points. For now, I have kept the warn log message, as I don't like the idea of silently ignoring bad values. From a usability perspective, I would like the DN to still start if an operator puts a bad value in the config rather than fail completely. However it is easy to argue this the other way too, that we should fail on a bad value rather than trying to be too clever.

I have taken on the default point and applied it to both parameters in the DatanodeConfig class.

Copy link
Contributor

@adoroszlai adoroszlai left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @sodonnel for addressing my nits.

@anuengineer anuengineer merged commit 6186cf9 into apache:master Nov 19, 2019
@anuengineer
Copy link
Contributor

Thanks for the contribution. I have committed this to the master branch. @adoroszlai Thanks for the reviews.

ptlrs pushed a commit to ptlrs/ozone that referenced this pull request Mar 8, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants