You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by lavacat <gi...@git.apache.org> on 2018/04/08 09:05:10 UTC

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

GitHub user lavacat opened a pull request:

    https://github.com/apache/zookeeper/pull/500

    ZOOKEEPER-2959: ignore accepted epoch and LEADERINFO ack from observers

    https://issues.apache.org/jira/browse/ZOOKEEPER-2959
    - add getVotingView check for id in getEpochToPropose and waitForEpochAck
    - refactor waitForNewLeaderAck to use getVotingView
    - unit tests
    
    credit: Xiang Yongqiang (https://github.com/xyq000) for original PR and reporting the issue

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/lavacat/zookeeper branch-3.4

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/500.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #500
    
----
commit 98c40dac60951c61b3b1922d0038461d81b843a1
Author: Bogdan Kanivets <bk...@...>
Date:   2018-04-08T08:46:37Z

    ZOOKEEPER-2959: ignore accepted epoch and LEADERINFO ack from observers
    
    https://issues.apache.org/jira/browse/ZOOKEEPER-2959
    - add getVotingView check for id in getEpochToPropose and waitForEpochAck
    - refactor waitForNewLeaderAck to use getVotingView
    - unit tests
    
    credit: Xiang Yongqiang (https://github.com/xyq000) for original PR and reporting the issue

----


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/500#discussion_r181267836
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -868,8 +868,8 @@ synchronized public long startForwarding(LearnerHandler handler,
                     
             return lastProposed;
         }
    -
    -    private HashSet<Long> connectingFollowers = new HashSet<Long>();
    +    // VisibleForTesting
    +    protected HashSet<Long> connectingFollowers = new HashSet<Long>();
    --- End diff --
    
    `protected Set<Long> connectingFollowers = new HashSet<>();`
    



---

[GitHub] zookeeper issue #500: ZOOKEEPER-2959: ignore accepted epoch and LEADERINFO a...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on the issue:

    https://github.com/apache/zookeeper/pull/500
  
    @lavacat I think either moving these methods/classes to a base class or creating a separate `ZabUtils` makes sense in this case to get cleaner code.


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by lavacat <gi...@git.apache.org>.
Github user lavacat commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/500#discussion_r181001854
  
    --- Diff: src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java ---
    @@ -245,6 +245,180 @@ public void testLastAcceptedEpoch() throws Exception {
                 recursiveDelete(tmpDir);
             }
         }
    +
    +    @Test
    +    public void testGetEpochToProposeWithObserver() throws Exception {
    +        File tmpDir = File.createTempFile("test", "dir", testData);
    +        tmpDir.delete();
    +        tmpDir.mkdir();
    +        Leader leader = null;
    +        try {
    +            QuorumPeer peer = createQuorumPeer(tmpDir);
    +            long participantId = 1;
    +            long observerId = peer.quorumPeers.size();
    +            peer.quorumPeers.put(observerId, new QuorumServer(observerId, "0.0.0.0", 33225,
    --- End diff --
    
    Do you mean using PortAssignment.unique() and "127.0.0.1"? Changed it.


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by lavacat <gi...@git.apache.org>.
Github user lavacat closed the pull request at:

    https://github.com/apache/zookeeper/pull/500


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by lavacat <gi...@git.apache.org>.
Github user lavacat commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/500#discussion_r182922147
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -900,9 +902,10 @@ public long getEpochToPropose(long sid, long lastAcceptedEpoch) throws Interrupt
                 return epoch;
             }
         }
    -
    -    private HashSet<Long> electingFollowers = new HashSet<Long>();
    -    private boolean electionFinished = false;
    +    // VisibleForTesting
    +    protected HashSet<Long> electingFollowers = new HashSet<Long>();
    --- End diff --
    
    updated


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/500#discussion_r181267252
  
    --- Diff: src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java ---
    @@ -161,8 +127,8 @@ public void testLeaderInConnectingFollowers() throws Exception {
             tmpDir.mkdir();
             Leader leader = null;
             try {
    -            QuorumPeer peer = createQuorumPeer(tmpDir);
    -            leader = createLeader(tmpDir, peer);
    +            QuorumPeer peer = ZabUtils.createQuorumPeer(tmpDir);
    --- End diff --
    
    `import static org.apache.zookeeper.server.quorum. ZabUtil.*` then you can simplify method invocation by using `createQuorumPeer(tmpDir);` instead of `ZabUtils.createQuorumPeer(tmpDir);`


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/500#discussion_r181564082
  
    --- 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 --
    
    `private` ?


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/500#discussion_r181268803
  
    --- 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"));
    --- End diff --
    
    Could join lines 52 and 53.


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by lavacat <gi...@git.apache.org>.
Github user lavacat commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/500#discussion_r181002193
  
    --- Diff: src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java ---
    @@ -245,6 +245,180 @@ public void testLastAcceptedEpoch() throws Exception {
                 recursiveDelete(tmpDir);
             }
         }
    +
    +    @Test
    +    public void testGetEpochToProposeWithObserver() throws Exception {
    +        File tmpDir = File.createTempFile("test", "dir", testData);
    --- End diff --
    
    Refactored


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by edwardoliveira <gi...@git.apache.org>.
Github user edwardoliveira commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/500#discussion_r182180406
  
    --- Diff: src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java ---
    @@ -161,8 +127,8 @@ public void testLeaderInConnectingFollowers() throws Exception {
             tmpDir.mkdir();
             Leader leader = null;
             try {
    -            QuorumPeer peer = createQuorumPeer(tmpDir);
    -            leader = createLeader(tmpDir, peer);
    +            QuorumPeer peer = ZabUtils.createQuorumPeer(tmpDir);
    --- End diff --
    
    Yup, you right. Sorry about that. :)


---

[GitHub] zookeeper issue #500: ZOOKEEPER-2959: ignore accepted epoch and LEADERINFO a...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on the issue:

    https://github.com/apache/zookeeper/pull/500
  
    Given that this change affects leader election I think it'd be very beneficial if @fpj could take a look by any chance.


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by lavacat <gi...@git.apache.org>.
Github user lavacat commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/500#discussion_r182889043
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -900,9 +902,10 @@ public long getEpochToPropose(long sid, long lastAcceptedEpoch) throws Interrupt
                 return epoch;
             }
         }
    -
    -    private HashSet<Long> electingFollowers = new HashSet<Long>();
    -    private boolean electionFinished = false;
    +    // VisibleForTesting
    +    protected HashSet<Long> electingFollowers = new HashSet<Long>();
    --- End diff --
    
    Can't use Set, because QuorumVerifier uses HashSet param. 
    QuorumVerifier.containsQuorum(HashSet<Long> set);
    
    I can refactor it all, but then I'll need to touch QuorumVerifier.java, QuorumMaj.java and QuorumHierarchical.java


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/500#discussion_r181564184
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -916,7 +919,9 @@ public void waitForEpochAck(long id, StateSummary ss) throws IOException, Interr
                                                         + leaderStateSummary.getLastZxid()
                                                         + " (last zxid)");
                     }
    -                electingFollowers.add(id);
    +                if (self.getVotingView().containsKey(id)) {
    --- End diff --
    
    I would suggest to encapsulate the `self.getVotingView().containsKey(id)` into a private method as below, if nothing else, for the sake of readability
    
    ```
    private boolean isParticipant(long sid) {
       return self.getVotingView().containsKey(id);
    }
    ```


---

[GitHub] zookeeper issue #500: ZOOKEEPER-2959: ignore accepted epoch and LEADERINFO a...

Posted by shralex <gi...@git.apache.org>.
Github user shralex commented on the issue:

    https://github.com/apache/zookeeper/pull/500
  
    I'm +1. Thanks Bogdan for making the PR.


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/500#discussion_r182128913
  
    --- Diff: src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java ---
    @@ -161,8 +127,8 @@ public void testLeaderInConnectingFollowers() throws Exception {
             tmpDir.mkdir();
             Leader leader = null;
             try {
    -            QuorumPeer peer = createQuorumPeer(tmpDir);
    -            leader = createLeader(tmpDir, peer);
    +            QuorumPeer peer = ZabUtils.createQuorumPeer(tmpDir);
    --- End diff --
    
    Agreed, but please don't use asterisk (*) import. We avoid wildcard imports in Zk project.


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/500#discussion_r181267704
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -900,9 +902,10 @@ public long getEpochToPropose(long sid, long lastAcceptedEpoch) throws Interrupt
                 return epoch;
             }
         }
    -
    -    private HashSet<Long> electingFollowers = new HashSet<Long>();
    -    private boolean electionFinished = false;
    +    // VisibleForTesting
    +    protected HashSet<Long> electingFollowers = new HashSet<Long>();
    --- End diff --
    
    `protected Set<Long> electingFollowers = new HashSet<>()`


---

[GitHub] zookeeper issue #500: ZOOKEEPER-2959: ignore accepted epoch and LEADERINFO a...

Posted by lavacat <gi...@git.apache.org>.
Github user lavacat commented on the issue:

    https://github.com/apache/zookeeper/pull/500
  
    Moved these 3 new tests into new class - LeaderWithObserverTest. Had to make createLeader and createQuorumPeer 'public static' in Zab1_0Test. Happy to refactor into common base class


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/500#discussion_r180789703
  
    --- Diff: src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java ---
    @@ -245,6 +245,180 @@ public void testLastAcceptedEpoch() throws Exception {
                 recursiveDelete(tmpDir);
             }
         }
    +
    +    @Test
    +    public void testGetEpochToProposeWithObserver() throws Exception {
    +        File tmpDir = File.createTempFile("test", "dir", testData);
    --- End diff --
    
    Have you considered using ClientBase.createEmptyTestDir() instead?


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by lavacat <gi...@git.apache.org>.
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


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/500#discussion_r180793047
  
    --- Diff: src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java ---
    @@ -245,6 +245,180 @@ public void testLastAcceptedEpoch() throws Exception {
                 recursiveDelete(tmpDir);
             }
         }
    +
    +    @Test
    +    public void testGetEpochToProposeWithObserver() throws Exception {
    +        File tmpDir = File.createTempFile("test", "dir", testData);
    +        tmpDir.delete();
    +        tmpDir.mkdir();
    +        Leader leader = null;
    +        try {
    +            QuorumPeer peer = createQuorumPeer(tmpDir);
    +            long participantId = 1;
    +            long observerId = peer.quorumPeers.size();
    +            peer.quorumPeers.put(observerId, new QuorumServer(observerId, "0.0.0.0", 33225,
    --- End diff --
    
    I think to be consistent with `createQuorumPeer()` method this should be something like:
    ```
    peers.put(observerId, new QuorumServer(observerId, new InetSocketAddress("127.0.0.1", PortAssignment.unique()), 
                   new InetSocketAddress("127.0.0.1", PortAssignment.unique()),
                   new InetSocketAddress("127.0.0.1", PortAssignment.unique()),
                   QuorumPeer.LearnerType.OBSERVER));
    ```



---

[GitHub] zookeeper issue #500: ZOOKEEPER-2959: ignore accepted epoch and LEADERINFO a...

Posted by lavacat <gi...@git.apache.org>.
Github user lavacat commented on the issue:

    https://github.com/apache/zookeeper/pull/500
  
    @anmolnar added ZabUtils


---

[GitHub] zookeeper pull request #500: ZOOKEEPER-2959: ignore accepted epoch and LEADE...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/500#discussion_r181266640
  
    --- 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;
    +
    --- End diff --
    
    If this is a helper class that doesn't require instantiation then create a private constructor: this makes it "final" and prevents instantiation.


---