ZOOKEEPER-3437: Improve sync throttling on a learner master#995
ZOOKEEPER-3437: Improve sync throttling on a learner master#995jhuan31 wants to merge 4 commits intoapache:masterfrom
Conversation
|
Thank you for working on this! I left a comment based on my understanding. :) |
| private static final Logger LOG = LoggerFactory.getLogger(LearnerMaster.class); | ||
|
|
||
| // Throttle when there are too many concurrent snapshots being sent to observers | ||
| private static final String MAX_CONCURRENT_SNAPSYNCS = "zookeeper.leader.maxConcurrentSnapSyncs"; |
There was a problem hiding this comment.
Would this configuration be shared between leader and observer master? If so, I think there are a few assumptions that worth a closer look.
- Leader and Followers are running on machines with similar capability (CPU, network bandwidth). This assumption is usually true.
- ObserverMasters and Observers are also running on machines with similar capabilities (CPU, network bandwidth). This assumption may not be true. E.g. If we deploy some Observers in a different data center than the ObserverMaster, then the effective network bandwidth between ObserverMaster and Observer could be significantly smaller than the bandwidth between Leader and Followers. In this case, we might want to set a different maxConcurrentSnapSyncs for ObserverMaster than Leader.
Do those assumptions make sense here?
There was a problem hiding this comment.
The thing is, we don't know what the appropriate thresholds are until we roll the feature out and then tune the settings. If it turns out that we do need different settings for Leader and ObserverMaster, we can add it, instead of introducing a setting that might never be used. What do you think? (BTW, in our deployment, Leader and Followers can be in different data centers too.)
There was a problem hiding this comment.
Thanks for the details! I think we can keep this configuration just one parameter for now. I'd love to learn the actual system behavior once this is used in production.
| return maxConcurrentSnapSyncs; | ||
| } | ||
|
|
||
| public void setMaxConcurrentSnapSyncs(int maxConcurrentSnapSyncs) { |
There was a problem hiding this comment.
I have not seen any usage of setMaxConcurrentSnapSyncs, and I am not sure how we can set different values for Leader vs ObserverMaster. How do we plan to use setMaxConcurrentSnapSyncs?
There was a problem hiding this comment.
This function is called in FollwerBean and LeaderBean. No, we can't set different values for Leader and ObserverMaster. We did have a discussion whether to support different settings for Leader and ObserverMaster. Since we don't see a need for different settings and since we have too many flags/settings already, we decide to go with one setting for both Leader and ObserverMaster
|
We don't have any design docs :( But you just remind me that I need to update the ZooKeeper Administrator's Guide :) |
| public void shutdown() { | ||
| // Send the packet of death | ||
| try { | ||
| queuedPackets.clear(); |
There was a problem hiding this comment.
This seems unrelated to this patch, but it looks the right thing to do. Just want to make sure this was not accidentally introduced.
There was a problem hiding this comment.
Good catch! It is actually added in this patch to handle diff sync throttling. Diff txns are queued in syncFollower() before the throttle. Later when we decide that this diff sync should be throttled, the txns are in the queues already so we need to remove them.
|
|
||
| /** | ||
| * Utility class to limit the number of concurrent syncs from a leader to | ||
| * observers and followers. {@link LearnerHandler} objects should call |
There was a problem hiding this comment.
nit: might add "or syncs from a follower to observers".
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerHandler.java
Show resolved
Hide resolved
Yes it'll be great to include the update as part of this patch. |
hanm
left a comment
There was a problem hiding this comment.
LGTM with some nits (and after doc is updated).
eolivelli
left a comment
There was a problem hiding this comment.
Looks good to me as well.
but I left a very few minor comments, please fix them
|
|
||
| syncLimitCheck.start(); | ||
| // sync ends when NEWLEADER-ACK is received | ||
| syncThrottler.endSync(); |
There was a problem hiding this comment.
Is this endSync() call to be put in a finally block?
There was a problem hiding this comment.
endSync() is called in the finally block line 720-722, if not called earlier.
| } | ||
|
|
||
| @Test(expected = SyncThrottleException.class) | ||
| public void testTryWithResourceThrottle() throws Exception { |
There was a problem hiding this comment.
It is not very clear that we are testing that the second call to beginSync is the one that throws the exception
I suggest to use an explicit catch block and not use the 'expected' attribute
|
|
||
| final LearnerSyncThrottler throttler = | ||
| new LearnerSyncThrottler(numThreads, syncType); | ||
| ExecutorService threadPool = Executors.newFixedThreadPool(numThreads); |
There was a problem hiding this comment.
Please shutdown this pool in a finally block
|
retest maven build |
Author: Jie Huang <jiehuang@fb.com> Reviewers: Michael Han <lhan@twitter.com>, Enrico Olivelli <eolivelli@apache.org>, Fangmin Lyu <fangmin@apache.org> Closes apache#995 from jhuan31/ZOOKEEPER-3437
Author: Jie Huang <jiehuang@fb.com> Reviewers: Michael Han <lhan@twitter.com>, Enrico Olivelli <eolivelli@apache.org>, Fangmin Lyu <fangmin@apache.org> Closes apache#995 from jhuan31/ZOOKEEPER-3437
Author: Jie Huang <jiehuang@fb.com> Reviewers: Michael Han <lhan@twitter.com>, Enrico Olivelli <eolivelli@apache.org>, Fangmin Lyu <fangmin@apache.org> Closes apache#995 from jhuan31/ZOOKEEPER-3437
No description provided.