summaryrefslogtreecommitdiff
path: root/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java
blob: dcf7819b12b74e839c4744a1cd5d79fe2c713d97 (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
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
/*
 * 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 static org.apache.zookeeper.server.quorum.ZabUtils.MockLeader;
import static org.apache.zookeeper.server.quorum.ZabUtils.createLeader;
import static org.apache.zookeeper.server.quorum.ZabUtils.createMockLeader;
import static org.apache.zookeeper.server.quorum.ZabUtils.createQuorumPeer;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.verify;
import java.io.BufferedInputStream;
import java.io.BufferedReader;
import java.io.ByteArrayOutputStream;
import java.io.EOFException;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.ServerSocket;
import java.net.Socket;
import java.nio.ByteBuffer;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.jute.BinaryInputArchive;
import org.apache.jute.BinaryOutputArchive;
import org.apache.jute.InputArchive;
import org.apache.jute.OutputArchive;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.Watcher.Event.EventType;
import org.apache.zookeeper.ZKTestCase;
import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.server.ByteBufferInputStream;
import org.apache.zookeeper.server.ByteBufferOutputStream;
import org.apache.zookeeper.server.DataTree;
import org.apache.zookeeper.server.Request;
import org.apache.zookeeper.server.ZKDatabase;
import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
import org.apache.zookeeper.server.util.ZxidUtils;
import org.apache.zookeeper.test.ClientBase;
import org.apache.zookeeper.test.TestUtils;
import org.apache.zookeeper.txn.CreateSessionTxn;
import org.apache.zookeeper.txn.CreateTxn;
import org.apache.zookeeper.txn.ErrorTxn;
import org.apache.zookeeper.txn.SetDataTxn;
import org.apache.zookeeper.txn.TxnHeader;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class Zab1_0Test extends ZKTestCase {

    private static final Logger LOG = LoggerFactory.getLogger(Zab1_0Test.class);

    private static final File testData = new File(System.getProperty("test.data.dir", "src/test/resources/data"));

    @BeforeEach
    public void setUp() {
        System.setProperty("zookeeper.admin.enableServer", "false");
    }

    private static final class LeadThread extends Thread {

        private final Leader leader;

        private LeadThread(Leader leader) {
            this.leader = leader;
        }

        public void run() {
            try {
                leader.lead();
            } catch (InterruptedException e) {
                LOG.info("Leader thread interrupted", e);
            } catch (Exception e) {
                LOG.warn("Unexpected exception in leader thread", e);
            } finally {
                leader.shutdown("lead ended");
            }
        }

    }

    public static final class FollowerMockThread extends Thread {

        private final Leader leader;
        private final long followerSid;
        public long epoch = -1;
        public String msg = null;
        private boolean onlyGetEpochToPropose;

        private FollowerMockThread(long followerSid, Leader leader, boolean onlyGetEpochToPropose) {
            this.leader = leader;
            this.followerSid = followerSid;
            this.onlyGetEpochToPropose = onlyGetEpochToPropose;
        }

        public void run() {
            if (onlyGetEpochToPropose) {
                try {
                    epoch = leader.getEpochToPropose(followerSid, 0);
                } catch (Exception e) {
                }
            } else {
                try {
                    leader.waitForEpochAck(followerSid, new StateSummary(0, 0));
                    msg = "FollowerMockThread (id = " + followerSid + ")  returned from waitForEpochAck";
                } catch (Exception e) {
                }
            }
        }

    }
    @Test
    public void testLeaderInConnectingFollowers() throws Exception {
        File tmpDir = File.createTempFile("test", "dir", testData);
        tmpDir.delete();
        tmpDir.mkdir();
        Leader leader = null;
        try {
            QuorumPeer peer = createQuorumPeer(tmpDir);
            leader = createLeader(tmpDir, peer);
            peer.leader = leader;
            peer.setAcceptedEpoch(5);

            FollowerMockThread f1 = new FollowerMockThread(1, leader, true);
            FollowerMockThread f2 = new FollowerMockThread(2, leader, true);
            f1.start();
            f2.start();

            // wait until followers time out in getEpochToPropose - they shouldn't return
            // normally because the leader didn't execute getEpochToPropose and so its epoch was not
            // accounted for
            f1.join(leader.self.getInitLimit() * leader.self.getTickTime() + 5000);
            f2.join(leader.self.getInitLimit() * leader.self.getTickTime() + 5000);

            // even though followers timed out, their ids are in connectingFollowers, and their
            // epoch were accounted for, so the leader should not block and since it started with
            // accepted epoch = 5 it should now have 6
            try {
                long epoch = leader.getEpochToPropose(leader.self.getMyId(), leader.self.getAcceptedEpoch());
                assertEquals(6, epoch, "leader got wrong epoch from getEpochToPropose");
            } catch (Exception e) {
                fail("leader timed out in getEpochToPropose");
            }
        } finally {
            if (leader != null) {
                leader.shutdown("end of test");
            }
            TestUtils.deleteFileRecursively(tmpDir);
        }
    }

    /**
     * In this test, the leader sets the last accepted epoch to 5. The call
     * to getEpochToPropose should set epoch to 6 and wait until another
     * follower executes it. If in getEpochToPropose we don't check if
     * lastAcceptedEpoch == epoch, then the call from the subsequent
     * follower with lastAcceptedEpoch = 6 doesn't change the value
     * of epoch, and the test fails. It passes with the fix to predicate.
     *
     * https://issues.apache.org/jira/browse/ZOOKEEPER-1343
     *
     *
     * @throws Exception
     */

    @Test
    public void testLastAcceptedEpoch() throws Exception {
        File tmpDir = File.createTempFile("test", "dir", testData);
        tmpDir.delete();
        tmpDir.mkdir();
        Leader leader = null;
        LeadThread leadThread = null;
        try {
            QuorumPeer peer = createQuorumPeer(tmpDir);
            leader = createMockLeader(tmpDir, peer);
            peer.leader = leader;
            peer.setAcceptedEpoch(5);
            leadThread = new LeadThread(leader);
            leadThread.start();

            while (((MockLeader) leader).getCurrentEpochToPropose() != 6) {
                Thread.sleep(20);
            }

            try {
                long epoch = leader.getEpochToPropose(1, 6);
                assertEquals(7, epoch, "New proposed epoch is wrong");
            } catch (Exception e) {
                fail("Timed out in getEpochToPropose");
            }

        } finally {
            if (leader != null) {
                leader.shutdown("end of test");
            }
            if (leadThread != null) {
                leadThread.interrupt();
                leadThread.join();
            }
            TestUtils.deleteFileRecursively(tmpDir);
        }
    }

    @Test
    public void testLeaderInElectingFollowers() throws Exception {
        File tmpDir = File.createTempFile("test", "dir", testData);
        tmpDir.delete();
        tmpDir.mkdir();
        Leader leader = null;
        try {
            QuorumPeer peer = createQuorumPeer(tmpDir);
            leader = createLeader(tmpDir, peer);
            peer.leader = leader;

            FollowerMockThread f1 = new FollowerMockThread(1, leader, false);
            FollowerMockThread f2 = new FollowerMockThread(2, leader, false);

            // things needed for waitForEpochAck to run (usually in leader.lead(), but we're not running leader here)
            leader.leaderStateSummary = new StateSummary(leader.self.getCurrentEpoch(), leader.zk.getLastProcessedZxid());

            f1.start();
            f2.start();

            // wait until followers time out in waitForEpochAck - they shouldn't return
            // normally because the leader didn't execute waitForEpochAck
            f1.join(leader.self.getInitLimit() * leader.self.getTickTime() + 5000);
            f2.join(leader.self.getInitLimit() * leader.self.getTickTime() + 5000);

            // make sure that they timed out and didn't return normally
            assertTrue(f1.msg == null, f1.msg + " without waiting for leader");
            assertTrue(f2.msg == null, f2.msg + " without waiting for leader");
        } finally {
            if (leader != null) {
                leader.shutdown("end of test");
            }
            TestUtils.deleteFileRecursively(tmpDir);
        }
    }

    static Socket[] getSocketPair() throws IOException {
        ServerSocket ss = new ServerSocket(0, 50, InetAddress.getByName("127.0.0.1"));
        InetSocketAddress endPoint = (InetSocketAddress) ss.getLocalSocketAddress();
        Socket s = new Socket(endPoint.getAddress(), endPoint.getPort());
        return new Socket[]{s, ss.accept()};
    }
    static void readPacketSkippingPing(InputArchive ia, QuorumPacket qp) throws IOException {
        while (true) {
            ia.readRecord(qp, null);
            if (qp.getType() != Leader.PING) {
                return;
            }
        }
    }

    public interface LeaderConversation {

        void converseWithLeader(InputArchive ia, OutputArchive oa, Leader l) throws Exception;

    }

    public interface PopulatedLeaderConversation {

        void converseWithLeader(InputArchive ia, OutputArchive oa, Leader l, long zxid) throws Exception;

    }

    public interface FollowerConversation {

        void converseWithFollower(InputArchive ia, OutputArchive oa, Follower f) throws Exception;

    }

    public interface ObserverConversation {

        void converseWithObserver(InputArchive ia, OutputArchive oa, Observer o) throws Exception;

    }

    public void testLeaderConversation(LeaderConversation conversation) throws Exception {
        Socket[] pair = getSocketPair();
        Socket leaderSocket = pair[0];
        Socket followerSocket = pair[1];
        File tmpDir = File.createTempFile("test", "dir", testData);
        tmpDir.delete();
        tmpDir.mkdir();
        LeadThread leadThread = null;
        Leader leader = null;
        try {
            QuorumPeer peer = createQuorumPeer(tmpDir);
            leader = createLeader(tmpDir, peer);
            peer.leader = leader;
            leadThread = new LeadThread(leader);
            leadThread.start();

            while (leader.cnxAcceptor == null || !leader.cnxAcceptor.isAlive()) {
                Thread.sleep(20);
            }

            LearnerHandler lh = new LearnerHandler(leaderSocket, new BufferedInputStream(leaderSocket.getInputStream()), leader);
            lh.start();
            leaderSocket.setSoTimeout(4000);

            InputArchive ia = BinaryInputArchive.getArchive(followerSocket.getInputStream());
            OutputArchive oa = BinaryOutputArchive.getArchive(followerSocket.getOutputStream());

            conversation.converseWithLeader(ia, oa, leader);
        } finally {
            if (leader != null) {
                leader.shutdown("end of test");
            }
            if (leadThread != null) {
                leadThread.interrupt();
                leadThread.join();
            }
            TestUtils.deleteFileRecursively(tmpDir);
        }
    }

    public void testPopulatedLeaderConversation(PopulatedLeaderConversation conversation, int ops) throws Exception {
        Socket[] pair = getSocketPair();
        Socket leaderSocket = pair[0];
        Socket followerSocket = pair[1];
        File tmpDir = File.createTempFile("test", "dir", testData);
        tmpDir.delete();
        tmpDir.mkdir();
        LeadThread leadThread = null;
        Leader leader = null;
        try {
            // Setup a database with two znodes
            FileTxnSnapLog snapLog = new FileTxnSnapLog(tmpDir, tmpDir);
            ZKDatabase zkDb = new ZKDatabase(snapLog);

            assertTrue(ops >= 1);
            long zxid = ZxidUtils.makeZxid(1, 0);
            for (int i = 1; i <= ops; i++) {
                zxid = ZxidUtils.makeZxid(1, i);
                String path = "/foo-" + i;
                zkDb.processTxn(new TxnHeader(13, 1000 + i, zxid, 30 + i, ZooDefs.OpCode.create),
                        new CreateTxn(path, "fpjwasalsohere".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 1), null);
                Stat stat = new Stat();
                assertEquals("fpjwasalsohere", new String(zkDb.getData(path, stat, null)));
            }
            assertTrue(zxid > ZxidUtils.makeZxid(1, 0));

            // Generate snapshot and close files.
            snapLog.save(zkDb.getDataTree(), zkDb.getSessionWithTimeOuts(), false);
            snapLog.close();

            QuorumPeer peer = createQuorumPeer(tmpDir);

            leader = createLeader(tmpDir, peer);
            peer.leader = leader;

            // Set the last accepted epoch and current epochs to be 1
            peer.setAcceptedEpoch(1);
            peer.setCurrentEpoch(1);

            leadThread = new LeadThread(leader);
            leadThread.start();

            while (leader.cnxAcceptor == null || !leader.cnxAcceptor.isAlive()) {
                Thread.sleep(20);
            }

            LearnerHandler lh = new LearnerHandler(leaderSocket, new BufferedInputStream(leaderSocket.getInputStream()), leader);
            lh.start();
            leaderSocket.setSoTimeout(4000);

            InputArchive ia = BinaryInputArchive.getArchive(followerSocket.getInputStream());
            OutputArchive oa = BinaryOutputArchive.getArchive(followerSocket.getOutputStream());

            conversation.converseWithLeader(ia, oa, leader, zxid);
        } finally {
            if (leader != null) {
                leader.shutdown("end of test");
            }
            if (leadThread != null) {
                leadThread.interrupt();
                leadThread.join();
            }
            TestUtils.deleteFileRecursively(tmpDir);
        }
    }

    public void testFollowerConversation(FollowerConversation conversation) throws Exception {
        File tmpDir = File.createTempFile("test", "dir", testData);
        tmpDir.delete();
        tmpDir.mkdir();
        Thread followerThread = null;
        ConversableFollower follower = null;
        QuorumPeer peer = null;
        try {
            peer = createQuorumPeer(tmpDir);
            follower = createFollower(tmpDir, peer);
            peer.follower = follower;

            ServerSocket ss = new ServerSocket(0, 50, InetAddress.getByName("127.0.0.1"));
            QuorumServer leaderQS = new QuorumServer(1, (InetSocketAddress) ss.getLocalSocketAddress());
            follower.setLeaderQuorumServer(leaderQS);
            final Follower followerForThread = follower;

            followerThread = new Thread() {
                public void run() {
                    try {
                        followerForThread.followLeader();
                    } catch (InterruptedException e) {
                        LOG.info("Follower thread interrupted", e);
                    } catch (Exception e) {
                        LOG.warn("Unexpected exception in follower thread", e);
                    }
                }
            };
            followerThread.start();
            Socket leaderSocket = ss.accept();

            InputArchive ia = BinaryInputArchive.getArchive(leaderSocket.getInputStream());
            OutputArchive oa = BinaryOutputArchive.getArchive(leaderSocket.getOutputStream());

            conversation.converseWithFollower(ia, oa, follower);
        } finally {
            if (follower != null) {
                follower.shutdown();
            }
            if (followerThread != null) {
                followerThread.interrupt();
                followerThread.join();
            }
            if (peer != null) {
                peer.shutdown();
            }
            TestUtils.deleteFileRecursively(tmpDir);
        }
    }

    public void testObserverConversation(ObserverConversation conversation) throws Exception {
        File tmpDir = File.createTempFile("test", "dir", testData);
        tmpDir.delete();
        tmpDir.mkdir();
        Thread observerThread = null;
        ConversableObserver observer = null;
        QuorumPeer peer = null;
        try {
            peer = createQuorumPeer(tmpDir);
            peer.setSyncEnabled(true);
            observer = createObserver(tmpDir, peer);
            peer.observer = observer;

            ServerSocket ss = new ServerSocket(0, 50, InetAddress.getByName("127.0.0.1"));
            QuorumServer leaderQS = new QuorumServer(1, (InetSocketAddress) ss.getLocalSocketAddress());
            observer.setLeaderQuorumServer(leaderQS);
            final Observer observerForThread = observer;

            observerThread = new Thread() {
                public void run() {
                    try {
                        observerForThread.observeLeader();
                    } catch (Exception e) {
                        e.printStackTrace();
                    }
                }
            };
            observerThread.start();
            Socket leaderSocket = ss.accept();

            InputArchive ia = BinaryInputArchive.getArchive(leaderSocket.getInputStream());
            OutputArchive oa = BinaryOutputArchive.getArchive(leaderSocket.getOutputStream());

            conversation.converseWithObserver(ia, oa, observer);
        } finally {
            if (observer != null) {
                observer.shutdown();
            }
            if (observerThread != null) {
                observerThread.interrupt();
                observerThread.join();
            }
            if (peer != null) {
                peer.shutdown();
            }
            TestUtils.deleteFileRecursively(tmpDir);
        }
    }

    @Test
    public void testUnnecessarySnap() throws Exception {
        testPopulatedLeaderConversation(new PopulatedLeaderConversation() {
            @Override
            public void converseWithLeader(InputArchive ia, OutputArchive oa, Leader l, long zxid) throws Exception {

                assertEquals(1, l.self.getAcceptedEpoch());
                assertEquals(1, l.self.getCurrentEpoch());

                /* we test a normal run. everything should work out well. */
                LearnerInfo li = new LearnerInfo(1, 0x10000, 0);
                byte[] liBytes = new byte[20];
                ByteBufferOutputStream.record2ByteBuffer(li, ByteBuffer.wrap(liBytes));
                QuorumPacket qp = new QuorumPacket(Leader.FOLLOWERINFO, 1, liBytes, null);
                oa.writeRecord(qp, null);

                readPacketSkippingPing(ia, qp);
                assertEquals(Leader.LEADERINFO, qp.getType());
                assertEquals(ZxidUtils.makeZxid(2, 0), qp.getZxid());
                assertEquals(ByteBuffer.wrap(qp.getData()).getInt(), 0x10000);
                assertEquals(2, l.self.getAcceptedEpoch());
                assertEquals(1, l.self.getCurrentEpoch());

                byte[] epochBytes = new byte[4];
                final ByteBuffer wrappedEpochBytes = ByteBuffer.wrap(epochBytes);
                wrappedEpochBytes.putInt(1);
                qp = new QuorumPacket(Leader.ACKEPOCH, zxid, epochBytes, null);
                oa.writeRecord(qp, null);

                readPacketSkippingPing(ia, qp);
                assertEquals(Leader.DIFF, qp.getType());

            }
        }, 2);
    }

    // We want to track the change with a callback rather than depending on timing
    class TrackerWatcher implements Watcher {

        boolean changed;
        synchronized void waitForChange() throws InterruptedException {
            while (!changed) {
                wait();
            }
        }
        @Override
        public void process(WatchedEvent event) {
            if (event.getType() == EventType.NodeDataChanged) {
                synchronized (this) {
                    changed = true;
                    notifyAll();
                }
            }
        }
        public synchronized boolean changed() {
            return changed;
        }

    }

    @Test
    public void testNormalFollowerRun() throws Exception {
        testFollowerConversation(new FollowerConversation() {
            @Override
            public void converseWithFollower(InputArchive ia, OutputArchive oa, Follower f) throws Exception {
                File tmpDir = File.createTempFile("test", "dir", testData);
                tmpDir.delete();
                tmpDir.mkdir();
                File logDir = f.fzk.getTxnLogFactory().getDataDir().getParentFile();
                File snapDir = f.fzk.getTxnLogFactory().getSnapDir().getParentFile();
                //Spy on ZK so we can check if a snapshot happened or not.
                f.zk = spy(f.zk);
                try {
                    assertEquals(0, f.self.getAcceptedEpoch());
                    assertEquals(0, f.self.getCurrentEpoch());

                    // Setup a database with a single /foo node
                    ZKDatabase zkDb = new ZKDatabase(new FileTxnSnapLog(tmpDir, tmpDir));
                    final long firstZxid = ZxidUtils.makeZxid(1, 1);
                    zkDb.processTxn(new TxnHeader(13, 1313, firstZxid, 33, ZooDefs.OpCode.create), new CreateTxn("/foo", "data1".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 1), null);
                    Stat stat = new Stat();
                    assertEquals("data1", new String(zkDb.getData("/foo", stat, null)));

                    QuorumPacket qp = new QuorumPacket();
                    readPacketSkippingPing(ia, qp);
                    assertEquals(Leader.FOLLOWERINFO, qp.getType());
                    assertEquals(qp.getZxid(), 0);
                    LearnerInfo learnInfo = new LearnerInfo();
                    ByteBufferInputStream.byteBuffer2Record(ByteBuffer.wrap(qp.getData()), learnInfo);
                    assertEquals(learnInfo.getProtocolVersion(), 0x10000);
                    assertEquals(learnInfo.getServerid(), 0);

                    // We are simulating an established leader, so the epoch is 1
                    qp.setType(Leader.LEADERINFO);
                    qp.setZxid(ZxidUtils.makeZxid(1, 0));
                    byte[] protoBytes = new byte[4];
                    ByteBuffer.wrap(protoBytes).putInt(0x10000);
                    qp.setData(protoBytes);
                    oa.writeRecord(qp, null);

                    readPacketSkippingPing(ia, qp);
                    assertEquals(Leader.ACKEPOCH, qp.getType());
                    assertEquals(0, qp.getZxid());
                    assertEquals(ZxidUtils.makeZxid(0, 0), ByteBuffer.wrap(qp.getData()).getInt());
                    assertEquals(1, f.self.getAcceptedEpoch());
                    assertEquals(0, f.self.getCurrentEpoch());

                    // Send the snapshot we created earlier
                    qp.setType(Leader.SNAP);
                    qp.setData(new byte[0]);
                    qp.setZxid(zkDb.getDataTreeLastProcessedZxid());
                    oa.writeRecord(qp, null);
                    zkDb.serializeSnapshot(oa);
                    oa.writeString("BenWasHere", null);
                    Thread.sleep(10); //Give it some time to process the snap
                    //No Snapshot taken yet, the SNAP was applied in memory
                    verify(f.zk, never()).takeSnapshot();

                    qp.setType(Leader.NEWLEADER);
                    qp.setZxid(ZxidUtils.makeZxid(1, 0));
                    oa.writeRecord(qp, null);

                    // Get the ack of the new leader
                    readPacketSkippingPing(ia, qp);
                    assertEquals(Leader.ACK, qp.getType());
                    assertEquals(ZxidUtils.makeZxid(1, 0), qp.getZxid());
                    assertEquals(1, f.self.getAcceptedEpoch());
                    assertEquals(1, f.self.getCurrentEpoch());
                    //Make sure that we did take the snapshot now
                    verify(f.zk).takeSnapshot(true);
                    assertEquals(firstZxid, f.fzk.getLastProcessedZxid());

                    // Make sure the data was recorded in the filesystem ok
                    ZKDatabase zkDb2 = new ZKDatabase(new FileTxnSnapLog(logDir, snapDir));
                    long lastZxid = zkDb2.loadDataBase();
                    assertEquals("data1", new String(zkDb2.getData("/foo", stat, null)));
                    assertEquals(firstZxid, lastZxid);

                    // Propose an update
                    long proposalZxid = ZxidUtils.makeZxid(1, 1000);
                    proposeSetData(qp, proposalZxid, "data2", 2);
                    oa.writeRecord(qp, null);

                    TrackerWatcher watcher = new TrackerWatcher();

                    // The change should not have happened yet, since we haven't committed
                    assertEquals("data1", new String(f.fzk.getZKDatabase().getData("/foo", stat, watcher)));

                    // The change should happen now
                    qp.setType(Leader.COMMIT);
                    qp.setZxid(proposalZxid);
                    oa.writeRecord(qp, null);

                    qp.setType(Leader.UPTODATE);
                    qp.setZxid(0);
                    oa.writeRecord(qp, null);

                    // Read the uptodate ack
                    readPacketSkippingPing(ia, qp);
                    assertEquals(Leader.ACK, qp.getType());
                    assertEquals(ZxidUtils.makeZxid(1, 0), qp.getZxid());

                    readPacketSkippingPing(ia, qp);
                    assertEquals(Leader.ACK, qp.getType());
                    assertEquals(proposalZxid, qp.getZxid());

                    watcher.waitForChange();
                    assertEquals("data2", new String(f.fzk.getZKDatabase().getData("/foo", stat, null)));

                    // check and make sure the change is persisted
                    zkDb2 = new ZKDatabase(new FileTxnSnapLog(logDir, snapDir));
                    lastZxid = zkDb2.loadDataBase();
                    assertEquals("data2", new String(zkDb2.getData("/foo", stat, null)));
                    assertEquals(proposalZxid, lastZxid);
                } finally {
                    TestUtils.deleteFileRecursively(tmpDir);
                }

            }

            private void proposeSetData(QuorumPacket qp, long zxid, String data, int version) throws IOException {
                qp.setType(Leader.PROPOSAL);
                qp.setZxid(zxid);
                TxnHeader hdr = new TxnHeader(4, 1414, qp.getZxid(), 55, ZooDefs.OpCode.setData);
                SetDataTxn sdt = new SetDataTxn("/foo", data.getBytes(), version);
                ByteArrayOutputStream baos = new ByteArrayOutputStream();
                OutputArchive boa = BinaryOutputArchive.getArchive(baos);
                boa.writeRecord(hdr, null);
                boa.writeRecord(sdt, null);
                qp.setData(baos.toByteArray());
            }
        });
    }

    @Test
    public void testNormalFollowerRunWithDiff() throws Exception {
        testFollowerConversation(new FollowerConversation() {
            @Override
            public void converseWithFollower(InputArchive ia, OutputArchive oa, Follower f) throws Exception {
                File tmpDir = File.createTempFile("test", "dir", testData);
                tmpDir.delete();
                tmpDir.mkdir();
                File logDir = f.fzk.getTxnLogFactory().getDataDir().getParentFile();
                File snapDir = f.fzk.getTxnLogFactory().getSnapDir().getParentFile();
                //Spy on ZK so we can check if a snapshot happened or not.
                f.zk = spy(f.zk);
                try {
                    assertEquals(0, f.self.getAcceptedEpoch());
                    assertEquals(0, f.self.getCurrentEpoch());

                    // Setup a database with a single /foo node
                    ZKDatabase zkDb = new ZKDatabase(new FileTxnSnapLog(tmpDir, tmpDir));
                    final long firstZxid = ZxidUtils.makeZxid(1, 1);
                    zkDb.processTxn(new TxnHeader(13, 1313, firstZxid, 33, ZooDefs.OpCode.create), new CreateTxn("/foo", "data1".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 1), null);
                    Stat stat = new Stat();
                    assertEquals("data1", new String(zkDb.getData("/foo", stat, null)));

                    QuorumPacket qp = new QuorumPacket();
                    readPacketSkippingPing(ia, qp);
                    assertEquals(Leader.FOLLOWERINFO, qp.getType());
                    assertEquals(qp.getZxid(), 0);
                    LearnerInfo learnInfo = new LearnerInfo();
                    ByteBufferInputStream.byteBuffer2Record(ByteBuffer.wrap(qp.getData()), learnInfo);
                    assertEquals(learnInfo.getProtocolVersion(), 0x10000);
                    assertEquals(learnInfo.getServerid(), 0);

                    // We are simulating an established leader, so the epoch is 1
                    qp.setType(Leader.LEADERINFO);
                    qp.setZxid(ZxidUtils.makeZxid(1, 0));
                    byte[] protoBytes = new byte[4];
                    ByteBuffer.wrap(protoBytes).putInt(0x10000);
                    qp.setData(protoBytes);
                    oa.writeRecord(qp, null);

                    readPacketSkippingPing(ia, qp);
                    assertEquals(Leader.ACKEPOCH, qp.getType());
                    assertEquals(0, qp.getZxid());
                    assertEquals(ZxidUtils.makeZxid(0, 0), ByteBuffer.wrap(qp.getData()).getInt());
                    assertEquals(1, f.self.getAcceptedEpoch());
                    assertEquals(0, f.self.getCurrentEpoch());

                    // Send a diff
                    qp.setType(Leader.DIFF);
                    qp.setData(new byte[0]);
                    qp.setZxid(zkDb.getDataTreeLastProcessedZxid());
                    oa.writeRecord(qp, null);
                    final long createSessionZxid = ZxidUtils.makeZxid(1, 2);
                    proposeNewSession(qp, createSessionZxid, 0x333);
                    oa.writeRecord(qp, null);
                    qp.setType(Leader.COMMIT);
                    qp.setZxid(createSessionZxid);
                    oa.writeRecord(qp, null);
                    qp.setType(Leader.NEWLEADER);
                    qp.setZxid(ZxidUtils.makeZxid(1, 0));
                    qp.setData(null);
                    oa.writeRecord(qp, null);
                    qp.setType(Leader.UPTODATE);
                    qp.setZxid(0);
                    oa.writeRecord(qp, null);

                    // Read the uptodate ack
                    readPacketSkippingPing(ia, qp);
                    assertEquals(Leader.ACK, qp.getType());
                    assertEquals(ZxidUtils.makeZxid(1, 0), qp.getZxid());

                    // Get the ack of the new leader
                    readPacketSkippingPing(ia, qp);
                    assertEquals(Leader.ACK, qp.getType());
                    assertEquals(ZxidUtils.makeZxid(1, 0), qp.getZxid());
                    assertEquals(1, f.self.getAcceptedEpoch());
                    assertEquals(1, f.self.getCurrentEpoch());

                    //Wait for the transactions to be written out. The thread that writes them out
                    // does not send anything back when it is done.
                    long start = System.currentTimeMillis();
                    while (createSessionZxid != f.fzk.getLastProcessedZxid()
                                   && (System.currentTimeMillis() - start) < 50) {
                        Thread.sleep(1);
                    }

                    assertEquals(createSessionZxid, f.fzk.getLastProcessedZxid());

                    // Make sure the data was recorded in the filesystem ok
                    ZKDatabase zkDb2 = new ZKDatabase(new FileTxnSnapLog(logDir, snapDir));
                    start = System.currentTimeMillis();
                    zkDb2.loadDataBase();
                    while (zkDb2.getSessionWithTimeOuts().isEmpty() && (System.currentTimeMillis() - start) < 50) {
                        Thread.sleep(1);
                        zkDb2.loadDataBase();
                    }
                    LOG.info("zkdb2 sessions:{}", zkDb2.getSessions());
                    LOG.info("zkdb2 with timeouts:{}", zkDb2.getSessionWithTimeOuts());
                    assertNotNull(zkDb2.getSessionWithTimeOuts().get(4L));
                    //Snapshot was never taken during very simple sync
                    verify(f.zk, never()).takeSnapshot();
                } finally {
                    TestUtils.deleteFileRecursively(tmpDir);
                }

            }

            private void proposeNewSession(QuorumPacket qp, long zxid, long sessionId) throws IOException {
                qp.setType(Leader.PROPOSAL);
                qp.setZxid(zxid);
                TxnHeader hdr = new TxnHeader(4, 1414, qp.getZxid(), 55, ZooDefs.OpCode.createSession);
                CreateSessionTxn cst = new CreateSessionTxn(30000);
                ByteArrayOutputStream baos = new ByteArrayOutputStream();
                OutputArchive boa = BinaryOutputArchive.getArchive(baos);
                boa.writeRecord(hdr, null);
                boa.writeRecord(cst, null);
                qp.setData(baos.toByteArray());
            }
        });
    }

    @Test
    public void testNormalRun() throws Exception {
        testLeaderConversation(new LeaderConversation() {
            public void converseWithLeader(InputArchive ia, OutputArchive oa, Leader l) throws IOException {
                assertEquals(0, l.self.getAcceptedEpoch());
                assertEquals(0, l.self.getCurrentEpoch());

                /* we test a normal run. everything should work out well. */
                LearnerInfo li = new LearnerInfo(1, 0x10000, 0);
                byte[] liBytes = new byte[20];
                ByteBufferOutputStream.record2ByteBuffer(li, ByteBuffer.wrap(liBytes));
                QuorumPacket qp = new QuorumPacket(Leader.FOLLOWERINFO, 0, liBytes, null);
                oa.writeRecord(qp, null);

                readPacketSkippingPing(ia, qp);
                assertEquals(Leader.LEADERINFO, qp.getType());
                assertEquals(ZxidUtils.makeZxid(1, 0), qp.getZxid());
                assertEquals(ByteBuffer.wrap(qp.getData()).getInt(), 0x10000);
                assertEquals(1, l.self.getAcceptedEpoch());
                assertEquals(0, l.self.getCurrentEpoch());

                qp = new QuorumPacket(Leader.ACKEPOCH, 0, new byte[4], null);
                oa.writeRecord(qp, null);

                readPacketSkippingPing(ia, qp);
                assertEquals(Leader.DIFF, qp.getType());

                readPacketSkippingPing(ia, qp);
                assertEquals(Leader.NEWLEADER, qp.getType());
                assertEquals(ZxidUtils.makeZxid(1, 0), qp.getZxid());
                assertEquals(1, l.self.getAcceptedEpoch());
                assertCurrentEpochGotUpdated(1, l.self, ClientBase.CONNECTION_TIMEOUT);

                qp = new QuorumPacket(Leader.ACK, qp.getZxid(), null, null);
                oa.writeRecord(qp, null);

                readPacketSkippingPing(ia, qp);
                assertEquals(Leader.UPTODATE, qp.getType());
            }
        });
    }

    @Test
    public void testTxnTimeout() throws Exception {
        testLeaderConversation(new LeaderConversation() {
            public void converseWithLeader(InputArchive ia, OutputArchive oa, Leader l) throws IOException, InterruptedException, org.apache.zookeeper.server.quorum.Leader.XidRolloverException {
                assertEquals(0, l.self.getAcceptedEpoch());
                assertEquals(0, l.self.getCurrentEpoch());

                LearnerInfo li = new LearnerInfo(1, 0x10000, 0);
                byte[] liBytes = new byte[20];
                ByteBufferOutputStream.record2ByteBuffer(li, ByteBuffer.wrap(liBytes));
                QuorumPacket qp = new QuorumPacket(Leader.FOLLOWERINFO, 0, liBytes, null);
                oa.writeRecord(qp, null);

                readPacketSkippingPing(ia, qp);
                assertEquals(Leader.LEADERINFO, qp.getType());
                assertEquals(ZxidUtils.makeZxid(1, 0), qp.getZxid());
                assertEquals(ByteBuffer.wrap(qp.getData()).getInt(), 0x10000);
                assertEquals(1, l.self.getAcceptedEpoch());
                assertEquals(0, l.self.getCurrentEpoch());

                qp = new QuorumPacket(Leader.ACKEPOCH, 0, new byte[4], null);
                oa.writeRecord(qp, null);

                readPacketSkippingPing(ia, qp);
                assertEquals(Leader.DIFF, qp.getType());

                readPacketSkippingPing(ia, qp);
                assertEquals(Leader.NEWLEADER, qp.getType());
                assertEquals(ZxidUtils.makeZxid(1, 0), qp.getZxid());
                assertEquals(1, l.self.getAcceptedEpoch());
                assertCurrentEpochGotUpdated(1, l.self, ClientBase.CONNECTION_TIMEOUT);

                qp = new QuorumPacket(Leader.ACK, qp.getZxid(), null, null);
                oa.writeRecord(qp, null);

                readPacketSkippingPing(ia, qp);
                assertEquals(Leader.UPTODATE, qp.getType());

                long zxid = l.zk.getZxid();
                l.propose(new Request(1, 1, ZooDefs.OpCode.create, new TxnHeader(1, 1, zxid, 1, ZooDefs.OpCode.create), new CreateTxn("/test", "hola".getBytes(), null, true, 0), zxid));

                readPacketSkippingPing(ia, qp);
                assertEquals(Leader.PROPOSAL, qp.getType());

                LOG.info("Proposal sent.");

                for (int i = 0; i < (2 * ZabUtils.SYNC_LIMIT) + 2; i++) {
                    try {
                        ia.readRecord(qp, null);
                        LOG.info("Ping received: {}", i);
                        qp = new QuorumPacket(Leader.PING, qp.getZxid(), "".getBytes(), null);
                        oa.writeRecord(qp, null);
                    } catch (EOFException e) {
                        return;
                    }
                }
                fail("Connection hasn't been closed by leader after transaction times out.");
            }
        });
    }

    private void deserializeSnapshot(InputArchive ia) throws IOException {
        ZKDatabase zkdb = new ZKDatabase(null);
        zkdb.deserializeSnapshot(ia);
        String signature = ia.readString("signature");
        assertEquals("BenWasHere", signature);
    }

    @Test
    public void testNormalObserverRun() throws Exception {
        testObserverConversation(new ObserverConversation() {
            @Override
            public void converseWithObserver(InputArchive ia, OutputArchive oa, Observer o) throws Exception {
                File tmpDir = File.createTempFile("test", "dir", testData);
                tmpDir.delete();
                tmpDir.mkdir();
                File logDir = o.zk.getTxnLogFactory().getDataDir().getParentFile();
                File snapDir = o.zk.getTxnLogFactory().getSnapDir().getParentFile();
                try {
                    assertEquals(0, o.self.getAcceptedEpoch());
                    assertEquals(0, o.self.getCurrentEpoch());

                    // Setup a database with a single /foo node
                    ZKDatabase zkDb = new ZKDatabase(new FileTxnSnapLog(tmpDir, tmpDir));
                    final long foo1Zxid = ZxidUtils.makeZxid(1, 1);
                    final long foo2Zxid = ZxidUtils.makeZxid(1, 2);
                    zkDb.processTxn(new TxnHeader(13, 1313, foo1Zxid, 33, ZooDefs.OpCode.create), new CreateTxn("/foo1", "data1".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 1), null);
                    zkDb.processTxn(new TxnHeader(13, 1313, foo2Zxid, 33, ZooDefs.OpCode.create), new CreateTxn("/foo2", "data1".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 1), null);
                    Stat stat = new Stat();
                    assertEquals("data1", new String(zkDb.getData("/foo1", stat, null)));
                    assertEquals("data1", new String(zkDb.getData("/foo2", stat, null)));

                    QuorumPacket qp = new QuorumPacket();
                    readPacketSkippingPing(ia, qp);
                    assertEquals(Leader.OBSERVERINFO, qp.getType());
                    assertEquals(qp.getZxid(), 0);
                    LearnerInfo learnInfo = new LearnerInfo();
                    ByteBufferInputStream.byteBuffer2Record(ByteBuffer.wrap(qp.getData()), learnInfo);
                    assertEquals(learnInfo.getProtocolVersion(), 0x10000);
                    assertEquals(learnInfo.getServerid(), 0);

                    // We are simulating an established leader, so the epoch is 1
                    qp.setType(Leader.LEADERINFO);
                    qp.setZxid(ZxidUtils.makeZxid(1, 0));
                    byte[] protoBytes = new byte[4];
                    ByteBuffer.wrap(protoBytes).putInt(0x10000);
                    qp.setData(protoBytes);
                    oa.writeRecord(qp, null);

                    readPacketSkippingPing(ia, qp);
                    assertEquals(Leader.ACKEPOCH, qp.getType());
                    assertEquals(0, qp.getZxid());
                    assertEquals(ZxidUtils.makeZxid(0, 0), ByteBuffer.wrap(qp.getData()).getInt());
                    assertEquals(1, o.self.getAcceptedEpoch());
                    assertEquals(0, o.self.getCurrentEpoch());

                    // Send the snapshot we created earlier
                    qp.setType(Leader.SNAP);
                    qp.setData(new byte[0]);
                    qp.setZxid(zkDb.getDataTreeLastProcessedZxid());
                    oa.writeRecord(qp, null);
                    zkDb.serializeSnapshot(oa);
                    oa.writeString("BenWasHere", null);
                    qp.setType(Leader.NEWLEADER);
                    qp.setZxid(ZxidUtils.makeZxid(1, 0));
                    oa.writeRecord(qp, null);

                    // Get the ack of the new leader
                    readPacketSkippingPing(ia, qp);
                    assertEquals(Leader.ACK, qp.getType());
                    assertEquals(ZxidUtils.makeZxid(1, 0), qp.getZxid());
                    assertEquals(1, o.self.getAcceptedEpoch());
                    assertEquals(1, o.self.getCurrentEpoch());

                    assertEquals(foo2Zxid, o.zk.getLastProcessedZxid());

                    // Make sure the data was recorded in the filesystem ok
                    ZKDatabase zkDb2 = new ZKDatabase(new FileTxnSnapLog(logDir, snapDir));
                    long lastZxid = zkDb2.loadDataBase();
                    assertEquals("data1", new String(zkDb2.getData("/foo1", stat, null)));
                    assertEquals(foo2Zxid, lastZxid);

                    // Register watch
                    TrackerWatcher watcher = new TrackerWatcher();
                    assertEquals("data1", new String(o.zk.getZKDatabase().getData("/foo2", stat, watcher)));

                    // Propose /foo1 update
                    long proposalZxid = ZxidUtils.makeZxid(1, 1000);
                    proposeSetData(qp, "/foo1", proposalZxid, "data2", 2);
                    oa.writeRecord(qp, null);

                    // Commit /foo1 update
                    qp.setType(Leader.COMMIT);
                    qp.setZxid(proposalZxid);
                    oa.writeRecord(qp, null);

                    // Inform /foo2 update
                    long informZxid = ZxidUtils.makeZxid(1, 1001);
                    proposeSetData(qp, "/foo2", informZxid, "data2", 2);
                    qp.setType(Leader.INFORM);
                    oa.writeRecord(qp, null);

                    qp.setType(Leader.UPTODATE);
                    qp.setZxid(0);
                    oa.writeRecord(qp, null);

                    // Read the uptodate ack
                    readPacketSkippingPing(ia, qp);
                    assertEquals(Leader.ACK, qp.getType());
                    assertEquals(ZxidUtils.makeZxid(1, 0), qp.getZxid());

                    // Data should get updated
                    watcher.waitForChange();
                    assertEquals("data2", new String(o.zk.getZKDatabase().getData("/foo1", stat, null)));
                    assertEquals("data2", new String(o.zk.getZKDatabase().getData("/foo2", stat, null)));

                    // Shutdown sequence guarantee that all pending requests
                    // in sync request processor get flush to disk
                    o.zk.shutdown();

                    zkDb2 = new ZKDatabase(new FileTxnSnapLog(logDir, snapDir));
                    lastZxid = zkDb2.loadDataBase();
                    assertEquals("data2", new String(zkDb2.getData("/foo1", stat, null)));
                    assertEquals("data2", new String(zkDb2.getData("/foo2", stat, null)));
                    assertEquals(informZxid, lastZxid);
                } finally {
                    TestUtils.deleteFileRecursively(tmpDir);
                }

            }

            private void proposeSetData(QuorumPacket qp, String path, long zxid, String data, int version) throws IOException {
                qp.setType(Leader.PROPOSAL);
                qp.setZxid(zxid);
                TxnHeader hdr = new TxnHeader(4, 1414, qp.getZxid(), 55, ZooDefs.OpCode.setData);
                SetDataTxn sdt = new SetDataTxn(path, data.getBytes(), version);
                ByteArrayOutputStream baos = new ByteArrayOutputStream();
                OutputArchive boa = BinaryOutputArchive.getArchive(baos);
                boa.writeRecord(hdr, null);
                boa.writeRecord(sdt, null);
                qp.setData(baos.toByteArray());
            }
        });
    }

    @Test
    public void testLeaderBehind() throws Exception {
        testLeaderConversation(new LeaderConversation() {
            public void converseWithLeader(InputArchive ia, OutputArchive oa, Leader l) throws IOException {
                /* we test a normal run. everything should work out well. */
                LearnerInfo li = new LearnerInfo(1, 0x10000, 0);
                byte[] liBytes = new byte[20];
                ByteBufferOutputStream.record2ByteBuffer(li, ByteBuffer.wrap(liBytes));
                /* we are going to say we last acked epoch 20 */
                QuorumPacket qp = new QuorumPacket(Leader.FOLLOWERINFO, ZxidUtils.makeZxid(20, 0), liBytes, null);
                oa.writeRecord(qp, null);
                readPacketSkippingPing(ia, qp);
                assertEquals(Leader.LEADERINFO, qp.getType());
                assertEquals(ZxidUtils.makeZxid(21, 0), qp.getZxid());
                assertEquals(ByteBuffer.wrap(qp.getData()).getInt(), 0x10000);
                qp = new QuorumPacket(Leader.ACKEPOCH, 0, new byte[4], null);
                oa.writeRecord(qp, null);
                readPacketSkippingPing(ia, qp);
                assertEquals(Leader.DIFF, qp.getType());
                readPacketSkippingPing(ia, qp);
                assertEquals(Leader.NEWLEADER, qp.getType());
                assertEquals(ZxidUtils.makeZxid(21, 0), qp.getZxid());

                qp = new QuorumPacket(Leader.ACK, qp.getZxid(), null, null);
                oa.writeRecord(qp, null);

                readPacketSkippingPing(ia, qp);
                assertEquals(Leader.UPTODATE, qp.getType());
            }
        });
    }

    /**
     * Tests that when a quorum of followers send LearnerInfo but do not ack the epoch (which is sent
     * by the leader upon receipt of LearnerInfo from a quorum), the leader does not start using this epoch
     * as it would in the normal case (when a quorum do ack the epoch). This tests ZK-1192
     * @throws Exception
     */
    @Test
    public void testAbandonBeforeACKEpoch() throws Exception {
        testLeaderConversation(new LeaderConversation() {
            public void converseWithLeader(InputArchive ia, OutputArchive oa, Leader l) throws IOException, InterruptedException {
                /* we test a normal run. everything should work out well. */
                LearnerInfo li = new LearnerInfo(1, 0x10000, 0);
                byte[] liBytes = new byte[20];
                ByteBufferOutputStream.record2ByteBuffer(li, ByteBuffer.wrap(liBytes));
                QuorumPacket qp = new QuorumPacket(Leader.FOLLOWERINFO, 0, liBytes, null);
                oa.writeRecord(qp, null);
                readPacketSkippingPing(ia, qp);
                assertEquals(Leader.LEADERINFO, qp.getType());
                assertEquals(ZxidUtils.makeZxid(1, 0), qp.getZxid());
                assertEquals(ByteBuffer.wrap(qp.getData()).getInt(), 0x10000);
                Thread.sleep(l.self.getInitLimit() * l.self.getTickTime() + 5000);

                // The leader didn't get a quorum of acks - make sure that leader's current epoch is not advanced
                assertEquals(0, l.self.getCurrentEpoch());
            }
        });
    }

    static class ConversableFollower extends Follower {

        ConversableFollower(QuorumPeer self, FollowerZooKeeperServer zk) {
            super(self, zk);
        }

        QuorumServer leaderQuorumServer;
        public void setLeaderQuorumServer(QuorumServer quorumServer) {
            leaderQuorumServer = quorumServer;
        }

        @Override
        protected QuorumServer findLeader() {
            return leaderQuorumServer;
        }

    }
    private ConversableFollower createFollower(File tmpDir, QuorumPeer peer) throws IOException {
        FileTxnSnapLog logFactory = new FileTxnSnapLog(tmpDir, tmpDir);
        peer.setTxnFactory(logFactory);
        ZKDatabase zkDb = new ZKDatabase(logFactory);
        FollowerZooKeeperServer zk = new FollowerZooKeeperServer(logFactory, peer, zkDb);
        peer.setZKDatabase(zkDb);
        return new ConversableFollower(peer, zk);
    }

    static class ConversableObserver extends Observer {

        ConversableObserver(QuorumPeer self, ObserverZooKeeperServer zk) {
            super(self, zk);
        }

        QuorumServer leaderQuorumServer;
        public void setLeaderQuorumServer(QuorumServer quorumServer) {
            leaderQuorumServer = quorumServer;
        }

        @Override
        protected QuorumServer findLeader() {
            return leaderQuorumServer;
        }

    }

    private ConversableObserver createObserver(File tmpDir, QuorumPeer peer) throws IOException {
        FileTxnSnapLog logFactory = new FileTxnSnapLog(tmpDir, tmpDir);
        peer.setTxnFactory(logFactory);
        ZKDatabase zkDb = new ZKDatabase(logFactory);
        ObserverZooKeeperServer zk = new ObserverZooKeeperServer(logFactory, peer, zkDb);
        peer.setZKDatabase(zkDb);
        return new ConversableObserver(peer, zk);
    }

    private String readContentsOfFile(File f) throws IOException {
        return new BufferedReader(new FileReader(f)).readLine();
    }

    @Test
    public void testInitialAcceptedCurrent() throws Exception {
        File tmpDir = File.createTempFile("test", ".dir", testData);
        tmpDir.delete();
        tmpDir.mkdir();
        try {
            FileTxnSnapLog logFactory = new FileTxnSnapLog(tmpDir, tmpDir);
            File version2 = new File(tmpDir, "version-2");
            version2.mkdir();
            logFactory.save(new DataTree(), new ConcurrentHashMap<>(), false);
            long zxid = ZxidUtils.makeZxid(3, 3);
            logFactory.append(new Request(1, 1, ZooDefs.OpCode.error, new TxnHeader(1, 1, zxid, 1, ZooDefs.OpCode.error), new ErrorTxn(1), zxid));
            logFactory.commit();
            ZKDatabase zkDb = new ZKDatabase(logFactory);
            QuorumPeer peer = QuorumPeer.testingQuorumPeer();
            peer.setZKDatabase(zkDb);
            peer.setTxnFactory(logFactory);
            peer.getLastLoggedZxid();
            assertEquals(3, peer.getAcceptedEpoch());
            assertEquals(3, peer.getCurrentEpoch());
            assertEquals(3, Integer.parseInt(readContentsOfFile(new File(version2, QuorumPeer.CURRENT_EPOCH_FILENAME))));
            assertEquals(3, Integer.parseInt(readContentsOfFile(new File(version2, QuorumPeer.ACCEPTED_EPOCH_FILENAME))));
        } finally {
            TestUtils.deleteFileRecursively(tmpDir);
        }
    }

    /*
     * Epoch is first written to file then updated in memory. Give some time to
     * write the epoch in file and then go for assert.
     */
    private void assertCurrentEpochGotUpdated(int expected, QuorumPeer self, long timeout)
            throws IOException {
        long elapsedTime = 0;
        long waitInterval = 10;
        while (self.getCurrentEpoch() != expected && elapsedTime < timeout) {
            try {
                Thread.sleep(waitInterval);
            } catch (InterruptedException e) {
                fail("CurrentEpoch update failed");
            }
            elapsedTime = elapsedTime + waitInterval;
        }
        assertEquals(expected, self.getCurrentEpoch(), "CurrentEpoch update failed");
    }
}