Skip to content

Commit

Permalink
review: drop quorumSync config to gain consistent behavior
Browse files Browse the repository at this point in the history
  • Loading branch information
kezhuw committed Sep 30, 2023
1 parent 9364951 commit 9f790f4
Show file tree
Hide file tree
Showing 3 changed files with 0 additions and 26 deletions.
6 changes: 0 additions & 6 deletions zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
Original file line number Diff line number Diff line change
Expand Up @@ -1209,12 +1209,6 @@ property, when available, is noted below.
leader.

The default value is false.
* *quorumSync*
(Java system property: **zookeeper.quorumSync**)
**New in 3.10.0**
When this property is set, `sync` will be a quorum operation.

The default value is false.

* *serializeLastProcessedZxid.enabled*
(Jave system property: **zookeeper.serializeLastProcessedZxid.enabled**)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,14 +123,6 @@ public static int getAckLoggingFrequency() {
return ackLoggingFrequency;
}

public static final String QUORUM_SYNC = "zookeeper.quorumSync";
private static final boolean quorumSync;

static {
quorumSync = Boolean.parseBoolean(System.getProperty(QUORUM_SYNC));
LOG.info("{} = {}", QUORUM_SYNC, quorumSync);
}

final LeaderZooKeeperServer zk;

final QuorumPeer self;
Expand Down Expand Up @@ -1383,8 +1375,6 @@ public synchronized void processSync(Request r) {
if (p != null) {
p.pendingSyncs.add(r);
pendingSyncs.incrementAndGet();
} else if (!quorumSync) {
sendSync(r);
} else {
p = createQuorumSyncProposal(r);
outstandingProposals.put(lastProposed, p);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,17 +27,9 @@
import org.apache.zookeeper.server.quorum.Leader;
import org.apache.zookeeper.server.quorum.LearnerHandler;
import org.apache.zookeeper.server.quorum.QuorumPeer;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;

public class QuorumSyncTest extends QuorumBase {
@BeforeEach
@Override
public void setUp() throws Exception {
System.setProperty("zookeeper.quorumSync", "true");
super.setUp();
}

@Test
public void testReadAfterSync() throws Exception {
int leaderPort = getLeaderClientPort();
Expand All @@ -59,7 +51,6 @@ public void testReadAfterSync() throws Exception {
followerWriter.setData("/test", "test1".getBytes(), -1);
break;
} catch (KeeperException.ConnectionLossException ignored) {
ignored.printStackTrace();
}
}

Expand All @@ -73,7 +64,6 @@ public void testReadAfterSync() throws Exception {
assertArrayEquals("test1".getBytes(), test1);
break;
} catch (Exception ignored) {
ignored.printStackTrace();
}
}
}
Expand Down

0 comments on commit 9f790f4

Please sign in to comment.