org.apache.zookeeper
Class ZooKeeper

java.lang.Object
  extended by org.apache.zookeeper.ZooKeeper

public class ZooKeeper
extends Object

This is the main class of ZooKeeper client library. To use a ZooKeeper service, an application must first instantiate an object of ZooKeeper class. All the iterations will be done by calling the methods of ZooKeeper class.

Once a connection to a server is established, a session ID is assigned to the client. The client will send heart beats to the server periodically to keep the session valid.

The application can call ZooKeeper APIs through a client as long as the session ID of the client remains valid.

If for some reason, the client fails to send heart beats to the server for a prolonged period of time (exceeding the sessionTimeout value, for instance), the server will expire the session, and the session ID will become invalid. The client object will no longer be usable. To make ZooKeeper API calls, the application must create a new client object.

If the ZooKeeper server the client currently connects to fails or otherwise does not respond, the client will automatically try to connect to another server before its session ID expires. If successful, the application can continue to use the client.

Some successful ZooKeeper API calls can leave watches on the "data nodes" in the ZooKeeper server. Other successful ZooKeeper API calls can trigger those watches. Once a watch is triggered, an event will be delivered to the client which left the watch at the first place. Each watch can be triggered only once. Thus, up to one event will be delivered to a client for every watch it leaves.

A client needs an object of a class implementing Watcher interface for processing the events delivered to the client. When a client drops current connection and re-connects to a server, all the existing watches are considered as being triggered but the undelivered events are lost. To emulate this, the client will generate a special event to tell the event handler a connection has been dropped. This special event has type EventNone and state sKeeperStateDisconnected.


Nested Class Summary
static class ZooKeeper.States
           
 
Constructor Summary
ZooKeeper(String host, int sessionTimeout, Watcher watcher)
          To create a client(ZooKeeper) object, the application needs to pass a string containing a comma separated list of host:port pairs, each corresponding to a ZooKeeper server.
ZooKeeper(String host, int sessionTimeout, Watcher watcher, long sessionId, byte[] sessionPasswd)
          To create a client(ZooKeeper) object, the application needs to pass a string containing a comma separated list of host:port pairs, each corresponding to a ZooKeeper server.
 
Method Summary
 void addAuthInfo(String scheme, byte[] auth)
           
 void close()
          Close this client object.
 String create(String path, byte[] data, List<ACL> acl, CreateMode createMode)
          Create a node with the given path.
 void create(String path, byte[] data, List<ACL> acl, CreateMode createMode, AsyncCallback.StringCallback cb, Object ctx)
          The Asynchronous version of create.
 void delete(String path, int version)
          Delete the node with the given path.
 void delete(String path, int version, AsyncCallback.VoidCallback cb, Object ctx)
          The Asynchronous version of delete.
 Stat exists(String path, boolean watch)
          Return the stat of the node of the given path.
 void exists(String path, boolean watch, AsyncCallback.StatCallback cb, Object ctx)
          The Asynchronous version of exists.
 Stat exists(String path, Watcher watcher)
          Return the stat of the node of the given path.
 void exists(String path, Watcher watcher, AsyncCallback.StatCallback cb, Object ctx)
          The Asynchronous version of exists.
 List<ACL> getACL(String path, Stat stat)
          Return the ACL and stat of the node of the given path.
 void getACL(String path, Stat stat, AsyncCallback.ACLCallback cb, Object ctx)
          The Asynchronous version of getACL.
 List<String> getChildren(String path, boolean watch)
          Return the list of the children of the node of the given path.
 void getChildren(String path, boolean watch, AsyncCallback.ChildrenCallback cb, Object ctx)
          The Asynchronous version of getChildren.
 List<String> getChildren(String path, Watcher watcher)
          Return the list of the children of the node of the given path.
 void getChildren(String path, Watcher watcher, AsyncCallback.ChildrenCallback cb, Object ctx)
          The Asynchronous version of getChildren.
 void getData(String path, boolean watch, AsyncCallback.DataCallback cb, Object ctx)
          The Asynchronous version of getData.
 byte[] getData(String path, boolean watch, Stat stat)
          Return the data and the stat of the node of the given path.
 void getData(String path, Watcher watcher, AsyncCallback.DataCallback cb, Object ctx)
          The Asynchronous version of getData.
 byte[] getData(String path, Watcher watcher, Stat stat)
          Return the data and the stat of the node of the given path.
 long getSessionId()
          The session id for this ZooKeeper client instance.
 byte[] getSessionPasswd()
          The session password for this ZooKeeper client instance.
 ZooKeeper.States getState()
           
 void register(Watcher watcher)
           
 Stat setACL(String path, List<ACL> acl, int version)
          Set the ACL for the node of the given path if such a node exists and the given version matches the version of the node.
 void setACL(String path, List<ACL> acl, int version, AsyncCallback.StatCallback cb, Object ctx)
          The Asynchronous version of setACL.
 Stat setData(String path, byte[] data, int version)
          Set the data for the node of the given path if such a node exists and the given version matches the version of the node (if the given version is -1, it matches any node's versions).
 void setData(String path, byte[] data, int version, AsyncCallback.StatCallback cb, Object ctx)
          The Asynchronous version of setData.
 void sync(String path, AsyncCallback.VoidCallback cb, Object ctx)
          Asynchronous sync.
static void validatePath(String path)
          Validate the provided znode path string
 
Methods inherited from class java.lang.Object
equals, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
 

Constructor Detail

ZooKeeper

public ZooKeeper(String host,
                 int sessionTimeout,
                 Watcher watcher)
          throws IOException
To create a client(ZooKeeper) object, the application needs to pass a string containing a comma separated list of host:port pairs, each corresponding to a ZooKeeper server.

The client object will pick an arbitrary server and try to connect to it. If failed, it will try the next one in the list, until a connection is established, or all the servers have been tried.

Parameters:
host - comma separated host:port pairs, each corresponding to a zk server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
sessionTimeout - session timeout in milliseconds
watcher - a watcher object which will be notified of state changes, may also be notified for node events
Throws:
IOException - in cases of network failure

ZooKeeper

public ZooKeeper(String host,
                 int sessionTimeout,
                 Watcher watcher,
                 long sessionId,
                 byte[] sessionPasswd)
          throws IOException
To create a client(ZooKeeper) object, the application needs to pass a string containing a comma separated list of host:port pairs, each corresponding to a ZooKeeper server.

The client object will pick an arbitrary server and try to connect to it. If failed, it will try the next one in the list, until a connection is established, or all the servers have been tried.

Use getSessionId() and getSessionPasswd() on an established client connection, these values must be passed as sessionId and sessionPasswd respectively if reconnecting. Otherwise, if not reconnecting, use the other constructor which does not require these parameters.

Parameters:
host - comma separated host:port pairs, each corresponding to a zk server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"
sessionTimeout - session timeout in milliseconds
watcher - a watcher object which will be notified of state changes, may also be notified for node events
sessionId - specific session id to use if reconnecting
sessionPasswd - password for this session
Throws:
IOException - in cases of network failure
Method Detail

getSessionId

public long getSessionId()
The session id for this ZooKeeper client instance. The value returned is not valid until the client connects to a server and may change after a re-connect.

Returns:
current session id

getSessionPasswd

public byte[] getSessionPasswd()
The session password for this ZooKeeper client instance. The value returned is not valid until the client connects to a server and may change after a re-connect.

Returns:
current session password

addAuthInfo

public void addAuthInfo(String scheme,
                        byte[] auth)

register

public void register(Watcher watcher)

close

public void close()
           throws InterruptedException
Close this client object. Once the client is closed, its session becomes invalid. All the ephemeral nodes in the ZooKeeper server associated with the session will be removed. The watches left on those nodes (and on their parents) will be triggered.

Throws:
InterruptedException
IOException
InterruptedException

create

public String create(String path,
                     byte[] data,
                     List<ACL> acl,
                     CreateMode createMode)
              throws KeeperException,
                     InterruptedException
Create a node with the given path. The node data will be the given data, and node acl will be the given acl.

The flags argument specifies whether the created node will be ephemeral or not.

An ephemeral node will be removed by the ZooKeeper automatically when the session associated with the creation of the node expires.

The flags argument can also specify to create a sequential node. The actual path name of a sequential node will be the given path plus a suffix "_i" where i is the current sequential number of the node. Once such a node is created, the sequential number will be incremented by one.

If a node with the same actual path already exists in the ZooKeeper, a KeeperException with error code KeeperException.NodeExists will be thrown. Note that since a different actual path is used for each invocation of creating sequential node with the same path argument, the call will never throw "file exists" KeeperException.

If the parent node does not exist in the ZooKeeper, a KeeperException with error code KeeperException.NoNode will be thrown.

An ephemeral node cannot have children. If the parent node of the given path is ephemeral, a KeeperException with error code KeeperException.NoChildrenForEphemerals will be thrown.

This operation, if successful, will trigger all the watches left on the node of the given path by exists and getData API calls, and the watches left on the parent node by getChildren API calls.

If a node is created successfully, the ZooKeeper server will trigger the watches on the path left by exists calls, and the watches on the parent of the node by getChildren calls.

The maximum allowable size of the data array is 1 MB (1,048,576 bytes). Arrays larger than this will cause a KeeperExecption to be thrown.

Parameters:
path - the path for the node
data - the initial data for the node
acl - the acl for the node
flags - specifying whether the node to be created is ephemeral and/or sequential
Returns:
the actual path of the created node
Throws:
KeeperException - if the server returns a non-zero error code
KeeperException.InvalidACLException - if the ACL is invalid
InterruptedException - if the transaction is interrupted
IllegalArgumentException - if an invalid path is specified

validatePath

public static void validatePath(String path)
                         throws IllegalArgumentException
Validate the provided znode path string

Parameters:
path - znode path string
Throws:
IllegalArgumentException - if the path is invalid

create

public void create(String path,
                   byte[] data,
                   List<ACL> acl,
                   CreateMode createMode,
                   AsyncCallback.StringCallback cb,
                   Object ctx)
The Asynchronous version of create. The request doesn't actually until the asynchronous callback is called.

See Also:
#create(String, byte[], List, CreateMode)

delete

public void delete(String path,
                   int version)
            throws InterruptedException,
                   KeeperException
Delete the node with the given path. The call will succeed if such a node exists, and the given version matches the node's version (if the given version is -1, it matches any node's versions).

A KeeperException with error code KeeperException.NoNode will be thrown if the nodes does not exist.

A KeeperException with error code KeeperException.BadVersion will be thrown if the given version does not match the node's version.

A KeeperException with error code KeeperException.NotEmpty will be thrown if the node has children.

This operation, if successful, will trigger all the watches on the node of the given path left by exists API calls, and the watches on the parent node left by getChildren API calls.

Parameters:
path - the path of the node to be deleted.
version - the expected node version.
Throws:
InterruptedException - IF the server transaction is interrupted
KeeperException - If the server signals an error with a non-zero return code.
IllegalArgumentException - if an invalid path is specified

delete

public void delete(String path,
                   int version,
                   AsyncCallback.VoidCallback cb,
                   Object ctx)
The Asynchronous version of delete. The request doesn't actually until the asynchronous callback is called.

See Also:
delete(String, int)

exists

public Stat exists(String path,
                   Watcher watcher)
            throws KeeperException,
                   InterruptedException
Return the stat of the node of the given path. Return null if no such a node exists.

If the watch is non-null and the call is successful (no exception is thrown), a watch will be left on the node with the given path. The watch will be triggered by a successful operation that creates/delete the node or sets the data on the node.

Parameters:
path - the node path
watcher - explicit watcher
Returns:
the stat of the node of the given path; return null if no such a node exists.
Throws:
KeeperException - If the server signals an error
InterruptedException - If the server transaction is interrupted.
IllegalArgumentException - if an invalid path is specified

exists

public Stat exists(String path,
                   boolean watch)
            throws KeeperException,
                   InterruptedException
Return the stat of the node of the given path. Return null if no such a node exists.

If the watch is true and the call is successful (no exception is thrown), a watch will be left on the node with the given path. The watch will be triggered by a successful operation that creates/delete the node or sets the data on the node.

Parameters:
path - the node path
watch - whether need to watch this node
Returns:
the stat of the node of the given path; return null if no such a node exists.
Throws:
KeeperException - If the server signals an error
InterruptedException - If the server transaction is interrupted.

exists

public void exists(String path,
                   Watcher watcher,
                   AsyncCallback.StatCallback cb,
                   Object ctx)
The Asynchronous version of exists. The request doesn't actually until the asynchronous callback is called.

See Also:
exists(String, boolean)

exists

public void exists(String path,
                   boolean watch,
                   AsyncCallback.StatCallback cb,
                   Object ctx)
The Asynchronous version of exists. The request doesn't actually until the asynchronous callback is called.

See Also:
exists(String, boolean)

getData

public byte[] getData(String path,
                      Watcher watcher,
                      Stat stat)
               throws KeeperException,
                      InterruptedException
Return the data and the stat of the node of the given path.

If the watch is non-null and the call is successful (no exception is thrown), a watch will be left on the node with the given path. The watch will be triggered by a successful operation that sets data on the node, or deletes the node.

A KeeperException with error code KeeperException.NoNode will be thrown if no node with the given path exists.

Parameters:
path - the given path
watcher - explicit watcher
stat - the stat of the node
Returns:
the data of the node
Throws:
KeeperException - If the server signals an error with a non-zero error code
InterruptedException - If the server transaction is interrupted.
IllegalArgumentException - if an invalid path is specified

getData

public byte[] getData(String path,
                      boolean watch,
                      Stat stat)
               throws KeeperException,
                      InterruptedException
Return the data and the stat of the node of the given path.

If the watch is true and the call is successful (no exception is thrown), a watch will be left on the node with the given path. The watch will be triggered by a successful operation that sets data on the node, or deletes the node.

A KeeperException with error code KeeperException.NoNode will be thrown if no node with the given path exists.

Parameters:
path - the given path
watch - whether need to watch this node
stat - the stat of the node
Returns:
the data of the node
Throws:
KeeperException - If the server signals an error with a non-zero error code
InterruptedException - If the server transaction is interrupted.

getData

public void getData(String path,
                    Watcher watcher,
                    AsyncCallback.DataCallback cb,
                    Object ctx)
The Asynchronous version of getData. The request doesn't actually until the asynchronous callback is called.

See Also:
getData(String, Watcher, Stat)

getData

public void getData(String path,
                    boolean watch,
                    AsyncCallback.DataCallback cb,
                    Object ctx)
The Asynchronous version of getData. The request doesn't actually until the asynchronous callback is called.

See Also:
getData(String, boolean, Stat)

setData

public Stat setData(String path,
                    byte[] data,
                    int version)
             throws KeeperException,
                    InterruptedException
Set the data for the node of the given path if such a node exists and the given version matches the version of the node (if the given version is -1, it matches any node's versions). Return the stat of the node.

This operation, if successful, will trigger all the watches on the node of the given path left by getData calls.

A KeeperException with error code KeeperException.NoNode will be thrown if no node with the given path exists.

A KeeperException with error code KeeperException.BadVersion will be thrown if the given version does not match the node's version.

The maximum allowable size of the data array is 1 MB (1,048,576 bytes). Arrays larger than this will cause a KeeperExecption to be thrown.

Parameters:
path - the path of the node
data - the data to set
version - the expected matching version
Returns:
the state of the node
Throws:
InterruptedException - If the server transaction is interrupted.
KeeperException - If the server signals an error with a non-zero error code.
IllegalArgumentException - if an invalid path is specified

setData

public void setData(String path,
                    byte[] data,
                    int version,
                    AsyncCallback.StatCallback cb,
                    Object ctx)
The Asynchronous version of setData. The request doesn't actually until the asynchronous callback is called.

See Also:
setData(String, byte[], int)

getACL

public List<ACL> getACL(String path,
                        Stat stat)
                 throws KeeperException,
                        InterruptedException
Return the ACL and stat of the node of the given path.

A KeeperException with error code KeeperException.NoNode will be thrown if no node with the given path exists.

Parameters:
path - the given path for the node
stat - the stat of the node will be copied to this parameter.
Returns:
the ACL array of the given node.
Throws:
InterruptedException - If the server transaction is interrupted.
KeeperException - If the server signals an error with a non-zero error code.
IllegalArgumentException - if an invalid path is specified

getACL

public void getACL(String path,
                   Stat stat,
                   AsyncCallback.ACLCallback cb,
                   Object ctx)
The Asynchronous version of getACL. The request doesn't actually until the asynchronous callback is called.

See Also:
getACL(String, Stat)

setACL

public Stat setACL(String path,
                   List<ACL> acl,
                   int version)
            throws KeeperException,
                   InterruptedException
Set the ACL for the node of the given path if such a node exists and the given version matches the version of the node. Return the stat of the node.

A KeeperException with error code KeeperException.NoNode will be thrown if no node with the given path exists.

A KeeperException with error code KeeperException.BadVersion will be thrown if the given version does not match the node's version.

Parameters:
path -
acl -
version -
Returns:
the stat of the node.
Throws:
InterruptedException - If the server transaction is interrupted.
KeeperException - If the server signals an error with a non-zero error code.
KeeperException.InvalidACLException - If the acl is invalide.
IllegalArgumentException - if an invalid path is specified

setACL

public void setACL(String path,
                   List<ACL> acl,
                   int version,
                   AsyncCallback.StatCallback cb,
                   Object ctx)
The Asynchronous version of setACL. The request doesn't actually until the asynchronous callback is called.

See Also:
setACL(String, List, int)

getChildren

public List<String> getChildren(String path,
                                Watcher watcher)
                         throws KeeperException,
                                InterruptedException
Return the list of the children of the node of the given path.

If the watch is non-null and the call is successful (no exception is thrown), a watch will be left on the node with the given path. The watch willbe triggered by a successful operation that deletes the node of the given path or creates/delete a child under the node.

The list of children returned is not sorted and no guarantee is provided as to its natural or lexical order.

A KeeperException with error code KeeperException.NoNode will be thrown if no node with the given path exists.

Parameters:
path -
watcher - explicit watcher
Returns:
an unordered array of children of the node with the given path
Throws:
InterruptedException - If the server transaction is interrupted.
KeeperException - If the server signals an error with a non-zero error code.
IllegalArgumentException - if an invalid path is specified

getChildren

public List<String> getChildren(String path,
                                boolean watch)
                         throws KeeperException,
                                InterruptedException
Return the list of the children of the node of the given path.

If the watch is true and the call is successful (no exception is thrown), a watch will be left on the node with the given path. The watch willbe triggered by a successful operation that deletes the node of the given path or creates/delete a child under the node.

The list of children returned is not sorted and no guarantee is provided as to its natural or lexical order.

A KeeperException with error code KeeperException.NoNode will be thrown if no node with the given path exists.

Parameters:
path -
watch -
Returns:
an unordered array of children of the node with the given path
Throws:
InterruptedException - If the server transaction is interrupted.
KeeperException - If the server signals an error with a non-zero error code.

getChildren

public void getChildren(String path,
                        Watcher watcher,
                        AsyncCallback.ChildrenCallback cb,
                        Object ctx)
The Asynchronous version of getChildren. The request doesn't actually until the asynchronous callback is called.

See Also:
getChildren(String, Watcher)

getChildren

public void getChildren(String path,
                        boolean watch,
                        AsyncCallback.ChildrenCallback cb,
                        Object ctx)
The Asynchronous version of getChildren. The request doesn't actually until the asynchronous callback is called.

See Also:
getChildren(String, boolean)

sync

public void sync(String path,
                 AsyncCallback.VoidCallback cb,
                 Object ctx)
Asynchronous sync. Flushes channel between process and leader.

Parameters:
path -
cb - a handler for the callback
ctx - context to be provided to the callback
Throws:
IllegalArgumentException - if an invalid path is specified

getState

public ZooKeeper.States getState()


Copyright © 2009 The Apache Software Foundation