[ZOOKEEPER-3657] Implementing snapshot schedule to avoid high latency issue due to disk contention#1191
[ZOOKEEPER-3657] Implementing snapshot schedule to avoid high latency issue due to disk contention#1191lvfangmin wants to merge 1 commit intoapache:masterfrom
Conversation
|
Refer to this link for build results (access rights to CI server needed): Failed Tests: 1PreCommit-ZOOKEEPER-github-pr-build-maven/org.apache.zookeeper:zookeeper: 1 |
|
retest maven build |
|
interesting. two quick questions:
|
|
@lvfangmin I second @hanm 's questions. This patch is quite huge, are you still working on it? |
|
@hanm the randomness snapshot could introduce high latency if majority taking snapshot at the same time, when the total DataTree size increasing, it will take longer time to do snapshot, which means it's more likely majority will take snapshot at the same time with longer period. Which will be a problem when running ZK on a single disk driver. From what we saw in benchmark, the write throughput within SLA for 6GB DataTree size is more than 10X smaller than 100MB DataTree. That's why we introduced this feature. Observer don't need to handle SNAPPING, since the quorum ack latency is only affected by participants. @anmolnar this feature is complete, it has been on our prod for more than 6 months. |
anmolnar
left a comment
There was a problem hiding this comment.
+1 lgtm.
Just a few nitpicks and also rebase please.
| handshakes. Set it to something like 250 is good enough to avoid herd effect. | ||
|
|
||
| * *leader.snapPingIntervalInSeconds* | ||
| (Jave system property only: **zookeeper.leader.snapPingIntervalInSeconds**) |
| scheduler if it's enabled, and send SNAPPING to the quorum. If the follower is | ||
| running old code, it will ignore that packet. When follower with new code received | ||
| SNAPPING packet, it will turn off the periodically snapshot locally, and only | ||
| taking safety snapshot if the if the txns since last snapshot is much larger than |
There was a problem hiding this comment.
Fix typo: 'if the if the'
| Also there is a JMX setting on leader to turn it on and off in flight. | ||
|
|
||
| * *leader.snapTxnsThreshold* | ||
| (Jave system property only: **zookeeper.leader.snapTxnsThreshold**) |
| default value is 100,000 which is the suggested value. | ||
|
|
||
| * *leader.snapTxnsSizeThresholdKB* | ||
| (Jave system property only: **zookeeper.leader.snapTxnsSizeThresholdKB**) |
There was a problem hiding this comment.
Typo: 'Jave' (looks like a copy-paste problem)
| }); | ||
| return true; | ||
| } else { | ||
| LOG.warn("Too busy to snap, skipping"); |
There was a problem hiding this comment.
Is this log message accurate? Getting here means previous snapshot is still running.
| + "compatible with ours {}, will skip", peerSnapPingVersion, | ||
| SnapPingManager.SNAP_PING_VERSION); | ||
| if (fzk.syncProcessor.isOnlySnapWhenSafetyIsThreatened()) { | ||
| LOG.info("SnapPing version imcompatible, start self snapshot"); |
|
Rebased and addressed the nit suggestion from @anmolnar. |
| return; | ||
| } | ||
|
|
||
| if (snapCode == SnapPingCode.CANCEL.ordinal()) { |
There was a problem hiding this comment.
Relying on ordinal() can lead to problems in case that someone refactor the class, adds items, reorders them.
What about having well defined constants?
There was a problem hiding this comment.
That's a good point, it seems more natural to use enum for these code, I'll add a comment to the enum to WARN ordering changes.
…issue due to disk contention
hanm
left a comment
There was a problem hiding this comment.
back to review mode, left some comments. Haven't finished review all parts, bear with my snail speed please.
| The default value is false. | ||
|
|
||
| * *leader.snapPingIntervalInSeconds* | ||
| (Java system property only: **zookeeper.leader.snapPingIntervalInSeconds**) |
There was a problem hiding this comment.
are these really java system properties only? Put a config foo in zoo.cfg and ZK will parse them and generate a zookeeoer.foo. similar for other "java only system properties" listed here. might need update doc.
| public static final String ZOOKEEPER_NODE_SUBTREE = "/zookeeper/"; | ||
|
|
||
| /** | ||
| * WARN: please don't retain the order, which is used to check |
There was a problem hiding this comment.
did you mean "don't change" or "retain", as opposed to "don't retain"? my understanding is the order must be preserved here for the code to work.
| purgeAfterSnapshot = Boolean.getBoolean(PURGE_AFTER_SNAPSHOT); | ||
| LOG.info("{} = {}", PURGE_AFTER_SNAPSHOT, purgeAfterSnapshot); | ||
|
|
||
| fsyncSnapshotFromScheduler = Boolean.parseBoolean( |
There was a problem hiding this comment.
why not use Boolean.getBoolean so it's consistent with previous property parsing code (also less verbose)?
|
|
||
| if (onlySnapWhenSafetyIsThreatened) { | ||
| if (safetySnapThreshold.meet(zkDB.getTxnsSinceLastSnap(), zkDB.getTxnsSizeSinceLastSnap())) { | ||
| snapGenerator.takeSnapshot(false); |
There was a problem hiding this comment.
should we pass snapGenerator.fsyncSnapshotFromScheduler as parameter here instead of hardcoding a false?
There was a problem hiding this comment.
also if we want to hard code a value, it seems true is more safe than false here for durability guarantees - but i lose track of what our default options were when taking snapshot (fsync it or not, especially if IIRC we lost the fsync parameter when introducing SnapStream..)
| } | ||
| } | ||
| }.start(); | ||
| snapGenerator.takeSnapshot(false); |
There was a problem hiding this comment.
similar here - should the parameter be hardcoded or taken from fsyncSnapshotFromScheduler
| if (snapCode == SnapPingCode.CANCEL.ordinal()) { | ||
| if (fzk.syncProcessor.isOnlySnapWhenSafetyIsThreatened()) { | ||
| LOG.info("Snapshot schedule cancelled by leader, start self snapshot"); | ||
| fzk.syncProcessor.setOnlySnapWhenSafetyIsThreatened(false); |
There was a problem hiding this comment.
did we consider that the CANCEL packet could potentially lost so a server in schedule snap mode will never be snapping again? Is there any built in defense mechanism for that case (didn't read all part of code yet) to make sure a server will not end up in not snapping state?
hanm
left a comment
There was a problem hiding this comment.
finishing reviewing rest of the pull request.
| try { | ||
| listener.snapPing(SnapPingListener.SNAP_PING_ID_DONT_CARE, | ||
| sid == learnerSnapCandidate | ||
| ? SnapPingCode.SNAP : SnapPingCode.SKIP); |
There was a problem hiding this comment.
we can simply skip this learner if its not the candidate. this saves sending a SnapPingCode.SKIP packet, as on learner side that code doesn't do anything. This also raise a question on why SnapPingCode.SKIP exists in first place - if we don't want a learner snap we can just skip sending it a command instead of sending it a no-op command.
| } | ||
|
|
||
| if (++snapPingId < 0) { | ||
| snapPingId = 1; |
There was a problem hiding this comment.
is this to deal with overflow? might worth to add a comment here.
|
|
||
| @Before | ||
| public void setup() throws Exception { | ||
| System.setProperty( |
There was a problem hiding this comment.
I don't see where zookeeper.leader.snapPingIntervalInSeconds is set - isn't it require to explicit set this to enable the snap schedule feature?
No description provided.