Skip to content

[ZOOKEEPER-3657] Implementing snapshot schedule to avoid high latency issue due to disk contention#1191

Open
lvfangmin wants to merge 1 commit intoapache:masterfrom
lvfangmin:ZOOKEEPER-3657
Open

[ZOOKEEPER-3657] Implementing snapshot schedule to avoid high latency issue due to disk contention#1191
lvfangmin wants to merge 1 commit intoapache:masterfrom
lvfangmin:ZOOKEEPER-3657

Conversation

@lvfangmin
Copy link
Contributor

No description provided.

@asf-ci
Copy link

asf-ci commented Dec 20, 2019

@lvfangmin
Copy link
Contributor Author

retest maven build

@hanm
Copy link
Contributor

hanm commented Dec 21, 2019

interesting. two quick questions:

  • zookeeper has some randomness in today's sync processor when snapshotting already, i am curious to know what's the short comings of existing approach that motivates this change.

  • i don't see observers processing snap ping packet in this pull request so only quorum servers will be scheduled by leader for snap shot generation?

@anmolnar
Copy link
Contributor

@lvfangmin I second @hanm 's questions. This patch is quite huge, are you still working on it?

@lvfangmin
Copy link
Contributor Author

@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.

Copy link
Contributor

@anmolnar anmolnar 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.
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**)
Copy link
Contributor

Choose a reason for hiding this comment

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

Fix typo please: 'Jave'

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
Copy link
Contributor

Choose a reason for hiding this comment

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

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**)
Copy link
Contributor

Choose a reason for hiding this comment

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

Typo: 'Jave'

default value is 100,000 which is the suggested value.

* *leader.snapTxnsSizeThresholdKB*
(Jave system property only: **zookeeper.leader.snapTxnsSizeThresholdKB**)
Copy link
Contributor

Choose a reason for hiding this comment

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

Typo: 'Jave' (looks like a copy-paste problem)

});
return true;
} else {
LOG.warn("Too busy to snap, skipping");
Copy link
Contributor

Choose a reason for hiding this comment

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

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");
Copy link
Contributor

Choose a reason for hiding this comment

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

Typo: 'imcompatible'

@lvfangmin
Copy link
Contributor Author

Rebased and addressed the nit suggestion from @anmolnar.

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

Great work

return;
}

if (snapCode == SnapPingCode.CANCEL.ordinal()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Relying on ordinal() can lead to problems in case that someone refactor the class, adds items, reorders them.
What about having well defined constants?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

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.

Copy link
Contributor

@hanm hanm left a comment

Choose a reason for hiding this comment

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

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**)
Copy link
Contributor

Choose a reason for hiding this comment

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

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
Copy link
Contributor

Choose a reason for hiding this comment

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

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(
Copy link
Contributor

Choose a reason for hiding this comment

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

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);
Copy link
Contributor

Choose a reason for hiding this comment

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

should we pass snapGenerator.fsyncSnapshotFromScheduler as parameter here instead of hardcoding a false?

Copy link
Contributor

Choose a reason for hiding this comment

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

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);
Copy link
Contributor

Choose a reason for hiding this comment

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

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);
Copy link
Contributor

Choose a reason for hiding this comment

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

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?

Copy link
Contributor

@hanm hanm left a comment

Choose a reason for hiding this comment

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

finishing reviewing rest of the pull request.

try {
listener.snapPing(SnapPingListener.SNAP_PING_ID_DONT_CARE,
sid == learnerSnapCandidate
? SnapPingCode.SNAP : SnapPingCode.SKIP);
Copy link
Contributor

Choose a reason for hiding this comment

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

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;
Copy link
Contributor

Choose a reason for hiding this comment

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

is this to deal with overflow? might worth to add a comment here.


@Before
public void setup() throws Exception {
System.setProperty(
Copy link
Contributor

Choose a reason for hiding this comment

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

I don't see where zookeeper.leader.snapPingIntervalInSeconds is set - isn't it require to explicit set this to enable the snap schedule feature?

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.

5 participants