blob: c99228283f143868b5754c6c87e01f1acc51d3dd [file] [log] [blame]
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<!DOCTYPE article PUBLIC "-//OASIS//DTD Simplified DocBook XML V1.0//EN"
"http://www.oasis-open.org/docbook/xml/simple/1.0/sdocbook.dtd">
<article id="ar_JavaExample">
<title>ZooKeeper Java Example</title>
<articleinfo>
<legalnotice>
<para>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 <ulink
url="http://www.apache.org/licenses/LICENSE-2.0">http://www.apache.org/licenses/LICENSE-2.0</ulink>.</para>
<para>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.</para>
</legalnotice>
<abstract>
<para>This article contains sample Java code for a simple watch client.</para>
</abstract>
</articleinfo>
<section id="ch_Introduction">
<title>A Simple Watch Client</title>
<para>To introduce you to the ZooKeeper Java API, we develop here a very simple
watch client. This ZooKeeper client watches a ZooKeeper node for changes
and responds to by starting or stopping a program.</para>
<section id="sc_requirements"><title>Requirements</title>
<para>The client has four requirements:</para>
<itemizedlist><listitem><para>It takes as parameters:</para>
<itemizedlist>
<listitem><para>the address of the ZooKeeper service</para></listitem>
<listitem> <para>then name of a znode - the one to be watched</para></listitem>
<listitem><para>an executable with arguments.</para></listitem></itemizedlist></listitem>
<listitem><para>It fetches the data associated with the znode and starts the executable.</para></listitem>
<listitem><para>If the znode changes, the client refetches the contents and restarts the executable.</para></listitem>
<listitem><para>If the znode disappears, the client kills the executable.</para></listitem></itemizedlist>
</section>
<section id="sc_design">
<title>Program Design</title>
<para>Conventionally, ZooKeeper applications are broken into two units, one which maintains the connection,
and the other which monitors data. In this application, the class called the <emphasis role="bold">Executor</emphasis>
maintains the ZooKeeper connection, and the class called the <emphasis role="bold">DataMonitor</emphasis> monitors the data
in the ZooKeeper tree. Also, Executor contains the main thread and contains the execution logic.
It is responsible for what little user interaction there is, as well as interaction with the exectuable program you
pass in as an argument and which the sample (per the requirements) shuts down and restarts, according to the
state of the znode.</para>
</section>
</section>
<section id="sc_executor"><title>The Executor Class</title>
<para>The Executor object is the primary container of the sample application. It contains
both the <emphasis role="bold">ZooKeeper</emphasis> object, <emphasis role="bold">DataMonitor</emphasis>, as described above in
<xref linkend="sc_design"/>. </para>
<programlisting>
// from the Executor class...
public static void main(String[] args) {
if (args.length &lt; 4) {
System.err
.println("USAGE: Executor hostPort znode filename program [args ...]");
System.exit(2);
}
String hostPort = args[0];
String znode = args[1];
String filename = args[2];
String exec[] = new String[args.length - 3];
System.arraycopy(args, 3, exec, 0, exec.length);
try {
new Executor(hostPort, znode, filename, exec).run();
} catch (Exception e) {
e.printStackTrace();
}
}
public Executor(String hostPort, String znode, String filename,
String exec[]) throws KeeperException, IOException {
this.filename = filename;
this.exec = exec;
zk = new ZooKeeper(hostPort, 3000, this);
dm = new DataMonitor(zk, znode, null, this);
}
public void run() {
try {
synchronized (this) {
while (!dm.dead) {
wait();
}
}
} catch (InterruptedException e) {
}
}
</programlisting>
<para>
Recall that the Executor's job is to start and stop the executable whose name you pass in on the command line.
It does this in response to events fired by the ZooKeeper object. As you can see in the code above, the Executor passes
a reference to itself as the Watcher argument in the ZooKeeper constructor. It also passes a reference to itself
as DataMonitorListener argument to the DataMonitor constructor. Per the Executor's definition, it implements both these
interfaces:
</para>
<programlisting>
public class Executor implements Watcher, Runnable, DataMonitor.DataMonitorListener {
...</programlisting>
<para>The <emphasis role="bold">Watcher</emphasis> interface is defined by the ZooKeeper Java API.
ZooKeeper uses it to communicate back to its container. It supports only one method, <command>process()</command>, and ZooKeeper uses
it to communciates generic events that the main thread would be intersted in, such as the state of the ZooKeeper connection or the ZooKeeper session.The Executor
in this example simply forwards those events down to the DataMonitor to decide what to do with them. It does this simply to illustrate
the point that, by convention, the Executor or some Executor-like object "owns" the ZooKeeper connection, but it is free to delegate the events to other
events to other objects. It also uses this as the default channel on which to fire watch events. (More on this later.)</para>
<programlisting>
public void process(WatchedEvent event) {
dm.process(event);
}
</programlisting>
<para>The <emphasis role="bold">DataMonitorListener</emphasis>
interface, on the other hand, is not part of the the ZooKeeper API. It is a completely custom interface,
designed for this sample application. The DataMonitor object uses it to communicate back to its container, which
is also the the Executor object.The DataMonitorListener interface looks like this:</para>
<programlisting>
public interface DataMonitorListener {
/**
* The existence status of the node has changed.
*/
void exists(byte data[]);
/**
* The ZooKeeper session is no longer valid.
*
* @param rc
* the ZooKeeper reason code
*/
void closing(int rc);
}
</programlisting>
<para>This interface is defined in the DataMonitor class and implemented in the Executor class.
When <command>Executor.exists()</command> is invoked,
the Executor decides whether to start up or shut down per the requirements. Recall that the requires say to kill the executable when the
znode ceases to <emphasis>exist</emphasis>. </para>
<para>When <command>Executor.closing()</command>
is invoked, the Executor decides whether or not to shut itself down in response to the ZooKeeper connection permanently disappearing.</para>
<para>As you might have guessed, DataMonitor is the object that invokes
these methods, in response to changes in ZooKeeper's state.</para>
<para>Here are Executor's implementation of
<command>DataMonitorListener.exists()</command> and <command>DataMonitorListener.closing</command>:
</para>
<programlisting>
public void exists( byte[] data ) {
if (data == null) {
if (child != null) {
System.out.println("Killing process");
child.destroy();
try {
child.waitFor();
} catch (InterruptedException e) {
}
}
child = null;
} else {
if (child != null) {
System.out.println("Stopping child");
child.destroy();
try {
child.waitFor();
} catch (InterruptedException e) {
e.printStackTrace();
}
}
try {
FileOutputStream fos = new FileOutputStream(filename);
fos.write(data);
fos.close();
} catch (IOException e) {
e.printStackTrace();
}
try {
System.out.println("Starting child");
child = Runtime.getRuntime().exec(exec);
new StreamWriter(child.getInputStream(), System.out);
new StreamWriter(child.getErrorStream(), System.err);
} catch (IOException e) {
e.printStackTrace();
}
}
}
public void closing(int rc) {
synchronized (this) {
notifyAll();
}
}
</programlisting>
</section>
<section id="sc_DataMonitor"><title>The DataMonitor Class</title>
<para>
The DataMonitor class has the meat of the ZooKeeper logic. It is mostly
asynchronous and event driven. DataMonitor kicks things off in the constructor with:</para>
<programlisting>
public DataMonitor(ZooKeeper zk, String znode, Watcher chainedWatcher,
DataMonitorListener listener) {
this.zk = zk;
this.znode = znode;
this.chainedWatcher = chainedWatcher;
this.listener = listener;
// Get things started by checking if the node exists. We are going
// to be completely event driven
<emphasis role="bold">zk.exists(znode, true, this, null);</emphasis>
}
</programlisting>
<para>The call to <command>ZooKeeper.exists()</command> checks for the existence of the znode,
sets a watch, and passes a reference to itself (<command>this</command>)
as the completion callback object. In this sense, it kicks things off, since the
real processing happens when the watch is triggered.</para>
<note>
<para>Don't confuse the completion callback with the watch callback. The <command>ZooKeeper.exists()</command>
completion callback, which happens to be the method <command>StatCallback.processResult()</command> implemented
in the DataMonitor object, is invoked when the asynchronous <emphasis>setting of the watch</emphasis> operation
(by <command>ZooKeeper.exists()</command>) completes on the server. </para>
<para>
The triggering of the watch, on the other hand, sends an event to the <emphasis>Executor</emphasis> object, since
the Executor registered as the Watcher of the ZooKeeper object.</para>
<para>As an aside, you might note that the DataMonitor could also register itself as the Watcher
for this particular watch event. This is new to ZooKeeper 3.0.0 (the support of multiple Watchers). In this
example, however, DataMonitor does not register as the Watcher.</para>
</note>
<para>When the <command>ZooKeeper.exists()</command> operation completes on the server, the ZooKeeper API invokes this completion callback on
the client:</para>
<programlisting>
public void processResult(int rc, String path, Object ctx, Stat stat) {
boolean exists;
switch (rc) {
case Code.Ok:
exists = true;
break;
case Code.NoNode:
exists = false;
break;
case Code.SessionExpired:
case Code.NoAuth:
dead = true;
listener.closing(rc);
return;
default:
// Retry errors
zk.exists(znode, true, this, null);
return;
}
byte b[] = null;
if (exists) {
try {
<emphasis role="bold">b = zk.getData(znode, false, null);</emphasis>
} catch (KeeperException e) {
// We don't need to worry about recovering now. The watch
// callbacks will kick off any exception handling
e.printStackTrace();
} catch (InterruptedException e) {
return;
}
}
if ((b == null &amp;&amp; b != prevData)
|| (b != null &amp;&amp; !Arrays.equals(prevData, b))) {
<emphasis role="bold">listener.exists(b);</emphasis>
prevData = b;
}
}
</programlisting>
<para>
The code first checks the error codes for znode existence, fatal errors, and
recoverable errors. If the file (or znode) exists, it gets the data from the znode, and
then invoke the exists() callback of Executor if the state has changed. Note,
it doesn't have to do any Exception processing for the getData call because it
has watches pending for anything that could cause an error: if the node is deleted
before it calls <command>ZooKeeper.getData()</command>, the watch event set by
the <command>ZooKeeper.exists()</command> triggers a callback;
if there is a communication error, a connection watch event fires when
the connection comes back up.
</para>
<para>Finally, notice how DataMonitor processes watch events: </para>
<programlisting>
public void process(WatchedEvent event) {
String path = event.getPath();
if (event.getType() == Event.EventType.None) {
// We are are being told that the state of the
// connection has changed
switch (event.getState()) {
case SyncConnected:
// In this particular example we don't need to do anything
// here - watches are automatically re-registered with
// server and any watches triggered while the client was
// disconnected will be delivered (in order of course)
break;
case Expired:
// It's all over
dead = true;
listener.closing(KeeperException.Code.SessionExpired);
break;
}
} else {
if (path != null &amp;&amp; path.equals(znode)) {
// Something has changed on the node, let's find out
zk.exists(znode, true, this, null);
}
}
if (chainedWatcher != null) {
chainedWatcher.process(event);
}
}
</programlisting>
<para>
If the client-side ZooKeeper libraries can re-establish the
communication channel (SyncConnected event) to ZooKeeper before
session expiration (Expired event) all of the session's watches will
automatically be re-established with the server (auto-reset of watches
is new in ZooKeeper 3.0.0). See <ulink
url="zookeeperProgrammers.html#ch_zkWatches">ZooKeeper Watches</ulink>
in the programmer guide for more on this. A bit lower down in this
function, when DataMonitor gets an event for a znode, it calls
<command>ZooKeeper.exists()</command> to find out what has changed.
</para>
</section>
<section id="sc_completeSourceCode">
<title>Complete Source Listings</title>
<example id="eg_Executor_java"><title>Executor.java</title><programlisting>
/**
* A simple example program to use DataMonitor to start and
* stop executables based on a znode. The program watches the
* specified znode and saves the data that corresponds to the
* znode in the filesystem. It also starts the specified program
* with the specified arguments when the znode exists and kills
* the program if the znode goes away.
*/
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooKeeper;
public class Executor
implements Watcher, Runnable, DataMonitor.DataMonitorListener
{
String znode;
DataMonitor dm;
ZooKeeper zk;
String filename;
String exec[];
Process child;
public Executor(String hostPort, String znode, String filename,
String exec[]) throws KeeperException, IOException {
this.filename = filename;
this.exec = exec;
zk = new ZooKeeper(hostPort, 3000, this);
dm = new DataMonitor(zk, znode, null, this);
}
/**
* @param args
*/
public static void main(String[] args) {
if (args.length &lt; 4) {
System.err
.println("USAGE: Executor hostPort znode filename program [args ...]");
System.exit(2);
}
String hostPort = args[0];
String znode = args[1];
String filename = args[2];
String exec[] = new String[args.length - 3];
System.arraycopy(args, 3, exec, 0, exec.length);
try {
new Executor(hostPort, znode, filename, exec).run();
} catch (Exception e) {
e.printStackTrace();
}
}
/***************************************************************************
* We do process any events ourselves, we just need to forward them on.
*
* @see org.apache.zookeeper.Watcher#process(org.apache.zookeeper.proto.WatcherEvent)
*/
public void process(WatchedEvent event) {
dm.process(event);
}
public void run() {
try {
synchronized (this) {
while (!dm.dead) {
wait();
}
}
} catch (InterruptedException e) {
}
}
public void closing(int rc) {
synchronized (this) {
notifyAll();
}
}
static class StreamWriter extends Thread {
OutputStream os;
InputStream is;
StreamWriter(InputStream is, OutputStream os) {
this.is = is;
this.os = os;
start();
}
public void run() {
byte b[] = new byte[80];
int rc;
try {
while ((rc = is.read(b)) > 0) {
os.write(b, 0, rc);
}
} catch (IOException e) {
}
}
}
public void exists(byte[] data) {
if (data == null) {
if (child != null) {
System.out.println("Killing process");
child.destroy();
try {
child.waitFor();
} catch (InterruptedException e) {
}
}
child = null;
} else {
if (child != null) {
System.out.println("Stopping child");
child.destroy();
try {
child.waitFor();
} catch (InterruptedException e) {
e.printStackTrace();
}
}
try {
FileOutputStream fos = new FileOutputStream(filename);
fos.write(data);
fos.close();
} catch (IOException e) {
e.printStackTrace();
}
try {
System.out.println("Starting child");
child = Runtime.getRuntime().exec(exec);
new StreamWriter(child.getInputStream(), System.out);
new StreamWriter(child.getErrorStream(), System.err);
} catch (IOException e) {
e.printStackTrace();
}
}
}
}
</programlisting>
</example>
<example id="eg_DataMonitor_java">
<title>DataMonitor.java</title>
<programlisting>
/**
* A simple class that monitors the data and existence of a ZooKeeper
* node. It uses asynchronous ZooKeeper APIs.
*/
import java.util.Arrays;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.AsyncCallback.StatCallback;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.data.Stat;
public class DataMonitor implements Watcher, StatCallback {
ZooKeeper zk;
String znode;
Watcher chainedWatcher;
boolean dead;
DataMonitorListener listener;
byte prevData[];
public DataMonitor(ZooKeeper zk, String znode, Watcher chainedWatcher,
DataMonitorListener listener) {
this.zk = zk;
this.znode = znode;
this.chainedWatcher = chainedWatcher;
this.listener = listener;
// Get things started by checking if the node exists. We are going
// to be completely event driven
zk.exists(znode, true, this, null);
}
/**
* Other classes use the DataMonitor by implementing this method
*/
public interface DataMonitorListener {
/**
* The existence status of the node has changed.
*/
void exists(byte data[]);
/**
* The ZooKeeper session is no longer valid.
*
* @param rc
* the ZooKeeper reason code
*/
void closing(int rc);
}
public void process(WatchedEvent event) {
String path = event.getPath();
if (event.getType() == Event.EventType.None) {
// We are are being told that the state of the
// connection has changed
switch (event.getState()) {
case SyncConnected:
// In this particular example we don't need to do anything
// here - watches are automatically re-registered with
// server and any watches triggered while the client was
// disconnected will be delivered (in order of course)
break;
case Expired:
// It's all over
dead = true;
listener.closing(KeeperException.Code.SessionExpired);
break;
}
} else {
if (path != null &amp;&amp; path.equals(znode)) {
// Something has changed on the node, let's find out
zk.exists(znode, true, this, null);
}
}
if (chainedWatcher != null) {
chainedWatcher.process(event);
}
}
public void processResult(int rc, String path, Object ctx, Stat stat) {
boolean exists;
switch (rc) {
case Code.Ok:
exists = true;
break;
case Code.NoNode:
exists = false;
break;
case Code.SessionExpired:
case Code.NoAuth:
dead = true;
listener.closing(rc);
return;
default:
// Retry errors
zk.exists(znode, true, this, null);
return;
}
byte b[] = null;
if (exists) {
try {
b = zk.getData(znode, false, null);
} catch (KeeperException e) {
// We don't need to worry about recovering now. The watch
// callbacks will kick off any exception handling
e.printStackTrace();
} catch (InterruptedException e) {
return;
}
}
if ((b == null &amp;&amp; b != prevData)
|| (b != null &amp;&amp; !Arrays.equals(prevData, b))) {
listener.exists(b);
prevData = b;
}
}
}
</programlisting>
</example>
</section>
</article>