Skip to content
Closed
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
23 changes: 17 additions & 6 deletions zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
Original file line number Diff line number Diff line change
Expand Up @@ -1151,14 +1151,25 @@ 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**)
**New in 3.6.2:**
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.
(Java system property: **zookeeper.learner.closeSocketAsync**)
(Java system property: **learner.closeSocketAsync**)(Added for backward compatibility)
**New in 3.7.0:**
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.

* *leader.closeSocketAsync*
(Java system property only: **leader.closeSocketAsync**)
**New in 3.6.2:**
When enabled, the leader will close a quorum socket asynchoronously. This is useful for TLS connections where closing a socket might take a long time. If disconnecting a follower is initiated in ping() because of a failed SyncLimitCheck then the long socket closing time will block the sending of pings to other followers. Without receiving pings, the other followers will not send session information to the leader, which causes sessions to expire. Setting this flag to true ensures that pings will be sent regularly. The default is false.
(Java system property: **zookeeper.leader.closeSocketAsync**)
(Java system property: **leader.closeSocketAsync**)(Added for backward compatibility)
**New in 3.7.0:**
When enabled, the leader will close a quorum socket asynchoronously. This is useful for TLS connections where closing a socket might take a long time. If disconnecting a follower is initiated in ping() because of a failed SyncLimitCheck then the long socket closing time will block the sending of pings to other followers. Without receiving pings, the other followers will not send session information to the leader, which causes sessions to expire. Setting this flag to true ensures that pings will be sent regularly.
The default is false.

* *learner.asyncSending*
(Java system property: **zookeeper.learner.asyncSending**)
(Java system property: **learner.asyncSending**)(Added for backward compatibility)
**New in 3.7.0:**
The sending and receiving packets in Learner were done synchronously in a critical section. An untimely network issue could cause the followers to hang (see [ZOOKEEPER-3575](https://issues.apache.org/jira/browse/ZOOKEEPER-3575) and [ZOOKEEPER-4074](https://issues.apache.org/jira/browse/ZOOKEEPER-4074)). The new design moves sending packets in Learner to a separate thread and sends the packets asynchronously. The new design is enabled with this parameter (learner.asyncSending).
The default is false.

* *forward_learner_requests_to_commit_processor_disabled*
(Jave system property: **zookeeper.forward_learner_requests_to_commit_processor_disabled**)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@
import org.apache.zookeeper.server.ZooTrace;
import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
import org.apache.zookeeper.server.util.ConfigUtils;
import org.apache.zookeeper.server.util.MessageTracker;
import org.apache.zookeeper.server.util.SerializeUtils;
import org.apache.zookeeper.server.util.ZxidUtils;
Expand Down Expand Up @@ -118,10 +119,12 @@ public Socket getSocket() {

private static final boolean nodelay = System.getProperty("follower.nodelay", "true").equals("true");

public static final String LEARNER_ASYNC_SENDING = "learner.asyncSending";
private static boolean asyncSending = Boolean.getBoolean(LEARNER_ASYNC_SENDING);
public static final String LEARNER_CLOSE_SOCKET_ASYNC = "learner.closeSocketAsync";
public static final boolean closeSocketAsync = Boolean.getBoolean(LEARNER_CLOSE_SOCKET_ASYNC);
public static final String LEARNER_ASYNC_SENDING = "zookeeper.learner.asyncSending";
private static boolean asyncSending =
Boolean.parseBoolean(ConfigUtils.getPropertyBackwardCompatibleWay(LEARNER_ASYNC_SENDING));
public static final String LEARNER_CLOSE_SOCKET_ASYNC = "zookeeper.learner.closeSocketAsync";
public static final boolean closeSocketAsync = Boolean
.parseBoolean(ConfigUtils.getPropertyBackwardCompatibleWay(LEARNER_CLOSE_SOCKET_ASYNC));

static {
LOG.info("leaderConnectDelayDuringRetryMs: {}", leaderConnectDelayDuringRetryMs);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@
import org.apache.zookeeper.server.quorum.Leader.Proposal;
import org.apache.zookeeper.server.quorum.QuorumPeer.LearnerType;
import org.apache.zookeeper.server.quorum.auth.QuorumAuthServer;
import org.apache.zookeeper.server.util.ConfigUtils;
import org.apache.zookeeper.server.util.MessageTracker;
import org.apache.zookeeper.server.util.ZxidUtils;
import org.slf4j.Logger;
Expand All @@ -65,8 +66,10 @@ public class LearnerHandler extends ZooKeeperThread {

private static final Logger LOG = LoggerFactory.getLogger(LearnerHandler.class);

public static final String LEADER_CLOSE_SOCKET_ASYNC = "leader.closeSocketAsync";
public static final boolean closeSocketAsync = Boolean.parseBoolean(System.getProperty(LEADER_CLOSE_SOCKET_ASYNC, "false"));
public static final String LEADER_CLOSE_SOCKET_ASYNC = "zookeeper.leader.closeSocketAsync";

public static final boolean closeSocketAsync = Boolean
.parseBoolean(ConfigUtils.getPropertyBackwardCompatibleWay(LEADER_CLOSE_SOCKET_ASYNC));

static {
LOG.info("{} = {}", LEADER_CLOSE_SOCKET_ASYNC, closeSocketAsync);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,4 +95,29 @@ public static String[] getHostAndPort(String s) throws ConfigException {
}
}

/**
* Some old configuration properties are not configurable in zookeeper configuration file
* zoo.cfg. To make these properties configurable in zoo.cfg old properties are prepended
* with zookeeper. For example prop.x.y.z changed to zookeeper.prop.x.y.z. But for backward
* compatibility both prop.x.y.z and zookeeper.prop.x.y.z should be supported.
* This method first gets value from new property, if first property is not configured
* then gets value from old property
*
* @param newPropertyKey new property key which starts with zookeeper.
* @return either new or old system property value. Null if none of the properties are set.
*/
public static String getPropertyBackwardCompatibleWay(String newPropertyKey) {
String newKeyValue = System.getProperty(newPropertyKey);
if (newKeyValue != null) {
return newKeyValue.trim();
}
String oldPropertyKey = newPropertyKey.replace("zookeeper.", "");
String oldKeyValue = System.getProperty(oldPropertyKey);

if (oldKeyValue != null) {
return oldKeyValue.trim();
}
return null;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.zookeeper.server.util;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
import org.junit.jupiter.api.Test;
Expand Down Expand Up @@ -74,4 +75,45 @@ public void testGetHostAndPortWithoutPort() throws ConfigException {
assertEquals(nsa.length, 1);
}

@Test
public void testGetPropertyBackwardCompatibleWay() throws ConfigException {
String newProp = "zookeeper.prop.x.y.z";
String oldProp = "prop.x.y.z";

// Null as both properties are not set
String result = ConfigUtils.getPropertyBackwardCompatibleWay(newProp);
assertNull(result);

// Return old property value when only old property is set
String oldPropValue = "oldPropertyValue";
System.setProperty(oldProp, oldPropValue);
result = ConfigUtils.getPropertyBackwardCompatibleWay(newProp);
assertEquals(oldPropValue, result);

// Return new property value when both properties are set
String newPropValue = "newPropertyValue";
System.setProperty(newProp, newPropValue);
result = ConfigUtils.getPropertyBackwardCompatibleWay(newProp);
assertEquals(newPropValue, result);

// cleanUp
clearProp(newProp, oldProp);

// Return trimmed value
System.setProperty(oldProp, oldPropValue + " ");
result = ConfigUtils.getPropertyBackwardCompatibleWay(newProp);
assertEquals(oldPropValue, result);

System.setProperty(newProp, " " + newPropValue);
result = ConfigUtils.getPropertyBackwardCompatibleWay(newProp);
assertEquals(newPropValue, result);

// cleanUp
clearProp(newProp, oldProp);
}

private void clearProp(String newProp, String oldProp) {
System.clearProperty(newProp);
System.clearProperty(oldProp);
}
}