[
https://issues.apache.org/jira/browse/ZOOKEEPER-2959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16444818#comment-16444818
]
ASF GitHub Bot commented on ZOOKEEPER-2959:
-------------------------------------------
Github user lavacat commented on a diff in the pull request:
https://github.com/apache/zookeeper/pull/500#discussion_r182887771
--- Diff: src/java/test/org/apache/zookeeper/server/quorum/ZabUtils.java ---
@@ -0,0 +1,140 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.quorum;
+
+import org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.server.ServerCnxn;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.ZKDatabase;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.quorum.flexible.QuorumMaj;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+
+public class ZabUtils {
+ public static final int SYNC_LIMIT = 2;
+
+ public static QuorumPeer createQuorumPeer(File tmpDir) throws
IOException{
+ QuorumPeer peer = new QuorumPeer();
+ peer.syncLimit = 2;
+ peer.initLimit = 2;
+ peer.tickTime = 2000;
+ peer.quorumPeers = new HashMap<Long, QuorumPeer.QuorumServer>();
+ peer.quorumPeers.put(0L, new QuorumPeer.QuorumServer(0,
"127.0.0.1", PortAssignment.unique(), 0, null));
+ peer.quorumPeers.put(1L, new QuorumPeer.QuorumServer(1,
"127.0.0.1", PortAssignment.unique(), 0, null));
+ peer.quorumPeers.put(2L, new QuorumPeer.QuorumServer(2,
"127.0.0.1", PortAssignment.unique(), 0, null));
+ peer.setQuorumVerifier(new QuorumMaj(peer.quorumPeers.size()));
+ peer.setCnxnFactory(new NullServerCnxnFactory());
+ File version2 = new File(tmpDir, "version-2");
+ version2.mkdir();
+ FileOutputStream fos;
+ fos = new FileOutputStream(new File(version2, "currentEpoch"));
+ fos.write("0\n".getBytes());
+ fos.close();
+ fos = new FileOutputStream(new File(version2, "acceptedEpoch"));
+ fos.write("0\n".getBytes());
+ fos.close();
+ return peer;
+ }
+
+ public static Leader createLeader(File tmpDir, QuorumPeer peer)
+ throws IOException, NoSuchFieldException,
IllegalAccessException{
+ LeaderZooKeeperServer zk = prepareLeader(tmpDir, peer);
+ return new Leader(peer, zk);
+ }
+
+ public static MockLeader createMockLeader(File tmpDir, QuorumPeer peer)
+ throws IOException, NoSuchFieldException,
IllegalAccessException{
+ LeaderZooKeeperServer zk = prepareLeader(tmpDir, peer);
+ return new MockLeader(peer, zk);
+ }
+
+ private static LeaderZooKeeperServer prepareLeader(File tmpDir,
QuorumPeer peer)
+ throws IOException, NoSuchFieldException,
IllegalAccessException {
+ FileTxnSnapLog logFactory = new FileTxnSnapLog(tmpDir, tmpDir);
+ peer.setTxnFactory(logFactory);
+ Field addrField = peer.getClass().getDeclaredField("myQuorumAddr");
+ addrField.setAccessible(true);
+ addrField.set(peer, new
InetSocketAddress(PortAssignment.unique()));
+ ZKDatabase zkDb = new ZKDatabase(logFactory);
+ return new LeaderZooKeeperServer(logFactory, peer, new
ZooKeeperServer.BasicDataTreeBuilder(), zkDb);
+ }
+
+ private static final class NullServerCnxnFactory extends
ServerCnxnFactory {
+ public void startup(ZooKeeperServer zkServer) throws IOException,
+ InterruptedException {
+ }
+ public void start() {
+ }
+ public void shutdown() {
+ }
+ public void setMaxClientCnxnsPerHost(int max) {
+ }
+ public void join() throws InterruptedException {
+ }
+ public int getMaxClientCnxnsPerHost() {
+ return 0;
+ }
+ public int getLocalPort() {
+ return 0;
+ }
+ public InetSocketAddress getLocalAddress() {
+ return null;
+ }
+ public Iterable<ServerCnxn> getConnections() {
+ return null;
+ }
+ public void configure(InetSocketAddress addr, int maxClientCnxns)
+ throws IOException {
+ }
+ public void closeSession(long sessionId) {
+ }
+ public void closeAll() {
+ }
+ @Override
+ public int getNumAliveConnections() {
+ return 0;
+ }
+ }
+
+ static final class MockLeader extends Leader {
--- End diff --
Can't make it private, this class is still used in Zab1_0Test
> ignore accepted epoch and LEADERINFO ack from observers when a newly elected
> leader computes new epoch
> ------------------------------------------------------------------------------------------------------
>
> Key: ZOOKEEPER-2959
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2959
> Project: ZooKeeper
> Issue Type: Bug
> Affects Versions: 3.4.10, 3.5.3
> Reporter: xiangyq000
> Assignee: Bogdan Kanivets
> Priority: Blocker
>
> Once the ZooKeeper cluster finishes the election for new leader, all learners
> report their accepted epoch to the leader for the computation of new cluster
> epoch.
> org.apache.zookeeper.server.quorum.Leader#getEpochToPropose
> {code:java}
> private final HashSet<Long> connectingFollowers = new HashSet<Long>();
> public long getEpochToPropose(long sid, long lastAcceptedEpoch) throws
> InterruptedException, IOException {
> synchronized(connectingFollowers) {
> if (!waitingForNewEpoch) {
> return epoch;
> }
> if (lastAcceptedEpoch >= epoch) {
> epoch = lastAcceptedEpoch+1;
> }
> connectingFollowers.add(sid);
> QuorumVerifier verifier = self.getQuorumVerifier();
> if (connectingFollowers.contains(self.getId()) &&
>
> verifier.containsQuorum(connectingFollowers)) {
> waitingForNewEpoch = false;
> self.setAcceptedEpoch(epoch);
> connectingFollowers.notifyAll();
> } else {
> long start = Time.currentElapsedTime();
> long cur = start;
> long end = start + self.getInitLimit()*self.getTickTime();
> while(waitingForNewEpoch && cur < end) {
> connectingFollowers.wait(end - cur);
> cur = Time.currentElapsedTime();
> }
> if (waitingForNewEpoch) {
> throw new InterruptedException("Timeout while waiting for
> epoch from quorum");
> }
> }
> return epoch;
> }
> }
> {code}
> The computation will get an outcome once :
> # The leader has call method "getEpochToPropose"
> # The number of all reporters is greater than half of participants.
> The problem is, an observer server will also send its accepted epoch to the
> leader, while this procedure treat observers as participants.
> Supposed that the cluster consists of 1 leader, 2 followers and 1 observer,
> and now the leader and the observer have reported their accepted epochs while
> neither of the followers has. Thus, the connectingFollowers set consists of
> two elements, resulting in a size of 2, which is greater than half quorum,
> namely, 2. Then QuorumVerifier#containsQuorum will return true, because it
> does not check whether the elements of the parameter are participants.
> The same flaw exists in
> org.apache.zookeeper.server.quorum.Leader#waitForEpochAck
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)