Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
74 changes: 72 additions & 2 deletions zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
Original file line number Diff line number Diff line change
Expand Up @@ -1128,7 +1128,7 @@ property, when available, is noted below.
The size threshold after which a request is considered a large request. If it is -1, then all requests are considered small, effectively turning off large request throttling. The default is -1.

* *outstandingHandshake.limit*
(Jave system property only: **zookeeper.netty.server.outstandingHandshake.limit**)
(Java system property only: **zookeeper.netty.server.outstandingHandshake.limit**)
The maximum in-flight TLS handshake connections could have in ZooKeeper,
the connections exceed this limit will be rejected before starting handshake.
This setting doesn't limit the max TLS concurrency, but helps avoid herd
Expand All @@ -1145,7 +1145,7 @@ property, when available, is noted below.
When set to 0, no requests will be throttled. The default is 0.

* *learner.closeSocketAsync*
(Jave system property only: **learner.closeSocketAsync**)
(Java system property only: **learner.closeSocketAsync**)
When enabled, a learner will close the quorum socket asynchronously. This is useful for TLS connections where closing a socket might take a long time, block the shutdown process, potentially delay a new leader election, and leave the quorum unavailabe. Closing the socket asynchronously avoids blocking the shutdown process despite the long socket closing time and a new leader election can be started while the socket being closed. The default is false.

* *forward_learner_requests_to_commit_processor_disabled*
Expand All @@ -1156,6 +1156,76 @@ property, when available, is noted below.

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.

Set the interval of snapshot scheduler, this is also the switch for
enabling/disabling snapshot scheduler.

Snapshot scheduler is the feature used to coordinate the time of snapshot
happens in the quorum, which avoid high latency issue due to majority of
servers taking snapshot at the same time when running on a single disk
driver.

A new quorum packet is added: SNAPPING, but it's backwards compatible and can be
rolled out safely with rolling restart. Leader will check and start the snapshot
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 txns since last snapshot is much larger than
the threshold defined in SyncRequestProcessor. This is used to avoid issues like
the follower accumulated too many txns before it is scheduled to take snapshot.

The default value is -1, which disables the central snapshot scheduler in
quorum. The suggest value would be 20s, which means it checks and schedule
the next round of snapshot every 20s. Note that each round will only schedule
at most one server to take snapshot.

Also there is a JMX setting on leader to turn it on and off in flight.

* *leader.snapTxnsThreshold*
(Java system property only: **zookeeper.leader.snapTxnsThreshold**)
The minimal number of txns to schedule snapshot since last snapshot. The
default value is 100,000 which is the suggested value.

* *leader.snapTxnsSizeThresholdKB*
(Java system property only: **zookeeper.leader.snapTxnsSizeThresholdKB**)
The minimal size of txns to scheduler snapshot since last snapshot. The
default value is 4GB, which is the suggested value.

* *flushLatencyDrainThreshold*
(Java system property only: **zookeeper.flushLatencyDrainThreshold**)
The threshold used to decide if the learner is having high fsync time,
which might due to draining the previous snapshot data. The default value
is 200ms, and when a server exceeds this threshold, it will be considered
as draining, being excluded from the idle snapshot server set.

* *learner.queueSizeDrainThreshold*
(Java system property only: **zookeeper.queueSizeDrainThreshold**)
The threshold used to decide if the learner is having long queue in
SyncRequestProcessor, the default value is 10,000. The server will be
considered as draining if it exceeds this threshold, and being excluded
from the idle snapshot server set.

* *snapsync.unscheduledSnapshotThreshold*
(Java system property only: **zookeeper.snapsync.unscheduledSnapshotThreshold**)
If no snapshot is scheduled in the last N runs due to things like slow disk
on majority for a while, leader will go ahead and schedule the snapshot to
avoid out of disk issue.

By default this value is 3, which means if no snapshot scheduled in 3 rounds,
leader will ignore the majority rule and schedule anyway.

* *purgeAfterSnapshot.enabled*
(Java system property: **zookeeper.purgeAfterSnapshot.enabled**)
Purge or not after snapshot, from our test it's better to enable this to
include purge in the snapshot schedule to avoid high latency due to purge.

The default value is false.

* *fsyncSnapshotFromScheduler*
(Java system property: **zookeeper.fsyncSnapshotFromScheduler**)
Fsync the snapshot or not. The default value is true, which seems better
from our testing.

<a name="sc_clusterOptions"></a>

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,14 @@ public class ZooDefs {

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.

* the op during snapshot schedule.
*/
public enum SnapPingCode {
CHECK, SNAP, SKIP, CANCEL;
}

@InterfaceAudience.Public
public interface OpCode {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,12 +73,14 @@ static void printUsage() {
* @throws IOException
*/
public static void purge(File dataDir, File snapDir, int num) throws IOException {
purge(new FileTxnSnapLog(dataDir, snapDir), num);
}

public static void purge(FileTxnSnapLog txnLog, int num) throws IOException {
if (num < 3) {
throw new IllegalArgumentException(COUNT_ERR_MSG);
}

FileTxnSnapLog txnLog = new FileTxnSnapLog(dataDir, snapDir);

List<File> snaps = txnLog.findNValidSnapshots(num);
int numSnaps = snaps.size();
if (numSnaps > 0) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -243,6 +243,8 @@ private ServerMetrics(MetricsProvider metricsProvider) {

REQUESTS_NOT_FORWARDED_TO_COMMIT_PROCESSOR = metricsContext.getCounter(
"requests_not_forwarded_to_commit_processor");
TAKING_SAFE_SNAPSHOT = metricsContext.getCounter("taking_safe_snapshot");
MANAGER_INITIATED_SAFE_SNAPSHOT = metricsContext.getCounter("manager_initiated_safe_snapshot");
}

/**
Expand Down Expand Up @@ -471,6 +473,9 @@ private ServerMetrics(MetricsProvider metricsProvider) {

private final MetricsProvider metricsProvider;

public final Counter TAKING_SAFE_SNAPSHOT;
public final Counter MANAGER_INITIATED_SAFE_SNAPSHOT;

public void resetAll() {
metricsProvider.resetAllValues();
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,105 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.zookeeper.server;

import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicBoolean;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* Util class Used to control the behavior abouthow we take snapshot.
*/
public class SnapshotGenerator {
private static final Logger LOG = LoggerFactory.getLogger(SnapshotGenerator.class);

public static final String PURGE_AFTER_SNAPSHOT = "zookeeper.purgeAfterSnapshot.enabled";
private static boolean purgeAfterSnapshot;

public static final String FSYNC_SNAPSHOT_FROM_SCHEDULER = "zookeeper.fsyncSnapshotFromScheduler";
private static boolean fsyncSnapshotFromScheduler;

static {
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)?

System.getProperty(FSYNC_SNAPSHOT_FROM_SCHEDULER, "true"));
LOG.info("{} = {}", FSYNC_SNAPSHOT_FROM_SCHEDULER, fsyncSnapshotFromScheduler);
}

public static boolean getPurgeAfterSnapshot() {
return purgeAfterSnapshot;
}

public static void setPurgeAfterSnapshot(boolean enabled) {
purgeAfterSnapshot = enabled;
LOG.info("{} = {}", PURGE_AFTER_SNAPSHOT, purgeAfterSnapshot);
}

public static void setFsyncSnapshotFromScheduler(boolean fsync) {
fsyncSnapshotFromScheduler = fsync;
LOG.info("{} = {}", FSYNC_SNAPSHOT_FROM_SCHEDULER, fsyncSnapshotFromScheduler);
}

public static boolean getFsyncSnapshotFromScheduler() {
return fsyncSnapshotFromScheduler;
}

private final ZooKeeperServer zks;
private final ExecutorService worker;
private final AtomicBoolean isTakingSnapshot;

public SnapshotGenerator(final ZooKeeperServer zks) {
this.zks = zks;
this.worker = Executors.newFixedThreadPool(1);
this.isTakingSnapshot = new AtomicBoolean(false);
}

public boolean takeSnapshot(boolean syncSnap) {
// Only allow a single snapshot in progress.
if (isTakingSnapshot.compareAndSet(false, true)) {
this.worker.execute(new Runnable() {
@Override
public void run() {
try {
zks.takeSnapshot(syncSnap);
if (purgeAfterSnapshot) {
zks.purge();
}
} catch (Exception e) {
LOG.warn("Unexpected exception", e);
} finally {
isTakingSnapshot.compareAndSet(true, false);
}
}
});
return true;
} else {
LOG.warn("Previous snapshot is still in-flight, too busy to snap, skipping");
return false;
}
}

public boolean isSnapInProgress() {
return isTakingSnapshot.get();
}

}
Loading