Conversation
* Tear down nodeAnnouncer * Remove useless Logger and ExecutorService * Init CuratorListener by lambda * Improve explicit type * Using CuratorMultiTransaction instead of CuratorTransaction * Add @GuardedBy("toAnnounce") for toUpdate field * Improve docs
leventov
left a comment
There was a problem hiding this comment.
Please also apply the same improvements to Announcer when possible, as the ones suggested for NodeAnnouncer here, as well as these improvements already applied by @kaijianding and you to this moment, in the course of #6683.
| import java.util.concurrent.CopyOnWriteArrayList; | ||
|
|
||
| /** | ||
| * NodeAnnouncer announces single node on Zookeeper and only watches this node, |
There was a problem hiding this comment.
Please add a mirroring comment in Announcer
| private final ConcurrentMap<String, NodeCache> listeners = new ConcurrentHashMap<>(); | ||
| private final ConcurrentMap<String, byte[]> announcedPaths = new ConcurrentHashMap<>(); | ||
| /** | ||
| * Only the one created the parent path can drop the parent path, so should remember these created parents. |
There was a problem hiding this comment.
This comment sounds confusing, shouldn't one of "parent path" occurrences in it be something else?
|
|
||
| started = false; | ||
|
|
||
| Closer closer = Closer.create(); |
There was a problem hiding this comment.
I think you can add unannouncements, and pathsCreatedInThisAnnouncer deletion operation to the Closer too, so they all attempted in case of problems.
There was a problem hiding this comment.
Sorry, I don't understand this comment. These delete operations are not closable, can they be registered in Closer?
There was a problem hiding this comment.
Like closer.register(() -> unannounce(path))
There was a problem hiding this comment.
Thanks, I will patch this comment 👍
There was a problem hiding this comment.
I guess this change hasn't been applied yet?
| unannounce(announcementPath); | ||
| } | ||
|
|
||
| if (!pathsCreatedInThisAnnouncer.isEmpty()) { |
There was a problem hiding this comment.
Please extract this as a method
There was a problem hiding this comment.
Is it really necessary? 😰
There was a problem hiding this comment.
Okay, let's stay the original way 😄
| } | ||
| } | ||
| catch (Exception e) { | ||
| log.debug(e, "Problem checking if the parent existed, ignoring."); |
There was a problem hiding this comment.
Please change the comment ", assuming it doesn't exist."
|
|
||
| final byte[] billy = StringUtils.toUtf8("billy"); | ||
| final String testPath = "/somewhere/test2"; | ||
| final String parent = ZKPaths.getPathAndNode(testPath).getPath(); |
| * NodeAnnouncer announces single node on Zookeeper and only watches this node, | ||
| * while {@link Announcer} watches all child paths, not only this node | ||
| */ | ||
| public class NodeAnnouncer |
There was a problem hiding this comment.
Please add a concurrent control flow documentation explaining how and why somebody may call announce() and update() concurrently or before start().
There was a problem hiding this comment.
Okay, I will improve this doc later.
| } | ||
| } | ||
|
|
||
| private void createPath(String parentPath, boolean removeParentsIfCreated) |
There was a problem hiding this comment.
Please annotate @GuardedBy("toAnnounce")
| /** | ||
| * Only the one created the parent path can drop the parent path, so should remember these created parents. | ||
| */ | ||
| private final List<String> pathsCreatedInThisAnnouncer = new CopyOnWriteArrayList<>(); |
There was a problem hiding this comment.
- Please annotate
@GuardedBy("toAnnounce") - Doesn't need to be
CopyOnWriteArrayList, can be a simpleArrayList.
| try { | ||
| curator.create().creatingParentsIfNeeded().forPath(parentPath); | ||
| if (removeParentsIfCreated) { | ||
| pathsCreatedInThisAnnouncer.add(parentPath); |
There was a problem hiding this comment.
The path is added pathsCreatedInThisAnnouncer regardless of whether it was actually created two lines above or already existed?
There was a problem hiding this comment.
I think so, what's your concern?
There was a problem hiding this comment.
Because then the path is not actually paths**Created**InThisAnnouncer
There was a problem hiding this comment.
This should be resolved before merge
|
this is similar to #6683? |
|
@pzhdfy Yep, the original discussion is here: https://lists.apache.org/thread.html/r92fcfa896418b941dd4aa1eed7b60aaf5b7e2ea55137600d844ff4a4%40%3Cdev.druid.apache.org%3E |
We use #6683 in our production environment, and it works great fine. The watch count decrease more than 90% . So I raise this pr to be merged. |
jihoonson
left a comment
There was a problem hiding this comment.
Hi @asdf2014, thank you for taking up this issue! I left a couple of comments. Besides them, I would like to say, it could be better if we share some common codes between NodeAnnouncer and Announcer. But I would regard this comment as a nit and don't mind if the refactoring would have done in a followup PR.
| import java.util.concurrent.ConcurrentMap; | ||
|
|
||
| /** | ||
| * {@link NodeAnnouncer} announces single node on Zookeeper and only watches this node, |
There was a problem hiding this comment.
typo: announces a single node.
| @@ -64,6 +67,7 @@ public class Announcer | |||
| private final ExecutorService pathChildrenCacheExecutor; | |||
|
|
|||
| private final List<Announceable> toAnnounce = new ArrayList<>(); | |||
There was a problem hiding this comment.
Please add @GuaredBy("toAnnounce") for this list too.
| * In case a path is added to this collection in {@link #announce} before zk is connected, | ||
| * should remember the path and do announce in {@link #start} later. | ||
| */ | ||
| private final List<Announceable> toAnnounce = new ArrayList<>(); |
There was a problem hiding this comment.
Please add @GuaredBy("toAnnounce") for this list too.
| private final ConcurrentMap<String, NodeCache> listeners = new ConcurrentHashMap<>(); | ||
| private final ConcurrentMap<String, byte[]> announcedPaths = new ConcurrentHashMap<>(); | ||
| /** | ||
| * Only the one created the parent path can drop it, so should remember these created parents. |
There was a problem hiding this comment.
How about rephrasing such as This list is to remember all paths this node announcer has created. On {@list #stop}, the node announcer is responsible for deleting all paths in this list.
There was a problem hiding this comment.
Thanks. It sounds better. Also, I use {@link #stop} instead of {@list #stop}.
|
|
||
| started = false; | ||
|
|
||
| Closer closer = Closer.create(); |
There was a problem hiding this comment.
I guess this change hasn't been applied yet?
| try { | ||
| if (!Arrays.equals(oldBytes, bytes)) { | ||
| announcedPaths.put(path, bytes); | ||
| updateAnnouncement(path, bytes); |
There was a problem hiding this comment.
I guess it would be probably worth to define the concurrency control more precisely across all variables and methods in this class. I'm not sure why listeners is a concurrentHashMap since it's guarded by toAnnounce. Similar question for announcedPaths. Seems like it's a concurrentHashMap to allow reads without locking toAnnounce. However, unannounce() updates announcedPaths without the lock on toAnnounce. Maybe they don't have to be concurrentHashMaps. Or maybe there is another way to provide a better concurrency control.
|
Hi, @jihoonson. Thank you and @leventov for your comments. They are very helpful. I would prefer to raise another one for a larger refactoring, so we can resolve this issue ASAP. |
Ok. Please consider the latest comments from me and @leventov. |
|
@jihoonson Okay, no problem. Thanks also to @leventov for creating the #9244 issue. |
|
@asdf2014 thanks. Let me know when this PR is ready for another review. |
|
@jihoonson Sure, I will try to continue to address these comments. Thanks for reminding. |
|
@asdf2014 thank you! |
|
@jihoonson You are welcome! |
|
This pull request has been marked as stale due to 60 days of inactivity. It will be closed in 4 weeks if no further activity occurs. If you think that's incorrect or this pull request should instead be reviewed, please simply write any comment. Even if closed, you can still revive the PR at any time or discuss it on the dev@druid.apache.org list. Thank you for your contributions. |
|
This pull request/issue has been closed due to lack of activity. If you think that is incorrect, or the pull request requires review, you can revive the PR at any time. |
|
This pull request/issue is no longer marked as stale. |
|
This pull request has been marked as stale due to 60 days of inactivity. It will be closed in 4 weeks if no further activity occurs. If you think that's incorrect or this pull request should instead be reviewed, please simply write any comment. Even if closed, you can still revive the PR at any time or discuss it on the dev@druid.apache.org list. Thank you for your contributions. |
|
This pull request/issue has been closed due to lack of activity. If you think that is incorrect, or the pull request requires review, you can revive the PR at any time. |
|
Hi @asdf2014 , are you planning to continue working on this PR? |
Description
Fix huge number of watches in zk
Tear down nodeAnnouncer
Remove useless Logger and ExecutorService
Init CuratorListener by lambda
Improve explicit type
Using CuratorMultiTransaction instead of CuratorTransaction
Add @GuardedBy("toAnnounce") for toUpdate field
Improve docs
Related to #6683
This PR has: