summaryrefslogtreecommitdiff
path: root/zookeeper-server/src/test/java/org/apache/zookeeper/test/WatcherTest.java
blob: 22da89a2e67f0de96ba565e9a4b933e68c63e2fa (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
/*
 * Licensed to the Apache Software Foundation (ASF) under one
 * or more contributor license agreements.  See the NOTICE file
 * distributed with this work for additional information
 * regarding copyright ownership.  The ASF licenses this file
 * to you under the Apache License, Version 2.0 (the
 * "License"); you may not use this file except in compliance
 * with the License.  You may obtain a copy of the License at
 *
 *     http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.zookeeper.test;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import java.io.IOException;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.zookeeper.AsyncCallback.StatCallback;
import org.apache.zookeeper.AsyncCallback.VoidCallback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.TestableZooKeeper;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.Watcher.Event;
import org.apache.zookeeper.Watcher.Event.EventType;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.client.ZKClientConfig;
import org.apache.zookeeper.data.Stat;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class WatcherTest extends ClientBase {

    protected static final Logger LOG = LoggerFactory.getLogger(WatcherTest.class);

    private long timeOfLastWatcherInvocation;

    private static final class MyStatCallback implements StatCallback {

        int rc;
        public void processResult(int rc, String path, Object ctx, Stat stat) {
            ((int[]) ctx)[0]++;
            this.rc = rc;
        }

    }

    private class MyWatcher extends CountdownWatcher {

        LinkedBlockingQueue<WatchedEvent> events = new LinkedBlockingQueue<>();

        public void process(WatchedEvent event) {
            super.process(event);
            if (event.getType() != Event.EventType.None) {
                timeOfLastWatcherInvocation = System.currentTimeMillis();
                try {
                    events.put(event);
                } catch (InterruptedException e) {
                    LOG.warn("ignoring interrupt during event.put");
                }
            }
        }

    }

    @BeforeEach
    public void setUp() throws Exception {
        super.setUp();
        // Reset to default value since some test cases set this to true.
        // Needed for JDK7 since unit test can run is random order
        System.setProperty(ZKClientConfig.DISABLE_AUTO_WATCH_RESET, "false");
    }

    /**
     * Verify that we get all of the events we expect to get. This particular
     * case verifies that we see all of the data events on a particular node.
     * There was a bug (ZOOKEEPER-137) that resulted in events being dropped
     * in some cases (timing).
     *
     * @throws IOException
     * @throws InterruptedException
     * @throws KeeperException
     */
    @Test
    public void testWatcherCorrectness() throws IOException, InterruptedException, KeeperException {
        ZooKeeper zk = null;
        try {
            MyWatcher watcher = new MyWatcher();
            zk = createClient(watcher, hostPort);

            StatCallback scb = new StatCallback() {
                public void processResult(int rc, String path, Object ctx, Stat stat) {
                    // don't do anything
                }
            };
            VoidCallback vcb = new VoidCallback() {
                public void processResult(int rc, String path, Object ctx) {
                    // don't do anything
                }
            };

            String[] names = new String[10];
            for (int i = 0; i < names.length; i++) {
                String name = zk.create("/tc-", "initialvalue".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_SEQUENTIAL);
                names[i] = name;

                Stat stat = new Stat();
                zk.getData(name, watcher, stat);
                zk.setData(name, "new".getBytes(), stat.getVersion(), scb, null);
                stat = zk.exists(name, watcher);
                zk.delete(name, stat.getVersion(), vcb, null);
            }

            for (int i = 0; i < names.length; i++) {
                String name = names[i];
                WatchedEvent event = watcher.events.poll(10, TimeUnit.SECONDS);
                assertEquals(name, event.getPath());
                assertEquals(Event.EventType.NodeDataChanged, event.getType());
                assertEquals(Event.KeeperState.SyncConnected, event.getState());
                event = watcher.events.poll(10, TimeUnit.SECONDS);
                assertEquals(name, event.getPath());
                assertEquals(Event.EventType.NodeDeleted, event.getType());
                assertEquals(Event.KeeperState.SyncConnected, event.getState());
            }
        } finally {
            if (zk != null) {
                zk.close();
            }
        }
    }

    @Test
    public void testWatcherDisconnectOnClose() throws IOException, InterruptedException, KeeperException {
        ZooKeeper zk = null;
        try {
            final BlockingQueue<WatchedEvent> queue = new LinkedBlockingQueue<>();

            MyWatcher connWatcher = new MyWatcher();

            Watcher watcher = event -> {
                try {
                    queue.put(event);
                } catch (InterruptedException e) {
                    // Oh well, never mind
                }
            };

            zk = createClient(connWatcher, hostPort);

            StatCallback scb = new StatCallback() {
                public void processResult(int rc, String path, Object ctx, Stat stat) {
                    // don't do anything
                }
            };

            // Register a watch on the node
            zk.exists("/missing", watcher, scb, null);

            // Close the client without changing the node
            zk.close();

            WatchedEvent event = queue.poll(10, TimeUnit.SECONDS);

            assertNotNull(event, "No watch event was received after closing the Zookeeper client. A 'Closed' event should have occurred");
            assertEquals(Event.EventType.None, event.getType(), "Closed events are not generated by the server, and so should have a type of 'None'");
            assertEquals(Event.KeeperState.Closed, event.getState(), "A 'Closed' event was expected as the Zookeeper client was closed without altering the node it was watching");
        } finally {
            if (zk != null) {
                zk.close();
            }
        }

    }

    @Test
    public void testWatcherCount() throws IOException, InterruptedException, KeeperException {
        ZooKeeper zk1 = null, zk2 = null;
        try {
            MyWatcher w1 = new MyWatcher();
            zk1 = createClient(w1, hostPort);

            MyWatcher w2 = new MyWatcher();
            zk2 = createClient(w2, hostPort);

            Stat stat = new Stat();
            zk1.create("/watch-count-test", "value".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
            zk1.create("/watch-count-test-2", "value".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);

            zk1.getData("/watch-count-test", w1, stat);
            zk1.getData("/watch-count-test-2", w1, stat);
            zk2.getData("/watch-count-test", w2, stat);

            assertEquals(serverFactory.getZooKeeperServer().getZKDatabase().getDataTree().getWatchCount(), 3);

        } finally {
            if (zk1 != null) {
                zk1.close();
            }
            if (zk2 != null) {
                zk2.close();
            }
        }

    }

    static final int COUNT = 100;
    /**
     * This test checks that watches for pending requests do not get triggered,
     * but watches set by previous requests do.
     *
     * @throws Exception
     */
    @Test
    public void testWatchAutoResetWithPending() throws Exception {
        MyWatcher[] watches = new MyWatcher[COUNT];
        MyStatCallback[] cbs = new MyStatCallback[COUNT];
        MyWatcher watcher = new MyWatcher();
        int[] count = new int[1];
        TestableZooKeeper zk = createClient(watcher, hostPort, 6000);
        ZooKeeper zk2 = createClient(watcher, hostPort, 5000);
        zk2.create("/test", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
        for (int i = 0; i < COUNT / 2; i++) {
            watches[i] = new MyWatcher();
            cbs[i] = new MyStatCallback();
            zk.exists("/test", watches[i], cbs[i], count);
        }
        zk.exists("/test", false);
        assertTrue(zk.pauseCnxn(3000), "Failed to pause the connection!");
        zk2.close();
        stopServer();
        watches[0].waitForDisconnected(60000);
        for (int i = COUNT / 2; i < COUNT; i++) {
            watches[i] = new MyWatcher();
            cbs[i] = new MyStatCallback();
            zk.exists("/test", watches[i], cbs[i], count);
        }
        startServer();
        watches[COUNT / 2 - 1].waitForConnected(60000);
        assertEquals(null, zk.exists("/test", false));
        waitForAllWatchers();
        for (int i = 0; i < COUNT / 2; i++) {
            assertEquals(1, watches[i].events.size(), "For " + i);
        }
        for (int i = COUNT / 2; i < COUNT; i++) {
            if (cbs[i].rc == KeeperException.Code.OK.intValue()) {
                assertEquals(1, watches[i].events.size(), "For " + i);
            } else {
                assertEquals(0, watches[i].events.size(), "For " + i);
            }
        }
        assertEquals(COUNT, count[0]);
        zk.close();
    }

    /**
     * Wait until no watcher has been fired in the last second to ensure that all watches
     * that are waiting to be fired have been fired
     * @throws Exception
     */
    private void waitForAllWatchers() throws Exception {
        timeOfLastWatcherInvocation = System.currentTimeMillis();
        while (System.currentTimeMillis() - timeOfLastWatcherInvocation < 1000) {
            Thread.sleep(1000);
        }
    }

    final int TIMEOUT = 5000;

    @Test
    public void testWatcherAutoResetWithGlobal() throws Exception {
        ZooKeeper zk = null;
        MyWatcher watcher = new MyWatcher();
        zk = createClient(watcher, hostPort, TIMEOUT);
        testWatcherAutoReset(zk, watcher, watcher);
        zk.close();
    }

    @Test
    public void testWatcherAutoResetWithLocal() throws Exception {
        ZooKeeper zk = null;
        MyWatcher watcher = new MyWatcher();
        zk = createClient(watcher, hostPort, TIMEOUT);
        testWatcherAutoReset(zk, watcher, new MyWatcher());
        zk.close();
    }

    @Test
    public void testWatcherAutoResetDisabledWithGlobal() throws Exception {
        /**
         * When ZooKeeper is created this property will get used.
         */
        System.setProperty(ZKClientConfig.DISABLE_AUTO_WATCH_RESET, "true");
        testWatcherAutoResetWithGlobal();
    }

    @Test
    public void testWatcherAutoResetDisabledWithLocal() throws Exception {
        System.setProperty(ZKClientConfig.DISABLE_AUTO_WATCH_RESET, "true");
        testWatcherAutoResetWithLocal();
    }

    private void testWatcherAutoReset(ZooKeeper zk, MyWatcher globalWatcher, MyWatcher localWatcher) throws Exception {
        boolean isGlobal = (localWatcher == globalWatcher);
        // First test to see if the watch survives across reconnects
        zk.create("/watchtest", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        zk.create("/watchtest/child", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
        if (isGlobal) {
            zk.getChildren("/watchtest", true);
            zk.getData("/watchtest/child", true, new Stat());
            zk.exists("/watchtest/child2", true);
        } else {
            zk.getChildren("/watchtest", localWatcher);
            zk.getData("/watchtest/child", localWatcher, new Stat());
            zk.exists("/watchtest/child2", localWatcher);
        }

        assertTrue(localWatcher.events.isEmpty());

        stopServer();
        globalWatcher.waitForDisconnected(3000);
        localWatcher.waitForDisconnected(500);
        startServer();
        globalWatcher.waitForConnected(3000);
        boolean disableAutoWatchReset = zk.getClientConfig().getBoolean(ZKClientConfig.DISABLE_AUTO_WATCH_RESET);
        if (!isGlobal && !disableAutoWatchReset) {
            localWatcher.waitForConnected(500);
        }

        assertTrue(localWatcher.events.isEmpty());
        zk.setData("/watchtest/child", new byte[1], -1);
        zk.create("/watchtest/child2", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);

        WatchedEvent e;
        if (!disableAutoWatchReset) {
            e = localWatcher.events.poll(TIMEOUT, TimeUnit.MILLISECONDS);
            assertEquals(EventType.NodeDataChanged, e.getType(), e.getPath());
            assertEquals("/watchtest/child", e.getPath());
        } else {
            // we'll catch this later if it does happen after timeout, so
            // why waste the time on poll
        }

        if (!disableAutoWatchReset) {
            e = localWatcher.events.poll(TIMEOUT, TimeUnit.MILLISECONDS);
            // The create will trigger the get children and the exist
            // watches
            assertEquals(EventType.NodeCreated, e.getType());
            assertEquals("/watchtest/child2", e.getPath());
        } else {
            // we'll catch this later if it does happen after timeout, so
            // why waste the time on poll
        }

        if (!disableAutoWatchReset) {
            e = localWatcher.events.poll(TIMEOUT, TimeUnit.MILLISECONDS);
            assertEquals(EventType.NodeChildrenChanged, e.getType());
            assertEquals("/watchtest", e.getPath());
        } else {
            // we'll catch this later if it does happen after timeout, so
            // why waste the time on poll
        }

        assertTrue(localWatcher.events.isEmpty()); // ensure no late arrivals
        stopServer();
        globalWatcher.waitForDisconnected(TIMEOUT);
        try {
            try {
                localWatcher.waitForDisconnected(500);
                if (!isGlobal && !disableAutoWatchReset) {
                    fail("Got an event when I shouldn't have");
                }
            } catch (TimeoutException toe) {
                if (disableAutoWatchReset) {
                    fail("Didn't get an event when I should have");
                }
                // Else what we are expecting since there are no outstanding watches
            }
        } catch (Exception e1) {
            LOG.error("bad", e1);
            throw new RuntimeException(e1);
        }
        startServer();
        globalWatcher.waitForConnected(TIMEOUT);

        if (isGlobal) {
            zk.getChildren("/watchtest", true);
            zk.getData("/watchtest/child", true, new Stat());
            zk.exists("/watchtest/child2", true);
        } else {
            zk.getChildren("/watchtest", localWatcher);
            zk.getData("/watchtest/child", localWatcher, new Stat());
            zk.exists("/watchtest/child2", localWatcher);
        }

        // Do trigger an event to make sure that we do not get
        // it later
        zk.delete("/watchtest/child2", -1);

        e = localWatcher.events.poll(TIMEOUT, TimeUnit.MILLISECONDS);
        assertEquals(EventType.NodeDeleted, e.getType());
        assertEquals("/watchtest/child2", e.getPath());

        e = localWatcher.events.poll(TIMEOUT, TimeUnit.MILLISECONDS);
        assertEquals(EventType.NodeChildrenChanged, e.getType());
        assertEquals("/watchtest", e.getPath());

        assertTrue(localWatcher.events.isEmpty());

        stopServer();
        globalWatcher.waitForDisconnected(TIMEOUT);
        localWatcher.waitForDisconnected(500);
        startServer();
        globalWatcher.waitForConnected(TIMEOUT);
        if (!isGlobal && !disableAutoWatchReset) {
            localWatcher.waitForConnected(500);
        }

        zk.delete("/watchtest/child", -1);
        zk.delete("/watchtest", -1);

        if (!disableAutoWatchReset) {
            e = localWatcher.events.poll(TIMEOUT, TimeUnit.MILLISECONDS);
            assertEquals(EventType.NodeDeleted, e.getType());
            assertEquals("/watchtest/child", e.getPath());
        } else {
            // we'll catch this later if it does happen after timeout, so
            // why waste the time on poll
        }

        // Make sure nothing is straggling!
        Thread.sleep(1000);
        assertTrue(localWatcher.events.isEmpty());
    }

}