File: FLECompatibilityTest.java

package info (click to toggle)
zookeeper 3.4.9-3%2Bdeb9u2
  • links: PTS, VCS
  • area: main
  • in suites: stretch
  • size: 18,388 kB
  • sloc: java: 57,077; xml: 17,979; cpp: 11,970; ansic: 9,383; sh: 2,662; python: 2,236; makefile: 257; perl: 114
file content (350 lines) | stat: -rw-r--r-- 15,365 bytes parent folder | download | duplicates (4)
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
/**
 * 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 java.io.File;
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;

import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.ZKTestCase;
import org.apache.zookeeper.server.quorum.FastLeaderElection.Notification;
import org.apache.zookeeper.server.quorum.FastLeaderElection.ToSend;
import org.apache.zookeeper.server.quorum.FastLeaderElection.Messenger.WorkerReceiver;
import org.apache.zookeeper.server.quorum.QuorumCnxManager.Message;
import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState;
import org.apache.zookeeper.server.util.ZxidUtils;
import org.apache.zookeeper.test.ClientBase;
import org.apache.zookeeper.test.FLETest;
import org.apache.zookeeper.test.QuorumBase;

import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;


public class FLECompatibilityTest extends ZKTestCase {
    protected static final Logger LOG = LoggerFactory.getLogger(FLECompatibilityTest.class);

    int count;
    HashMap<Long,QuorumServer> peers;
    File tmpdir[];
    int port[];
    
    @Before
    public void setUp() throws Exception {
        count = 3;
        peers = new HashMap<Long,QuorumServer>(count);
        tmpdir = new File[count];
        port = new int[count];
    }
    
    @After
    public void tearDown() throws Exception {
        
    }
    
    class MockFLEMessengerBackward {   
        QuorumCnxManager manager;
        QuorumPeer self;
        long logicalclock = 1L;
        LinkedBlockingQueue<ToSend> sendqueue = new LinkedBlockingQueue<ToSend>();
        LinkedBlockingQueue<ToSend> internalqueue = new LinkedBlockingQueue<ToSend>();
        LinkedBlockingQueue<Notification> recvqueue = new LinkedBlockingQueue<Notification>();
        WorkerReceiver wr;
        
        MockFLEMessengerBackward(QuorumPeer self, QuorumCnxManager manager){
            this.manager = manager;
            this.self = self;
            
            this.wr = new WorkerReceiver(manager);

            Thread t = new Thread(this.wr,
                    "WorkerReceiver[myid=" + self.getId() + "]");
            t.setDaemon(true);
            t.start();
        }
        
        void halt() {
            wr.stop = true;
        }
        
        /*
         * This class has been copied from before adding versions to notifications.
         * 
         * {@see https://issues.apache.org/jira/browse/ZOOKEEPER-1808}
         */
        class WorkerReceiver implements Runnable {
            volatile boolean stop;
            QuorumCnxManager manager;
            final long proposedLeader = 2;
            final long proposedZxid = 0x1;
            final long proposedEpoch = 1;

            WorkerReceiver(QuorumCnxManager manager) {
                this.stop = false;
                this.manager = manager;
            }

            /*
             * The vote we return here is fixed for test purposes.
             */
            Vote getVote(){
                return new Vote(proposedLeader, proposedZxid, proposedEpoch);
            }
            
            public void run() {

                Message response;
                while (!stop) {
                    // Sleeps on receive
                    try{
                        response = manager.pollRecvQueue(3000, TimeUnit.MILLISECONDS);
                        if(response == null) continue;

                        /*
                         * If it is from an observer, respond right away.
                         * Note that the following predicate assumes that
                         * if a server is not a follower, then it must be
                         * an observer. If we ever have any other type of
                         * learner in the future, we'll have to change the
                         * way we check for observers.
                         */
                        if(!self.getVotingView().containsKey(response.sid)){
                            Vote current = self.getCurrentVote();
                            ToSend notmsg = new ToSend(ToSend.mType.notification,
                                    current.getId(),
                                    current.getZxid(),
                                    logicalclock,
                                    self.getPeerState(),
                                    response.sid,
                                    current.getPeerEpoch());

                            internalqueue.offer(notmsg);
                        } else {
                            // Receive new message
                            if (LOG.isDebugEnabled()) {
                                LOG.debug("Receive new notification message. My id = "
                                        + self.getId());
                            }

                            /*
                             * We check for 28 bytes for backward compatibility
                             */
                            if (response.buffer.capacity() < 28) {
                                LOG.error("Got a short response: "
                                        + response.buffer.capacity());
                                continue;
                            }
                            boolean backCompatibility = (response.buffer.capacity() == 28);
                            response.buffer.clear();

                            // State of peer that sent this message
                            QuorumPeer.ServerState ackstate = QuorumPeer.ServerState.LOOKING;
                            switch (response.buffer.getInt()) {
                            case 0:
                                ackstate = QuorumPeer.ServerState.LOOKING;
                                break;
                            case 1:
                                ackstate = QuorumPeer.ServerState.FOLLOWING;
                                break;
                            case 2:
                                ackstate = QuorumPeer.ServerState.LEADING;
                                break;
                            case 3:
                                ackstate = QuorumPeer.ServerState.OBSERVING;
                                break;
                            }

                            // Instantiate Notification and set its attributes
                            Notification n = new Notification();
                            n.leader = response.buffer.getLong();
                            n.zxid = response.buffer.getLong();
                            n.electionEpoch = response.buffer.getLong();
                            n.state = ackstate;
                            n.sid = response.sid;
                            if(!backCompatibility){
                                n.peerEpoch = response.buffer.getLong();
                            } else {
                                if(LOG.isInfoEnabled()){
                                    LOG.info("Backward compatibility mode, server id=" + n.sid);
                                }
                                n.peerEpoch = ZxidUtils.getEpochFromZxid(n.zxid);
                            }

                            /*
                             * If this server is looking, then send proposed leader
                             */

                            if(self.getPeerState() == QuorumPeer.ServerState.LOOKING){
                                recvqueue.offer(n);

                                /*
                                 * Send a notification back if the peer that sent this
                                 * message is also looking and its logical clock is
                                 * lagging behind.
                                 */
                                if((ackstate == QuorumPeer.ServerState.LOOKING)
                                        && (n.electionEpoch < logicalclock)){
                                    Vote v = getVote();
                                    ToSend notmsg = new ToSend(ToSend.mType.notification,
                                            v.getId(),
                                            v.getZxid(),
                                            logicalclock,
                                            self.getPeerState(),
                                            response.sid,
                                            v.getPeerEpoch());
                                    internalqueue.offer(notmsg);
                                }
                            } else {
                                /*
                                 * If this server is not looking, but the one that sent the ack
                                 * is looking, then send back what it believes to be the leader.
                                 */
                                Vote current = self.getCurrentVote();
                                if(ackstate == QuorumPeer.ServerState.LOOKING){
                                    if(LOG.isDebugEnabled()){
                                        LOG.debug("Sending new notification. My id =  " +
                                                self.getId() + " recipient=" +
                                                response.sid + " zxid=0x" +
                                                Long.toHexString(current.getZxid()) +
                                                " leader=" + current.getId());
                                    }
                                    ToSend notmsg = new ToSend(
                                            ToSend.mType.notification,
                                            current.getId(),
                                            current.getZxid(),
                                            current.getElectionEpoch(),
                                            self.getPeerState(),
                                            response.sid,
                                            current.getPeerEpoch());
                                    internalqueue.offer(notmsg);
                                }
                            }
                        }
                    } catch (InterruptedException e) {
                        System.out.println("Interrupted Exception while waiting for new message" +
                                e.toString());
                    }
                }
                LOG.info("WorkerReceiver is down");
            }
        }
    }
    
    class MockFLEMessengerForward extends FastLeaderElection {
        
        MockFLEMessengerForward(QuorumPeer self, QuorumCnxManager manager){
            super( self, manager );
        }
        
        void halt() {
            super.shutdown();
        }
    }
    
    void populate()
    throws Exception {
        for (int i = 0; i < count; i++) {
            peers.put(Long.valueOf(i),
                      new QuorumServer(i, "0.0.0.0",
                                       PortAssignment.unique(),
                                       PortAssignment.unique(), null));
            tmpdir[i] = ClientBase.createTmpDir();
            port[i] = PortAssignment.unique();
        }
    }
    
    @Test(timeout=20000)
    public void testBackwardCompatibility() 
    throws Exception {
        populate();
        
        QuorumPeer peer = new QuorumPeer(peers, tmpdir[0], tmpdir[0], port[0], 3, 0, 1000, 2, 2);
        peer.setPeerState(ServerState.LOOKING);
        QuorumCnxManager mng = peer.createCnxnManager();
        
        /*
         * Check that it generates an internal notification correctly
         */
        MockFLEMessengerBackward fle = new MockFLEMessengerBackward(peer, mng);
        ByteBuffer buffer = FastLeaderElection.buildMsg(ServerState.LOOKING.ordinal(), 2, 0x1, 1, 1);
        fle.manager.recvQueue.add(new Message(buffer, 2));
        Notification n = fle.recvqueue.take();
        Assert.assertTrue("Wrong state", n.state == ServerState.LOOKING);
        Assert.assertTrue("Wrong leader", n.leader == 2);
        Assert.assertTrue("Wrong zxid", n.zxid == 0x1);
        Assert.assertTrue("Wrong epoch", n.electionEpoch == 1);
        Assert.assertTrue("Wrong epoch", n.peerEpoch == 1);
        
        /*
         * Check that it sends a notification back to the sender
         */
        peer.setPeerState(ServerState.FOLLOWING);
        peer.setCurrentVote( new Vote(2, 0x1, 1, 1, ServerState.LOOKING) );
        buffer = FastLeaderElection.buildMsg(ServerState.LOOKING.ordinal(), 1, 0x1, 1, 1);
        fle.manager.recvQueue.add(new Message(buffer, 1));
        ToSend m = fle.internalqueue.take();
        Assert.assertTrue("Wrong state", m.state == ServerState.FOLLOWING);
        Assert.assertTrue("Wrong sid", m.sid == 1);
        Assert.assertTrue("Wrong leader", m.leader == 2);
        Assert.assertTrue("Wrong epoch", m.electionEpoch == 1);
        Assert.assertTrue("Wrong epoch", m.peerEpoch == 1);
    }
    
    @Test(timeout=20000)
    public void testForwardCompatibility() 
    throws Exception {
        populate();

        QuorumPeer peer = new QuorumPeer(peers, tmpdir[0], tmpdir[0], port[0], 3, 0, 1000, 2, 2);
        peer.setPeerState(ServerState.LOOKING);
        QuorumCnxManager mng = peer.createCnxnManager();
        
        /*
         * Check that it generates an internal notification correctly
         */
        MockFLEMessengerForward fle = new MockFLEMessengerForward(peer, mng);
        ByteBuffer notBuffer = FastLeaderElection.buildMsg(ServerState.LOOKING.ordinal(), 2, 0x1, 1, 1);
        ByteBuffer buffer = ByteBuffer.allocate( notBuffer.capacity() + 8 );
        notBuffer.flip();
        buffer.put(notBuffer);
        buffer.putLong( Long.MAX_VALUE );
        buffer.flip();
        
        fle.manager.recvQueue.add(new Message(buffer, 2));
        Notification n = fle.recvqueue.take();
        Assert.assertTrue("Wrong state", n.state == ServerState.LOOKING);
        Assert.assertTrue("Wrong leader", n.leader == 2);
        Assert.assertTrue("Wrong zxid", n.zxid == 0x1);
        Assert.assertTrue("Wrong epoch", n.electionEpoch == 1);
        Assert.assertTrue("Wrong epoch", n.peerEpoch == 1);
        Assert.assertTrue("Wrong version", n.version == FastLeaderElection.Notification.CURRENTVERSION);
    }
}