summaryrefslogtreecommitdiff
path: root/_released_docs/r3.8.1/zookeeperTutorial.html
blob: e3dee65e04ae71eade30f23b27dad25acf7c042d (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

<!DOCTYPE html>
<html>
<head>
    <META http-equiv="Content-Type" content="text/html; charset=UTF-8">
    <title>ZooKeeper: Because Coordinating Distributed Systems is a Zoo</title>
    <link type="text/css" href="skin/basic.css" rel="stylesheet">
    <link media="screen" type="text/css" href="skin/screen.css" rel="stylesheet">
    <link media="print" type="text/css" href="skin/print.css" rel="stylesheet">
    <link type="text/css" href="skin/profile.css" rel="stylesheet">
    <script src="skin/getBlank.js" language="javascript" type="text/javascript"></script>
    <script src="skin/getMenu.js" language="javascript" type="text/javascript"></script>
    <script src="skin/init.js" language="javascript" type="text/javascript"></script>
    <link rel="shortcut icon" href="images/favicon.ico">
</head>
<body onload="init();">
<div id="top">
    <div class="breadtrail">
        <a href="http://www.apache.org/">Apache</a> &gt; <a href="http://zookeeper.apache.org/">ZooKeeper</a>
    </div>
    <div class="header">
        <div class="projectlogo">
            <a href="http://zookeeper.apache.org/"><img class="logoImage" alt="ZooKeeper" src="images/zookeeper_small.gif" title="ZooKeeper: distributed coordination"></a>
        </div>
        <div class="searchbox">
            <form action="http://www.google.com/search" method="get">
                <input value="zookeeper.apache.org" name="sitesearch" type="hidden"><input onFocus="getBlank (this, 'Search the site with google');" size="25" name="q" id="query" type="text" value="Search the site with google">&nbsp;
                <input name="Search" value="Search" type="submit">
            </form>
        </div>
        <ul id="tabs">
            <li>
                <a class="unselected" href="http://zookeeper.apache.org/">Project</a>
            </li>
            <li>
                <a class="unselected" href="https://cwiki.apache.org/confluence/display/ZOOKEEPER/">Wiki</a>
            </li>
            <li class="current">
                <a class="selected" href="index.html">ZooKeeper 3.8 Documentation</a>
            </li>
        </ul>
    </div>
</div>
<div id="main">
    <div id="publishedStrip">
        <div id="level2tabs"></div>
        <script type="text/javascript"><!--
document.write("Last Published: " + document.lastModified);
//  --></script>
    </div>
    <div class="breadtrail">
        &nbsp;
    </div>
    <div id="menu">
        <div onclick="SwitchMenu('menu_1', 'skin/')" id="menu_1Title" class="menutitle">Overview</div>
        <div id="menu_1" class="menuitemgroup">
            <div class="menuitem">
                <a href="index.html">Welcome</a>
            </div>
            <div class="menuitem">
                <a href="zookeeperOver.html">Overview</a>
            </div>
            <div class="menuitem">
                <a href="zookeeperStarted.html">Getting Started</a>
            </div>
            <div class="menuitem">
                <a href="releasenotes.html">Release Notes</a>
            </div>
        </div>
        <div onclick="SwitchMenu('menu_2', 'skin/')" id="menu_2Title" class="menutitle">Developer</div>
        <div id="menu_2" class="menuitemgroup">
            <div class="menuitem">
                <a href="apidocs/zookeeper-server/index.html">API Docs</a>
            </div>
            <div class="menuitem">
                <a href="zookeeperProgrammers.html">Programmer's Guide</a>
            </div>
            <div class="menuitem">
                <a href="zookeeperUseCases.html">Use Cases</a>
            </div>
            <div class="menuitem">
                <a href="javaExample.html">Java Example</a>
            </div>
            <div class="menuitem">
                <a href="zookeeperTutorial.html">Barrier and Queue Tutorial</a>
            </div>
            <div class="menuitem">
                <a href="recipes.html">Recipes</a>
            </div>
        </div>
        <div onclick="SwitchMenu('menu_3', 'skin/')" id="menu_3Title" class="menutitle">Admin &amp; Ops</div>
        <div id="menu_3" class="menuitemgroup">
            <div class="menuitem">
                <a href="zookeeperAdmin.html">Administrator's Guide</a>
            </div>
            <div class="menuitem">
                <a href="zookeeperQuotas.html">Quota Guide</a>
            </div>
            <div class="menuitem">
                <a href="zookeeperJMX.html">JMX</a>
            </div>
            <div class="menuitem">
                <a href="zookeeperHierarchicalQuorums.html">Hierarchical Quorums</a>
            </div>
            <div class="menuitem">
                <a href="zookeeperOracleQuorums.html">Oracle Quorum</a>
            </div>
            <div class="menuitem">
                <a href="zookeeperObservers.html">Observers Guide</a>
            </div>
            <div class="menuitem">
                <a href="zookeeperReconfig.html">Dynamic Reconfiguration</a>
            </div>
            <div class="menuitem">
                <a href="zookeeperCLI.html">ZooKeeper CLI</a>
            </div>
            <div class="menuitem">
                <a href="zookeeperTools.html">ZooKeeper Tools</a>
            </div>
            <div class="menuitem">
                <a href="zookeeperMonitor.html">ZooKeeper Monitor</a>
            </div>
			<div class="menuitem">
                <a href="zookeeperAuditLogs.html">Audit Logs</a>
            </div>
        </div>
        <div onclick="SwitchMenu('menu_4', 'skin/')" id="menu_4Title" class="menutitle">Contributor</div>
        <div id="menu_4" class="menuitemgroup">
            <div class="menuitem">
                <a href="zookeeperInternals.html">ZooKeeper Internals</a>
            </div>
        </div>
        <div onclick="SwitchMenu('menu_5', 'skin/')" id="menu_5Title" class="menutitle">Miscellaneous</div>
        <div id="menu_5" class="menuitemgroup">
            <div class="menuitem">
                <a href="https://cwiki.apache.org/confluence/display/ZOOKEEPER">Wiki</a>
            </div>
            <div class="menuitem">
                <a href="https://cwiki.apache.org/confluence/display/ZOOKEEPER/FAQ">FAQ</a>
            </div>
            <div class="menuitem">
                <a href="http://zookeeper.apache.org/mailing_lists.html">Mailing Lists</a>
            </div>
        </div>
    </div>
    <div id="content">
<!--
Copyright 2002-2004 The Apache Software Foundation

Licensed 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.
//-->
<h1>Programming with ZooKeeper - A basic tutorial</h1>
<ul>
<li><a href="#ch_Introduction">Introduction</a></li>
<li><a href="#sc_barriers">Barriers</a></li>
<li><a href="#sc_producerConsumerQueues">Producer-Consumer Queues</a></li>
<li><a href="#Complete+example">Complete example</a>
<ul>
<li><a href="#Queue+test">Queue test</a></li>
<li><a href="#Barrier+test">Barrier test</a></li>
<li><a href="#sc_sourceListing">Source Listing</a></li>
</ul>
</li>
</ul>
<p><a name="ch_Introduction"></a></p>
<h2>Introduction</h2>
<p>In this tutorial, we show simple implementations of barriers and producer-consumer queues using ZooKeeper. We call the respective classes Barrier and Queue. These examples assume that you have at least one ZooKeeper server running.</p>
<p>Both primitives use the following common excerpt of code:</p>
<pre><code>static ZooKeeper zk = null;
static Integer mutex;

String root;

SyncPrimitive(String address) {
    if(zk == null){
        try {
            System.out.println(&quot;Starting ZK:&quot;);
            zk = new ZooKeeper(address, 3000, this);
            mutex = new Integer(-1);
            System.out.println(&quot;Finished starting ZK: &quot; + zk);
        } catch (IOException e) {
            System.out.println(e.toString());
            zk = null;
        }
    }
}

synchronized public void process(WatchedEvent event) {
    synchronized (mutex) {
        mutex.notify();
    }
}
</code></pre>
<p>Both classes extend SyncPrimitive. In this way, we execute steps that are common to all primitives in the constructor of SyncPrimitive. To keep the examples simple, we create a ZooKeeper object the first time we instantiate either a barrier object or a queue object, and we declare a static variable that is a reference to this object. The subsequent instances of Barrier and Queue check whether a ZooKeeper object exists. Alternatively, we could have the application creating a ZooKeeper object and passing it to the constructor of Barrier and Queue.</p>
<p>We use the process() method to process notifications triggered due to watches. In the following discussion, we present code that sets watches. A watch is internal structure that enables ZooKeeper to notify a client of a change to a node. For example, if a client is waiting for other clients to leave a barrier, then it can set a watch and wait for modifications to a particular node, which can indicate that it is the end of the wait. This point becomes clear once we go over the examples.</p>
<p><a name="sc_barriers"></a></p>
<h2>Barriers</h2>
<p>A barrier is a primitive that enables a group of processes to synchronize the beginning and the end of a computation. The general idea of this implementation is to have a barrier node that serves the purpose of being a parent for individual process nodes. Suppose that we call the barrier node &quot;/b1&quot;. Each process &quot;p&quot; then creates a node &quot;/b1/p&quot;. Once enough processes have created their corresponding nodes, joined processes can start the computation.</p>
<p>In this example, each process instantiates a Barrier object, and its constructor takes as parameters:</p>
<ul>
<li>the address of a ZooKeeper server (e.g., &quot;zoo1.foo.com:2181&quot;)</li>
<li>the path of the barrier node on ZooKeeper (e.g., &quot;/b1&quot;)</li>
<li>the size of the group of processes</li>
</ul>
<p>The constructor of Barrier passes the address of the Zookeeper server to the constructor of the parent class. The parent class creates a ZooKeeper instance if one does not exist. The constructor of Barrier then creates a barrier node on ZooKeeper, which is the parent node of all process nodes, and we call root (<strong>Note:</strong> This is not the ZooKeeper root &quot;/&quot;).</p>
<pre><code>/**
 * Barrier constructor
 *
 * @param address
 * @param root
 * @param size
 */
Barrier(String address, String root, int size) {
    super(address);
    this.root = root;
    this.size = size;
    // Create barrier node
    if (zk != null) {
        try {
            Stat s = zk.exists(root, false);
            if (s == null) {
                zk.create(root, new byte[0], Ids.OPEN_ACL_UNSAFE,
                        CreateMode.PERSISTENT);
            }
        } catch (KeeperException e) {
            System.out
                    .println(&quot;Keeper exception when instantiating queue: &quot;
                            + e.toString());
        } catch (InterruptedException e) {
            System.out.println(&quot;Interrupted exception&quot;);
        }
    }

    // My node name
    try {
        name = new String(InetAddress.getLocalHost().getCanonicalHostName().toString());
    } catch (UnknownHostException e) {
        System.out.println(e.toString());
    }
}
</code></pre>
<p>To enter the barrier, a process calls enter(). The process creates a node under the root to represent it, using its host name to form the node name. It then wait until enough processes have entered the barrier. A process does it by checking the number of children the root node has with &quot;getChildren()&quot;, and waiting for notifications in the case it does not have enough. To receive a notification when there is a change to the root node, a process has to set a watch, and does it through the call to &quot;getChildren()&quot;. In the code, we have that &quot;getChildren()&quot; has two parameters. The first one states the node to read from, and the second is a boolean flag that enables the process to set a watch. In the code the flag is true.</p>
<pre><code>/**
 * Join barrier
 *
 * @return
 * @throws KeeperException
 * @throws InterruptedException
 */

boolean enter() throws KeeperException, InterruptedException{
    zk.create(root + &quot;/&quot; + name, new byte[0], Ids.OPEN_ACL_UNSAFE,
            CreateMode.EPHEMERAL);
    while (true) {
        synchronized (mutex) {
            List&lt;String&gt; list = zk.getChildren(root, true);

            if (list.size() &lt; size) {
                mutex.wait();
            } else {
                return true;
            }
        }
    }
}
</code></pre>
<p>Note that enter() throws both KeeperException and InterruptedException, so it is the responsibility of the application to catch and handle such exceptions.</p>
<p>Once the computation is finished, a process calls leave() to leave the barrier. First it deletes its corresponding node, and then it gets the children of the root node. If there is at least one child, then it waits for a notification (obs: note that the second parameter of the call to getChildren() is true, meaning that ZooKeeper has to set a watch on the root node). Upon reception of a notification, it checks once more whether the root node has any children.</p>
<pre><code>/**
 * Wait until all reach barrier
 *
 * @return
 * @throws KeeperException
 * @throws InterruptedException
 */

boolean leave() throws KeeperException, InterruptedException {
    zk.delete(root + &quot;/&quot; + name, 0);
    while (true) {
        synchronized (mutex) {
            List&lt;String&gt; list = zk.getChildren(root, true);
                if (list.size() &gt; 0) {
                    mutex.wait();
                } else {
                    return true;
                }
            }
        }
    }
</code></pre>
<p><a name="sc_producerConsumerQueues"></a></p>
<h2>Producer-Consumer Queues</h2>
<p>A producer-consumer queue is a distributed data structure that groups of processes use to generate and consume items. Producer processes create new elements and add them to the queue. Consumer processes remove elements from the list, and process them. In this implementation, the elements are simple integers. The queue is represented by a root node, and to add an element to the queue, a producer process creates a new node, a child of the root node.</p>
<p>The following excerpt of code corresponds to the constructor of the object. As with Barrier objects, it first calls the constructor of the parent class, SyncPrimitive, that creates a ZooKeeper object if one doesn't exist. It then verifies if the root node of the queue exists, and creates if it doesn't.</p>
<pre><code>/**
 * Constructor of producer-consumer queue
 *
 * @param address
 * @param name
 */
Queue(String address, String name) {
    super(address);
    this.root = name;
    // Create ZK node name
    if (zk != null) {
        try {
            Stat s = zk.exists(root, false);
            if (s == null) {
                zk.create(root, new byte[0], Ids.OPEN_ACL_UNSAFE,
                        CreateMode.PERSISTENT);
            }
        } catch (KeeperException e) {
            System.out
                    .println(&quot;Keeper exception when instantiating queue: &quot;
                            + e.toString());
        } catch (InterruptedException e) {
            System.out.println(&quot;Interrupted exception&quot;);
        }
    }
}
</code></pre>
<p>A producer process calls &quot;produce()&quot; to add an element to the queue, and passes an integer as an argument. To add an element to the queue, the method creates a new node using &quot;create()&quot;, and uses the SEQUENCE flag to instruct ZooKeeper to append the value of the sequencer counter associated to the root node. In this way, we impose a total order on the elements of the queue, thus guaranteeing that the oldest element of the queue is the next one consumed.</p>
<pre><code>/**
 * Add element to the queue.
 *
 * @param i
 * @return
 */

boolean produce(int i) throws KeeperException, InterruptedException{
    ByteBuffer b = ByteBuffer.allocate(4);
    byte[] value;

    // Add child with value i
    b.putInt(i);
    value = b.array();
    zk.create(root + &quot;/element&quot;, value, Ids.OPEN_ACL_UNSAFE,
                CreateMode.PERSISTENT_SEQUENTIAL);

    return true;
}
</code></pre>
<p>To consume an element, a consumer process obtains the children of the root node, reads the node with smallest counter value, and returns the element. Note that if there is a conflict, then one of the two contending processes won't be able to delete the node and the delete operation will throw an exception.</p>
<p>A call to getChildren() returns the list of children in lexicographic order. As lexicographic order does not necessarily follow the numerical order of the counter values, we need to decide which element is the smallest. To decide which one has the smallest counter value, we traverse the list, and remove the prefix &quot;element&quot; from each one.</p>
<pre><code>/**
 * Remove first element from the queue.
 *
 * @return
 * @throws KeeperException
 * @throws InterruptedException
 */
int consume() throws KeeperException, InterruptedException{
    int retvalue = -1;
    Stat stat = null;

    // Get the first element available
    while (true) {
        synchronized (mutex) {
            List&lt;String&gt; list = zk.getChildren(root, true);
            if (list.size() == 0) {
                System.out.println(&quot;Going to wait&quot;);
                mutex.wait();
            } else {
                Integer min = new Integer(list.get(0).substring(7));
                for(String s : list){
                    Integer tempValue = new Integer(s.substring(7));
                    //System.out.println(&quot;Temporary value: &quot; + tempValue);
                    if(tempValue &lt; min) min = tempValue;
                }
                System.out.println(&quot;Temporary value: &quot; + root + &quot;/element&quot; + min);
                byte[] b = zk.getData(root + &quot;/element&quot; + min,
                            false, stat);
                zk.delete(root + &quot;/element&quot; + min, 0);
                ByteBuffer buffer = ByteBuffer.wrap(b);
                retvalue = buffer.getInt();

                return retvalue;
                }
            }
        }
    }
}
</code></pre>
<p><a name="Complete+example"></a></p>
<h2>Complete example</h2>
<p>In the following section you can find a complete command line application to demonstrate the above mentioned recipes. Use the following command to run it.</p>
<pre><code>ZOOBINDIR=&quot;[path_to_distro]/bin&quot;
. &quot;$ZOOBINDIR&quot;/zkEnv.sh
java SyncPrimitive [Test Type] [ZK server] [No of elements] [Client type]
</code></pre>
<p><a name="Queue+test"></a></p>
<h3>Queue test</h3>
<p>Start a producer to create 100 elements</p>
<pre><code>java SyncPrimitive qTest localhost 100 p
</code></pre>
<p>Start a consumer to consume 100 elements</p>
<pre><code>java SyncPrimitive qTest localhost 100 c
</code></pre>
<p><a name="Barrier+test"></a></p>
<h3>Barrier test</h3>
<p>Start a barrier with 2 participants (start as many times as many participants you'd like to enter)</p>
<pre><code>java SyncPrimitive bTest localhost 2
</code></pre>
<p><a name="sc_sourceListing"></a></p>
<h3>Source Listing</h3>
<h4>SyncPrimitive.Java</h4>
<pre><code>import java.io.IOException;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.Random;

import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.Stat;

public class SyncPrimitive implements Watcher {

    static ZooKeeper zk = null;
    static Integer mutex;
    String root;

    SyncPrimitive(String address) {
        if(zk == null){
            try {
                System.out.println(&quot;Starting ZK:&quot;);
                zk = new ZooKeeper(address, 3000, this);
                mutex = new Integer(-1);
                System.out.println(&quot;Finished starting ZK: &quot; + zk);
            } catch (IOException e) {
                System.out.println(e.toString());
                zk = null;
            }
        }
        //else mutex = new Integer(-1);
    }

    synchronized public void process(WatchedEvent event) {
        synchronized (mutex) {
            //System.out.println(&quot;Process: &quot; + event.getType());
            mutex.notify();
        }
    }

    /**
     * Barrier
     */
    static public class Barrier extends SyncPrimitive {
        int size;
        String name;

        /**
         * Barrier constructor
         *
         * @param address
         * @param root
         * @param size
         */
        Barrier(String address, String root, int size) {
            super(address);
            this.root = root;
            this.size = size;

            // Create barrier node
            if (zk != null) {
                try {
                    Stat s = zk.exists(root, false);
                    if (s == null) {
                        zk.create(root, new byte[0], Ids.OPEN_ACL_UNSAFE,
                                CreateMode.PERSISTENT);
                    }
                } catch (KeeperException e) {
                    System.out
                            .println(&quot;Keeper exception when instantiating queue: &quot;
                                    + e.toString());
                } catch (InterruptedException e) {
                    System.out.println(&quot;Interrupted exception&quot;);
                }
            }

            // My node name
            try {
                name = new String(InetAddress.getLocalHost().getCanonicalHostName().toString());
            } catch (UnknownHostException e) {
                System.out.println(e.toString());
            }

        }

        /**
         * Join barrier
         *
         * @return
         * @throws KeeperException
         * @throws InterruptedException
         */

        boolean enter() throws KeeperException, InterruptedException{
            zk.create(root + &quot;/&quot; + name, new byte[0], Ids.OPEN_ACL_UNSAFE,
                    CreateMode.EPHEMERAL);
            while (true) {
                synchronized (mutex) {
                    List&lt;String&gt; list = zk.getChildren(root, true);

                    if (list.size() &lt; size) {
                        mutex.wait();
                    } else {
                        return true;
                    }
                }
            }
        }

        /**
         * Wait until all reach barrier
         *
         * @return
         * @throws KeeperException
         * @throws InterruptedException
         */
        boolean leave() throws KeeperException, InterruptedException{
            zk.delete(root + &quot;/&quot; + name, 0);
            while (true) {
                synchronized (mutex) {
                    List&lt;String&gt; list = zk.getChildren(root, true);
                        if (list.size() &gt; 0) {
                            mutex.wait();
                        } else {
                            return true;
                        }
                    }
                }
            }
        }

    /**
     * Producer-Consumer queue
     */
    static public class Queue extends SyncPrimitive {

        /**
         * Constructor of producer-consumer queue
         *
         * @param address
         * @param name
         */
        Queue(String address, String name) {
            super(address);
            this.root = name;
            // Create ZK node name
            if (zk != null) {
                try {
                    Stat s = zk.exists(root, false);
                    if (s == null) {
                        zk.create(root, new byte[0], Ids.OPEN_ACL_UNSAFE,
                                CreateMode.PERSISTENT);
                    }
                } catch (KeeperException e) {
                    System.out
                            .println(&quot;Keeper exception when instantiating queue: &quot;
                                    + e.toString());
                } catch (InterruptedException e) {
                    System.out.println(&quot;Interrupted exception&quot;);
                }
            }
        }

        /**
         * Add element to the queue.
         *
         * @param i
         * @return
         */

        boolean produce(int i) throws KeeperException, InterruptedException{
            ByteBuffer b = ByteBuffer.allocate(4);
            byte[] value;

            // Add child with value i
            b.putInt(i);
            value = b.array();
            zk.create(root + &quot;/element&quot;, value, Ids.OPEN_ACL_UNSAFE,
                        CreateMode.PERSISTENT_SEQUENTIAL);

            return true;
        }

        /**
         * Remove first element from the queue.
         *
         * @return
         * @throws KeeperException
         * @throws InterruptedException
         */
        int consume() throws KeeperException, InterruptedException{
            int retvalue = -1;
            Stat stat = null;

            // Get the first element available
            while (true) {
                synchronized (mutex) {
                    List&lt;String&gt; list = zk.getChildren(root, true);
                    if (list.size() == 0) {
                        System.out.println(&quot;Going to wait&quot;);
                        mutex.wait();
                    } else {
                        Integer min = new Integer(list.get(0).substring(7));
                        String minNode = list.get(0);
                        for(String s : list){
                            Integer tempValue = new Integer(s.substring(7));
                            //System.out.println(&quot;Temporary value: &quot; + tempValue);
                            if(tempValue &lt; min) {
                                min = tempValue;
                                minNode = s;
                            }
                        }
                        System.out.println(&quot;Temporary value: &quot; + root + &quot;/&quot; + minNode);
                        byte[] b = zk.getData(root + &quot;/&quot; + minNode,
                        false, stat);
                        zk.delete(root + &quot;/&quot; + minNode, 0);
                        ByteBuffer buffer = ByteBuffer.wrap(b);
                        retvalue = buffer.getInt();

                        return retvalue;
                    }
                }
            }
        }
    }

    public static void main(String args[]) {
        if (args[0].equals(&quot;qTest&quot;))
            queueTest(args);
        else
            barrierTest(args);
    }

    public static void queueTest(String args[]) {
        Queue q = new Queue(args[1], &quot;/app1&quot;);

        System.out.println(&quot;Input: &quot; + args[1]);
        int i;
        Integer max = new Integer(args[2]);

        if (args[3].equals(&quot;p&quot;)) {
            System.out.println(&quot;Producer&quot;);
            for (i = 0; i &lt; max; i++)
                try{
                    q.produce(10 + i);
                } catch (KeeperException e){

                } catch (InterruptedException e){

                }
        } else {
            System.out.println(&quot;Consumer&quot;);

            for (i = 0; i &lt; max; i++) {
                try{
                    int r = q.consume();
                    System.out.println(&quot;Item: &quot; + r);
                } catch (KeeperException e){
                    i--;
                } catch (InterruptedException e){
                }
            }
        }
    }

    public static void barrierTest(String args[]) {
        Barrier b = new Barrier(args[1], &quot;/b1&quot;, new Integer(args[2]));
        try{
            boolean flag = b.enter();
            System.out.println(&quot;Entered barrier: &quot; + args[2]);
            if(!flag) System.out.println(&quot;Error when entering the barrier&quot;);
        } catch (KeeperException e){
        } catch (InterruptedException e){
        }

        // Generate random integer
        Random rand = new Random();
        int r = rand.nextInt(100);
        // Loop for rand iterations
        for (int i = 0; i &lt; r; i++) {
            try {
                Thread.sleep(100);
            } catch (InterruptedException e) {
            }
        }
        try{
            b.leave();
        } catch (KeeperException e){

        } catch (InterruptedException e){

        }
        System.out.println(&quot;Left barrier&quot;);
    }
}
</code></pre>
</div>
<div class="clearboth">&nbsp;</div>
</div>
<div id="footer">
    <div class="lastmodified">
        <script type="text/javascript">
        <!--
            document.write("Last Published: " + document.lastModified);
        //  -->
        </script>
    </div>
    <div class="copyright">
        Copyright &copy; <a href="http://www.apache.org/licenses/">The Apache Software Foundation.</a>
    </div>
    <div id="logos"></div>
</div>
</body>
</html>