summaryrefslogtreecommitdiff
path: root/zookeeper-server/src/test/java/org/apache/zookeeper/test/FLETest.java
blob: 15713e423cc0ac6edda9e07ec801cd74b8d78517 (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
/*
 * 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.test;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import java.io.File;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.ZKTestCase;
import org.apache.zookeeper.server.quorum.FastLeaderElection;
import org.apache.zookeeper.server.quorum.QuorumPeer;
import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState;
import org.apache.zookeeper.server.quorum.Vote;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class FLETest extends ZKTestCase {

    protected static final Logger LOG = LoggerFactory.getLogger(FLETest.class);
    private final int MAX_LOOP_COUNTER = 300;
    private FLETest.LEThread leThread;

    static class TestVote {

        TestVote(int id, long leader) {
            this.leader = leader;
        }

        long leader;

    }

    int countVotes(HashSet<TestVote> hs, long id) {
        int counter = 0;
        for (TestVote v : hs) {
            if (v.leader == id) {
                counter++;
            }
        }

        return counter;
    }

    int count;
    Map<Long, QuorumServer> peers;
    ArrayList<LEThread> threads;
    Map<Integer, HashSet<TestVote>> voteMap;
    Map<Long, LEThread> quora;
    File[] tmpdir;
    int[] port;
    int successCount;

    volatile Vote[] votes;
    volatile long leader = -1;
    //volatile int round = 1;
    Random rand = new Random();
    Set<Long> joinedThreads;

    @BeforeEach
    public void setUp() throws Exception {
        count = 7;

        peers = new HashMap<Long, QuorumServer>(count);
        threads = new ArrayList<LEThread>(count);
        voteMap = new HashMap<Integer, HashSet<TestVote>>();
        votes = new Vote[count];
        tmpdir = new File[count];
        port = new int[count];
        successCount = 0;
        joinedThreads = new HashSet<Long>();
    }

    @AfterEach
    public void tearDown() throws Exception {
        for (int i = 0; i < threads.size(); i++) {
            leThread = threads.get(i);
            QuorumBase.shutdown(leThread.peer);
        }
    }

    /**
     * Implements the behavior of a peer during the leader election rounds
     * of tests.
     */
    class LEThread extends Thread {

        FLETest self;
        int i;
        QuorumPeer peer;
        int totalRounds;
        ConcurrentHashMap<Long, HashSet<Integer>> quora;

        LEThread(FLETest self, QuorumPeer peer, int i, int rounds, ConcurrentHashMap<Long, HashSet<Integer>> quora) {
            this.self = self;
            this.i = i;
            this.peer = peer;
            this.totalRounds = rounds;
            this.quora = quora;

            LOG.info("Constructor: {}", getName());
        }

        public void run() {
            try {
                Vote v = null;
                while (true) {

                    /*
                     * Set the state of the peer to LOOKING and look for leader
                     */
                    peer.setPeerState(ServerState.LOOKING);
                    LOG.info("Going to call leader election again.");
                    v = peer.getElectionAlg().lookForLeader();
                    if (v == null) {
                        LOG.info("Thread {} got a null vote", i);
                        break;
                    }

                    /*
                     * Done with the election round, so now we set the vote in
                     * the peer. A real zookeeper would take care of setting the
                     * current vote. Here we do it manually.
                     */
                    peer.setCurrentVote(v);

                    LOG.info("Finished election: {}, {}", i, v.getId());
                    votes[i] = v;

                    /*
                     * Get the current value of the logical clock for this peer
                     * so that we know in which round this peer has executed.
                     */
                    int lc = (int) ((FastLeaderElection) peer.getElectionAlg()).getLogicalClock();

                    /*
                     * The leader executes the following block, which essentially shuts down
                     * the peer if it is not the last round.
                     */
                    if (v.getId() == i) {
                        LOG.info("I'm the leader: {}", i);
                        if (lc < this.totalRounds) {
                            LOG.info("Leader {} dying", i);
                            FastLeaderElection election = (FastLeaderElection) peer.getElectionAlg();
                            election.shutdown();
                            // Make sure the vote is reset to -1 after shutdown.
                            assertEquals(-1, election.getVote().getId());
                            LOG.info("Leader {} dead", i);

                            break;
                        }
                    }

                    /*
                     * If the peer has done enough rounds, then consider joining. The thread
                     * will only join if it is part of a quorum supporting the current
                     * leader. Otherwise it will try again.
                     */
                    if (lc >= this.totalRounds) {
                        /*
                         * quora keeps the supporters of a given leader, so
                         * we first update it with the vote of this peer.
                         */
                        if (quora.get(v.getId()) == null) {
                            quora.put(v.getId(), new HashSet<Integer>());
                        }
                        quora.get(v.getId()).add(i);

                        /*
                         * we now wait until a quorum supports the same leader.
                         */
                        if (waitForQuorum(v.getId())) {
                            synchronized (self) {

                                /*
                                 * Assert that the state of the thread is the one expected.
                                 */
                                if (v.getId() == i) {
                                    assertTrue(peer.getPeerState() == ServerState.LEADING, "Wrong state" + peer.getPeerState());
                                    leader = i;
                                } else {
                                    assertTrue(peer.getPeerState() == ServerState.FOLLOWING, "Wrong state" + peer.getPeerState());
                                }

                                /*
                                 * Global variable keeping track of
                                 * how many peers have successfully
                                 * joined.
                                 */
                                successCount++;
                                joinedThreads.add((long) i);
                                self.notify();
                            }

                            /*
                             * I'm done so joining.
                             */
                            break;
                        } else {
                            quora.get(v.getId()).remove(i);
                        }
                    }

                    /*
                     * This sleep time represents the time a follower
                     * would take to declare the leader dead and start
                     * a new leader election.
                     */
                    Thread.sleep(100);

                }
                LOG.debug("Thread {} votes {}", i, v);
            } catch (InterruptedException e) {
                fail(e.toString());
            }
        }

        /**
         * Auxiliary method to make sure that enough followers terminated.
         *
         * @return boolean  followers successfully joined.
         */
        boolean waitForQuorum(long id) throws InterruptedException {
            int loopCounter = 0;
            while ((quora.get(id).size() <= count / 2) && (loopCounter < MAX_LOOP_COUNTER)) {
                Thread.sleep(100);
                loopCounter++;
            }

            return (loopCounter < MAX_LOOP_COUNTER) || (quora.get(id).size() > count / 2);
        }

    }

    @Test
    public void testSingleElection() throws Exception {
        try {
            runElection(1);
        } catch (Exception e) {
            fail(e.toString());
        }
    }

    @Test
    public void testDoubleElection() throws Exception {
        try {
            runElection(2);
        } catch (Exception e) {
            fail(e.toString());
        }
    }

    @Test
    public void testTripleElection() throws Exception {
        try {
            runElection(3);
        } catch (Exception e) {
            fail(e.toString());
        }
    }

    /**
     * Test leader election for a number of rounds. In all rounds but the last one
     * we kill the leader.
     *
     * @param rounds
     * @throws Exception
     */
    private void runElection(int rounds) throws Exception {
        ConcurrentHashMap<Long, HashSet<Integer>> quora = new ConcurrentHashMap<Long, HashSet<Integer>>();

        LOG.info("TestLE: {}, {}", getTestName(), count);

        /*
         * Creates list of peers.
         */
        for (int i = 0; i < count; i++) {
            port[i] = PortAssignment.unique();
            peers.put(Long.valueOf(i), new QuorumServer(i, new InetSocketAddress("127.0.0.1", PortAssignment.unique()), new InetSocketAddress("127.0.0.1", PortAssignment.unique()), new InetSocketAddress("127.0.0.1", port[i])));
            tmpdir[i] = ClientBase.createTmpDir();
        }

        /*
         * Start one LEThread for each peer we want to run.
         */
        for (int i = 0; i < count; i++) {
            QuorumPeer peer = new QuorumPeer(peers, tmpdir[i], tmpdir[i], port[i], 3, i, 1000, 2, 2, 2);
            peer.startLeaderElection();
            LEThread thread = new LEThread(this, peer, i, rounds, quora);
            thread.start();
            threads.add(thread);
        }
        LOG.info("Started threads {}", getTestName());

        int waitCounter = 0;
        synchronized (this) {
            while (((successCount <= count / 2) || (leader == -1)) && (waitCounter < MAX_LOOP_COUNTER)) {
                this.wait(200);
                waitCounter++;
            }
        }
        LOG.info("Success count: {}", successCount);

        /*
         * Lists what threads haven't joined. A thread doesn't join if
         * it hasn't decided upon a leader yet. It can happen that a
         * peer is slow or disconnected, and it can take longer to
         * nominate and connect to the current leader.
         */
        for (int i = 0; i < threads.size(); i++) {
            if (threads.get(i).isAlive()) {
                LOG.info("Threads didn't join: {}", i);
            }
        }

        /*
         * If we have a majority, then we are good to go.
         */
        if (successCount <= count / 2) {
            fail("Fewer than a a majority has joined");
        }

        /*
         * I'm done so joining.
         */
        if (!joinedThreads.contains(leader)) {
            fail("Leader hasn't joined: " + leader);
        }
    }

    /*
     * Class to verify of the thread has become a follower
     */
    static class VerifyState extends Thread {

        private volatile boolean success = false;
        private QuorumPeer peer;
        public VerifyState(QuorumPeer peer) {
            this.peer = peer;
        }
        public void run() {
            setName("VerifyState-" + peer.getId());
            while (true) {
                if (peer.getPeerState() == ServerState.FOLLOWING) {
                    LOG.info("I am following");
                    success = true;
                    break;
                } else if (peer.getPeerState() == ServerState.LEADING) {
                    LOG.info("I am leading");
                    success = false;
                    break;
                }
                try {
                    Thread.sleep(250);
                } catch (Exception e) {
                    LOG.warn("Sleep failed ", e);
                }
            }
        }
        public boolean isSuccess() {
            return success;
        }

    }

    /*
     * For ZOOKEEPER-975 verify that a peer joining an established cluster
     * does not go in LEADING state.
     */
    @Test
    public void testJoin() throws Exception {
        int sid;
        QuorumPeer peer;
        int waitTime = 10 * 1000;
        ArrayList<QuorumPeer> peerList = new ArrayList<QuorumPeer>();
        for (sid = 0; sid < 3; sid++) {
            port[sid] = PortAssignment.unique();
            peers.put(Long.valueOf(sid), new QuorumServer(sid, new InetSocketAddress("127.0.0.1", PortAssignment.unique()), new InetSocketAddress("127.0.0.1", PortAssignment.unique()), new InetSocketAddress("127.0.0.1", port[sid])));
            tmpdir[sid] = ClientBase.createTmpDir();
        }
        // start 2 peers and verify if they form the cluster
        for (sid = 0; sid < 2; sid++) {
            peer = new QuorumPeer(peers, tmpdir[sid], tmpdir[sid], port[sid], 3, sid, 2000, 2, 2, 2);
            LOG.info("Starting peer {}", peer.getId());
            peer.start();
            peerList.add(sid, peer);
        }
        peer = peerList.get(0);
        VerifyState v1 = new VerifyState(peerList.get(0));
        v1.start();
        v1.join(waitTime);
        assertFalse(!v1.isSuccess(), "Unable to form cluster in " + waitTime + " ms");
        // Start 3rd peer and check if it goes in LEADING state
        peer = new QuorumPeer(peers, tmpdir[sid], tmpdir[sid], port[sid], 3, sid, 2000, 2, 2, 2);
        LOG.info("Starting peer {}", peer.getId());
        peer.start();
        peerList.add(sid, peer);
        v1 = new VerifyState(peer);
        v1.start();
        v1.join(waitTime);
        if (v1.isAlive()) {
            fail("Peer " + peer.getId() + " failed to join the cluster " + "within " + waitTime + " ms");
        } else if (!v1.isSuccess()) {
            fail("Incorrect LEADING state for peer " + peer.getId());
        }
        // cleanup
        for (int id = 0; id < 3; id++) {
            peer = peerList.get(id);
            if (peer != null) {
                peer.shutdown();
            }
        }
    }

    /*
     * For ZOOKEEPER-1732 verify that it is possible to join an ensemble with
     * inconsistent election round information.
     */
    @Test
    public void testJoinInconsistentEnsemble() throws Exception {
        int sid;
        QuorumPeer peer;
        int waitTime = 10 * 1000;
        ArrayList<QuorumPeer> peerList = new ArrayList<QuorumPeer>();
        for (sid = 0; sid < 3; sid++) {
            peers.put(Long.valueOf(sid), new QuorumServer(sid, new InetSocketAddress("127.0.0.1", PortAssignment.unique()), new InetSocketAddress("127.0.0.1", PortAssignment.unique())));
            tmpdir[sid] = ClientBase.createTmpDir();
            port[sid] = PortAssignment.unique();
        }
        // start 2 peers and verify if they form the cluster
        for (sid = 0; sid < 2; sid++) {
            peer = new QuorumPeer(peers, tmpdir[sid], tmpdir[sid], port[sid], 3, sid, 2000, 2, 2, 2);
            LOG.info("Starting peer {}", peer.getId());
            peer.start();
            peerList.add(sid, peer);
        }
        peer = peerList.get(0);
        VerifyState v1 = new VerifyState(peerList.get(0));
        v1.start();
        v1.join(waitTime);
        assertFalse(!v1.isSuccess(), "Unable to form cluster in " + waitTime + " ms");
        // Change the election round for one of the members of the ensemble
        long leaderSid = peer.getCurrentVote().getId();
        long zxid = peer.getCurrentVote().getZxid();
        long electionEpoch = peer.getCurrentVote().getElectionEpoch();
        ServerState state = peer.getCurrentVote().getState();
        long peerEpoch = peer.getCurrentVote().getPeerEpoch();
        Vote newVote = new Vote(leaderSid, zxid + 100, electionEpoch + 100, peerEpoch, state);
        peer.setCurrentVote(newVote);
        // Start 3rd peer and check if it joins the quorum
        peer = new QuorumPeer(peers, tmpdir[2], tmpdir[2], port[2], 3, 2, 2000, 2, 2, 2);
        LOG.info("Starting peer {}", peer.getId());
        peer.start();
        peerList.add(sid, peer);
        v1 = new VerifyState(peer);
        v1.start();
        v1.join(waitTime);
        if (v1.isAlive()) {
            fail("Peer " + peer.getId() + " failed to join the cluster " + "within " + waitTime + " ms");
        }
        // cleanup
        for (int id = 0; id < 3; id++) {
            peer = peerList.get(id);
            if (peer != null) {
                peer.shutdown();
            }
        }
    }

    @Test
    public void testElectionTimeUnit() throws Exception {
        assertEquals("MS", QuorumPeer.FLE_TIME_UNIT);
    }

}