summaryrefslogtreecommitdiff
path: root/zookeeper-server/src/test/java/org/apache/zookeeper/RemoveWatchesTest.java
blob: 93ab353abb884431d86736b4597d8d7a9820866f (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
/*
 * 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;

import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.spy;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.Watcher.Event.EventType;
import org.apache.zookeeper.Watcher.WatcherType;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.server.ServerCnxn;
import org.apache.zookeeper.test.ClientBase;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
 * Verifies removing watches using ZooKeeper client apis
 */
public class RemoveWatchesTest extends ClientBase {

    private static final Logger LOG = LoggerFactory.getLogger(RemoveWatchesTest.class);
    private ZooKeeper zk1 = null;
    private ZooKeeper zk2 = null;

    @BeforeEach
    @Override
    public void setUp() throws Exception {
        super.setUp();
        zk1 = createClient();
        zk2 = createClient();
    }

    @AfterEach
    @Override
    public void tearDown() throws Exception {
        if (zk1 != null) {
            zk1.close();
        }
        if (zk2 != null) {
            zk2.close();
        }
        super.tearDown();
    }

    private void removeWatches(
        ZooKeeper zk,
        String path,
        Watcher watcher,
        WatcherType watcherType,
        boolean local,
        KeeperException.Code rc,
        boolean useAsync) throws InterruptedException, KeeperException {
        LOG.info("Sending removeWatches req using zk {} path: {} type: {} watcher: {} ", zk, path, watcherType, watcher);
        if (useAsync) {
            MyCallback c1 = new MyCallback(rc.intValue(), path);
            zk.removeWatches(path, watcher, watcherType, local, c1, null);
            assertTrue(c1.matches(), "Didn't succeeds removeWatch operation");
            if (KeeperException.Code.OK.intValue() != c1.rc) {
                KeeperException ke = KeeperException.create(KeeperException.Code.get(c1.rc));
                throw ke;
            }
        } else {
            zk.removeWatches(path, watcher, watcherType, local);
        }
    }

    private void removeAllWatches(
        ZooKeeper zk,
        String path,
        WatcherType watcherType,
        boolean local,
        KeeperException.Code rc,
        boolean useAsync) throws InterruptedException, KeeperException {
        LOG.info("Sending removeWatches req using zk {} path: {} type: {} ", zk, path, watcherType);
        if (useAsync) {
            MyCallback c1 = new MyCallback(rc.intValue(), path);
            zk.removeAllWatches(path, watcherType, local, c1, null);
            assertTrue(c1.matches(), "Didn't succeeds removeWatch operation");
            if (KeeperException.Code.OK.intValue() != c1.rc) {
                KeeperException ke = KeeperException.create(KeeperException.Code.get(c1.rc));
                throw ke;
            }
        } else {
            zk.removeAllWatches(path, watcherType, local);
        }
    }

    /**
     * Test verifies removal of single watcher when there is server connection
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testRemoveSingleWatcher(boolean useAsync) throws Exception {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
        zk1.create("/node2", null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
        MyWatcher w1 = new MyWatcher("/node1", 1);
        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
        assertNotNull(zk2.exists("/node1", w1), "Didn't set data watches");
        MyWatcher w2 = new MyWatcher("/node2", 1);
        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
        assertNotNull(zk2.exists("/node2", w2), "Didn't set data watches");
        removeWatches(zk2, "/node1", w1, WatcherType.Data, false, Code.OK, useAsync);
        assertEquals(1, zk2.getDataWatches().size(), "Didn't find data watcher");
        assertEquals("/node2", zk2.getDataWatches().get(0), "Didn't find data watcher");
        removeWatches(zk2, "/node2", w2, WatcherType.Any, false, Code.OK, useAsync);
        assertTrue(w2.matches(), "Didn't remove data watcher");
        // closing session should remove ephemeral nodes and trigger data
        // watches if any
        if (zk1 != null) {
            zk1.close();
            zk1 = null;
        }

        List<EventType> events = w1.getEventsAfterWatchRemoval();
        assertFalse(events.contains(EventType.NodeDeleted), "Shouldn't get NodeDeletedEvent after watch removal");
        assertEquals(0, events.size(), "Shouldn't get NodeDeletedEvent after watch removal");
    }

    /**
     * Test verifies removal of multiple data watchers when there is server
     * connection
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testMultipleDataWatchers(boolean useAsync) throws IOException, InterruptedException, KeeperException {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
        MyWatcher w1 = new MyWatcher("/node1", 1);
        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
        assertNotNull(zk2.exists("/node1", w1), "Didn't set data watches");
        MyWatcher w2 = new MyWatcher("/node1", 1);
        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
        assertNotNull(zk2.exists("/node1", w2), "Didn't set data watches");
        removeWatches(zk2, "/node1", w2, WatcherType.Data, false, Code.OK, useAsync);
        assertEquals(1, zk2.getDataWatches().size(), "Didn't find data watcher");
        assertEquals("/node1", zk2.getDataWatches().get(0), "Didn't find data watcher");
        removeWatches(zk2, "/node1", w1, WatcherType.Any, false, Code.OK, useAsync);
        assertTrue(w2.matches(), "Didn't remove data watcher");
        // closing session should remove ephemeral nodes and trigger data
        // watches if any
        if (zk1 != null) {
            zk1.close();
            zk1 = null;
        }

        List<EventType> events = w2.getEventsAfterWatchRemoval();
        assertEquals(0, events.size(), "Shouldn't get NodeDeletedEvent after watch removal");
    }

    /**
     * Test verifies removal of multiple child watchers when there is server
     * connection
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testMultipleChildWatchers(boolean useAsync) throws IOException, InterruptedException, KeeperException {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        MyWatcher w1 = new MyWatcher("/node1", 1);
        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
        zk2.getChildren("/node1", w1);
        MyWatcher w2 = new MyWatcher("/node1", 1);
        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
        zk2.getChildren("/node1", w2);
        removeWatches(zk2, "/node1", w2, WatcherType.Children, false, Code.OK, useAsync);
        assertTrue(w2.matches(), "Didn't remove child watcher");
        assertEquals(1, zk2.getChildWatches().size(), "Didn't find child watcher");
        removeWatches(zk2, "/node1", w1, WatcherType.Any, false, Code.OK, useAsync);
        assertTrue(w1.matches(), "Didn't remove child watcher");
        // create child to see NodeChildren notification
        zk1.create("/node1/node2", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        // waiting for child watchers to be notified
        int count = 30;
        while (count > 0) {
            if (w1.getEventsAfterWatchRemoval().size() > 0) {
                break;
            }
            count--;
            Thread.sleep(100);
        }
        // watcher2
        List<EventType> events = w2.getEventsAfterWatchRemoval();
        assertEquals(0, events.size(), "Shouldn't get NodeChildrenChanged event");
    }

    /**
     * Test verifies null watcher with WatcherType.Any - remove all the watchers
     * data, child, exists
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testRemoveAllWatchers(boolean useAsync) throws IOException, InterruptedException, KeeperException {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        MyWatcher w1 = new MyWatcher("/node1", 2);
        MyWatcher w2 = new MyWatcher("/node1", 2);
        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
        assertNotNull(zk2.exists("/node1", w1), "Didn't set data watches");
        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
        assertNotNull(zk2.exists("/node1", w2), "Didn't set data watches");
        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
        zk2.getChildren("/node1", w1);
        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
        zk2.getChildren("/node1", w2);
        removeWatches(zk2, "/node1", w1, WatcherType.Any, false, Code.OK, useAsync);
        removeWatches(zk2, "/node1", w2, WatcherType.Any, false, Code.OK, useAsync);
        zk1.create("/node1/child", null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
        assertTrue(w1.matches(), "Didn't remove data watcher");
        assertTrue(w2.matches(), "Didn't remove child watcher");
    }

    /**
     * Test verifies null watcher with WatcherType.Data - remove all data
     * watchers. Child watchers shouldn't be removed
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testRemoveAllDataWatchers(boolean useAsync) throws IOException, InterruptedException, KeeperException {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        MyWatcher w1 = new MyWatcher("/node1", 1);
        MyWatcher w2 = new MyWatcher("/node1", 1);
        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
        assertNotNull(zk2.exists("/node1", w1), "Didn't set data watches");
        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
        assertNotNull(zk2.exists("/node1", w2), "Didn't set data watches");
        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
        zk2.getChildren("/node1", w1);
        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
        zk2.getChildren("/node1", w2);
        removeWatches(zk2, "/node1", w1, WatcherType.Data, false, Code.OK, useAsync);
        removeWatches(zk2, "/node1", w2, WatcherType.Data, false, Code.OK, useAsync);
        zk1.create("/node1/child", null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
        assertTrue(w1.matches(), "Didn't remove data watcher");
        assertTrue(w2.matches(), "Didn't remove data watcher");
        // waiting for child watchers to be notified
        int count = 10;
        while (count > 0) {
            if (w1.getEventsAfterWatchRemoval().size() > 0 && w2.getEventsAfterWatchRemoval().size() > 0) {
                break;
            }
            count--;
            Thread.sleep(1000);
        }
        // watcher1
        List<EventType> events = w1.getEventsAfterWatchRemoval();
        assertEquals(1, events.size(), "Didn't get NodeChildrenChanged event");
        assertTrue(events.contains(EventType.NodeChildrenChanged), "Didn't get NodeChildrenChanged event");
        // watcher2
        events = w2.getEventsAfterWatchRemoval();
        assertEquals(1, events.size(), "Didn't get NodeChildrenChanged event");
        assertTrue(events.contains(EventType.NodeChildrenChanged), "Didn't get NodeChildrenChanged event");
    }

    /**
     * Test verifies null watcher with WatcherType.Children - remove all child
     * watchers. Data watchers shouldn't be removed
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testRemoveAllChildWatchers(boolean useAsync) throws IOException, InterruptedException, KeeperException {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        MyWatcher w1 = new MyWatcher("/node1", 1);
        MyWatcher w2 = new MyWatcher("/node1", 1);
        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
        assertNotNull(zk2.exists("/node1", w1), "Didn't set data watches");
        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
        assertNotNull(zk2.exists("/node1", w2), "Didn't set data watches");
        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
        zk2.getChildren("/node1", w1);
        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
        zk2.getChildren("/node1", w2);
        removeWatches(zk2, "/node1", w1, WatcherType.Children, false, Code.OK, useAsync);
        removeWatches(zk2, "/node1", w2, WatcherType.Children, false, Code.OK, useAsync);
        zk1.setData("/node1", "test".getBytes(), -1);
        assertTrue(w1.matches(), "Didn't remove child watcher");
        assertTrue(w2.matches(), "Didn't remove child watcher");
        // waiting for child watchers to be notified
        int count = 10;
        while (count > 0) {
            if (w1.getEventsAfterWatchRemoval().size() > 0 && w2.getEventsAfterWatchRemoval().size() > 0) {
                break;
            }
            count--;
            Thread.sleep(1000);
        }
        // watcher1
        List<EventType> events = w1.getEventsAfterWatchRemoval();
        assertEquals(1, events.size(), "Didn't get NodeDataChanged event");
        assertTrue(events.contains(EventType.NodeDataChanged), "Didn't get NodeDataChanged event");
        // watcher2
        events = w2.getEventsAfterWatchRemoval();
        assertEquals(1, events.size(), "Didn't get NodeDataChanged event");
        assertTrue(events.contains(EventType.NodeDataChanged), "Didn't get NodeDataChanged event");
    }

    /**
     * Test verifies given watcher doesn't exists!
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testNoWatcherException(boolean useAsync) throws IOException, InterruptedException, KeeperException {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        MyWatcher w1 = new MyWatcher("/node1", 2);
        MyWatcher w2 = new MyWatcher("/node1", 2);
        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
        assertNotNull(zk2.exists("/node1", w1), "Didn't set data watches");
        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
        assertNull(zk2.exists("/node2", w2), "Didn't set data watches");
        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
        zk2.getChildren("/node1", w1);
        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
        zk2.getChildren("/node1", w2);

        // New Watcher which will be used for removal
        MyWatcher w3 = new MyWatcher("/node1", 2);

        try {
            removeWatches(zk2, "/node1", w3, WatcherType.Any, false, Code.NOWATCHER, useAsync);
            fail("Should throw exception as given watcher doesn't exists");
        } catch (KeeperException.NoWatcherException nwe) {
            // expected
        }
        try {
            removeWatches(zk2, "/node1", w3, WatcherType.Children, false, Code.NOWATCHER, useAsync);
            fail("Should throw exception as given watcher doesn't exists");
        } catch (KeeperException.NoWatcherException nwe) {
            // expected
        }
        try {
            removeWatches(zk2, "/node1", w3, WatcherType.Data, false, Code.NOWATCHER, useAsync);
            fail("Should throw exception as given watcher doesn't exists");
        } catch (KeeperException.NoWatcherException nwe) {
            // expected
        }
        try {
            removeWatches(zk2, "/nonexists", w3, WatcherType.Data, false, Code.NOWATCHER, useAsync);
            fail("Should throw exception as given watcher doesn't exists");
        } catch (KeeperException.NoWatcherException nwe) {
            // expected
        }
    }

    /**
     * Test verifies WatcherType.Any - removes only the configured data watcher
     * function
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testRemoveAnyDataWatcher(boolean useAsync) throws Exception {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        MyWatcher w1 = new MyWatcher("/node1", 1);
        MyWatcher w2 = new MyWatcher("/node1", 2);
        // Add multiple data watches
        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
        assertNotNull(zk2.exists("/node1", w1), "Didn't set data watches");
        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
        assertNotNull(zk2.exists("/node1", w2), "Didn't set data watches");
        // Add child watch
        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
        zk2.getChildren("/node1", w2);
        removeWatches(zk2, "/node1", w1, WatcherType.Any, false, Code.OK, useAsync);
        assertTrue(w1.matches(), "Didn't remove data watcher");
        assertEquals(1, zk2.getChildWatches().size(), "Didn't find child watcher");
        assertEquals(1, zk2.getDataWatches().size(), "Didn't find data watcher");
        removeWatches(zk2, "/node1", w2, WatcherType.Any, false, Code.OK, useAsync);
        assertTrue(w2.matches(), "Didn't remove child watcher");
    }

    /**
     * Test verifies WatcherType.Any - removes only the configured child watcher
     * function
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testRemoveAnyChildWatcher(boolean useAsync) throws Exception {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        MyWatcher w1 = new MyWatcher("/node1", 2);
        MyWatcher w2 = new MyWatcher("/node1", 1);
        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
        assertNotNull(zk2.exists("/node1", w1), "Didn't set data watches");
        // Add multiple child watches
        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
        zk2.getChildren("/node1", w2);
        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
        zk2.getChildren("/node1", w1);
        removeWatches(zk2, "/node1", w2, WatcherType.Any, false, Code.OK, useAsync);
        assertTrue(w2.matches(), "Didn't remove child watcher");
        assertEquals(1, zk2.getChildWatches().size(), "Didn't find child watcher");
        assertEquals(1, zk2.getDataWatches().size(), "Didn't find data watcher");
        removeWatches(zk2, "/node1", w1, WatcherType.Any, false, Code.OK, useAsync);
        assertTrue(w1.matches(), "Didn't remove watchers");
    }

    /**
     * Test verifies when there is no server connection. Remove watches when
     * local=true, otw should retain it
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testRemoveWatcherWhenNoConnection(boolean useAsync) throws Exception {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        MyWatcher w1 = new MyWatcher("/node1", 2);
        MyWatcher w2 = new MyWatcher("/node1", 1);
        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
        assertNotNull(zk2.exists("/node1", w1), "Didn't set data watches");
        // Add multiple child watches
        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
        zk2.getChildren("/node1", w1);
        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
        zk2.getChildren("/node1", w2);
        stopServer();
        removeWatches(zk2, "/node1", w2, WatcherType.Any, true, Code.OK, useAsync);
        assertTrue(w2.matches(), "Didn't remove child watcher");
        assertFalse(w1.matches(), "Shouldn't remove data watcher");
        try {
            removeWatches(zk2, "/node1", w1, WatcherType.Any, false, Code.CONNECTIONLOSS, useAsync);
            fail("Should throw exception as last watch removal requires server connection");
        } catch (KeeperException.ConnectionLossException nwe) {
            // expected
        }
        assertFalse(w1.matches(), "Shouldn't remove data watcher");

        // when local=true, here if connection not available, simply removes
        // from local session
        removeWatches(zk2, "/node1", w1, WatcherType.Any, true, Code.OK, useAsync);
        assertTrue(w1.matches(), "Didn't remove data watcher");
    }

    /**
     * Test verifies many pre-node watchers. Also, verifies internal
     * datastructure 'watchManager.existWatches'
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testManyPreNodeWatchers(boolean useAsync) throws Exception {
        int count = 50;
        List<MyWatcher> wList = new ArrayList<MyWatcher>(count);
        MyWatcher w;
        String path = "/node";
        // Exists watcher
        for (int i = 0; i < count; i++) {
            final String nodePath = path + i;
            w = new MyWatcher(nodePath, 1);
            wList.add(w);
            LOG.info("Adding pre node watcher {} on path {}", w, nodePath);
            zk1.exists(nodePath, w);
        }
        assertEquals(count, zk1.getExistWatches().size(), "Failed to add watchers!");
        for (int i = 0; i < count; i++) {
            final MyWatcher watcher = wList.get(i);
            removeWatches(zk1, path + i, watcher, WatcherType.Data, false, Code.OK, useAsync);
            assertTrue(watcher.matches(), "Didn't remove data watcher");
        }
        assertEquals(0, zk1.getExistWatches().size(), "Didn't remove watch references!");
    }

    /**
     * Test verifies many child watchers. Also, verifies internal datastructure
     * 'watchManager.childWatches'
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testManyChildWatchers(boolean useAsync) throws Exception {
        int count = 50;
        List<MyWatcher> wList = new ArrayList<MyWatcher>(count);
        MyWatcher w;
        String path = "/node";

        // Child watcher
        for (int i = 0; i < count; i++) {
            String nodePath = path + i;
            zk1.create(nodePath, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
            nodePath += "/";
        }
        for (int i = 0; i < count; i++) {
            String nodePath = path + i;
            w = new MyWatcher(path + i, 1);
            wList.add(w);
            LOG.info("Adding child watcher {} on path {}", w, nodePath);
            zk1.getChildren(nodePath, w);
            nodePath += "/";
        }
        assertEquals(count, zk1.getChildWatches().size(), "Failed to add watchers!");
        for (int i = 0; i < count; i++) {
            final MyWatcher watcher = wList.get(i);
            removeWatches(zk1, path + i, watcher, WatcherType.Children, false, Code.OK, useAsync);
            assertTrue(watcher.matches(), "Didn't remove child watcher");
        }
        assertEquals(0, zk1.getChildWatches().size(), "Didn't remove watch references!");
    }

    /**
     * Test verifies many data watchers. Also, verifies internal datastructure
     * 'watchManager.dataWatches'
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testManyDataWatchers(boolean useAsync) throws Exception {
        int count = 50;
        List<MyWatcher> wList = new ArrayList<MyWatcher>(count);
        MyWatcher w;
        String path = "/node";

        // Data watcher
        for (int i = 0; i < count; i++) {
            String nodePath = path + i;
            w = new MyWatcher(path + i, 1);
            wList.add(w);
            zk1.create(nodePath, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
            LOG.info("Adding data watcher {} on path {}", w, nodePath);
            zk1.getData(nodePath, w, null);
            nodePath += "/";
        }
        assertEquals(count, zk1.getDataWatches().size(), "Failed to add watchers!");
        for (int i = 0; i < count; i++) {
            final MyWatcher watcher = wList.get(i);
            removeWatches(zk1, path + i, watcher, WatcherType.Data, false, Code.OK, useAsync);
            assertTrue(watcher.matches(), "Didn't remove data watcher");
        }
        assertEquals(0, zk1.getDataWatches().size(), "Didn't remove watch references!");
    }

    /**
     * Test verifies removal of many watchers locally when no connection and
     * WatcherType#Any. Also, verifies internal watchManager datastructures
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testManyWatchersWhenNoConnection(boolean useAsync) throws Exception {
        int count = 3;
        List<MyWatcher> wList = new ArrayList<MyWatcher>(count);
        MyWatcher w;
        String path = "/node";

        // Child watcher
        for (int i = 0; i < count; i++) {
            String nodePath = path + i;
            zk1.create(nodePath, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
            nodePath += "/";
        }
        for (int i = 0; i < count; i++) {
            String nodePath = path + i;
            w = new MyWatcher(path + i, 2);
            wList.add(w);
            LOG.info("Adding child watcher {} on path {}", w, nodePath);
            zk1.getChildren(nodePath, w);
            nodePath += "/";
        }
        assertEquals(count, zk1.getChildWatches().size(), "Failed to add watchers!");

        // Data watcher
        for (int i = 0; i < count; i++) {
            String nodePath = path + i;
            w = wList.get(i);
            LOG.info("Adding data watcher {} on path {}", w, nodePath);
            zk1.getData(nodePath, w, null);
            nodePath += "/";
        }
        assertEquals(count, zk1.getDataWatches().size(), "Failed to add watchers!");
        stopServer();
        for (int i = 0; i < count; i++) {
            final MyWatcher watcher = wList.get(i);
            removeWatches(zk1, path + i, watcher, WatcherType.Any, true, Code.OK, useAsync);
            assertTrue(watcher.matches(), "Didn't remove watcher");
        }
        assertEquals(0, zk1.getChildWatches().size(), "Didn't remove watch references!");
        assertEquals(0, zk1.getDataWatches().size(), "Didn't remove watch references!");
    }

    /**
     * Test verifies removing watcher having namespace
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testChRootRemoveWatcher(boolean useAsync) throws Exception {
        // creating the subtree for chRoot clients.
        String chRoot = "/appsX";
        zk1.create("/appsX", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        if (zk1 != null) {
            zk1.close();
        }
        if (zk2 != null) {
            zk2.close();
        }
        // Creating chRoot client.
        zk1 = createClient(this.hostPort + chRoot);
        zk2 = createClient(this.hostPort + chRoot);

        LOG.info("Creating child znode /node1 using chRoot client");
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        MyWatcher w1 = new MyWatcher("/node1", 2);
        MyWatcher w2 = new MyWatcher("/node1", 1);
        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
        assertNotNull(zk2.exists("/node1", w1), "Didn't set data watches");
        // Add multiple child watches
        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
        zk2.getChildren("/node1", w2);
        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
        zk2.getChildren("/node1", w1);
        removeWatches(zk2, "/node1", w1, WatcherType.Any, false, Code.OK, useAsync);
        assertTrue(w1.matches(), "Didn't remove child watcher");
        assertEquals(1, zk2.getChildWatches().size(), "Didn't find child watcher");
        removeWatches(zk2, "/node1", w2, WatcherType.Any, false, Code.OK, useAsync);
        assertTrue(w2.matches(), "Didn't remove child watcher");
    }

    /**
     * Verify that if a given watcher doesn't exist, the server properly
     * returns an error code for it.
     *
     * In our Java client implementation, we check that a given watch exists at
     * two points:
     *
     * 1) before submitting the RemoveWatches request
     * 2) after a successful server response, when the watcher needs to be
     *    removed
     *
     * Since this can be racy (i.e. a watch can fire while a RemoveWatches
     * request is in-flight), we need to verify that the watch was actually
     * removed (i.e. from ZKDatabase and DataTree) and return NOWATCHER if
     * needed.
     *
     * Also, other implementations might not do a client side check before
     * submitting a RemoveWatches request. If we don't do a server side check,
     * we would just return ZOK even if no watch was removed.
     *
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testNoWatcherServerException(boolean useAsync) throws InterruptedException, IOException, TimeoutException {
        CountdownWatcher watcher = new CountdownWatcher();
        ZooKeeper zk = spy(new ZooKeeper(hostPort, CONNECTION_TIMEOUT, watcher));
        MyWatchManager watchManager = new MyWatchManager(false, watcher);
        doReturn(watchManager).when(zk).getWatchManager();
        boolean nw = false;

        watcher.waitForConnected(CONNECTION_TIMEOUT);

        try {
            zk.removeWatches("/nowatchhere", watcher, WatcherType.Data, false);
        } catch (KeeperException nwe) {
            if (nwe.code().intValue() == Code.NOWATCHER.intValue()) {
                nw = true;
            }
        }

        assertThat("Server didn't return NOWATCHER", watchManager.lastReturnCode, is(Code.NOWATCHER.intValue()));
        assertThat("NoWatcherException didn't happen", nw, is(true));
    }

    /**
     * Test verifies given watcher doesn't exists!
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testRemoveAllNoWatcherException(boolean useAsync) throws IOException, InterruptedException, KeeperException {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        try {
            removeAllWatches(zk2, "/node1", WatcherType.Any, false, Code.NOWATCHER, useAsync);
            fail("Should throw exception as given watcher doesn't exists");
        } catch (KeeperException.NoWatcherException nwe) {
            // expected
        }
    }

    /**
     * Test verifies null watcher
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 30)
    public void testNullWatcherReference(boolean useAsync) throws Exception {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        try {
            if (useAsync) {
                zk1.removeWatches("/node1", null, WatcherType.Data, false, null, null);
            } else {
                zk1.removeWatches("/node1", null, WatcherType.Data, false);
            }
            fail("Must throw IllegalArgumentException as watcher is null!");
        } catch (IllegalArgumentException iae) {
            // expected
        }
    }

    /**
     * Test verifies WatcherType.Data - removes only the configured data watcher
     * function
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testRemoveWhenMultipleDataWatchesOnAPath(boolean useAsync) throws Exception {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        final CountDownLatch dataWatchCount = new CountDownLatch(1);
        final CountDownLatch rmWatchCount = new CountDownLatch(1);
        Watcher w1 = event -> {
            if (event.getType() == EventType.DataWatchRemoved) {
                rmWatchCount.countDown();
            }
        };
        Watcher w2 = event -> {
            if (event.getType() == EventType.NodeDataChanged) {
                dataWatchCount.countDown();
            }
        };
        // Add multiple data watches
        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
        assertNotNull(zk2.exists("/node1", w1), "Didn't set data watches");
        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
        assertNotNull(zk2.exists("/node1", w2), "Didn't set data watches");

        removeWatches(zk2, "/node1", w1, WatcherType.Data, false, Code.OK, useAsync);
        assertTrue(rmWatchCount.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS), "Didn't remove data watcher");

        zk1.setData("/node1", "test".getBytes(), -1);
        LOG.info("Waiting for data watchers to be notified");
        assertTrue(dataWatchCount.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS), "Didn't get data watch notification!");
    }

    /**
     * Test verifies WatcherType.Children - removes only the configured child
     * watcher function
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testRemoveWhenMultipleChildWatchesOnAPath(boolean useAsync) throws Exception {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        final CountDownLatch childWatchCount = new CountDownLatch(1);
        final CountDownLatch rmWatchCount = new CountDownLatch(1);
        Watcher w1 = event -> {
            if (event.getType() == EventType.ChildWatchRemoved) {
                rmWatchCount.countDown();
            }
        };
        Watcher w2 = event -> {
            if (event.getType() == EventType.NodeChildrenChanged) {
                childWatchCount.countDown();
            }
        };
        // Add multiple child watches
        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
        assertEquals(0, zk2.getChildren("/node1", w1).size(), "Didn't set child watches");
        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
        assertEquals(0, zk2.getChildren("/node1", w2).size(), "Didn't set child watches");

        removeWatches(zk2, "/node1", w1, WatcherType.Children, false, Code.OK, useAsync);
        assertTrue(rmWatchCount.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS), "Didn't remove child watcher");

        zk1.create("/node1/node2", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        LOG.info("Waiting for child watchers to be notified");
        assertTrue(childWatchCount.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS), "Didn't get child watch notification!");
    }

    /**
     * Test verifies WatcherType.Data - removes only the configured data watcher
     * function
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testRemoveAllDataWatchesOnAPath(boolean useAsync) throws Exception {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        final CountDownLatch dWatchCount = new CountDownLatch(2);
        final CountDownLatch rmWatchCount = new CountDownLatch(2);
        Watcher w1 = event -> {
            switch (event.getType()) {
            case DataWatchRemoved:
                rmWatchCount.countDown();
                break;
            case NodeDataChanged:
                dWatchCount.countDown();
                break;
            default:
                break;
            }
        };
        Watcher w2 = event -> {
            switch (event.getType()) {
            case DataWatchRemoved:
                rmWatchCount.countDown();
                break;
            case NodeDataChanged:
                dWatchCount.countDown();
                break;
            default:
                break;
            }
        };
        // Add multiple data watches
        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
        assertNotNull(zk2.exists("/node1", w1), "Didn't set data watches");
        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
        assertNotNull(zk2.exists("/node1", w2), "Didn't set data watches");

        assertTrue(isServerSessionWatcher(zk2.getSessionId(), "/node1", WatcherType.Data), "Server session is not a watcher");
        removeAllWatches(zk2, "/node1", WatcherType.Data, false, Code.OK, useAsync);
        assertTrue(rmWatchCount.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS), "Didn't remove data watcher");

        assertFalse(isServerSessionWatcher(zk2.getSessionId(), "/node1", WatcherType.Data), "Server session is still a watcher after removal");
    }

    /**
     * Test verifies WatcherType.Children - removes only the configured child
     * watcher function
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testRemoveAllChildWatchesOnAPath(boolean useAsync) throws Exception {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        final CountDownLatch cWatchCount = new CountDownLatch(2);
        final CountDownLatch rmWatchCount = new CountDownLatch(2);
        Watcher w1 = event -> {
            switch (event.getType()) {
            case ChildWatchRemoved:
                rmWatchCount.countDown();
                break;
            case NodeChildrenChanged:
                cWatchCount.countDown();
                break;
            default:
                break;
            }
        };
        Watcher w2 = event -> {
            switch (event.getType()) {
            case ChildWatchRemoved:
                rmWatchCount.countDown();
                break;
            case NodeChildrenChanged:
                cWatchCount.countDown();
                break;
            default:
                break;
            }
        };
        // Add multiple child watches
        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
        assertEquals(0, zk2.getChildren("/node1", w1).size(), "Didn't set child watches");
        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
        assertEquals(0, zk2.getChildren("/node1", w2).size(), "Didn't set child watches");

        assertTrue(isServerSessionWatcher(zk2.getSessionId(), "/node1", WatcherType.Children), "Server session is not a watcher");
        removeAllWatches(zk2, "/node1", WatcherType.Children, false, Code.OK, useAsync);
        assertTrue(rmWatchCount.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS), "Didn't remove child watcher");

        assertFalse(isServerSessionWatcher(zk2.getSessionId(), "/node1", WatcherType.Children), "Server session is still a watcher after removal");
    }

    /**
     * Test verifies WatcherType.Any - removes all the configured child,data
     * watcher functions
     */
    @ParameterizedTest
    @ValueSource(booleans = {true, false})
    @Timeout(value = 90)
    public void testRemoveAllWatchesOnAPath(boolean useAsync) throws Exception {
        zk1.create("/node1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        final CountDownLatch watchCount = new CountDownLatch(2);
        final CountDownLatch rmWatchCount = new CountDownLatch(4);
        Watcher w1 = event -> {
            switch (event.getType()) {
            case ChildWatchRemoved:
            case DataWatchRemoved:
                rmWatchCount.countDown();
                break;
            case NodeChildrenChanged:
            case NodeDataChanged:
                watchCount.countDown();
                break;
            default:
                break;
            }
        };
        Watcher w2 = event -> {
            switch (event.getType()) {
            case ChildWatchRemoved:
            case DataWatchRemoved:
                rmWatchCount.countDown();
                break;
            case NodeChildrenChanged:
            case NodeDataChanged:
                watchCount.countDown();
                break;
            default:
                break;
            }
        };
        // Add multiple child watches
        LOG.info("Adding child watcher {} on path {}", w1, "/node1");
        assertEquals(0, zk2.getChildren("/node1", w1).size(), "Didn't set child watches");
        LOG.info("Adding child watcher {} on path {}", w2, "/node1");
        assertEquals(0, zk2.getChildren("/node1", w2).size(), "Didn't set child watches");

        // Add multiple data watches
        LOG.info("Adding data watcher {} on path {}", w1, "/node1");
        assertNotNull(zk2.exists("/node1", w1), "Didn't set data watches");
        LOG.info("Adding data watcher {} on path {}", w2, "/node1");
        assertNotNull(zk2.exists("/node1", w2), "Didn't set data watches");

        assertTrue(isServerSessionWatcher(zk2.getSessionId(), "/node1", WatcherType.Data), "Server session is not a watcher");
        removeAllWatches(zk2, "/node1", WatcherType.Any, false, Code.OK, useAsync);
        assertTrue(rmWatchCount.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS), "Didn't remove data watcher");
        assertFalse(isServerSessionWatcher(zk2.getSessionId(), "/node1", WatcherType.Data), "Server session is still a watcher after removal");
        assertEquals(2, watchCount.getCount(), "Received watch notification after removal!");
    }

    private static class MyWatchManager extends ZKWatchManager {

        int lastReturnCode;

        MyWatchManager(boolean disableAutoWatchReset, Watcher defaultWatcher) {
            super(disableAutoWatchReset, defaultWatcher);
        }

        void containsWatcher(String path, Watcher watcher, WatcherType watcherType) {
            // prevent contains watcher
        }

        @Override
        protected boolean removeWatches(
            Map<String, Set<Watcher>> pathVsWatcher,
            Watcher watcher,
            String path,
            boolean local,
            int rc,
            Set<Watcher> removedWatchers) {
            lastReturnCode = rc;
            return false;
        }
    }

    private static class MyWatcher implements Watcher {

        private final String path;
        private String eventPath;
        private CountDownLatch latch;
        private List<EventType> eventsAfterWatchRemoval = new ArrayList<EventType>();
        MyWatcher(String path, int count) {
            this.path = path;
            latch = new CountDownLatch(count);
        }

        public void process(WatchedEvent event) {
            LOG.debug("Event path : {}, eventPath : {}", path, event.getPath());
            this.eventPath = event.getPath();
            // notifies watcher removal
            if (latch.getCount() == 0) {
                if (event.getType() != EventType.None) {
                    eventsAfterWatchRemoval.add(event.getType());
                }
            }
            if (event.getType() == EventType.ChildWatchRemoved || event.getType() == EventType.DataWatchRemoved) {
                latch.countDown();
            }
        }

        /**
         * Returns true if the watcher was triggered.  Try to avoid using this
         * method with assertFalse statements.  A false return depends on a timed
         * out wait on a latch, which makes tests run long.
         *
         * @return true if the watcher was triggered, false otherwise
         * @throws InterruptedException if interrupted while waiting on latch
         */
        public boolean matches() throws InterruptedException {
            if (!latch.await(CONNECTION_TIMEOUT / 5, TimeUnit.MILLISECONDS)) {
                LOG.error("Failed waiting to remove the watches");
                return false;
            }
            LOG.debug("Client path : {} eventPath : {}", path, eventPath);
            return path.equals(eventPath);
        }

        public List<EventType> getEventsAfterWatchRemoval() {
            return eventsAfterWatchRemoval;
        }

    }

    private class MyCallback implements AsyncCallback.VoidCallback {

        private final String path;
        private final int rc;
        private String eventPath;
        int eventRc;
        private CountDownLatch latch = new CountDownLatch(1);

        public MyCallback(int rc, String path) {
            this.rc = rc;
            this.path = path;
        }

        @Override
        public void processResult(int rc, String eventPath, Object ctx) {
            System.out.println("latch:" + path + " " + eventPath);
            this.eventPath = eventPath;
            this.eventRc = rc;
            this.latch.countDown();
        }

        /**
         * Returns true if the callback was triggered.  Try to avoid using this
         * method with assertFalse statements.  A false return depends on a timed
         * out wait on a latch, which makes tests run long.
         *
         * @return true if the watcher was triggered, false otherwise
         * @throws InterruptedException if interrupted while waiting on latch
         */
        public boolean matches() throws InterruptedException {
            if (!latch.await(CONNECTION_TIMEOUT / 5, TimeUnit.MILLISECONDS)) {
                return false;
            }
            return path.equals(eventPath) && rc == eventRc;
        }

    }

    /**
     * Checks if a session is registered with the server as a watcher.
     *
     * @param sessionId the session ID to check
     * @param path the path to check for watchers
     * @param type the type of watcher
     * @return true if the client session is a watcher on path for the type
     */
    private boolean isServerSessionWatcher(long sessionId, String path, WatcherType type) {
        Set<ServerCnxn> cnxns = new HashSet<>();
        CollectionUtils.addAll(cnxns, serverFactory.getConnections().iterator());
        for (ServerCnxn cnxn : cnxns) {
            if (cnxn.getSessionId() == sessionId) {
                return serverFactory.getZooKeeperServer().getZKDatabase().getDataTree().containsWatcher(path, type, cnxn);
            }
        }
        return false;
    }

}