Browse Source

[ 1949196 ] KeeperException should be sub-typed

git-svn-id: https://svn.apache.org/repos/asf/hadoop/zookeeper/trunk@670958 13f79535-47bb-0310-9956-ffa450edef68
Benjamin Reed 17 years ago
parent
commit
ffce835123
23 changed files with 3678 additions and 3545 deletions
  1. 840 844
      zookeeper/java/src/com/yahoo/zookeeper/ClientCnxn.java
  2. 246 50
      zookeeper/java/src/com/yahoo/zookeeper/KeeperException.java
  3. 859 874
      zookeeper/java/src/com/yahoo/zookeeper/ZooKeeper.java
  4. 18 21
      zookeeper/java/src/com/yahoo/zookeeper/server/DataTree.java
  5. 243 243
      zookeeper/java/src/com/yahoo/zookeeper/server/FinalRequestProcessor.java
  6. 443 445
      zookeeper/java/src/com/yahoo/zookeeper/server/PrepRequestProcessor.java
  7. 1 1
      zookeeper/java/src/com/yahoo/zookeeper/server/SessionTracker.java
  8. 3 3
      zookeeper/java/src/com/yahoo/zookeeper/server/SessionTrackerImpl.java
  9. 925 927
      zookeeper/java/src/com/yahoo/zookeeper/server/ZooKeeperServer.java
  10. 1 1
      zookeeper/java/src/com/yahoo/zookeeper/server/auth/HostAuthenticationProvider.java
  11. 1 1
      zookeeper/java/src/com/yahoo/zookeeper/server/auth/IPAuthenticationProvider.java
  12. 1 2
      zookeeper/java/src/com/yahoo/zookeeper/server/quorum/FollowerSessionTracker.java
  13. 17 18
      zookeeper/test/com/yahoo/zookeeper/server/DeserializationPerfTest.java
  14. 19 30
      zookeeper/test/com/yahoo/zookeeper/server/SerializationPerfTest.java
  15. 3 3
      zookeeper/test/com/yahoo/zookeeper/test/AsyncTest.java
  16. 21 23
      zookeeper/test/com/yahoo/zookeeper/test/ClientTest.java
  17. 0 3
      zookeeper/test/com/yahoo/zookeeper/test/GenerateLoad.java
  18. 8 12
      zookeeper/test/com/yahoo/zookeeper/test/IntegrityCheck.java
  19. 7 11
      zookeeper/test/com/yahoo/zookeeper/test/OOMTest.java
  20. 3 6
      zookeeper/test/com/yahoo/zookeeper/test/SessionTest.java
  21. 5 5
      zookeeper/test/com/yahoo/zookeeper/test/SledgeHammer.java
  22. 0 4
      zookeeper/test/com/yahoo/zookeeper/test/TestHammer.java
  23. 14 18
      zookeeper/test/com/yahoo/zookeeper/test/ZooKeeperTestClient.java

+ 840 - 844
zookeeper/java/src/com/yahoo/zookeeper/ClientCnxn.java

@@ -1,844 +1,840 @@
-/*
- * Copyright 2008, Yahoo! Inc.
- *
- * 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.
- */
-
-package com.yahoo.zookeeper;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.lang.Thread.UncaughtExceptionHandler;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.Selector;
-import java.nio.channels.SocketChannel;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.log4j.Logger;
-
-import com.yahoo.jute.BinaryInputArchive;
-import com.yahoo.jute.BinaryOutputArchive;
-import com.yahoo.jute.Record;
-import com.yahoo.zookeeper.AsyncCallback.ACLCallback;
-import com.yahoo.zookeeper.AsyncCallback.ChildrenCallback;
-import com.yahoo.zookeeper.AsyncCallback.DataCallback;
-import com.yahoo.zookeeper.AsyncCallback.StatCallback;
-import com.yahoo.zookeeper.AsyncCallback.StringCallback;
-import com.yahoo.zookeeper.AsyncCallback.VoidCallback;
-import com.yahoo.zookeeper.Watcher.Event;
-import com.yahoo.zookeeper.ZooDefs.OpCode;
-import com.yahoo.zookeeper.ZooKeeper.States;
-import com.yahoo.zookeeper.proto.AuthPacket;
-import com.yahoo.zookeeper.proto.ConnectRequest;
-import com.yahoo.zookeeper.proto.ConnectResponse;
-import com.yahoo.zookeeper.proto.CreateResponse;
-import com.yahoo.zookeeper.proto.ExistsResponse;
-import com.yahoo.zookeeper.proto.GetACLResponse;
-import com.yahoo.zookeeper.proto.GetChildrenResponse;
-import com.yahoo.zookeeper.proto.GetDataResponse;
-import com.yahoo.zookeeper.proto.ReplyHeader;
-import com.yahoo.zookeeper.proto.RequestHeader;
-import com.yahoo.zookeeper.proto.SetACLResponse;
-import com.yahoo.zookeeper.proto.SetDataResponse;
-import com.yahoo.zookeeper.proto.WatcherEvent;
-import com.yahoo.zookeeper.server.ByteBufferInputStream;
-import com.yahoo.zookeeper.server.ZooKeeperServer;
-import com.yahoo.zookeeper.server.ZooTrace;
-
-/**
- * This class manages the socket i/o for the client. ClientCnxn maintains a list
- * of available servers to connect to and "transparently" switches servers it is
- * connected to as needed.
- *
- */
-class ClientCnxn {
-    private static final Logger LOG = Logger.getLogger(ZooKeeperServer.class);
-
-    private ArrayList<InetSocketAddress> serverAddrs = new ArrayList<InetSocketAddress>();
-
-    static class AuthData {
-        AuthData(String scheme, byte data[]) {
-            this.scheme = scheme;
-            this.data = data;
-        }
-
-        String scheme;
-
-        byte data[];
-    }
-
-    private ArrayList<AuthData> authInfo = new ArrayList<AuthData>();
-
-    /**
-     * These are the packets that have been sent and are waiting for a response.
-     */
-    private LinkedList<Packet> pendingQueue = new LinkedList<Packet>();
-
-    private LinkedBlockingQueue waitingEvents = new LinkedBlockingQueue();
-
-    /**
-     * These are the packets that need to be sent.
-     */
-    private LinkedList<Packet> outgoingQueue = new LinkedList<Packet>();
-
-    private int nextAddrToTry = 0;
-
-    private int connectTimeout;
-
-    private int readTimeout;
-
-    private final int sessionTimeout;
-
-    private final ZooKeeper zooKeeper;
-
-    private long sessionId;
-
-    private byte sessionPasswd[] = new byte[16];
-
-    final SendThread sendThread;
-
-    final EventThread eventThread;
-
-    final Selector selector = Selector.open();
-
-    public long getSessionId() {
-        return sessionId;
-    }
-
-    public byte[] getSessionPasswd() {
-        return sessionPasswd;
-    }
-
-    public String toString() {
-        StringBuffer sb = new StringBuffer();
-        sb.append("sessionId: ").append(sessionId).append("\n");
-        sb.append("lastZxid: ").append(lastZxid).append("\n");
-        sb.append("xid: ").append(xid).append("\n");
-        sb.append("nextAddrToTry: ").append(nextAddrToTry).append("\n");
-        sb.append("serverAddrs: ").append(serverAddrs.get(nextAddrToTry))
-                .append("\n");
-        return sb.toString();
-    }
-
-    /**
-     * This class allows us to pass the headers and the relevant records around.
-     */
-    static class Packet {
-        RequestHeader header;
-
-        ByteBuffer bb;
-
-        String path;
-
-        ReplyHeader replyHeader;
-
-        Record request;
-
-        Record response;
-
-        boolean finished;
-
-        AsyncCallback cb;
-
-        Object ctx;
-
-        Packet(RequestHeader header, ReplyHeader replyHeader, Record record,
-                Record response, ByteBuffer bb) {
-            this.header = header;
-            this.replyHeader = replyHeader;
-            this.request = record;
-            this.response = response;
-            if (bb != null) {
-                this.bb = bb;
-            } else {
-                try {
-                    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-                    BinaryOutputArchive boa = BinaryOutputArchive
-                            .getArchive(baos);
-                    boa.writeInt(-1, "len"); // We'll fill this in later
-                    header.serialize(boa, "header");
-                    if (record != null) {
-                        record.serialize(boa, "request");
-                    }
-                    baos.close();
-                    this.bb = ByteBuffer.wrap(baos.toByteArray());
-                    this.bb.putInt(this.bb.capacity() - 4);
-                    this.bb.rewind();
-                } catch (IOException e) {
-                    LOG.warn("Unexpected exception",e);
-                }
-            }
-        }
-    }
-
-    public ClientCnxn(String hosts, int sessionTimeout, ZooKeeper zooKeeper)
-            throws KeeperException, IOException {
-        this(hosts, sessionTimeout, zooKeeper, 0, new byte[16]);
-    }
-
-    /**
-     * Creates a connection object. The actual network connect doesn't get
-     * established until needed.
-     *
-     * @param hosts
-     *                a comma separated list of hosts that can be connected to.
-     * @param connectTimeout
-     *                the timeout for connections.
-     * @param readTimeout
-     *                the read timeout.
-     * @param zooKeeper
-     *                the zookeeper object that this connection is related to.
-     * @throws KeeperException
-     * @throws IOException
-     */
-    public ClientCnxn(String hosts, int sessionTimeout, ZooKeeper zooKeeper,
-            long sessionId, byte[] sessionPasswd) throws KeeperException,
-            IOException {
-        this.zooKeeper = zooKeeper;
-        this.sessionId = sessionId;
-        this.sessionPasswd = sessionPasswd;
-        String hostsList[] = hosts.split(",");
-        for (String host : hostsList) {
-            int port = 2181;
-            String parts[] = host.split(":");
-            if (parts.length > 1) {
-                port = Integer.parseInt(parts[1]);
-                host = parts[0];
-            }
-            InetAddress addrs[] = InetAddress.getAllByName(host);
-            for (InetAddress addr : addrs) {
-                serverAddrs.add(new InetSocketAddress(addr, port));
-            }
-        }
-        this.sessionTimeout = sessionTimeout;
-        connectTimeout = sessionTimeout / hostsList.length;
-        readTimeout = sessionTimeout * 2 / 3;
-        Collections.shuffle(serverAddrs);
-        sendThread = new SendThread();
-        sendThread.start();
-        eventThread = new EventThread();
-        eventThread.start();
-    }
-
-    WatcherEvent eventOfDeath = new WatcherEvent();
-
-    final static UncaughtExceptionHandler uncaughtExceptionHandler = new UncaughtExceptionHandler() {
-        public void uncaughtException(Thread t, Throwable e) {
-            LOG.error("from " + t.getName(), e);
-        }
-    };
-
-    class EventThread extends Thread {
-        EventThread() {
-            super("EventThread");
-            setUncaughtExceptionHandler(uncaughtExceptionHandler);
-            setDaemon(true);
-        }
-
-        public void run() {
-            try {
-                while (true) {
-                    Object event = waitingEvents.take();
-                    if (event == eventOfDeath) {
-                        break;
-                    }
-                    if (event instanceof WatcherEvent) {
-                        zooKeeper.watcher.process((WatcherEvent) event);
-                    } else {
-                        Packet p = (Packet) event;
-                        int rc = 0;
-                        String path = p.path;
-                        if (p.replyHeader.getErr() != 0) {
-                            rc = p.replyHeader.getErr();
-                        }
-                        if (p.cb == null) {
-                            LOG.warn("Somehow a null cb got to EventThread!");
-                        } else if (p.response instanceof ExistsResponse
-                                || p.response instanceof SetDataResponse
-                                || p.response instanceof SetACLResponse) {
-                            StatCallback cb = (StatCallback) p.cb;
-                            if (rc == 0) {
-                                if (p.response instanceof ExistsResponse) {
-                                    cb.processResult(rc, path, p.ctx,
-                                            ((ExistsResponse) p.response)
-                                                    .getStat());
-                                } else if (p.response instanceof SetDataResponse) {
-                                    cb.processResult(rc, path, p.ctx,
-                                            ((SetDataResponse) p.response)
-                                                    .getStat());
-                                } else if (p.response instanceof SetACLResponse) {
-                                    cb.processResult(rc, path, p.ctx,
-                                            ((SetACLResponse) p.response)
-                                                    .getStat());
-                                }
-                            } else {
-                                cb.processResult(rc, path, p.ctx, null);
-                            }
-                        } else if (p.response instanceof GetDataResponse) {
-                            DataCallback cb = (DataCallback) p.cb;
-                            GetDataResponse rsp = (GetDataResponse) p.response;
-                            if (rc == 0) {
-                                cb.processResult(rc, path, p.ctx,
-                                        rsp.getData(), rsp.getStat());
-                            } else {
-                                cb.processResult(rc, path, p.ctx, null, null);
-                            }
-                        } else if (p.response instanceof GetACLResponse) {
-                            ACLCallback cb = (ACLCallback) p.cb;
-                            GetACLResponse rsp = (GetACLResponse) p.response;
-                            if (rc == 0) {
-                                cb.processResult(rc, path, p.ctx, rsp.getAcl(),
-                                        rsp.getStat());
-                            } else {
-                                cb.processResult(rc, path, p.ctx, null, null);
-                            }
-                        } else if (p.response instanceof GetChildrenResponse) {
-                            ChildrenCallback cb = (ChildrenCallback) p.cb;
-                            GetChildrenResponse rsp = (GetChildrenResponse) p.response;
-                            if (rc == 0) {
-                                cb.processResult(rc, path, p.ctx, rsp
-                                        .getChildren());
-                            } else {
-                                cb.processResult(rc, path, p.ctx, null);
-                            }
-                        } else if (p.response instanceof CreateResponse) {
-                            StringCallback cb = (StringCallback) p.cb;
-                            CreateResponse rsp = (CreateResponse) p.response;
-                            if (rc == 0) {
-                                cb
-                                        .processResult(rc, path, p.ctx, rsp
-                                                .getPath());
-                            } else {
-                                cb.processResult(rc, path, p.ctx, null);
-                            }
-                        } else if (p.cb instanceof VoidCallback) {
-                            VoidCallback cb = (VoidCallback) p.cb;
-                            cb.processResult(rc, path, p.ctx);
-                        }
-                    }
-                }
-            } catch (InterruptedException e) {
-            }
-        }
-    }
-
-    @SuppressWarnings("unchecked")
-    private void finishPacket(Packet p) {
-        p.finished = true;
-        if (p.cb == null) {
-            synchronized (p) {
-                p.notifyAll();
-            }
-        } else {
-            waitingEvents.add(p);
-        }
-    }
-
-    private void conLossPacket(Packet p) {
-        if (p.replyHeader == null) {
-            return;
-        }
-        switch(zooKeeper.state) {
-        case AUTH_FAILED:
-            p.replyHeader.setErr(KeeperException.Code.AuthFailed);
-            break;
-        case CLOSED:
-            p.replyHeader.setErr(KeeperException.Code.SessionExpired);
-            break;
-        default:
-            p.replyHeader.setErr(KeeperException.Code.ConnectionLoss);
-        }
-        finishPacket(p);
-    }
-
-    long lastZxid;
-
-    /**
-     * This class services the outgoing request queue and generates the heart
-     * beats. It also spawns the ReadThread.
-     */
-    class SendThread extends Thread {
-        SelectionKey sockKey;
-
-        ByteBuffer lenBuffer = ByteBuffer.allocateDirect(4);
-
-        ByteBuffer incomingBuffer = lenBuffer;
-
-        boolean initialized;
-
-        void readLength() throws IOException {
-            int len = incomingBuffer.getInt();
-            if (len < 0 || len >= 4096 * 1024) {
-                throw new IOException("Packet len" + len + " is out of range!");
-            }
-            incomingBuffer = ByteBuffer.allocate(len);
-        }
-
-        void readConnectResult() throws IOException {
-            ByteBufferInputStream bbis = new ByteBufferInputStream(
-                    incomingBuffer);
-            BinaryInputArchive bbia = BinaryInputArchive.getArchive(bbis);
-            ConnectResponse conRsp = new ConnectResponse();
-            conRsp.deserialize(bbia, "connect");
-            int sessionTimeout = conRsp.getTimeOut();
-            if (sessionTimeout <= 0) {
-                zooKeeper.state = States.CLOSED;
-                waitingEvents.add(new WatcherEvent(Watcher.Event.EventNone,
-                        Watcher.Event.KeeperStateExpired, null));
-                throw new IOException("Session Expired");
-            }
-            readTimeout = sessionTimeout * 2 / 3;
-            connectTimeout = sessionTimeout / serverAddrs.size();
-            sessionId = conRsp.getSessionId();
-            sessionPasswd = conRsp.getPasswd();
-            waitingEvents.add(new WatcherEvent(Watcher.Event.EventNone,
-                    Watcher.Event.KeeperStateSyncConnected, null));
-        }
-
-        @SuppressWarnings("unchecked")
-        void readResponse() throws IOException {
-            ByteBufferInputStream bbis = new ByteBufferInputStream(
-                    incomingBuffer);
-            BinaryInputArchive bbia = BinaryInputArchive.getArchive(bbis);
-            ReplyHeader r = new ReplyHeader();
-
-            r.deserialize(bbia, "header");
-            if (r.getXid() == -2) {
-                // -2 is the xid for pings
-                return;
-            }
-            if (r.getXid() == -4) {
-                // -2 is the xid for AuthPacket
-                // TODO: process AuthPacket here
-                return;
-            }
-            if (r.getXid() == -1) {
-                // -1 means notification
-                WatcherEvent event = new WatcherEvent();
-                event.deserialize(bbia, "response");
-                // System.out.println("Got an event: " + event + " for " +
-                // sessionId + " through" + _cnxn);
-                waitingEvents.add(event);
-                return;
-            }
-            if (pendingQueue.size() == 0) {
-                throw new IOException("Nothing in the queue, but got "
-                        + r.getXid());
-            }
-            Packet p = null;
-            synchronized (pendingQueue) {
-                p = pendingQueue.remove();
-            }
-            /*
-             * Since requests are processed in order, we better get a response
-             * to the first request!
-             */
-            if (p.header.getXid() != r.getXid()) {
-                throw new IOException("Xid out of order. Got " + r.getXid()
-                        + " expected " + p.header.getXid());
-            }
-            p.replyHeader.setXid(r.getXid());
-            p.replyHeader.setErr(r.getErr());
-            p.replyHeader.setZxid(r.getZxid());
-            lastZxid = r.getZxid();
-            if (p.response != null && r.getErr() == 0) {
-                p.response.deserialize(bbia, "response");
-            }
-            p.finished = true;
-            finishPacket(p);
-        }
-
-        /**
-         * @return true if a packet was received
-         * @param k
-         * @throws InterruptedException
-         * @throws IOException
-         */
-        boolean doIO() throws InterruptedException, IOException {
-            boolean packetReceived = false;
-            SocketChannel sock = (SocketChannel) sockKey.channel();
-            if (sock == null) {
-                throw new IOException("Socket is null!");
-            }
-            if (sockKey.isReadable()) {
-                int rc = sock.read(incomingBuffer);
-                if (rc < 0) {
-                    throw new IOException("Read error rc = " + rc + " "
-                            + incomingBuffer);
-                }
-                if (incomingBuffer.remaining() == 0) {
-                    incomingBuffer.flip();
-                    if (incomingBuffer == lenBuffer) {
-                        readLength();
-                    } else if (!initialized) {
-                        readConnectResult();
-                        enableRead();
-                        if (outgoingQueue.size() > 0) {
-                            enableWrite();
-                        }
-                        lenBuffer.clear();
-                        incomingBuffer = lenBuffer;
-                        packetReceived = true;
-                        initialized = true;
-                    } else {
-                        readResponse();
-                        lenBuffer.clear();
-                        incomingBuffer = lenBuffer;
-                        packetReceived = true;
-                    }
-                }
-            }
-            if (sockKey.isWritable()) {
-                synchronized (outgoingQueue) {
-                    if (outgoingQueue.size() > 0) {
-                        int rc = sock.write(outgoingQueue.getFirst().bb);
-                        if (outgoingQueue.getFirst().bb.remaining() == 0) {
-                            Packet p = outgoingQueue.removeFirst();
-                            if (p.header != null
-                                    && p.header.getType() != OpCode.ping
-                                    && p.header.getType() != OpCode.auth) {
-                                pendingQueue.add(p);
-                            }
-                        }
-                    }
-                }
-            }
-            if (outgoingQueue.size() == 0) {
-                disableWrite();
-            } else {
-                enableWrite();
-            }
-            return packetReceived;
-        }
-
-        synchronized private void enableWrite() {
-            int i = sockKey.interestOps();
-            if ((i & SelectionKey.OP_WRITE) == 0) {
-                sockKey.interestOps(i | SelectionKey.OP_WRITE);
-            }
-        }
-
-        synchronized private void disableWrite() {
-            int i = sockKey.interestOps();
-            if ((i & SelectionKey.OP_WRITE) != 0) {
-                sockKey.interestOps(i & (~SelectionKey.OP_WRITE));
-            }
-        }
-
-        synchronized private void enableRead() {
-            int i = sockKey.interestOps();
-            if ((i & SelectionKey.OP_READ) == 0) {
-                sockKey.interestOps(i | SelectionKey.OP_READ);
-            }
-        }
-
-        synchronized private void disableRead() {
-            int i = sockKey.interestOps();
-            if ((i & SelectionKey.OP_READ) != 0) {
-                sockKey.interestOps(i & (~SelectionKey.OP_READ));
-            }
-        }
-
-        SendThread() {
-            super("SendThread");
-            zooKeeper.state = States.CONNECTING;
-            setUncaughtExceptionHandler(uncaughtExceptionHandler);
-            setDaemon(true);
-        }
-
-        private void primeConnection(SelectionKey k) throws IOException {
-            LOG.info("Priming connection to "
-                    + ((SocketChannel) sockKey.channel()));
-            lastConnectIndex = currentConnectIndex;
-            ConnectRequest conReq = new ConnectRequest(0, lastZxid,
-                    sessionTimeout, sessionId, sessionPasswd);
-            ByteArrayOutputStream baos = new ByteArrayOutputStream();
-            BinaryOutputArchive boa = BinaryOutputArchive.getArchive(baos);
-            boa.writeInt(-1, "len");
-            conReq.serialize(boa, "connect");
-            baos.close();
-            ByteBuffer bb = ByteBuffer.wrap(baos.toByteArray());
-            bb.putInt(bb.capacity() - 4);
-            bb.rewind();
-            synchronized (outgoingQueue) {
-                for (AuthData id : authInfo) {
-                    outgoingQueue.addFirst(new Packet(new RequestHeader(-4,
-                            OpCode.auth), null, new AuthPacket(0, id.scheme,
-                            id.data), null, null));
-                }
-                outgoingQueue
-                        .addFirst((new Packet(null, null, null, null, bb)));
-            }
-            synchronized (this) {
-                k.interestOps(SelectionKey.OP_READ | SelectionKey.OP_WRITE);
-            }
-        }
-
-        private void sendPing() {
-            RequestHeader h = new RequestHeader(-2, OpCode.ping);
-            queuePacket(h, null, null, null, null, null, null);
-        }
-
-        int lastConnectIndex = -1;
-
-        int currentConnectIndex;
-
-        Random r = new Random(System.nanoTime());
-
-        private void startConnect() throws IOException {
-            if (lastConnectIndex == -1) {
-                // We don't want to delay the first try at a connect, so we
-                // start with -1 the first time around
-                lastConnectIndex = 0;
-            } else {
-                try {
-                    Thread.sleep(r.nextInt(1000));
-                } catch (InterruptedException e1) {
-                    LOG.warn("Unexpected exception", e1);
-                }
-                if (nextAddrToTry == lastConnectIndex) {
-                    try {
-                        // Try not to spin too fast!
-                        Thread.sleep(1000);
-                    } catch (InterruptedException e) {
-                        LOG.warn("Unexpected exception", e);
-                    }
-                }
-            }
-            zooKeeper.state = States.CONNECTING;
-            currentConnectIndex = nextAddrToTry;
-            InetSocketAddress addr = serverAddrs.get(nextAddrToTry);
-            nextAddrToTry++;
-            if (nextAddrToTry == serverAddrs.size()) {
-                nextAddrToTry = 0;
-            }
-            SocketChannel sock;
-            sock = SocketChannel.open();
-            sock.configureBlocking(false);
-            sock.socket().setSoLinger(false, -1);
-            sock.socket().setTcpNoDelay(true);
-            LOG.info("Attempting connection to server " + addr);
-            sockKey = sock.register(selector, SelectionKey.OP_CONNECT);
-            if (sock.connect(addr)) {
-                primeConnection(sockKey);
-            }
-            initialized = false;
-        }
-
-        @Override
-        public void run() {
-            long now = System.currentTimeMillis();
-            long lastHeard = now;
-            long lastSend = now;
-            while (zooKeeper.state.isAlive()) {
-                try {
-                    if (sockKey == null) {
-                        startConnect();
-                        lastSend = now;
-                        lastHeard = now;
-                    }
-                    int idleRecv = (int) (now - lastHeard);
-                    int idleSend = (int) (now - lastSend);
-                    int to = readTimeout - idleRecv;
-                    if (zooKeeper.state != States.CONNECTED) {
-                        to = connectTimeout - idleRecv;
-                    }
-                    if (to <= 0) {
-                        throw new IOException("TIMED OUT");
-                    }
-                    if (zooKeeper.state == States.CONNECTED) {
-                        int timeToNextPing = readTimeout/2 - idleSend;
-                        if (timeToNextPing <= 0) {
-                            sendPing();
-                            lastSend = now;
-                            enableWrite();
-                        } else {
-                            if (timeToNextPing < to) {
-                                to = timeToNextPing;
-                            }
-                        }
-                    }
-
-                    selector.select(to);
-                    Set<SelectionKey> selected;
-                    synchronized (this) {
-                        selected = selector.selectedKeys();
-                    }
-                    // Everything below and until we get back to the select is
-                    // non blocking, so time is effectively a constant. That is
-                    // Why we just have to do this once, here
-                    now = System.currentTimeMillis();
-                    for (SelectionKey k : selected) {
-                        SocketChannel sc = ((SocketChannel) k.channel());
-                        if ((k.readyOps() & SelectionKey.OP_CONNECT) != 0) {
-                            if (sc.finishConnect()) {
-                                zooKeeper.state = States.CONNECTED;
-                                lastHeard = now;
-                                lastSend = now;
-                                primeConnection(k);
-                                LOG.info("Server connection successful");
-                            }
-                        } else if ((k.readyOps() & (SelectionKey.OP_READ | SelectionKey.OP_WRITE)) != 0) {
-                            if (outgoingQueue.size() > 0) {
-                                // We have something to send so it's the same
-                                // as if we do the send now.
-                                lastSend = now;
-                            }
-                            if (doIO()) {
-                                lastHeard = now;
-                            }
-                        }
-                    }
-                    if (zooKeeper.state == States.CONNECTED) {
-                        if (outgoingQueue.size() > 0) {
-                            enableWrite();
-                        } else {
-                            disableWrite();
-                        }
-                    }
-                    selected.clear();
-                } catch (Exception e) {
-                    LOG.warn("Closing: ", e);
-                    cleanup();
-                    if (zooKeeper.state.isAlive()) {
-                        waitingEvents.add(new WatcherEvent(Event.EventNone,
-                                Event.KeeperStateDisconnected, null));
-                    }
-
-                    now = System.currentTimeMillis();
-                    lastHeard = now;
-                    lastSend = now;
-                }
-            }
-            cleanup();
-            ZooTrace.logTraceMessage(LOG, ZooTrace.getTextTraceLevel(),
-                                     "SendThread exitedloop.");
-        }
-
-        private void cleanup() {
-            if (sockKey != null) {
-                SocketChannel sock = (SocketChannel) sockKey.channel();
-                sockKey.cancel();
-                try {
-                    sock.socket().shutdownInput();
-                } catch (IOException e2) {
-                }
-                try {
-                    sock.socket().shutdownOutput();
-                } catch (IOException e2) {
-                }
-                try {
-                    sock.socket().close();
-                } catch (IOException e1) {
-                }
-                try {
-                    sock.close();
-                } catch (IOException e1) {
-                }
-            }
-            try {
-                Thread.sleep(100);
-            } catch (InterruptedException e1) {
-                e1.printStackTrace();
-            }
-            sockKey = null;
-            synchronized (pendingQueue) {
-                for (Packet p : pendingQueue) {
-                    conLossPacket(p);
-                }
-                pendingQueue.clear();
-            }
-            synchronized (outgoingQueue) {
-                for (Packet p : outgoingQueue) {
-                    conLossPacket(p);
-                }
-                outgoingQueue.clear();
-            }
-        }
-
-        public void close() {
-            zooKeeper.state = States.CLOSED;
-            synchronized (this) {
-                selector.wakeup();
-            }
-        }
-    }
-
-    @SuppressWarnings("unchecked")
-    public void close() throws IOException {
-        long traceMask = ZooTrace.SESSION_TRACE_MASK;
-        if (ZooTrace.isTraceEnabled(LOG, traceMask)) {
-            ZooTrace.logTraceMessage(LOG, traceMask,
-                    "Close ClientCnxn for session: " + sessionId + "!");
-        }
-        sendThread.close();
-        waitingEvents.add(eventOfDeath);
-    }
-
-    private int xid = 1;
-
-    synchronized private int getXid() {
-        return xid++;
-    }
-
-    public ReplyHeader submitRequest(RequestHeader h, Record request,
-            Record response) throws InterruptedException {
-        ReplyHeader r = new ReplyHeader();
-        Packet packet = queuePacket(h, r, request, response, null, null, null);
-        synchronized (packet) {
-            while (!packet.finished) {
-                packet.wait();
-            }
-        }
-        return r;
-    }
-
-    Packet queuePacket(RequestHeader h, ReplyHeader r, Record request,
-            Record response, AsyncCallback cb, String path, Object ctx) {
-        Packet packet = null;
-        synchronized (outgoingQueue) {
-            if (h.getType() != OpCode.ping && h.getType() != OpCode.auth) {
-                h.setXid(getXid());
-            }
-            packet = new Packet(h, r, request, response, null);
-            packet.cb = cb;
-            packet.ctx = ctx;
-            packet.path = path;
-            if (!zooKeeper.state.isAlive()) {
-                conLossPacket(packet);
-            } else {
-                outgoingQueue.add(packet);
-            }
-        }
-        synchronized (sendThread) {
-            selector.wakeup();
-        }
-        return packet;
-    }
-
-    public void addAuthInfo(String scheme, byte auth[]) {
-        authInfo.add(new AuthData(scheme, auth));
-        if (zooKeeper.state == States.CONNECTED) {
-            queuePacket(new RequestHeader(-4, OpCode.auth), null,
-                    new AuthPacket(0, scheme, auth), null, null, null, null);
-        }
-    }
-}
+/*
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * 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.
+ */
+
+package com.yahoo.zookeeper;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.lang.Thread.UncaughtExceptionHandler;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.Selector;
+import java.nio.channels.SocketChannel;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.log4j.Logger;
+
+import com.yahoo.jute.BinaryInputArchive;
+import com.yahoo.jute.BinaryOutputArchive;
+import com.yahoo.jute.Record;
+import com.yahoo.zookeeper.AsyncCallback.ACLCallback;
+import com.yahoo.zookeeper.AsyncCallback.ChildrenCallback;
+import com.yahoo.zookeeper.AsyncCallback.DataCallback;
+import com.yahoo.zookeeper.AsyncCallback.StatCallback;
+import com.yahoo.zookeeper.AsyncCallback.StringCallback;
+import com.yahoo.zookeeper.AsyncCallback.VoidCallback;
+import com.yahoo.zookeeper.Watcher.Event;
+import com.yahoo.zookeeper.ZooDefs.OpCode;
+import com.yahoo.zookeeper.ZooKeeper.States;
+import com.yahoo.zookeeper.proto.AuthPacket;
+import com.yahoo.zookeeper.proto.ConnectRequest;
+import com.yahoo.zookeeper.proto.ConnectResponse;
+import com.yahoo.zookeeper.proto.CreateResponse;
+import com.yahoo.zookeeper.proto.ExistsResponse;
+import com.yahoo.zookeeper.proto.GetACLResponse;
+import com.yahoo.zookeeper.proto.GetChildrenResponse;
+import com.yahoo.zookeeper.proto.GetDataResponse;
+import com.yahoo.zookeeper.proto.ReplyHeader;
+import com.yahoo.zookeeper.proto.RequestHeader;
+import com.yahoo.zookeeper.proto.SetACLResponse;
+import com.yahoo.zookeeper.proto.SetDataResponse;
+import com.yahoo.zookeeper.proto.WatcherEvent;
+import com.yahoo.zookeeper.server.ByteBufferInputStream;
+import com.yahoo.zookeeper.server.ZooKeeperServer;
+import com.yahoo.zookeeper.server.ZooTrace;
+
+/**
+ * This class manages the socket i/o for the client. ClientCnxn maintains a list
+ * of available servers to connect to and "transparently" switches servers it is
+ * connected to as needed.
+ *
+ */
+class ClientCnxn {
+    private static final Logger LOG = Logger.getLogger(ZooKeeperServer.class);
+
+    private ArrayList<InetSocketAddress> serverAddrs = new ArrayList<InetSocketAddress>();
+
+    static class AuthData {
+        AuthData(String scheme, byte data[]) {
+            this.scheme = scheme;
+            this.data = data;
+        }
+
+        String scheme;
+
+        byte data[];
+    }
+
+    private ArrayList<AuthData> authInfo = new ArrayList<AuthData>();
+
+    /**
+     * These are the packets that have been sent and are waiting for a response.
+     */
+    private LinkedList<Packet> pendingQueue = new LinkedList<Packet>();
+
+    private LinkedBlockingQueue waitingEvents = new LinkedBlockingQueue();
+
+    /**
+     * These are the packets that need to be sent.
+     */
+    private LinkedList<Packet> outgoingQueue = new LinkedList<Packet>();
+
+    private int nextAddrToTry = 0;
+
+    private int connectTimeout;
+
+    private int readTimeout;
+
+    private final int sessionTimeout;
+
+    private final ZooKeeper zooKeeper;
+
+    private long sessionId;
+
+    private byte sessionPasswd[] = new byte[16];
+
+    final SendThread sendThread;
+
+    final EventThread eventThread;
+
+    final Selector selector = Selector.open();
+
+    public long getSessionId() {
+        return sessionId;
+    }
+
+    public byte[] getSessionPasswd() {
+        return sessionPasswd;
+    }
+
+    public String toString() {
+        StringBuffer sb = new StringBuffer();
+        sb.append("sessionId: ").append(sessionId).append("\n");
+        sb.append("lastZxid: ").append(lastZxid).append("\n");
+        sb.append("xid: ").append(xid).append("\n");
+        sb.append("nextAddrToTry: ").append(nextAddrToTry).append("\n");
+        sb.append("serverAddrs: ").append(serverAddrs.get(nextAddrToTry))
+                .append("\n");
+        return sb.toString();
+    }
+
+    /**
+     * This class allows us to pass the headers and the relevant records around.
+     */
+    static class Packet {
+        RequestHeader header;
+
+        ByteBuffer bb;
+
+        String path;
+
+        ReplyHeader replyHeader;
+
+        Record request;
+
+        Record response;
+
+        boolean finished;
+
+        AsyncCallback cb;
+
+        Object ctx;
+
+        Packet(RequestHeader header, ReplyHeader replyHeader, Record record,
+                Record response, ByteBuffer bb) {
+            this.header = header;
+            this.replyHeader = replyHeader;
+            this.request = record;
+            this.response = response;
+            if (bb != null) {
+                this.bb = bb;
+            } else {
+                try {
+                    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+                    BinaryOutputArchive boa = BinaryOutputArchive
+                            .getArchive(baos);
+                    boa.writeInt(-1, "len"); // We'll fill this in later
+                    header.serialize(boa, "header");
+                    if (record != null) {
+                        record.serialize(boa, "request");
+                    }
+                    baos.close();
+                    this.bb = ByteBuffer.wrap(baos.toByteArray());
+                    this.bb.putInt(this.bb.capacity() - 4);
+                    this.bb.rewind();
+                } catch (IOException e) {
+                    LOG.warn("Unexpected exception",e);
+                }
+            }
+        }
+    }
+
+    public ClientCnxn(String hosts, int sessionTimeout, ZooKeeper zooKeeper)
+            throws IOException {
+        this(hosts, sessionTimeout, zooKeeper, 0, new byte[16]);
+    }
+
+    /**
+     * Creates a connection object. The actual network connect doesn't get
+     * established until needed.
+     *
+     * @param hosts
+     *                a comma separated list of hosts that can be connected to.
+     * @param sessionTimeout
+     *                the timeout for connections.
+     * @param zooKeeper
+     *                the zookeeper object that this connection is related to.
+     * @throws KeeperException
+     * @throws IOException
+     */
+    public ClientCnxn(String hosts, int sessionTimeout, ZooKeeper zooKeeper,
+            long sessionId, byte[] sessionPasswd) throws IOException {
+        this.zooKeeper = zooKeeper;
+        this.sessionId = sessionId;
+        this.sessionPasswd = sessionPasswd;
+        String hostsList[] = hosts.split(",");
+        for (String host : hostsList) {
+            int port = 2181;
+            String parts[] = host.split(":");
+            if (parts.length > 1) {
+                port = Integer.parseInt(parts[1]);
+                host = parts[0];
+            }
+            InetAddress addrs[] = InetAddress.getAllByName(host);
+            for (InetAddress addr : addrs) {
+                serverAddrs.add(new InetSocketAddress(addr, port));
+            }
+        }
+        this.sessionTimeout = sessionTimeout;
+        connectTimeout = sessionTimeout / hostsList.length;
+        readTimeout = sessionTimeout * 2 / 3;
+        Collections.shuffle(serverAddrs);
+        sendThread = new SendThread();
+        sendThread.start();
+        eventThread = new EventThread();
+        eventThread.start();
+    }
+
+    WatcherEvent eventOfDeath = new WatcherEvent();
+
+    final static UncaughtExceptionHandler uncaughtExceptionHandler = new UncaughtExceptionHandler() {
+        public void uncaughtException(Thread t, Throwable e) {
+            LOG.error("from " + t.getName(), e);
+        }
+    };
+
+    class EventThread extends Thread {
+        EventThread() {
+            super("EventThread");
+            setUncaughtExceptionHandler(uncaughtExceptionHandler);
+            setDaemon(true);
+        }
+
+        public void run() {
+            try {
+                while (true) {
+                    Object event = waitingEvents.take();
+                    if (event == eventOfDeath) {
+                        break;
+                    }
+                    if (event instanceof WatcherEvent) {
+                        zooKeeper.watcher.process((WatcherEvent) event);
+                    } else {
+                        Packet p = (Packet) event;
+                        int rc = 0;
+                        String path = p.path;
+                        if (p.replyHeader.getErr() != 0) {
+                            rc = p.replyHeader.getErr();
+                        }
+                        if (p.cb == null) {
+                            LOG.warn("Somehow a null cb got to EventThread!");
+                        } else if (p.response instanceof ExistsResponse
+                                || p.response instanceof SetDataResponse
+                                || p.response instanceof SetACLResponse) {
+                            StatCallback cb = (StatCallback) p.cb;
+                            if (rc == 0) {
+                                if (p.response instanceof ExistsResponse) {
+                                    cb.processResult(rc, path, p.ctx,
+                                            ((ExistsResponse) p.response)
+                                                    .getStat());
+                                } else if (p.response instanceof SetDataResponse) {
+                                    cb.processResult(rc, path, p.ctx,
+                                            ((SetDataResponse) p.response)
+                                                    .getStat());
+                                } else if (p.response instanceof SetACLResponse) {
+                                    cb.processResult(rc, path, p.ctx,
+                                            ((SetACLResponse) p.response)
+                                                    .getStat());
+                                }
+                            } else {
+                                cb.processResult(rc, path, p.ctx, null);
+                            }
+                        } else if (p.response instanceof GetDataResponse) {
+                            DataCallback cb = (DataCallback) p.cb;
+                            GetDataResponse rsp = (GetDataResponse) p.response;
+                            if (rc == 0) {
+                                cb.processResult(rc, path, p.ctx,
+                                        rsp.getData(), rsp.getStat());
+                            } else {
+                                cb.processResult(rc, path, p.ctx, null, null);
+                            }
+                        } else if (p.response instanceof GetACLResponse) {
+                            ACLCallback cb = (ACLCallback) p.cb;
+                            GetACLResponse rsp = (GetACLResponse) p.response;
+                            if (rc == 0) {
+                                cb.processResult(rc, path, p.ctx, rsp.getAcl(),
+                                        rsp.getStat());
+                            } else {
+                                cb.processResult(rc, path, p.ctx, null, null);
+                            }
+                        } else if (p.response instanceof GetChildrenResponse) {
+                            ChildrenCallback cb = (ChildrenCallback) p.cb;
+                            GetChildrenResponse rsp = (GetChildrenResponse) p.response;
+                            if (rc == 0) {
+                                cb.processResult(rc, path, p.ctx, rsp
+                                        .getChildren());
+                            } else {
+                                cb.processResult(rc, path, p.ctx, null);
+                            }
+                        } else if (p.response instanceof CreateResponse) {
+                            StringCallback cb = (StringCallback) p.cb;
+                            CreateResponse rsp = (CreateResponse) p.response;
+                            if (rc == 0) {
+                                cb
+                                        .processResult(rc, path, p.ctx, rsp
+                                                .getPath());
+                            } else {
+                                cb.processResult(rc, path, p.ctx, null);
+                            }
+                        } else if (p.cb instanceof VoidCallback) {
+                            VoidCallback cb = (VoidCallback) p.cb;
+                            cb.processResult(rc, path, p.ctx);
+                        }
+                    }
+                }
+            } catch (InterruptedException e) {
+            }
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private void finishPacket(Packet p) {
+        p.finished = true;
+        if (p.cb == null) {
+            synchronized (p) {
+                p.notifyAll();
+            }
+        } else {
+            waitingEvents.add(p);
+        }
+    }
+
+    private void conLossPacket(Packet p) {
+        if (p.replyHeader == null) {
+            return;
+        }
+        switch(zooKeeper.state) {
+        case AUTH_FAILED:
+            p.replyHeader.setErr(KeeperException.Code.AuthFailed);
+            break;
+        case CLOSED:
+            p.replyHeader.setErr(KeeperException.Code.SessionExpired);
+            break;
+        default:
+            p.replyHeader.setErr(KeeperException.Code.ConnectionLoss);
+        }
+        finishPacket(p);
+    }
+
+    long lastZxid;
+
+    /**
+     * This class services the outgoing request queue and generates the heart
+     * beats. It also spawns the ReadThread.
+     */
+    class SendThread extends Thread {
+        SelectionKey sockKey;
+
+        ByteBuffer lenBuffer = ByteBuffer.allocateDirect(4);
+
+        ByteBuffer incomingBuffer = lenBuffer;
+
+        boolean initialized;
+
+        void readLength() throws IOException {
+            int len = incomingBuffer.getInt();
+            if (len < 0 || len >= 4096 * 1024) {
+                throw new IOException("Packet len" + len + " is out of range!");
+            }
+            incomingBuffer = ByteBuffer.allocate(len);
+        }
+
+        void readConnectResult() throws IOException {
+            ByteBufferInputStream bbis = new ByteBufferInputStream(
+                    incomingBuffer);
+            BinaryInputArchive bbia = BinaryInputArchive.getArchive(bbis);
+            ConnectResponse conRsp = new ConnectResponse();
+            conRsp.deserialize(bbia, "connect");
+            int sessionTimeout = conRsp.getTimeOut();
+            if (sessionTimeout <= 0) {
+                zooKeeper.state = States.CLOSED;
+                waitingEvents.add(new WatcherEvent(Watcher.Event.EventNone,
+                        Watcher.Event.KeeperStateExpired, null));
+                throw new IOException("Session Expired");
+            }
+            readTimeout = sessionTimeout * 2 / 3;
+            connectTimeout = sessionTimeout / serverAddrs.size();
+            sessionId = conRsp.getSessionId();
+            sessionPasswd = conRsp.getPasswd();
+            waitingEvents.add(new WatcherEvent(Watcher.Event.EventNone,
+                    Watcher.Event.KeeperStateSyncConnected, null));
+        }
+
+        @SuppressWarnings("unchecked")
+        void readResponse() throws IOException {
+            ByteBufferInputStream bbis = new ByteBufferInputStream(
+                    incomingBuffer);
+            BinaryInputArchive bbia = BinaryInputArchive.getArchive(bbis);
+            ReplyHeader r = new ReplyHeader();
+
+            r.deserialize(bbia, "header");
+            if (r.getXid() == -2) {
+                // -2 is the xid for pings
+                return;
+            }
+            if (r.getXid() == -4) {
+                // -2 is the xid for AuthPacket
+                // TODO: process AuthPacket here
+                return;
+            }
+            if (r.getXid() == -1) {
+                // -1 means notification
+                WatcherEvent event = new WatcherEvent();
+                event.deserialize(bbia, "response");
+                // System.out.println("Got an event: " + event + " for " +
+                // sessionId + " through" + _cnxn);
+                waitingEvents.add(event);
+                return;
+            }
+            if (pendingQueue.size() == 0) {
+                throw new IOException("Nothing in the queue, but got "
+                        + r.getXid());
+            }
+            Packet p = null;
+            synchronized (pendingQueue) {
+                p = pendingQueue.remove();
+            }
+            /*
+             * Since requests are processed in order, we better get a response
+             * to the first request!
+             */
+            if (p.header.getXid() != r.getXid()) {
+                throw new IOException("Xid out of order. Got " + r.getXid()
+                        + " expected " + p.header.getXid());
+            }
+            p.replyHeader.setXid(r.getXid());
+            p.replyHeader.setErr(r.getErr());
+            p.replyHeader.setZxid(r.getZxid());
+            lastZxid = r.getZxid();
+            if (p.response != null && r.getErr() == 0) {
+                p.response.deserialize(bbia, "response");
+            }
+            p.finished = true;
+            finishPacket(p);
+        }
+
+        /**
+         * @return true if a packet was received
+         * @throws InterruptedException
+         * @throws IOException
+         */
+        boolean doIO() throws InterruptedException, IOException {
+            boolean packetReceived = false;
+            SocketChannel sock = (SocketChannel) sockKey.channel();
+            if (sock == null) {
+                throw new IOException("Socket is null!");
+            }
+            if (sockKey.isReadable()) {
+                int rc = sock.read(incomingBuffer);
+                if (rc < 0) {
+                    throw new IOException("Read error rc = " + rc + " "
+                            + incomingBuffer);
+                }
+                if (incomingBuffer.remaining() == 0) {
+                    incomingBuffer.flip();
+                    if (incomingBuffer == lenBuffer) {
+                        readLength();
+                    } else if (!initialized) {
+                        readConnectResult();
+                        enableRead();
+                        if (outgoingQueue.size() > 0) {
+                            enableWrite();
+                        }
+                        lenBuffer.clear();
+                        incomingBuffer = lenBuffer;
+                        packetReceived = true;
+                        initialized = true;
+                    } else {
+                        readResponse();
+                        lenBuffer.clear();
+                        incomingBuffer = lenBuffer;
+                        packetReceived = true;
+                    }
+                }
+            }
+            if (sockKey.isWritable()) {
+                synchronized (outgoingQueue) {
+                    if (outgoingQueue.size() > 0) {
+                        int rc = sock.write(outgoingQueue.getFirst().bb);
+                        if (outgoingQueue.getFirst().bb.remaining() == 0) {
+                            Packet p = outgoingQueue.removeFirst();
+                            if (p.header != null
+                                    && p.header.getType() != OpCode.ping
+                                    && p.header.getType() != OpCode.auth) {
+                                pendingQueue.add(p);
+                            }
+                        }
+                    }
+                }
+            }
+            if (outgoingQueue.size() == 0) {
+                disableWrite();
+            } else {
+                enableWrite();
+            }
+            return packetReceived;
+        }
+
+        synchronized private void enableWrite() {
+            int i = sockKey.interestOps();
+            if ((i & SelectionKey.OP_WRITE) == 0) {
+                sockKey.interestOps(i | SelectionKey.OP_WRITE);
+            }
+        }
+
+        synchronized private void disableWrite() {
+            int i = sockKey.interestOps();
+            if ((i & SelectionKey.OP_WRITE) != 0) {
+                sockKey.interestOps(i & (~SelectionKey.OP_WRITE));
+            }
+        }
+
+        synchronized private void enableRead() {
+            int i = sockKey.interestOps();
+            if ((i & SelectionKey.OP_READ) == 0) {
+                sockKey.interestOps(i | SelectionKey.OP_READ);
+            }
+        }
+
+        synchronized private void disableRead() {
+            int i = sockKey.interestOps();
+            if ((i & SelectionKey.OP_READ) != 0) {
+                sockKey.interestOps(i & (~SelectionKey.OP_READ));
+            }
+        }
+
+        SendThread() {
+            super("SendThread");
+            zooKeeper.state = States.CONNECTING;
+            setUncaughtExceptionHandler(uncaughtExceptionHandler);
+            setDaemon(true);
+        }
+
+        private void primeConnection(SelectionKey k) throws IOException {
+            LOG.info("Priming connection to "
+                    + ((SocketChannel) sockKey.channel()));
+            lastConnectIndex = currentConnectIndex;
+            ConnectRequest conReq = new ConnectRequest(0, lastZxid,
+                    sessionTimeout, sessionId, sessionPasswd);
+            ByteArrayOutputStream baos = new ByteArrayOutputStream();
+            BinaryOutputArchive boa = BinaryOutputArchive.getArchive(baos);
+            boa.writeInt(-1, "len");
+            conReq.serialize(boa, "connect");
+            baos.close();
+            ByteBuffer bb = ByteBuffer.wrap(baos.toByteArray());
+            bb.putInt(bb.capacity() - 4);
+            bb.rewind();
+            synchronized (outgoingQueue) {
+                for (AuthData id : authInfo) {
+                    outgoingQueue.addFirst(new Packet(new RequestHeader(-4,
+                            OpCode.auth), null, new AuthPacket(0, id.scheme,
+                            id.data), null, null));
+                }
+                outgoingQueue
+                        .addFirst((new Packet(null, null, null, null, bb)));
+            }
+            synchronized (this) {
+                k.interestOps(SelectionKey.OP_READ | SelectionKey.OP_WRITE);
+            }
+        }
+
+        private void sendPing() {
+            RequestHeader h = new RequestHeader(-2, OpCode.ping);
+            queuePacket(h, null, null, null, null, null, null);
+        }
+
+        int lastConnectIndex = -1;
+
+        int currentConnectIndex;
+
+        Random r = new Random(System.nanoTime());
+
+        private void startConnect() throws IOException {
+            if (lastConnectIndex == -1) {
+                // We don't want to delay the first try at a connect, so we
+                // start with -1 the first time around
+                lastConnectIndex = 0;
+            } else {
+                try {
+                    Thread.sleep(r.nextInt(1000));
+                } catch (InterruptedException e1) {
+                    LOG.warn("Unexpected exception", e1);
+                }
+                if (nextAddrToTry == lastConnectIndex) {
+                    try {
+                        // Try not to spin too fast!
+                        Thread.sleep(1000);
+                    } catch (InterruptedException e) {
+                        LOG.warn("Unexpected exception", e);
+                    }
+                }
+            }
+            zooKeeper.state = States.CONNECTING;
+            currentConnectIndex = nextAddrToTry;
+            InetSocketAddress addr = serverAddrs.get(nextAddrToTry);
+            nextAddrToTry++;
+            if (nextAddrToTry == serverAddrs.size()) {
+                nextAddrToTry = 0;
+            }
+            SocketChannel sock;
+            sock = SocketChannel.open();
+            sock.configureBlocking(false);
+            sock.socket().setSoLinger(false, -1);
+            sock.socket().setTcpNoDelay(true);
+            LOG.info("Attempting connection to server " + addr);
+            sockKey = sock.register(selector, SelectionKey.OP_CONNECT);
+            if (sock.connect(addr)) {
+                primeConnection(sockKey);
+            }
+            initialized = false;
+        }
+
+        @Override
+        public void run() {
+            long now = System.currentTimeMillis();
+            long lastHeard = now;
+            long lastSend = now;
+            while (zooKeeper.state.isAlive()) {
+                try {
+                    if (sockKey == null) {
+                        startConnect();
+                        lastSend = now;
+                        lastHeard = now;
+                    }
+                    int idleRecv = (int) (now - lastHeard);
+                    int idleSend = (int) (now - lastSend);
+                    int to = readTimeout - idleRecv;
+                    if (zooKeeper.state != States.CONNECTED) {
+                        to = connectTimeout - idleRecv;
+                    }
+                    if (to <= 0) {
+                        throw new IOException("TIMED OUT");
+                    }
+                    if (zooKeeper.state == States.CONNECTED) {
+                        int timeToNextPing = readTimeout/2 - idleSend;
+                        if (timeToNextPing <= 0) {
+                            sendPing();
+                            lastSend = now;
+                            enableWrite();
+                        } else {
+                            if (timeToNextPing < to) {
+                                to = timeToNextPing;
+                            }
+                        }
+                    }
+
+                    selector.select(to);
+                    Set<SelectionKey> selected;
+                    synchronized (this) {
+                        selected = selector.selectedKeys();
+                    }
+                    // Everything below and until we get back to the select is
+                    // non blocking, so time is effectively a constant. That is
+                    // Why we just have to do this once, here
+                    now = System.currentTimeMillis();
+                    for (SelectionKey k : selected) {
+                        SocketChannel sc = ((SocketChannel) k.channel());
+                        if ((k.readyOps() & SelectionKey.OP_CONNECT) != 0) {
+                            if (sc.finishConnect()) {
+                                zooKeeper.state = States.CONNECTED;
+                                lastHeard = now;
+                                lastSend = now;
+                                primeConnection(k);
+                                LOG.info("Server connection successful");
+                            }
+                        } else if ((k.readyOps() & (SelectionKey.OP_READ | SelectionKey.OP_WRITE)) != 0) {
+                            if (outgoingQueue.size() > 0) {
+                                // We have something to send so it's the same
+                                // as if we do the send now.
+                                lastSend = now;
+                            }
+                            if (doIO()) {
+                                lastHeard = now;
+                            }
+                        }
+                    }
+                    if (zooKeeper.state == States.CONNECTED) {
+                        if (outgoingQueue.size() > 0) {
+                            enableWrite();
+                        } else {
+                            disableWrite();
+                        }
+                    }
+                    selected.clear();
+                } catch (Exception e) {
+                    LOG.warn("Closing: ", e);
+                    cleanup();
+                    if (zooKeeper.state.isAlive()) {
+                        waitingEvents.add(new WatcherEvent(Event.EventNone,
+                                Event.KeeperStateDisconnected, null));
+                    }
+
+                    now = System.currentTimeMillis();
+                    lastHeard = now;
+                    lastSend = now;
+                }
+            }
+            cleanup();
+            ZooTrace.logTraceMessage(LOG, ZooTrace.getTextTraceLevel(),
+                                     "SendThread exitedloop.");
+        }
+
+        private void cleanup() {
+            if (sockKey != null) {
+                SocketChannel sock = (SocketChannel) sockKey.channel();
+                sockKey.cancel();
+                try {
+                    sock.socket().shutdownInput();
+                } catch (IOException e2) {
+                }
+                try {
+                    sock.socket().shutdownOutput();
+                } catch (IOException e2) {
+                }
+                try {
+                    sock.socket().close();
+                } catch (IOException e1) {
+                }
+                try {
+                    sock.close();
+                } catch (IOException e1) {
+                }
+            }
+            try {
+                Thread.sleep(100);
+            } catch (InterruptedException e1) {
+                e1.printStackTrace();
+            }
+            sockKey = null;
+            synchronized (pendingQueue) {
+                for (Packet p : pendingQueue) {
+                    conLossPacket(p);
+                }
+                pendingQueue.clear();
+            }
+            synchronized (outgoingQueue) {
+                for (Packet p : outgoingQueue) {
+                    conLossPacket(p);
+                }
+                outgoingQueue.clear();
+            }
+        }
+
+        public void close() {
+            zooKeeper.state = States.CLOSED;
+            synchronized (this) {
+                selector.wakeup();
+            }
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    public void close() throws IOException {
+        long traceMask = ZooTrace.SESSION_TRACE_MASK;
+        if (ZooTrace.isTraceEnabled(LOG, traceMask)) {
+            ZooTrace.logTraceMessage(LOG, traceMask,
+                    "Close ClientCnxn for session: " + sessionId + "!");
+        }
+        sendThread.close();
+        waitingEvents.add(eventOfDeath);
+    }
+
+    private int xid = 1;
+
+    synchronized private int getXid() {
+        return xid++;
+    }
+
+    public ReplyHeader submitRequest(RequestHeader h, Record request,
+            Record response) throws InterruptedException {
+        ReplyHeader r = new ReplyHeader();
+        Packet packet = queuePacket(h, r, request, response, null, null, null);
+        synchronized (packet) {
+            while (!packet.finished) {
+                packet.wait();
+            }
+        }
+        return r;
+    }
+
+    Packet queuePacket(RequestHeader h, ReplyHeader r, Record request,
+            Record response, AsyncCallback cb, String path, Object ctx) {
+        Packet packet = null;
+        synchronized (outgoingQueue) {
+            if (h.getType() != OpCode.ping && h.getType() != OpCode.auth) {
+                h.setXid(getXid());
+            }
+            packet = new Packet(h, r, request, response, null);
+            packet.cb = cb;
+            packet.ctx = ctx;
+            packet.path = path;
+            if (!zooKeeper.state.isAlive()) {
+                conLossPacket(packet);
+            } else {
+                outgoingQueue.add(packet);
+            }
+        }
+        synchronized (sendThread) {
+            selector.wakeup();
+        }
+        return packet;
+    }
+
+    public void addAuthInfo(String scheme, byte auth[]) {
+        authInfo.add(new AuthData(scheme, auth));
+        if (zooKeeper.state == States.CONNECTED) {
+            queuePacket(new RequestHeader(-4, OpCode.auth), null,
+                    new AuthPacket(0, scheme, auth), null, null, null, null);
+        }
+    }
+}

+ 246 - 50
zookeeper/java/src/com/yahoo/zookeeper/KeeperException.java

@@ -17,7 +17,85 @@
 package com.yahoo.zookeeper;
 
 @SuppressWarnings("serial")
-public class KeeperException extends Exception {
+public abstract class KeeperException extends Exception {
+
+    /**
+     * All non-specific keeper exceptions should be constructed via this factory method
+     * in order to guarantee consistency in error codes and such.  If you know the error
+     * code, then you should construct the special purpose exception directly.  That will
+     * allow you to have the most specific possible declarations of what exceptions might
+     * actually be thrown.
+     * @param code  The error code.
+     * @param path  The zookeeper path being operated on.
+     * @return  The specialized exception, presumably to be thrown by the caller.
+     */
+    public static KeeperException create(int code, String path) {
+        KeeperException r = create(code);
+        r.path = path;
+        return r;
+    }
+
+    /**
+     * All non-specific keeper exceptions should be constructed via this factory method
+     * in order to guarantee consistency in error codes and such.  If you know the error
+     * code, then you should construct the special purpose exception directly.  That will
+     * allow you to have the most specific possible declarations of what exceptions might
+     * actually be thrown.
+     * @param code The error code of your new exception.  This will also determine the
+     * specific type of the exception that is returned.
+     * @return  The specialized exception, presumably to be thrown by the caller.
+     */
+    public static KeeperException create(int code) {
+        switch (code) {
+            case Code.SystemError:
+                return new SystemErrorException();
+            case Code.RuntimeInconsistency:
+                return new RuntimeInconsistencyException();
+            case Code.DataInconsistency:
+                return new DataInconsistencyException();
+            case Code.ConnectionLoss:
+                return new ConnectionLossException();
+            case Code.MarshallingError:
+                return new MarshallingErrorException();
+            case Code.Unimplemented:
+                return new UnimplementedException();
+            case Code.OperationTimeout:
+                return new OperationTimeoutException();
+            case Code.BadArguments:
+                return new BadArgumentsException();
+            case Code.APIError:
+                return new APIErrorException();
+            case Code.NoNode:
+                return new NoNodeException();
+            case Code.NoAuth:
+                return new NoAuthException();
+            case Code.BadVersion:
+                return new BadVersionException();
+            case Code.NoChildrenForEphemerals:
+                return new NoChildrenForEphemeralsException();
+            case Code.NodeExists:
+                return new NodeExistsException();
+            case Code.InvalidACL:
+                return new InvalidACLException();
+            case Code.AuthFailed:
+                return new AuthFailedException();
+            case Code.NotEmpty:
+                return new NotEmptyException();
+            case Code.SessionExpired:
+                return new SessionExpiredException();
+            case Code.InvalidCallback:
+                return new InvalidCallbackException();
+
+            case 0:
+            default:
+                throw new IllegalArgumentException("Invalid exception code");
+        }
+    }
+
+    public void setCode(int code) {
+        this.code = code;
+    }
+
     public interface Code {
         int Ok = 0;
 
@@ -40,7 +118,7 @@ public class KeeperException extends Exception {
 
         // API errors
         int APIError = -100; // Catch all, shouldn't be used other
-                                        // than range start
+        // than range start
 
         int NoNode = APIError - 1; // Node does not exist
 
@@ -66,62 +144,66 @@ public class KeeperException extends Exception {
 
     static String getCodeMessage(int code) {
         switch (code) {
-        case 0:
-            return "ok";
-        case Code.SystemError:
-            return "SystemError";
-        case Code.RuntimeInconsistency:
-            return "RuntimeInconsistency";
-        case Code.DataInconsistency:
-            return "DataInconsistency";
-        case Code.ConnectionLoss:
-            return "ConnectionLoss";
-        case Code.MarshallingError:
-            return "MarshallingError";
-        case Code.Unimplemented:
-            return "Unimplemented";
-        case Code.OperationTimeout:
-            return "OperationTimeout";
-        case Code.BadArguments:
-            return "BadArguments";
-        case Code.APIError:
-            return "APIError";
-        case Code.NoNode:
-            return "NoNode";
-        case Code.NoAuth:
-            return "NoAuth";
-        case Code.BadVersion:
-            return "BadVersion";
-        case Code.NoChildrenForEphemerals:
-            return "NoChildrenForEphemerals";
-        case Code.NodeExists:
-            return "NodeExists";
-        case Code.InvalidACL:
-            return "InvalidACL";
-        case Code.AuthFailed:
-            return "AuthFailed";
-        case Code.SessionExpired:
-            return "SessionExpired";
-        default:
-            return "Unknown error " + code;
-        }
-    }
-
-    private int _code;
+            case 0:
+                return "ok";
+            case Code.SystemError:
+                return "SystemError";
+            case Code.RuntimeInconsistency:
+                return "RuntimeInconsistency";
+            case Code.DataInconsistency:
+                return "DataInconsistency";
+            case Code.ConnectionLoss:
+                return "ConnectionLoss";
+            case Code.MarshallingError:
+                return "MarshallingError";
+            case Code.Unimplemented:
+                return "Unimplemented";
+            case Code.OperationTimeout:
+                return "OperationTimeout";
+            case Code.BadArguments:
+                return "BadArguments";
+            case Code.APIError:
+                return "APIError";
+            case Code.NoNode:
+                return "NoNode";
+            case Code.NoAuth:
+                return "NoAuth";
+            case Code.BadVersion:
+                return "BadVersion";
+            case Code.NoChildrenForEphemerals:
+                return "NoChildrenForEphemerals";
+            case Code.NodeExists:
+                return "NodeExists";
+            case Code.InvalidACL:
+                return "InvalidACL";
+            case Code.AuthFailed:
+                return "AuthFailed";
+            case Code.NotEmpty:
+                return "Directory not empty";
+            case Code.SessionExpired:
+                return "Session expired";
+            case Code.InvalidCallback:
+                return "Invalid callback";
+            default:
+                return "Unknown error " + code;
+        }
+    }
+
+    private int code;
 
     private String path;
 
     public KeeperException(int code) {
-        _code = code;
+        this.code = code;
     }
 
-    public KeeperException(int code, String path) {
-        _code = code;
+    KeeperException(int code, String path) {
+        this.code = code;
         this.path = path;
     }
 
     public int getCode() {
-        return _code;
+        return code;
     }
 
     public String getPath() {
@@ -131,8 +213,122 @@ public class KeeperException extends Exception {
     @Override
     public String getMessage() {
         if (path == null) {
-            return "KeeperErrorCode = " + getCodeMessage(_code);
+            return "KeeperErrorCode = " + getCodeMessage(code);
+        }
+        return "KeeperErrorCode = " + getCodeMessage(code) + " for " + path;
+    }
+
+    public static class APIErrorException extends KeeperException {
+        public APIErrorException() {
+            super(Code.APIError);
+        }
+    }
+
+    public static class AuthFailedException extends KeeperException {
+        public AuthFailedException() {
+            super(Code.AuthFailed);
+        }
+    }
+
+    public static class BadArgumentsException extends KeeperException {
+        public BadArgumentsException() {
+            super(Code.BadArguments);
+        }
+    }
+
+    public static class BadVersionException extends KeeperException {
+        public BadVersionException() {
+            super(Code.BadVersion);
+        }
+    }
+
+    public static class ConnectionLossException extends KeeperException {
+        public ConnectionLossException() {
+            super(Code.ConnectionLoss);
+        }
+    }
+
+    public static class DataInconsistencyException extends KeeperException {
+        public DataInconsistencyException() {
+            super(Code.DataInconsistency);
+        }
+    }
+
+    public static class InvalidACLException extends KeeperException {
+        public InvalidACLException() {
+            super(Code.InvalidACL);
+        }
+    }
+
+    public static class InvalidCallbackException extends KeeperException {
+        public InvalidCallbackException() {
+            super(Code.InvalidCallback);
+        }
+    }
+
+    public static class MarshallingErrorException extends KeeperException {
+        public MarshallingErrorException() {
+            super(Code.MarshallingError);
+        }
+    }
+
+    public static class NoAuthException extends KeeperException {
+        public NoAuthException() {
+            super(Code.NoAuth);
+        }
+    }
+
+    public static class NoChildrenForEphemeralsException extends KeeperException {
+        public NoChildrenForEphemeralsException() {
+            super(Code.NoChildrenForEphemerals);
+        }
+    }
+
+    public static class NodeExistsException extends KeeperException {
+        public NodeExistsException() {
+            super(Code.NodeExists);
+        }
+    }
+
+    public static class NoNodeException extends KeeperException {
+        public NoNodeException() {
+            super(Code.NoNode);
+        }
+    }
+
+    public static class NotEmptyException extends KeeperException {
+        public NotEmptyException() {
+            super(Code.NotEmpty);
+        }
+    }
+
+    public static class OperationTimeoutException extends KeeperException {
+        public OperationTimeoutException() {
+            super(Code.OperationTimeout);
+        }
+    }
+
+    public static class RuntimeInconsistencyException extends KeeperException {
+        public RuntimeInconsistencyException() {
+            super(Code.RuntimeInconsistency);
+        }
+    }
+
+    public static class SessionExpiredException extends KeeperException {
+        public SessionExpiredException() {
+            super(Code.SessionExpired);
+        }
+    }
+
+    public static class SystemErrorException extends KeeperException {
+        public SystemErrorException() {
+            super(Code.SystemError);
+        }
+    }
+
+    public static class UnimplementedException extends KeeperException {
+        public UnimplementedException() {
+            super(Code.Unimplemented);
         }
-        return "KeeperErrorCode = " + getCodeMessage(_code) + " for " + path;
     }
 }

+ 859 - 874
zookeeper/java/src/com/yahoo/zookeeper/ZooKeeper.java

@@ -1,874 +1,859 @@
-/**
- * Copyright 2008, Yahoo! Inc.
- *
- * 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.
- */
-
-package com.yahoo.zookeeper;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.util.ArrayList;
-import java.util.Date;
-
-import org.apache.log4j.Logger;
-
-import com.yahoo.zookeeper.AsyncCallback.ACLCallback;
-import com.yahoo.zookeeper.AsyncCallback.ChildrenCallback;
-import com.yahoo.zookeeper.AsyncCallback.DataCallback;
-import com.yahoo.zookeeper.AsyncCallback.StatCallback;
-import com.yahoo.zookeeper.AsyncCallback.StringCallback;
-import com.yahoo.zookeeper.AsyncCallback.VoidCallback;
-import com.yahoo.zookeeper.KeeperException.Code;
-import com.yahoo.zookeeper.ZooDefs.Ids;
-import com.yahoo.zookeeper.data.ACL;
-import com.yahoo.zookeeper.data.Id;
-import com.yahoo.zookeeper.data.Stat;
-import com.yahoo.zookeeper.proto.CreateRequest;
-import com.yahoo.zookeeper.proto.CreateResponse;
-import com.yahoo.zookeeper.proto.DeleteRequest;
-import com.yahoo.zookeeper.proto.ExistsRequest;
-import com.yahoo.zookeeper.proto.GetACLRequest;
-import com.yahoo.zookeeper.proto.GetACLResponse;
-import com.yahoo.zookeeper.proto.GetChildrenRequest;
-import com.yahoo.zookeeper.proto.GetChildrenResponse;
-import com.yahoo.zookeeper.proto.GetDataRequest;
-import com.yahoo.zookeeper.proto.GetDataResponse;
-import com.yahoo.zookeeper.proto.ReplyHeader;
-import com.yahoo.zookeeper.proto.RequestHeader;
-import com.yahoo.zookeeper.proto.SetACLRequest;
-import com.yahoo.zookeeper.proto.SetACLResponse;
-import com.yahoo.zookeeper.proto.SetDataRequest;
-import com.yahoo.zookeeper.proto.SetDataResponse;
-import com.yahoo.zookeeper.proto.SyncRequest;
-import com.yahoo.zookeeper.proto.SyncResponse;
-import com.yahoo.zookeeper.proto.WatcherEvent;
-import com.yahoo.zookeeper.server.DataTree;
-import com.yahoo.zookeeper.server.ZooKeeperServer;
-
-/**
- * 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.
- * <p>
- * To create a client(ZooKeeper) object, the application needs to pass a string
- * containing a list of host:port pairs, each corresponding to a ZooKeeper
- * server; a sessionTimeout; and an object of Watcher type.
- * <p>
- * 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.
- * <p>
- * 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.
- * <p>
- * The application can call ZooKeeper APIs through a client as long as the
- * session ID of the client remains valid.
- * <p>
- * 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.
- * <p>
- * 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.
- * <p>
- * 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.
- * <p>
- * 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.
- * 
- */
-public class ZooKeeper {
-    private static final Logger LOG = Logger.getLogger(ZooKeeper.class);
-    
-    volatile Watcher watcher;
-
-    public enum States {
-        CONNECTING, ASSOCIATING, CONNECTED, CLOSED, AUTH_FAILED;
-
-        public boolean isAlive() {
-            return this != CLOSED && this != AUTH_FAILED;
-        }
-    }
-
-    volatile States state;
-
-    ClientCnxn cnxn;
-
-    public ZooKeeper(String host, int sessionTimeout, Watcher watcher)
-            throws KeeperException, IOException {
-        this.watcher = watcher;
-        cnxn = new ClientCnxn(host, sessionTimeout, this);
-    }
-
-    public ZooKeeper(String host, int sessionTimeout, Watcher watcher,
-            long sessionId, byte[] sessionPasswd) throws KeeperException,
-            IOException {
-        this.watcher = watcher;
-        cnxn = new ClientCnxn(host, sessionTimeout, this, sessionId,
-                sessionPasswd);
-    }
-
-    public long getSessionId() {
-        return cnxn.getSessionId();
-    }
-
-    public byte[] getSessionPasswd() {
-        return cnxn.getSessionPasswd();
-    }
-
-    public void addAuthInfo(String scheme, byte auth[]) {
-        cnxn.addAuthInfo(scheme, auth);
-    }
-
-    public String describeCNXN() {
-        return cnxn.toString();
-    }
-
-    public synchronized void register(Watcher watcher) {
-        this.watcher = watcher;
-    }
-
-    /**
-     * 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
-     *
-     * @throws IOException
-     * @throws InterruptedException
-     * @throws KeeperException
-     */
-    public synchronized void close() throws InterruptedException {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.closeSession);
-        cnxn.submitRequest(h, null, null);
-        try {
-            cnxn.close();
-        } catch (IOException e) {
-            LOG.warn("Unexpected exception", e);
-        }
-    }
-
-    /**
-     * Create a node with the given path. The node data will be the given data,
-     * and node acl will be the given acl.
-     * <p>
-     * The flags argument specifies whether the created node will be ephemeral
-     * or not.
-     * <p>
-     * An ephemeral node will be removed by the ZooKeeper automatically when the
-     * session associated with the creation of the node expires.
-     * <p>
-     * 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.
-     * <p>
-     * 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.
-     * <p>
-     * If the parent node does not exist in the ZooKeeper, a KeeperException
-     * with error code KeeperException.NoNode will be thrown.
-     * <p>
-     * 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.
-     * <p>
-     * 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.
-     * <p>
-     * 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.
-     *
-     * @param path
-     *                the path for the node
-     * @param data
-     *                the initial data for the node
-     * @param acl
-     *                the acl for the node
-     * @param flags
-     *                specifying whether the node to be created is ephemeral
-     *                and/or sequential
-     * @return the actual path of the created node
-     * @throws KeeperException
-     *                 an exception with appropriate error code defined in
-     *                 KeeperException class..
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    public String create(String path, byte data[], ArrayList<ACL> acl, int flags)
-            throws KeeperException, InterruptedException {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.create);
-        CreateRequest request = new CreateRequest();
-        CreateResponse response = new CreateResponse();
-        request.setData(data);
-        request.setFlags(flags);
-        request.setPath(path);
-        if (acl != null && acl.size() == 0) {
-            throw new KeeperException(Code.InvalidACL);
-        }
-        request.setAcl(acl);
-        ReplyHeader r = cnxn.submitRequest(h, request, response);
-        if (r.getErr() != 0) {
-            throw new KeeperException(r.getErr(), path);
-        }
-        return response.getPath();
-    }
-
-    /**
-     * The Asynchronous version of create. The request doesn't actually until
-     * the asynchronous callback is called.
-     *
-     * @see #create(String, byte[], ArrayList, int)
-     */
-
-    public void create(String path, byte data[], ArrayList<ACL> acl, int flags,
-            StringCallback cb, Object ctx) {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.create);
-        CreateRequest request = new CreateRequest();
-        CreateResponse response = new CreateResponse();
-        ReplyHeader r = new ReplyHeader();
-        request.setData(data);
-        request.setFlags(flags);
-        request.setPath(path);
-        request.setAcl(acl);
-        cnxn.queuePacket(h, r, request, response, cb, path, ctx);
-    }
-
-    /**
-     * 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).
-     * <p>
-     * A KeeperException with error code KeeperException.NoNode will be thrown
-     * if the nodes does not exist.
-     * <p>
-     * A KeeperException with error code KeeperException.BadVersion will be
-     * thrown if the given version does not match the node's version.
-     * <p>
-     * A KeeperException with error code KeeperException.NotEmpty will be thrown
-     * if the node has children.
-     * <p>
-     * 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.
-     *
-     * @param path
-     *                the path of the node to be deleted.
-     * @param version
-     *                the expected node version.
-     * @throws KeeperException
-     *                 an KeeperException with appropriate error code.
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    public void delete(String path, int version) throws KeeperException,
-            InterruptedException {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.delete);
-        DeleteRequest request = new DeleteRequest();
-        request.setPath(path);
-        request.setVersion(version);
-        ReplyHeader r = cnxn.submitRequest(h, request, null);
-        if (r.getErr() != 0) {
-            throw new KeeperException(r.getErr());
-        }
-    }
-
-    /**
-     * The Asynchronous version of delete. The request doesn't actually until
-     * the asynchronous callback is called.
-     *
-     * @see #delete(String, int)
-     */
-    public void delete(String path, int version, VoidCallback cb, Object ctx) {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.delete);
-        DeleteRequest request = new DeleteRequest();
-        request.setPath(path);
-        request.setVersion(version);
-        cnxn.queuePacket(h, new ReplyHeader(), request, null, cb, path, ctx);
-    }
-
-    /**
-     * Return the stat of the node of the given path. Return null if no such a
-     * node exists.
-     * <p>
-     * 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.
-     *
-     * @param path
-     *                the node path
-     * @param watch
-     *                whether need to watch this node
-     * @return the stat of the node of the given path; return null if no such a
-     *         node exists.
-     * @throws KeeperException
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    public Stat exists(String path, boolean watch) throws KeeperException,
-            InterruptedException {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.exists);
-        ExistsRequest request = new ExistsRequest();
-        request.setPath(path);
-        request.setWatch(watch);
-        SetDataResponse response = new SetDataResponse();
-        ReplyHeader r = cnxn.submitRequest(h, request, response);
-        if (r.getErr() != 0) {
-            if (r.getErr() == KeeperException.Code.NoNode) {
-                return null;
-            }
-            throw new KeeperException(r.getErr());
-        }
-        return response.getStat().getCzxid() == -1 ? null : response.getStat();
-    }
-
-    /**
-     * The Asynchronous version of exists. The request doesn't actually until
-     * the asynchronous callback is called.
-     *
-     * @see #exists(String, boolean)
-     */
-    public void exists(String path, boolean watch, StatCallback cb, Object ctx) {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.exists);
-        ExistsRequest request = new ExistsRequest();
-        request.setPath(path);
-        request.setWatch(watch);
-        SetDataResponse response = new SetDataResponse();
-        cnxn
-                .queuePacket(h, new ReplyHeader(), request, response, cb, path,
-                        ctx);
-    }
-
-    /**
-     * Return the data and the stat of the node of the given path.
-     * <p>
-     * If the watch is true and the call is successfull (no exception is
-     * thrown), a watch will be left on the node with the given path. The watch
-     * will be triggered by a sucessful operation that sets data on the node, or
-     * deletes the node.
-     * <p>
-     * A KeeperException with error code KeeperException.NoNode will be thrown
-     * if no node with the given path exists.
-     *
-     * @param path
-     *                the given path
-     * @param watch
-     *                whether need to watch this node
-     * @param stat
-     *                teh stat of the node
-     * @return the data of the node
-     * @throws IOException
-     * @throws KeeperException
-     * @throws InterruptedException
-     */
-    public byte[] getData(String path, boolean watch, Stat stat)
-            throws KeeperException, InterruptedException {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.getData);
-        GetDataRequest request = new GetDataRequest();
-        request.setPath(path);
-        request.setWatch(watch);
-        GetDataResponse response = new GetDataResponse();
-        ReplyHeader r = cnxn.submitRequest(h, request, response);
-        if (r.getErr() != 0) {
-            throw new KeeperException(r.getErr());
-        }
-        if (stat != null) {
-            DataTree.copyStat(response.getStat(), stat);
-        }
-        return response.getData();
-    }
-
-    /**
-     * The Asynchronous version of getData. The request doesn't actually until
-     * the asynchronous callback is called.
-     *
-     * @see #getData(String, boolean, Stat)
-     */
-
-    public void getData(String path, boolean watch, DataCallback cb, Object ctx) {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.getData);
-        GetDataRequest request = new GetDataRequest();
-        request.setPath(path);
-        request.setWatch(watch);
-        GetDataResponse response = new GetDataResponse();
-        cnxn
-                .queuePacket(h, new ReplyHeader(), request, response, cb, path,
-                        ctx);
-    }
-
-    /**
-     * 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.
-     * <p>
-     * This operation, if successful, will trigger all the watches on the node
-     * of the given path left by getData calls.
-     * <p>
-     * 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.
-     *
-     * @param path
-     *                the path of the node
-     * @param data
-     *                the data to set
-     * @param version
-     *                the expected matching version
-     * @return the state of the node
-     * @throws KeeperException
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    public Stat setData(String path, byte data[], int version)
-            throws KeeperException, InterruptedException {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.setData);
-        SetDataRequest request = new SetDataRequest();
-        request.setPath(path);
-        request.setData(data);
-        request.setVersion(version);
-        SetDataResponse response = new SetDataResponse();
-        ReplyHeader r = cnxn.submitRequest(h, request, response);
-        if (r.getErr() != 0) {
-            throw new KeeperException(r.getErr());
-        }
-        return response.getStat();
-    }
-
-    /**
-     * The Asynchronous version of setData. The request doesn't actually until
-     * the asynchronous callback is called.
-     *
-     * @see #setData(String, byte[], int)
-     */
-    public void setData(String path, byte data[], int version, StatCallback cb,
-            Object ctx) {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.setData);
-        SetDataRequest request = new SetDataRequest();
-        request.setPath(path);
-        request.setData(data);
-        request.setVersion(version);
-        SetDataResponse response = new SetDataResponse();
-        cnxn
-                .queuePacket(h, new ReplyHeader(), request, response, cb, path,
-                        ctx);
-    }
-
-    /**
-     *
-     * Return the ACL and stat of the node of the given path.
-     * <p>
-     * A KeeperException with error code KeeperException.NoNode will be thrown
-     * if no node with the given path exists.
-     *
-     * @param path
-     *                the given path for the node
-     * @param stat
-     *                the stat of the node will be copied to this parameter.
-     * @return the ACL array of the given node.
-     * @throws KeeperException
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    public ArrayList<ACL> getACL(String path, Stat stat)
-            throws KeeperException, InterruptedException {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.getACL);
-        GetACLRequest request = new GetACLRequest();
-        request.setPath(path);
-        GetACLResponse response = new GetACLResponse();
-        ReplyHeader r = cnxn.submitRequest(h, request, response);
-        if (r.getErr() != 0) {
-            throw new KeeperException(r.getErr());
-        }
-        DataTree.copyStat(response.getStat(), stat);
-        return response.getAcl();
-    }
-
-    /**
-     * The Asynchronous version of getACL. The request doesn't actually until
-     * the asynchronous callback is called.
-     *
-     * @see #getACL(String, Stat)
-     */
-    public void getACL(String path, Stat stat, ACLCallback cb, Object ctx) {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.getACL);
-        GetACLRequest request = new GetACLRequest();
-        request.setPath(path);
-        GetACLResponse response = new GetACLResponse();
-        cnxn
-                .queuePacket(h, new ReplyHeader(), request, response, cb, path,
-                        ctx);
-    }
-
-    /**
-     * 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.
-     * <p>
-     * A KeeperException with error code KeeperException.NoNode will be thrown
-     * if no node with the given path exists.
-     * <p>
-     * A KeeperException with error code KeeperException.BadVersion will be
-     * thrown if the given version does not match the node's version.
-     *
-     * @param path
-     * @param acl
-     * @param version
-     * @return the stat of the node.
-     * @throws KeeperException
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    public Stat setACL(String path, ArrayList<ACL> acl, int version)
-            throws KeeperException, InterruptedException {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.setACL);
-        SetACLRequest request = new SetACLRequest();
-        request.setPath(path);
-        if (acl != null && acl.size() == 0) {
-            throw new KeeperException(Code.InvalidACL);
-        }
-        request.setAcl(acl);
-        request.setVersion(version);
-        SetACLResponse response = new SetACLResponse();
-        ReplyHeader r = cnxn.submitRequest(h, request, response);
-        if (r.getErr() != 0) {
-            throw new KeeperException(r.getErr());
-        }
-        return response.getStat();
-    }
-
-    /**
-     * The Asynchronous version of setACL. The request doesn't actually until
-     * the asynchronous callback is called.
-     *
-     * @see #setACL(String, ArrayList, int)
-     */
-    public void setACL(String path, ArrayList<ACL> acl, int version,
-            StatCallback cb, Object ctx) {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.setACL);
-        SetACLRequest request = new SetACLRequest();
-        request.setPath(path);
-        request.setAcl(acl);
-        request.setVersion(version);
-        SetACLResponse response = new SetACLResponse();
-        cnxn
-                .queuePacket(h, new ReplyHeader(), request, response, cb, path,
-                        ctx);
-    }
-
-    /**
-     * Return the list of the children of the node of the given path.
-     * <p>
-     * 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 sucessful operation that deletes the node of the given
-     * path or creates/delete a child under the node.
-     * <p>
-     * A KeeperException with error code KeeperException.NoNode will be thrown
-     * if no node with the given path exists.
-     *
-     * @param path
-     * @param watch
-     * @return an array of children of the node with the given path
-     * @throws IOException
-     * @throws KeeperException
-     * @throws InterruptedException
-     */
-    public ArrayList<String> getChildren(String path, boolean watch)
-            throws KeeperException, InterruptedException {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.getChildren);
-        GetChildrenRequest request = new GetChildrenRequest();
-        request.setPath(path);
-        request.setWatch(watch);
-        GetChildrenResponse response = new GetChildrenResponse();
-        ReplyHeader r = cnxn.submitRequest(h, request, response);
-        if (r.getErr() != 0) {
-            throw new KeeperException(r.getErr());
-        }
-        return response.getChildren();
-    }
-
-    /**
-     * The Asynchronous version of getChildren. The request doesn't actually
-     * until the asynchronous callback is called.
-     *
-     * @see #getChildren(String, boolean)
-     */
-    public void getChildren(String path, boolean watch, ChildrenCallback cb,
-            Object ctx) {
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.getChildren);
-        GetChildrenRequest request = new GetChildrenRequest();
-        request.setPath(path);
-        request.setWatch(watch);
-        GetChildrenResponse response = new GetChildrenResponse();
-        cnxn
-                .queuePacket(h, new ReplyHeader(), request, response, cb, path,
-                        ctx);
-    }
-
-    /**
-     * Asynchronous sync. Flushes channel between process and leader.
-     *
-     * @see #sync(String)
-     */
-    public void sync(String path, VoidCallback cb, Object ctx){
-        RequestHeader h = new RequestHeader();
-        h.setType(ZooDefs.OpCode.sync);
-        SyncRequest request = new SyncRequest();
-        SyncResponse response = new SyncResponse();
-        request.setPath(path);
-        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb, path, ctx);
-    }
-
-    public States getState() {
-        return state;
-    }
-
-    // Everything below this line is for testing!
-
-    static void usage() {
-        System.err.println("ZooKeeper host:port cmd args");
-        System.err.println("\tcreate path data acl");
-        System.err.println("\tdelete path [version]");
-        System.err.println("\tset path data [version]");
-        System.err.println("\tget path [watch]");
-        System.err.println("\tls path [watch]");
-        System.err.println("\tgetAcl path");
-        System.err.println("\tsetAcl path acl");
-        System.err.println("\tstat path [watch]");
-        System.err.println("\tsync path");
-    }
-
-    static private class MyWatcher implements Watcher {
-        public void process(WatcherEvent event) {
-            System.err.println(event.getPath() + ": " + event.getState() + "-"
-                    + event.getType());
-        }
-    }
-
-    static private int getPermFromString(String permString) {
-        int perm = 0;
-        for (int i = 0; i < permString.length(); i++) {
-            switch (permString.charAt(i)) {
-            case 'r':
-                perm |= ZooDefs.Perms.READ;
-                break;
-            case 'w':
-                perm |= ZooDefs.Perms.WRITE;
-                break;
-            case 'c':
-                perm |= ZooDefs.Perms.CREATE;
-                break;
-            case 'd':
-                perm |= ZooDefs.Perms.DELETE;
-                break;
-            case 'a':
-                perm |= ZooDefs.Perms.ADMIN;
-                break;
-            default:
-                System.err
-                        .println("Unknown perm type: " + permString.charAt(i));
-            }
-        }
-        return perm;
-    }
-
-    private static void printStat(Stat stat) {
-        System.err.println("ctime = " + new Date(stat.getCtime()).toString());
-        System.err.println("ctime = " + new Date(stat.getMtime()).toString());
-        System.err.println("cversion = " + stat.getCversion());
-        System.err.println("cZxid = " + stat.getCzxid());
-        System.err.println("mZxid = " + stat.getMzxid());
-        System.err.println("dataVersion = " + stat.getVersion());
-        System.err.println("aclVersion = " + stat.getAversion());
-    }
-
-    public static void main(String args[]) throws NumberFormatException,
-            KeeperException, IOException, InterruptedException {
-        if (args.length == 1) {
-            ZooKeeper zooKeeper = new ZooKeeper(args[0], 5000, new MyWatcher());
-            BufferedReader br = new BufferedReader(new InputStreamReader(
-                    System.in));
-            String line;
-            while ((line = br.readLine()) != null) {
-                line = "ignore " + line;
-                args = line.split(" ");
-                processCmd(args, zooKeeper);
-            }
-        } else if (args.length < 3) {
-            usage();
-        }
-
-        ZooKeeper zooKeeper = new ZooKeeper(args[0], 5000, new MyWatcher());
-        boolean watch = processCmd(args, zooKeeper);
-        if (!watch) {
-            System.exit(0);
-        }
-    }
-
-    private static DataCallback dataCallback = new DataCallback() {
-
-        public void processResult(int rc, String path, Object ctx, byte[] data,
-                Stat stat) {
-            System.out.println("rc = " + rc + " path = " + path + " data = "
-                    + (data == null ? "null" : new String(data)) + " stat = ");
-            printStat(stat);
-        }
-
-    };
-
-    private static boolean processCmd(String[] args, ZooKeeper zooKeeper)
-            throws KeeperException, IOException, InterruptedException {
-        Stat stat = new Stat();
-        if (args.length < 2) {
-            return false;
-        }
-        if (args.length < 3) {
-            usage();
-            return false;
-        }
-        String cmd = args[1];
-        boolean watch = args.length > 3;
-        String path = args[2];
-        ArrayList<ACL> acl = Ids.OPEN_ACL_UNSAFE;
-        System.out.println("Processing " + cmd);
-        if (cmd.equals("create") && args.length >= 4) {
-            if (args.length == 5) {
-                acl = parseACLs(args[4]);
-            }
-            String newPath = zooKeeper.create(path, args[3].getBytes(), acl, 0);
-            System.err.println("Created " + newPath);
-        } else if (cmd.equals("delete") && args.length >= 3) {
-            zooKeeper.delete(path, watch ? Integer.parseInt(args[3]) : -1);
-        } else if (cmd.equals("set") && args.length >= 4) {
-            stat = zooKeeper.setData(path, args[3].getBytes(),
-                    args.length > 4 ? Integer.parseInt(args[4]) : -1);
-            printStat(stat);
-        } else if (cmd.equals("aget") && args.length >= 3) {
-            zooKeeper.getData(path, watch, dataCallback, path);
-        } else if (cmd.equals("get") && args.length >= 3) {
-            byte data[] = zooKeeper.getData(path, watch, stat);
-            System.out.println(new String(data));
-            printStat(stat);
-        } else if (cmd.equals("ls") && args.length >= 3) {
-            ArrayList<String> children = zooKeeper.getChildren(path, watch);
-            System.out.println(children);
-        } else if (cmd.equals("getAcl") && args.length >= 2) {
-            acl = zooKeeper.getACL(path, stat);
-            for (ACL a : acl) {
-                System.out.println(a.getId() + ": "
-                        + getPermString(a.getPerms()));
-            }
-        } else if (cmd.equals("setAcl") && args.length >= 4) {
-
-            stat = zooKeeper.setACL(path, parseACLs(args[3]),
-                    args.length > 4 ? Integer.parseInt(args[4]) : -1);
-            printStat(stat);
-        } else if (cmd.equals("stat") && args.length >= 3) {
-            stat = zooKeeper.exists(path, watch);
-            printStat(stat);
-        } else {
-            usage();
-        }
-        return watch;
-    }
-
-    private static String getPermString(int perms) {
-        StringBuffer p = new StringBuffer();
-        if ((perms & ZooDefs.Perms.CREATE) != 0) {
-            p.append('c');
-        }
-        if ((perms & ZooDefs.Perms.DELETE) != 0) {
-            p.append('d');
-        }
-        if ((perms & ZooDefs.Perms.READ) != 0) {
-            p.append('r');
-        }
-        if ((perms & ZooDefs.Perms.WRITE) != 0) {
-            p.append('w');
-        }
-        if ((perms & ZooDefs.Perms.ADMIN) != 0) {
-            p.append('a');
-        }
-        return p.toString();
-    }
-
-    private static ArrayList<ACL> parseACLs(String aclString) {
-        ArrayList<ACL> acl;
-        String acls[] = aclString.split(",");
-        acl = new ArrayList<ACL>();
-        for (String a : acls) {
-            int firstColon = a.indexOf(':');
-            int lastColon = a.indexOf(':');
-            if (firstColon == -1 || lastColon == -1 || firstColon == lastColon) {
-                System.err
-                        .println(a + " does not have the form scheme:id:perm");
-                continue;
-            }
-            ACL newAcl = new ACL();
-            newAcl.setId(new Id(a.substring(0, firstColon), a.substring(
-                    firstColon + 1, lastColon)));
-            newAcl.setPerms(getPermFromString(a.substring(lastColon + 1)));
-            acl.add(newAcl);
-        }
-        return acl;
-    }
-
-    public void disconnect() throws IOException {
-        cnxn.close();
-    }
-}
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * 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.
+ */
+
+package com.yahoo.zookeeper;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Date;
+
+import org.apache.log4j.Logger;
+
+import com.yahoo.zookeeper.AsyncCallback.ACLCallback;
+import com.yahoo.zookeeper.AsyncCallback.ChildrenCallback;
+import com.yahoo.zookeeper.AsyncCallback.DataCallback;
+import com.yahoo.zookeeper.AsyncCallback.StatCallback;
+import com.yahoo.zookeeper.AsyncCallback.StringCallback;
+import com.yahoo.zookeeper.AsyncCallback.VoidCallback;
+import com.yahoo.zookeeper.ZooDefs.Ids;
+import com.yahoo.zookeeper.data.ACL;
+import com.yahoo.zookeeper.data.Id;
+import com.yahoo.zookeeper.data.Stat;
+import com.yahoo.zookeeper.proto.CreateRequest;
+import com.yahoo.zookeeper.proto.CreateResponse;
+import com.yahoo.zookeeper.proto.DeleteRequest;
+import com.yahoo.zookeeper.proto.ExistsRequest;
+import com.yahoo.zookeeper.proto.GetACLRequest;
+import com.yahoo.zookeeper.proto.GetACLResponse;
+import com.yahoo.zookeeper.proto.GetChildrenRequest;
+import com.yahoo.zookeeper.proto.GetChildrenResponse;
+import com.yahoo.zookeeper.proto.GetDataRequest;
+import com.yahoo.zookeeper.proto.GetDataResponse;
+import com.yahoo.zookeeper.proto.ReplyHeader;
+import com.yahoo.zookeeper.proto.RequestHeader;
+import com.yahoo.zookeeper.proto.SetACLRequest;
+import com.yahoo.zookeeper.proto.SetACLResponse;
+import com.yahoo.zookeeper.proto.SetDataRequest;
+import com.yahoo.zookeeper.proto.SetDataResponse;
+import com.yahoo.zookeeper.proto.SyncRequest;
+import com.yahoo.zookeeper.proto.SyncResponse;
+import com.yahoo.zookeeper.proto.WatcherEvent;
+import com.yahoo.zookeeper.server.DataTree;
+import com.yahoo.zookeeper.server.ZooKeeperServer;
+
+/**
+ * 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.
+ * <p>
+ * To create a client(ZooKeeper) object, the application needs to pass a string
+ * containing a list of host:port pairs, each corresponding to a ZooKeeper
+ * server; a sessionTimeout; and an object of Watcher type.
+ * <p>
+ * 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.
+ * <p>
+ * 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.
+ * <p>
+ * The application can call ZooKeeper APIs through a client as long as the
+ * session ID of the client remains valid.
+ * <p>
+ * 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.
+ * <p>
+ * 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.
+ * <p>
+ * 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.
+ * <p>
+ * 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.
+ *
+ */
+public class ZooKeeper {
+    private static final Logger LOG = Logger.getLogger(ZooKeeper.class);
+
+    volatile Watcher watcher;
+
+    public enum States {
+        CONNECTING, ASSOCIATING, CONNECTED, CLOSED, AUTH_FAILED;
+
+        public boolean isAlive() {
+            return this != CLOSED && this != AUTH_FAILED;
+        }
+    }
+
+    volatile States state;
+
+    ClientCnxn cnxn;
+
+    public ZooKeeper(String host, int sessionTimeout, Watcher watcher)
+            throws IOException {
+        this.watcher = watcher;
+        cnxn = new ClientCnxn(host, sessionTimeout, this);
+    }
+
+    public ZooKeeper(String host, int sessionTimeout, Watcher watcher,
+            long sessionId, byte[] sessionPasswd) throws IOException {
+        this.watcher = watcher;
+        cnxn = new ClientCnxn(host, sessionTimeout, this, sessionId,
+                sessionPasswd);
+    }
+
+    public long getSessionId() {
+        return cnxn.getSessionId();
+    }
+
+    public byte[] getSessionPasswd() {
+        return cnxn.getSessionPasswd();
+    }
+
+    public void addAuthInfo(String scheme, byte auth[]) {
+        cnxn.addAuthInfo(scheme, auth);
+    }
+
+    public String describeCNXN() {
+        return cnxn.toString();
+    }
+
+    public synchronized void register(Watcher watcher) {
+        this.watcher = watcher;
+    }
+
+    /**
+     * 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
+     *
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    public synchronized void close() throws InterruptedException {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.closeSession);
+        cnxn.submitRequest(h, null, null);
+        try {
+            cnxn.close();
+        } catch (IOException e) {
+            LOG.warn("Unexpected exception", e);
+        }
+    }
+
+    /**
+     * Create a node with the given path. The node data will be the given data,
+     * and node acl will be the given acl.
+     * <p>
+     * The flags argument specifies whether the created node will be ephemeral
+     * or not.
+     * <p>
+     * An ephemeral node will be removed by the ZooKeeper automatically when the
+     * session associated with the creation of the node expires.
+     * <p>
+     * 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.
+     * <p>
+     * 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.
+     * <p>
+     * If the parent node does not exist in the ZooKeeper, a KeeperException
+     * with error code KeeperException.NoNode will be thrown.
+     * <p>
+     * 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.
+     * <p>
+     * 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.
+     * <p>
+     * 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.
+     *
+     * @param path
+     *                the path for the node
+     * @param data
+     *                the initial data for the node
+     * @param acl
+     *                the acl for the node
+     * @param flags
+     *                specifying whether the node to be created is ephemeral
+     *                and/or sequential
+     * @return the actual path of the created node
+     * @throws KeeperException if the server returns a non-zero error code
+     * @throws com.yahoo.zookeeper.KeeperException.InvalidACLException if the ACL is invalid
+     * @throws InterruptedException if the transaction is interrrupted
+     */
+    public String create(String path, byte data[], ArrayList<ACL> acl, int flags)
+            throws KeeperException, InterruptedException {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.create);
+        CreateRequest request = new CreateRequest();
+        CreateResponse response = new CreateResponse();
+        request.setData(data);
+        request.setFlags(flags);
+        request.setPath(path);
+        if (acl != null && acl.size() == 0) {
+            throw new KeeperException.InvalidACLException();
+        }
+        request.setAcl(acl);
+        ReplyHeader r = cnxn.submitRequest(h, request, response);
+        if (r.getErr() != 0) {
+            throw KeeperException.create(r.getErr(), path);
+        }
+        return response.getPath();
+    }
+
+    /**
+     * The Asynchronous version of create. The request doesn't actually until
+     * the asynchronous callback is called.
+     *
+     * @see #create(String, byte[], ArrayList, int)
+     */
+
+    public void create(String path, byte data[], ArrayList<ACL> acl, int flags,
+            StringCallback cb, Object ctx) {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.create);
+        CreateRequest request = new CreateRequest();
+        CreateResponse response = new CreateResponse();
+        ReplyHeader r = new ReplyHeader();
+        request.setData(data);
+        request.setFlags(flags);
+        request.setPath(path);
+        request.setAcl(acl);
+        cnxn.queuePacket(h, r, request, response, cb, path, ctx);
+    }
+
+    /**
+     * 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).
+     * <p>
+     * A KeeperException with error code KeeperException.NoNode will be thrown
+     * if the nodes does not exist.
+     * <p>
+     * A KeeperException with error code KeeperException.BadVersion will be
+     * thrown if the given version does not match the node's version.
+     * <p>
+     * A KeeperException with error code KeeperException.NotEmpty will be thrown
+     * if the node has children.
+     * <p>
+     * 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.
+     *
+     * @param path
+     *                the path of the node to be deleted.
+     * @param version
+     *                the expected node version.
+     * @throws InterruptedException IF the server transaction is interrupted
+     * @throws KeeperException If the server signals an error with a non-zero return code.
+     */
+    public void delete(String path, int version) throws
+            InterruptedException, KeeperException {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.delete);
+        DeleteRequest request = new DeleteRequest();
+        request.setPath(path);
+        request.setVersion(version);
+        ReplyHeader r = cnxn.submitRequest(h, request, null);
+        if (r.getErr() != 0) {
+            throw KeeperException.create(r.getErr());
+        }
+    }
+
+    /**
+     * The Asynchronous version of delete. The request doesn't actually until
+     * the asynchronous callback is called.
+     *
+     * @see #delete(String, int)
+     */
+    public void delete(String path, int version, VoidCallback cb, Object ctx) {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.delete);
+        DeleteRequest request = new DeleteRequest();
+        request.setPath(path);
+        request.setVersion(version);
+        cnxn.queuePacket(h, new ReplyHeader(), request, null, cb, path, ctx);
+    }
+
+    /**
+     * Return the stat of the node of the given path. Return null if no such a
+     * node exists.
+     * <p>
+     * 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.
+     *
+     * @param path
+     *                the node path
+     * @param watch
+     *                whether need to watch this node
+     * @return 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
+     * @throws InterruptedException If the server transaction is interrupted.
+     */
+    public Stat exists(String path, boolean watch) throws KeeperException,
+            InterruptedException {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.exists);
+        ExistsRequest request = new ExistsRequest();
+        request.setPath(path);
+        request.setWatch(watch);
+        SetDataResponse response = new SetDataResponse();
+        ReplyHeader r = cnxn.submitRequest(h, request, response);
+        if (r.getErr() != 0) {
+            if (r.getErr() == KeeperException.Code.NoNode) {
+                return null;
+            }
+            throw KeeperException.create(r.getErr());
+        }
+        return response.getStat().getCzxid() == -1 ? null : response.getStat();
+    }
+
+    /**
+     * The Asynchronous version of exists. The request doesn't actually until
+     * the asynchronous callback is called.
+     *
+     * @see #exists(String, boolean)
+     */
+    public void exists(String path, boolean watch, StatCallback cb, Object ctx) {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.exists);
+        ExistsRequest request = new ExistsRequest();
+        request.setPath(path);
+        request.setWatch(watch);
+        SetDataResponse response = new SetDataResponse();
+        cnxn
+                .queuePacket(h, new ReplyHeader(), request, response, cb, path,
+                        ctx);
+    }
+
+    /**
+     * Return the data and the stat of the node of the given path.
+     * <p>
+     * If the watch is true and the call is successfull (no exception is
+     * thrown), a watch will be left on the node with the given path. The watch
+     * will be triggered by a sucessful operation that sets data on the node, or
+     * deletes the node.
+     * <p>
+     * A KeeperException with error code KeeperException.NoNode will be thrown
+     * if no node with the given path exists.
+     *
+     * @param path
+     *                the given path
+     * @param watch
+     *                whether need to watch this node
+     * @param stat
+     *                teh stat of the node
+     * @return the data of the node
+     * @throws KeeperException If the server signals an error with a non-zero error code
+     * @throws InterruptedException If the server transaction is interrupted.
+     */
+    public byte[] getData(String path, boolean watch, Stat stat)
+            throws KeeperException, InterruptedException {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.getData);
+        GetDataRequest request = new GetDataRequest();
+        request.setPath(path);
+        request.setWatch(watch);
+        GetDataResponse response = new GetDataResponse();
+        ReplyHeader r = cnxn.submitRequest(h, request, response);
+        if (r.getErr() != 0) {
+            throw KeeperException.create(r.getErr());
+        }
+        if (stat != null) {
+            DataTree.copyStat(response.getStat(), stat);
+        }
+        return response.getData();
+    }
+
+    /**
+     * The Asynchronous version of getData. The request doesn't actually until
+     * the asynchronous callback is called.
+     *
+     * @see #getData(String, boolean, Stat)
+     */
+
+    public void getData(String path, boolean watch, DataCallback cb, Object ctx) {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.getData);
+        GetDataRequest request = new GetDataRequest();
+        request.setPath(path);
+        request.setWatch(watch);
+        GetDataResponse response = new GetDataResponse();
+        cnxn
+                .queuePacket(h, new ReplyHeader(), request, response, cb, path,
+                        ctx);
+    }
+
+    /**
+     * 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.
+     * <p>
+     * This operation, if successful, will trigger all the watches on the node
+     * of the given path left by getData calls.
+     * <p>
+     * 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.
+     *
+     * @param path
+     *                the path of the node
+     * @param data
+     *                the data to set
+     * @param version
+     *                the expected matching version
+     * @return the state of the node
+     * @throws InterruptedException If the server transaction is interrupted.
+     * @throws KeeperException If the server signals an error with a non-zero error code.
+     */
+    public Stat setData(String path, byte data[], int version)
+            throws KeeperException, InterruptedException {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.setData);
+        SetDataRequest request = new SetDataRequest();
+        request.setPath(path);
+        request.setData(data);
+        request.setVersion(version);
+        SetDataResponse response = new SetDataResponse();
+        ReplyHeader r = cnxn.submitRequest(h, request, response);
+        if (r.getErr() != 0) {
+            throw KeeperException.create(r.getErr());
+        }
+        return response.getStat();
+    }
+
+    /**
+     * The Asynchronous version of setData. The request doesn't actually until
+     * the asynchronous callback is called.
+     *
+     * @see #setData(String, byte[], int)
+     */
+    public void setData(String path, byte data[], int version, StatCallback cb,
+            Object ctx) {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.setData);
+        SetDataRequest request = new SetDataRequest();
+        request.setPath(path);
+        request.setData(data);
+        request.setVersion(version);
+        SetDataResponse response = new SetDataResponse();
+        cnxn
+                .queuePacket(h, new ReplyHeader(), request, response, cb, path,
+                        ctx);
+    }
+
+    /**
+     *
+     * Return the ACL and stat of the node of the given path.
+     * <p>
+     * A KeeperException with error code KeeperException.NoNode will be thrown
+     * if no node with the given path exists.
+     *
+     * @param path
+     *                the given path for the node
+     * @param stat
+     *                the stat of the node will be copied to this parameter.
+     * @return the ACL array of the given node.
+     * @throws InterruptedException If the server transaction is interrupted.
+     * @throws KeeperException If the server signals an error with a non-zero error code.
+     */
+    public ArrayList<ACL> getACL(String path, Stat stat)
+            throws KeeperException, InterruptedException {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.getACL);
+        GetACLRequest request = new GetACLRequest();
+        request.setPath(path);
+        GetACLResponse response = new GetACLResponse();
+        ReplyHeader r = cnxn.submitRequest(h, request, response);
+        if (r.getErr() != 0) {
+            throw KeeperException.create(r.getErr());
+        }
+        DataTree.copyStat(response.getStat(), stat);
+        return response.getAcl();
+    }
+
+    /**
+     * The Asynchronous version of getACL. The request doesn't actually until
+     * the asynchronous callback is called.
+     *
+     * @see #getACL(String, Stat)
+     */
+    public void getACL(String path, Stat stat, ACLCallback cb, Object ctx) {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.getACL);
+        GetACLRequest request = new GetACLRequest();
+        request.setPath(path);
+        GetACLResponse response = new GetACLResponse();
+        cnxn
+                .queuePacket(h, new ReplyHeader(), request, response, cb, path,
+                        ctx);
+    }
+
+    /**
+     * 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.
+     * <p>
+     * A KeeperException with error code KeeperException.NoNode will be thrown
+     * if no node with the given path exists.
+     * <p>
+     * A KeeperException with error code KeeperException.BadVersion will be
+     * thrown if the given version does not match the node's version.
+     *
+     * @param path
+     * @param acl
+     * @param version
+     * @return the stat of the node.
+     * @throws InterruptedException If the server transaction is interrupted.
+     * @throws KeeperException If the server signals an error with a non-zero error code.
+     * @throws com.yahoo.zookeeper.KeeperException.InvalidACLException If the acl is invalide.
+     */
+    public Stat setACL(String path, ArrayList<ACL> acl, int version)
+            throws KeeperException, InterruptedException {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.setACL);
+        SetACLRequest request = new SetACLRequest();
+        request.setPath(path);
+        if (acl != null && acl.size() == 0) {
+            throw new KeeperException.InvalidACLException();
+        }
+        request.setAcl(acl);
+        request.setVersion(version);
+        SetACLResponse response = new SetACLResponse();
+        ReplyHeader r = cnxn.submitRequest(h, request, response);
+        if (r.getErr() != 0) {
+            throw KeeperException.create(r.getErr());
+        }
+        return response.getStat();
+    }
+
+    /**
+     * The Asynchronous version of setACL. The request doesn't actually until
+     * the asynchronous callback is called.
+     *
+     * @see #setACL(String, ArrayList, int)
+     */
+    public void setACL(String path, ArrayList<ACL> acl, int version,
+            StatCallback cb, Object ctx) {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.setACL);
+        SetACLRequest request = new SetACLRequest();
+        request.setPath(path);
+        request.setAcl(acl);
+        request.setVersion(version);
+        SetACLResponse response = new SetACLResponse();
+        cnxn
+                .queuePacket(h, new ReplyHeader(), request, response, cb, path,
+                        ctx);
+    }
+
+    /**
+     * Return the list of the children of the node of the given path.
+     * <p>
+     * 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 sucessful operation that deletes the node of the given
+     * path or creates/delete a child under the node.
+     * <p>
+     * A KeeperException with error code KeeperException.NoNode will be thrown
+     * if no node with the given path exists.
+     *
+     * @param path
+     * @param watch
+     * @return an array of children of the node with the given path
+     * @throws InterruptedException If the server transaction is interrupted.
+     * @throws KeeperException If the server signals an error with a non-zero error code.
+     */
+    public ArrayList<String> getChildren(String path, boolean watch)
+            throws KeeperException, InterruptedException {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.getChildren);
+        GetChildrenRequest request = new GetChildrenRequest();
+        request.setPath(path);
+        request.setWatch(watch);
+        GetChildrenResponse response = new GetChildrenResponse();
+        ReplyHeader r = cnxn.submitRequest(h, request, response);
+        if (r.getErr() != 0) {
+            throw KeeperException.create(r.getErr());
+        }
+        return response.getChildren();
+    }
+
+    /**
+     * The Asynchronous version of getChildren. The request doesn't actually
+     * until the asynchronous callback is called.
+     *
+     * @see #getChildren(String, boolean)
+     */
+    public void getChildren(String path, boolean watch, ChildrenCallback cb,
+            Object ctx) {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.getChildren);
+        GetChildrenRequest request = new GetChildrenRequest();
+        request.setPath(path);
+        request.setWatch(watch);
+        GetChildrenResponse response = new GetChildrenResponse();
+        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb, path,
+                        ctx);
+    }
+
+    /**
+     * Asynchronous sync. Flushes channel between process and leader.
+     */
+    public void sync(String path, VoidCallback cb, Object ctx){
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.sync);
+        SyncRequest request = new SyncRequest();
+        SyncResponse response = new SyncResponse();
+        request.setPath(path);
+        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb, path, ctx);
+    }
+
+    public States getState() {
+        return state;
+    }
+
+    // Everything below this line is for testing!
+
+    static void usage() {
+        System.err.println("ZooKeeper host:port cmd args");
+        System.err.println("\tcreate path data acl");
+        System.err.println("\tdelete path [version]");
+        System.err.println("\tset path data [version]");
+        System.err.println("\tget path [watch]");
+        System.err.println("\tls path [watch]");
+        System.err.println("\tgetAcl path");
+        System.err.println("\tsetAcl path acl");
+        System.err.println("\tstat path [watch]");
+        System.err.println("\tsync path");
+    }
+
+    static private class MyWatcher implements Watcher {
+        public void process(WatcherEvent event) {
+            System.err.println(event.getPath() + ": " + event.getState() + "-"
+                    + event.getType());
+        }
+    }
+
+    static private int getPermFromString(String permString) {
+        int perm = 0;
+        for (int i = 0; i < permString.length(); i++) {
+            switch (permString.charAt(i)) {
+            case 'r':
+                perm |= ZooDefs.Perms.READ;
+                break;
+            case 'w':
+                perm |= ZooDefs.Perms.WRITE;
+                break;
+            case 'c':
+                perm |= ZooDefs.Perms.CREATE;
+                break;
+            case 'd':
+                perm |= ZooDefs.Perms.DELETE;
+                break;
+            case 'a':
+                perm |= ZooDefs.Perms.ADMIN;
+                break;
+            default:
+                System.err
+                        .println("Unknown perm type: " + permString.charAt(i));
+            }
+        }
+        return perm;
+    }
+
+    private static void printStat(Stat stat) {
+        System.err.println("ctime = " + new Date(stat.getCtime()).toString());
+        System.err.println("ctime = " + new Date(stat.getMtime()).toString());
+        System.err.println("cversion = " + stat.getCversion());
+        System.err.println("cZxid = " + stat.getCzxid());
+        System.err.println("mZxid = " + stat.getMzxid());
+        System.err.println("dataVersion = " + stat.getVersion());
+        System.err.println("aclVersion = " + stat.getAversion());
+    }
+
+    public static void main(String args[]) throws NumberFormatException,
+            KeeperException, IOException, InterruptedException {
+        if (args.length == 1) {
+            ZooKeeper zooKeeper = new ZooKeeper(args[0], 5000, new MyWatcher());
+            BufferedReader br = new BufferedReader(new InputStreamReader(
+                    System.in));
+            String line;
+            while ((line = br.readLine()) != null) {
+                line = "ignore " + line;
+                args = line.split(" ");
+                processCmd(args, zooKeeper);
+            }
+        } else if (args.length < 3) {
+            usage();
+        }
+
+        ZooKeeper zooKeeper = new ZooKeeper(args[0], 5000, new MyWatcher());
+        boolean watch = processCmd(args, zooKeeper);
+        if (!watch) {
+            System.exit(0);
+        }
+    }
+
+    private static DataCallback dataCallback = new DataCallback() {
+
+        public void processResult(int rc, String path, Object ctx, byte[] data,
+                Stat stat) {
+            System.out.println("rc = " + rc + " path = " + path + " data = "
+                    + (data == null ? "null" : new String(data)) + " stat = ");
+            printStat(stat);
+        }
+
+    };
+
+    private static boolean processCmd(String[] args, ZooKeeper zooKeeper)
+            throws KeeperException, IOException, InterruptedException {
+        Stat stat = new Stat();
+        if (args.length < 2) {
+            return false;
+        }
+        if (args.length < 3) {
+            usage();
+            return false;
+        }
+        String cmd = args[1];
+        boolean watch = args.length > 3;
+        String path = args[2];
+        ArrayList<ACL> acl = Ids.OPEN_ACL_UNSAFE;
+        System.out.println("Processing " + cmd);
+        if (cmd.equals("create") && args.length >= 4) {
+            if (args.length == 5) {
+                acl = parseACLs(args[4]);
+            }
+            String newPath = zooKeeper.create(path, args[3].getBytes(), acl, 0);
+            System.err.println("Created " + newPath);
+        } else if (cmd.equals("delete") && args.length >= 3) {
+            zooKeeper.delete(path, watch ? Integer.parseInt(args[3]) : -1);
+        } else if (cmd.equals("set") && args.length >= 4) {
+            stat = zooKeeper.setData(path, args[3].getBytes(),
+                    args.length > 4 ? Integer.parseInt(args[4]) : -1);
+            printStat(stat);
+        } else if (cmd.equals("aget") && args.length >= 3) {
+            zooKeeper.getData(path, watch, dataCallback, path);
+        } else if (cmd.equals("get") && args.length >= 3) {
+            byte data[] = zooKeeper.getData(path, watch, stat);
+            System.out.println(new String(data));
+            printStat(stat);
+        } else if (cmd.equals("ls") && args.length >= 3) {
+            ArrayList<String> children = zooKeeper.getChildren(path, watch);
+            System.out.println(children);
+        } else if (cmd.equals("getAcl") && args.length >= 2) {
+            acl = zooKeeper.getACL(path, stat);
+            for (ACL a : acl) {
+                System.out.println(a.getId() + ": "
+                        + getPermString(a.getPerms()));
+            }
+        } else if (cmd.equals("setAcl") && args.length >= 4) {
+
+            stat = zooKeeper.setACL(path, parseACLs(args[3]),
+                    args.length > 4 ? Integer.parseInt(args[4]) : -1);
+            printStat(stat);
+        } else if (cmd.equals("stat") && args.length >= 3) {
+            stat = zooKeeper.exists(path, watch);
+            printStat(stat);
+        } else {
+            usage();
+        }
+        return watch;
+    }
+
+    private static String getPermString(int perms) {
+        StringBuffer p = new StringBuffer();
+        if ((perms & ZooDefs.Perms.CREATE) != 0) {
+            p.append('c');
+        }
+        if ((perms & ZooDefs.Perms.DELETE) != 0) {
+            p.append('d');
+        }
+        if ((perms & ZooDefs.Perms.READ) != 0) {
+            p.append('r');
+        }
+        if ((perms & ZooDefs.Perms.WRITE) != 0) {
+            p.append('w');
+        }
+        if ((perms & ZooDefs.Perms.ADMIN) != 0) {
+            p.append('a');
+        }
+        return p.toString();
+    }
+
+    private static ArrayList<ACL> parseACLs(String aclString) {
+        ArrayList<ACL> acl;
+        String acls[] = aclString.split(",");
+        acl = new ArrayList<ACL>();
+        for (String a : acls) {
+            int firstColon = a.indexOf(':');
+            int lastColon = a.indexOf(':');
+            if (firstColon == -1 || lastColon == -1 || firstColon == lastColon) {
+                System.err
+                        .println(a + " does not have the form scheme:id:perm");
+                continue;
+            }
+            ACL newAcl = new ACL();
+            newAcl.setId(new Id(a.substring(0, firstColon), a.substring(
+                    firstColon + 1, lastColon)));
+            newAcl.setPerms(getPermFromString(a.substring(lastColon + 1)));
+            acl.add(newAcl);
+        }
+        return acl;
+    }
+
+    public void disconnect() throws IOException {
+        cnxn.close();
+    }
+}

+ 18 - 21
zookeeper/java/src/com/yahoo/zookeeper/server/DataTree.java

@@ -148,7 +148,7 @@ public class DataTree {
      * @throws KeeperException
      */
     public String createNode(String path, byte data[], ArrayList<ACL> acl,
-            long ephemeralOwner, long zxid, long time) throws KeeperException {
+            long ephemeralOwner, long zxid, long time) throws KeeperException.NoNodeException, KeeperException.NodeExistsException {
         int lastSlash = path.lastIndexOf('/');
         String parentName = path.substring(0, lastSlash);
         String childName = path.substring(lastSlash + 1);
@@ -162,11 +162,11 @@ public class DataTree {
         stat.setEphemeralOwner(ephemeralOwner);
         DataNode parent = nodes.get(parentName);
         if (parent == null) {
-            throw new KeeperException(KeeperException.Code.NoNode);
+            throw new KeeperException.NoNodeException();
         }
         synchronized (parent) {
             if (parent.children.contains(childName)) {
-                throw new KeeperException(KeeperException.Code.NodeExists);
+                throw new KeeperException.NodeExistsException();
             }
             int cver = parent.stat.getCversion();
             cver++;
@@ -188,18 +188,18 @@ public class DataTree {
         return path;
     }
 
-    public void deleteNode(String path) throws KeeperException {
+    public void deleteNode(String path) throws KeeperException.NoNodeException {
         int lastSlash = path.lastIndexOf('/');
         String parentName = path.substring(0, lastSlash);
         String childName = path.substring(lastSlash + 1);
         DataNode node = nodes.get(path);
         if (node == null) {
-            throw new KeeperException(KeeperException.Code.NoNode);
+            throw new KeeperException.NoNodeException();
         }
         nodes.remove(path);
         DataNode parent = nodes.get(parentName);
         if (parent == null) {
-            throw new KeeperException(KeeperException.Code.NoNode);
+            throw new KeeperException.NoNodeException();
         }
         synchronized (parent) {
             parent.children.remove(childName);
@@ -224,11 +224,11 @@ public class DataTree {
     }
 
     public Stat setData(String path, byte data[], int version, long zxid,
-            long time) throws KeeperException {
+            long time) throws KeeperException.NoNodeException {
         Stat s = new Stat();
         DataNode n = nodes.get(path);
         if (n == null) {
-            throw new KeeperException(KeeperException.Code.NoNode);
+            throw new KeeperException.NoNodeException();
         }
         synchronized (n) {
             n.data = data;
@@ -241,11 +241,10 @@ public class DataTree {
         return s;
     }
 
-    public byte[] getData(String path, Stat stat, Watcher watcher)
-            throws KeeperException {
+    public byte[] getData(String path, Stat stat, Watcher watcher) throws KeeperException.NoNodeException {
         DataNode n = nodes.get(path);
         if (n == null) {
-            throw new KeeperException(KeeperException.Code.NoNode);
+            throw new KeeperException.NoNodeException();
         }
         synchronized (n) {
             copyStat(n.stat, stat);
@@ -256,14 +255,14 @@ public class DataTree {
         }
     }
 
-    public Stat statNode(String path, Watcher watcher) throws KeeperException {
+    public Stat statNode(String path, Watcher watcher) throws KeeperException.NoNodeException {
         Stat stat = new Stat();
         DataNode n = nodes.get(path);
         if (watcher != null) {
             dataWatches.addWatch(path, watcher);
         }
         if (n == null) {
-            throw new KeeperException(KeeperException.Code.NoNode);
+            throw new KeeperException.NoNodeException();
         }
         synchronized (n) {
             copyStat(n.stat, stat);
@@ -271,11 +270,10 @@ public class DataTree {
         }
     }
 
-    public ArrayList<String> getChildren(String path, Stat stat, Watcher watcher)
-            throws KeeperException {
+    public ArrayList<String> getChildren(String path, Stat stat, Watcher watcher) throws KeeperException.NoNodeException {
         DataNode n = nodes.get(path);
         if (n == null) {
-            throw new KeeperException(KeeperException.Code.NoNode);
+            throw new KeeperException.NoNodeException();
         }
         synchronized (n) {
             ArrayList<String> children = new ArrayList<String>();
@@ -288,12 +286,11 @@ public class DataTree {
         }
     }
 
-    public Stat setACL(String path, ArrayList<ACL> acl, int version)
-            throws KeeperException {
+    public Stat setACL(String path, ArrayList<ACL> acl, int version) throws KeeperException.NoNodeException {
         Stat stat = new Stat();
         DataNode n = nodes.get(path);
         if (n == null) {
-            throw new KeeperException(KeeperException.Code.NoNode);
+            throw new KeeperException.NoNodeException();
         }
         synchronized (n) {
             n.stat.setAversion(version);
@@ -304,10 +301,10 @@ public class DataTree {
     }
 
     @SuppressWarnings("unchecked")
-    public ArrayList<ACL> getACL(String path, Stat stat) throws KeeperException {
+    public ArrayList<ACL> getACL(String path, Stat stat) throws KeeperException.NoNodeException {
         DataNode n = nodes.get(path);
         if (n == null) {
-            throw new KeeperException(KeeperException.Code.NoNode);
+            throw new KeeperException.NoNodeException();
         }
         synchronized (n) {
             copyStat(n.stat, stat);

+ 243 - 243
zookeeper/java/src/com/yahoo/zookeeper/server/FinalRequestProcessor.java

@@ -1,243 +1,243 @@
-/*
- * Copyright 2008, Yahoo! Inc.
- *
- * 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.
- */
-
-package com.yahoo.zookeeper.server;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-
-import org.apache.log4j.Logger;
-
-import com.yahoo.jute.Record;
-import com.yahoo.zookeeper.KeeperException;
-import com.yahoo.zookeeper.ZooDefs;
-import com.yahoo.zookeeper.KeeperException.Code;
-import com.yahoo.zookeeper.ZooDefs.OpCode;
-import com.yahoo.zookeeper.data.ACL;
-import com.yahoo.zookeeper.data.Stat;
-import com.yahoo.zookeeper.proto.CreateResponse;
-import com.yahoo.zookeeper.proto.ExistsRequest;
-import com.yahoo.zookeeper.proto.ExistsResponse;
-import com.yahoo.zookeeper.proto.GetACLRequest;
-import com.yahoo.zookeeper.proto.GetACLResponse;
-import com.yahoo.zookeeper.proto.GetChildrenRequest;
-import com.yahoo.zookeeper.proto.GetChildrenResponse;
-import com.yahoo.zookeeper.proto.GetDataRequest;
-import com.yahoo.zookeeper.proto.GetDataResponse;
-import com.yahoo.zookeeper.proto.ReplyHeader;
-import com.yahoo.zookeeper.proto.SetACLResponse;
-import com.yahoo.zookeeper.proto.SetDataResponse;
-import com.yahoo.zookeeper.proto.SyncRequest;
-import com.yahoo.zookeeper.proto.SyncResponse;
-import com.yahoo.zookeeper.server.DataTree.ProcessTxnResult;
-import com.yahoo.zookeeper.server.NIOServerCnxn.Factory;
-import com.yahoo.zookeeper.txn.CreateSessionTxn;
-import com.yahoo.zookeeper.txn.ErrorTxn;
-
-/**
- * This Request processor actually applies any transaction associated with a
- * request and services any queries. It is always at the end of a
- * RequestProcessor chain (hence the name), so it does not have a nextProcessor
- * member.
- *
- * This RequestProcessor counts on ZooKeeperServer to populate the
- * outstandingRequests member of ZooKeeperServer.
- */
-public class FinalRequestProcessor implements RequestProcessor {
-    private static final Logger LOG = Logger.getLogger(FinalRequestProcessor.class);
-
-    ZooKeeperServer zks;
-
-    public FinalRequestProcessor(ZooKeeperServer zks) {
-        this.zks = zks;
-    }
-
-    public void processRequest(Request request) {
-        // LOG.info("Zoo>>> cxid = " + request.cxid + " type = " +
-        // request.type + " id = " + request.sessionId + " cnxn " +
-        // request.cnxn);
-        // request.addRQRec(">final");
-        long traceMask = ZooTrace.CLIENT_REQUEST_TRACE_MASK;
-        if (request.type == OpCode.ping) {
-            traceMask = ZooTrace.SERVER_PING_TRACE_MASK;
-        }
-        ZooTrace.logRequest(LOG, traceMask, 'E', request, "");
-        ProcessTxnResult rc = null;
-        synchronized (zks.outstandingChanges) {
-            while (!zks.outstandingChanges.isEmpty()
-                    && zks.outstandingChanges.get(0).zxid <= request.zxid) {
-                if (zks.outstandingChanges.get(0).zxid < request.zxid) {
-                    LOG.warn("Zxid outstanding "
-                            + zks.outstandingChanges.get(0).zxid
-                            + " is less than current " + request.zxid);
-                }
-                zks.outstandingChanges.remove(0);
-            }
-            if (request.hdr != null) {
-                rc = zks.dataTree.processTxn(request.hdr, request.txn);
-                if (request.type == OpCode.createSession) {
-                    if (request.txn instanceof CreateSessionTxn) {
-                        CreateSessionTxn cst = (CreateSessionTxn) request.txn;
-                        zks.sessionTracker.addSession(request.sessionId, cst
-                                .getTimeOut());
-                    } else {
-                        LOG.warn("*****>>>>> Got "
-                                + request.txn.getClass() + " "
-                                + request.txn.toString());
-                    }
-                } else if (request.type == OpCode.closeSession) {
-                    zks.sessionTracker.removeSession(request.sessionId);
-                }
-            }
-            // do not add non quorum packets to the queue.
-            if (Request.isQuorum(request.type)) {
-                zks.addCommittedProposal(request);
-            }
-        }
-
-        if (request.hdr != null && request.hdr.getType() == OpCode.closeSession) {
-            Factory scxn = zks.getServerCnxnFactory();
-            // this might be possible since
-            // we might just be playing diffs from the leader
-            if (scxn != null) {
-                scxn.closeSession(request.sessionId);
-            }
-        }
-
-        if (request.cnxn == null) {
-            return;
-        }
-        zks.decInProcess();
-        int err = 0;
-        Record rsp = null;
-        try {
-            if (request.hdr != null && request.hdr.getType() == OpCode.error) {
-                throw new KeeperException(((ErrorTxn) request.txn).getErr());
-            }
-            switch (request.type) {
-            case OpCode.ping:
-                request.cnxn.sendResponse(new ReplyHeader(-2,
-                        zks.dataTree.lastProcessedZxid, 0), null, "response");
-                return;
-            case OpCode.createSession:
-                request.cnxn.finishSessionInit(true);
-                return;
-            case OpCode.create:
-                rsp = new CreateResponse(rc.path);
-                err = rc.err;
-                break;
-            case OpCode.delete:
-                err = rc.err;
-                break;
-            case OpCode.setData:
-                rsp = new SetDataResponse(rc.stat);
-                err = rc.err;
-                break;
-            case OpCode.setACL:
-                rsp = new SetACLResponse(rc.stat);
-                err = rc.err;
-                break;
-            case OpCode.closeSession:
-                err = rc.err;
-                break;
-            case OpCode.sync:
-                SyncRequest syncRequest = new SyncRequest();
-                ZooKeeperServer.byteBuffer2Record(request.request,
-                        syncRequest);
-                rsp = new SyncResponse(syncRequest.getPath());
-                break;
-            case OpCode.exists:
-                // TODO we need to figure out the security requirement for this!
-                ExistsRequest existsRequest = new ExistsRequest();
-                ZooKeeperServer.byteBuffer2Record(request.request,
-                        existsRequest);
-                String path = existsRequest.getPath();
-                if (path.indexOf('\0') != -1) {
-                    throw new KeeperException(Code.BadArguments);
-                }
-                Stat stat = zks.dataTree.statNode(path, existsRequest
-                        .getWatch() ? request.cnxn : null);
-                rsp = new ExistsResponse(stat);
-                break;
-            case OpCode.getData:
-                GetDataRequest getDataRequest = new GetDataRequest();
-                ZooKeeperServer.byteBuffer2Record(request.request,
-                        getDataRequest);
-                DataNode n = zks.dataTree.getNode(getDataRequest.getPath());
-                if (n == null) {
-                    throw new KeeperException(Code.NoNode);
-                }
-                PrepRequestProcessor.checkACL(zks, n.acl, ZooDefs.Perms.READ,
-                        request.authInfo);
-                stat = new Stat();
-                byte b[] = zks.dataTree.getData(getDataRequest.getPath(), stat,
-                        getDataRequest.getWatch() ? request.cnxn : null);
-                rsp = new GetDataResponse(b, stat);
-                break;
-            case OpCode.getACL:
-                GetACLRequest getACLRequest = new GetACLRequest();
-                ZooKeeperServer.byteBuffer2Record(request.request,
-                        getACLRequest);
-                stat = new Stat();
-                ArrayList<ACL> acl = zks.dataTree.getACL(getACLRequest
-                        .getPath(), stat);
-                rsp = new GetACLResponse(acl, stat);
-                break;
-            case OpCode.getChildren:
-                GetChildrenRequest getChildrenRequest = new GetChildrenRequest();
-                ZooKeeperServer.byteBuffer2Record(request.request,
-                        getChildrenRequest);
-                stat = new Stat();
-                n = zks.dataTree.getNode(getChildrenRequest.getPath());
-                if (n == null) {
-                    throw new KeeperException(Code.NoNode);
-                }
-                PrepRequestProcessor.checkACL(zks, n.acl, ZooDefs.Perms.READ,
-                        request.authInfo);
-                ArrayList<String> children = zks.dataTree.getChildren(
-                        getChildrenRequest.getPath(), stat, getChildrenRequest
-                                .getWatch() ? request.cnxn : null);
-                rsp = new GetChildrenResponse(children);
-                break;
-            }
-        } catch (KeeperException e) {
-            err = e.getCode();
-        } catch (Exception e) {
-            LOG.error("****************************** " + request);
-            StringBuffer sb = new StringBuffer();
-            ByteBuffer bb = request.request;
-            bb.rewind();
-            while (bb.hasRemaining()) {
-                sb.append(Integer.toHexString(bb.get() & 0xff));
-            }
-            LOG.error(sb.toString());
-            LOG.error("Unexpected error while marshalling",e);
-            err = Code.MarshallingError;
-        }
-        ReplyHeader hdr = new ReplyHeader(request.cxid, request.zxid, err);
-        ServerStats.getInstance().updateLatency(request.createTime);
-        try {
-            request.cnxn.sendResponse(hdr, rsp, "response");
-        } catch (IOException e) {
-            LOG.warn("Unexpected exception",e);
-        }
-    }
-
-    public void shutdown() {
-    }
-
-}
+/*
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * 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.
+ */
+
+package com.yahoo.zookeeper.server;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+
+import org.apache.log4j.Logger;
+
+import com.yahoo.jute.Record;
+import com.yahoo.zookeeper.KeeperException;
+import com.yahoo.zookeeper.ZooDefs;
+import com.yahoo.zookeeper.KeeperException.Code;
+import com.yahoo.zookeeper.ZooDefs.OpCode;
+import com.yahoo.zookeeper.data.ACL;
+import com.yahoo.zookeeper.data.Stat;
+import com.yahoo.zookeeper.proto.CreateResponse;
+import com.yahoo.zookeeper.proto.ExistsRequest;
+import com.yahoo.zookeeper.proto.ExistsResponse;
+import com.yahoo.zookeeper.proto.GetACLRequest;
+import com.yahoo.zookeeper.proto.GetACLResponse;
+import com.yahoo.zookeeper.proto.GetChildrenRequest;
+import com.yahoo.zookeeper.proto.GetChildrenResponse;
+import com.yahoo.zookeeper.proto.GetDataRequest;
+import com.yahoo.zookeeper.proto.GetDataResponse;
+import com.yahoo.zookeeper.proto.ReplyHeader;
+import com.yahoo.zookeeper.proto.SetACLResponse;
+import com.yahoo.zookeeper.proto.SetDataResponse;
+import com.yahoo.zookeeper.proto.SyncRequest;
+import com.yahoo.zookeeper.proto.SyncResponse;
+import com.yahoo.zookeeper.server.DataTree.ProcessTxnResult;
+import com.yahoo.zookeeper.server.NIOServerCnxn.Factory;
+import com.yahoo.zookeeper.txn.CreateSessionTxn;
+import com.yahoo.zookeeper.txn.ErrorTxn;
+
+/**
+ * This Request processor actually applies any transaction associated with a
+ * request and services any queries. It is always at the end of a
+ * RequestProcessor chain (hence the name), so it does not have a nextProcessor
+ * member.
+ *
+ * This RequestProcessor counts on ZooKeeperServer to populate the
+ * outstandingRequests member of ZooKeeperServer.
+ */
+public class FinalRequestProcessor implements RequestProcessor {
+    private static final Logger LOG = Logger.getLogger(FinalRequestProcessor.class);
+
+    ZooKeeperServer zks;
+
+    public FinalRequestProcessor(ZooKeeperServer zks) {
+        this.zks = zks;
+    }
+
+    public void processRequest(Request request) {
+        // LOG.info("Zoo>>> cxid = " + request.cxid + " type = " +
+        // request.type + " id = " + request.sessionId + " cnxn " +
+        // request.cnxn);
+        // request.addRQRec(">final");
+        long traceMask = ZooTrace.CLIENT_REQUEST_TRACE_MASK;
+        if (request.type == OpCode.ping) {
+            traceMask = ZooTrace.SERVER_PING_TRACE_MASK;
+        }
+        ZooTrace.logRequest(LOG, traceMask, 'E', request, "");
+        ProcessTxnResult rc = null;
+        synchronized (zks.outstandingChanges) {
+            while (!zks.outstandingChanges.isEmpty()
+                    && zks.outstandingChanges.get(0).zxid <= request.zxid) {
+                if (zks.outstandingChanges.get(0).zxid < request.zxid) {
+                    LOG.warn("Zxid outstanding "
+                            + zks.outstandingChanges.get(0).zxid
+                            + " is less than current " + request.zxid);
+                }
+                zks.outstandingChanges.remove(0);
+            }
+            if (request.hdr != null) {
+                rc = zks.dataTree.processTxn(request.hdr, request.txn);
+                if (request.type == OpCode.createSession) {
+                    if (request.txn instanceof CreateSessionTxn) {
+                        CreateSessionTxn cst = (CreateSessionTxn) request.txn;
+                        zks.sessionTracker.addSession(request.sessionId, cst
+                                .getTimeOut());
+                    } else {
+                        LOG.warn("*****>>>>> Got "
+                                + request.txn.getClass() + " "
+                                + request.txn.toString());
+                    }
+                } else if (request.type == OpCode.closeSession) {
+                    zks.sessionTracker.removeSession(request.sessionId);
+                }
+            }
+            // do not add non quorum packets to the queue.
+            if (Request.isQuorum(request.type)) {
+                zks.addCommittedProposal(request);
+            }
+        }
+
+        if (request.hdr != null && request.hdr.getType() == OpCode.closeSession) {
+            Factory scxn = zks.getServerCnxnFactory();
+            // this might be possible since
+            // we might just be playing diffs from the leader
+            if (scxn != null) {
+                scxn.closeSession(request.sessionId);
+            }
+        }
+
+        if (request.cnxn == null) {
+            return;
+        }
+        zks.decInProcess();
+        int err = 0;
+        Record rsp = null;
+        try {
+            if (request.hdr != null && request.hdr.getType() == OpCode.error) {
+                throw KeeperException.create(((ErrorTxn) request.txn).getErr());
+            }
+            switch (request.type) {
+            case OpCode.ping:
+                request.cnxn.sendResponse(new ReplyHeader(-2,
+                        zks.dataTree.lastProcessedZxid, 0), null, "response");
+                return;
+            case OpCode.createSession:
+                request.cnxn.finishSessionInit(true);
+                return;
+            case OpCode.create:
+                rsp = new CreateResponse(rc.path);
+                err = rc.err;
+                break;
+            case OpCode.delete:
+                err = rc.err;
+                break;
+            case OpCode.setData:
+                rsp = new SetDataResponse(rc.stat);
+                err = rc.err;
+                break;
+            case OpCode.setACL:
+                rsp = new SetACLResponse(rc.stat);
+                err = rc.err;
+                break;
+            case OpCode.closeSession:
+                err = rc.err;
+                break;
+            case OpCode.sync:
+                SyncRequest syncRequest = new SyncRequest();
+                ZooKeeperServer.byteBuffer2Record(request.request,
+                        syncRequest);
+                rsp = new SyncResponse(syncRequest.getPath());
+                break;
+            case OpCode.exists:
+                // TODO we need to figure out the security requirement for this!
+                ExistsRequest existsRequest = new ExistsRequest();
+                ZooKeeperServer.byteBuffer2Record(request.request,
+                        existsRequest);
+                String path = existsRequest.getPath();
+                if (path.indexOf('\0') != -1) {
+                    throw new KeeperException.BadArgumentsException();
+                }
+                Stat stat = zks.dataTree.statNode(path, existsRequest
+                        .getWatch() ? request.cnxn : null);
+                rsp = new ExistsResponse(stat);
+                break;
+            case OpCode.getData:
+                GetDataRequest getDataRequest = new GetDataRequest();
+                ZooKeeperServer.byteBuffer2Record(request.request,
+                        getDataRequest);
+                DataNode n = zks.dataTree.getNode(getDataRequest.getPath());
+                if (n == null) {
+                    throw new KeeperException.NoNodeException();
+                }
+                PrepRequestProcessor.checkACL(zks, n.acl, ZooDefs.Perms.READ,
+                        request.authInfo);
+                stat = new Stat();
+                byte b[] = zks.dataTree.getData(getDataRequest.getPath(), stat,
+                        getDataRequest.getWatch() ? request.cnxn : null);
+                rsp = new GetDataResponse(b, stat);
+                break;
+            case OpCode.getACL:
+                GetACLRequest getACLRequest = new GetACLRequest();
+                ZooKeeperServer.byteBuffer2Record(request.request,
+                        getACLRequest);
+                stat = new Stat();
+                ArrayList<ACL> acl = zks.dataTree.getACL(getACLRequest
+                        .getPath(), stat);
+                rsp = new GetACLResponse(acl, stat);
+                break;
+            case OpCode.getChildren:
+                GetChildrenRequest getChildrenRequest = new GetChildrenRequest();
+                ZooKeeperServer.byteBuffer2Record(request.request,
+                        getChildrenRequest);
+                stat = new Stat();
+                n = zks.dataTree.getNode(getChildrenRequest.getPath());
+                if (n == null) {
+                    throw new KeeperException.NoNodeException();
+                }
+                PrepRequestProcessor.checkACL(zks, n.acl, ZooDefs.Perms.READ,
+                        request.authInfo);
+                ArrayList<String> children = zks.dataTree.getChildren(
+                        getChildrenRequest.getPath(), stat, getChildrenRequest
+                                .getWatch() ? request.cnxn : null);
+                rsp = new GetChildrenResponse(children);
+                break;
+            }
+        } catch (KeeperException e) {
+            err = e.getCode();
+        } catch (Exception e) {
+            LOG.warn("****************************** " + request);
+            StringBuffer sb = new StringBuffer();
+            ByteBuffer bb = request.request;
+            bb.rewind();
+            while (bb.hasRemaining()) {
+                sb.append(Integer.toHexString(bb.get() & 0xff));
+            }
+            LOG.warn(sb.toString());
+            LOG.error("FIXMSG",e);
+            err = Code.MarshallingError;
+        }
+        ReplyHeader hdr = new ReplyHeader(request.cxid, request.zxid, err);
+        ServerStats.getInstance().updateLatency(request.createTime);
+        try {
+            request.cnxn.sendResponse(hdr, rsp, "response");
+        } catch (IOException e) {
+            LOG.error("FIXMSG",e);
+        }
+    }
+
+    public void shutdown() {
+    }
+
+}

+ 443 - 445
zookeeper/java/src/com/yahoo/zookeeper/server/PrepRequestProcessor.java

@@ -1,445 +1,443 @@
-/*
- * Copyright 2008, Yahoo! Inc.
- *
- * 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.
- */
-
-package com.yahoo.zookeeper.server;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.log4j.Logger;
-
-import com.yahoo.jute.Record;
-import com.yahoo.zookeeper.KeeperException;
-import com.yahoo.zookeeper.ZooDefs;
-import com.yahoo.zookeeper.KeeperException.Code;
-import com.yahoo.zookeeper.ZooDefs.CreateFlags;
-import com.yahoo.zookeeper.ZooDefs.OpCode;
-import com.yahoo.zookeeper.data.ACL;
-import com.yahoo.zookeeper.data.Id;
-import com.yahoo.zookeeper.data.Stat;
-import com.yahoo.zookeeper.proto.CreateRequest;
-import com.yahoo.zookeeper.proto.DeleteRequest;
-import com.yahoo.zookeeper.proto.SetACLRequest;
-import com.yahoo.zookeeper.proto.SetDataRequest;
-import com.yahoo.zookeeper.server.ZooKeeperServer.ChangeRecord;
-import com.yahoo.zookeeper.server.auth.AuthenticationProvider;
-import com.yahoo.zookeeper.server.auth.ProviderRegistry;
-import com.yahoo.zookeeper.txn.CreateSessionTxn;
-import com.yahoo.zookeeper.txn.CreateTxn;
-import com.yahoo.zookeeper.txn.DeleteTxn;
-import com.yahoo.zookeeper.txn.ErrorTxn;
-import com.yahoo.zookeeper.txn.SetACLTxn;
-import com.yahoo.zookeeper.txn.SetDataTxn;
-import com.yahoo.zookeeper.txn.TxnHeader;
-
-/**
- * This request processor is generally at the start of a RequestProcessor
- * change. It sets up any transactions associated with requests that change the
- * state of the system. It counts on ZooKeeperServer to update
- * outstandingRequests, so that it can take into account transactions that are
- * in the queue to be applied when generating a transaction.
- */
-public class PrepRequestProcessor extends Thread implements RequestProcessor {
-    private static final Logger LOG = Logger.getLogger(PrepRequestProcessor.class);
-
-    static boolean skipACL;
-    static {
-        skipACL = System.getProperty("zookeeper.skipACL", "no").equals("yes");
-    }
-
-    LinkedBlockingQueue<Request> submittedRequests = new LinkedBlockingQueue<Request>();
-
-    RequestProcessor nextProcessor;
-
-    ZooKeeperServer zks;
-
-    public PrepRequestProcessor(ZooKeeperServer zks,
-            RequestProcessor nextProcessor) {
-        super("ProcessThread");
-        this.nextProcessor = nextProcessor;
-        this.zks = zks;
-
-        start();
-    }
-
-    public void run() {
-        try {
-            while (true) {
-                Request request = submittedRequests.take();
-                long traceMask = ZooTrace.CLIENT_REQUEST_TRACE_MASK;
-                if (request.type == OpCode.ping) {
-                    traceMask = ZooTrace.CLIENT_PING_TRACE_MASK;
-                }
-                ZooTrace.logRequest(LOG, traceMask, 'P', request, "");
-                if (Request.requestOfDeath == request) {
-                    break;
-                }
-                pRequest(request);
-            }
-        } catch (InterruptedException e) {
-            LOG.error("FIXMSG",e);
-        }
-        ZooTrace.logTraceMessage(LOG, ZooTrace.getTextTraceLevel(),
-                "PrepRequestProcessor exited loop!");
-    }
-
-    ChangeRecord getRecordForPath(String path) throws KeeperException {
-        ChangeRecord lastChange = null;
-        synchronized (zks.outstandingChanges) {
-            for (int i = 0; i < zks.outstandingChanges.size(); i++) {
-                ChangeRecord c = zks.outstandingChanges.get(i);
-                if (c.path.equals(path)) {
-                    lastChange = c;
-                }
-            }
-            if (lastChange == null) {
-                DataNode n = zks.dataTree.getNode(path);
-                if (n != null) {
-                    lastChange = new ChangeRecord(-1, path, n.stat, n.children
-                            .size(), n.acl);
-                }
-            }
-        }
-        if (lastChange == null || lastChange.stat == null) {
-            throw new KeeperException(KeeperException.Code.NoNode);
-        }
-        return lastChange;
-    }
-
-    void addChangeRecord(ChangeRecord c) {
-        synchronized (zks.outstandingChanges) {
-            zks.outstandingChanges.add(c);
-        }
-    }
-
-    static void checkACL(ZooKeeperServer zks, ArrayList<ACL> acl, int perm,
-            ArrayList<Id> ids) throws KeeperException {
-        if (skipACL) {
-            return;
-        }
-        if (acl == null || acl.size() == 0) {
-            return;
-        }
-        for (ACL a : acl) {
-            Id id = a.getId();
-            if ((a.getPerms() & perm) != 0) {
-                if (id.getScheme().equals("world")
-                        && id.getId().equals("anyone")) {
-                    return;
-                }
-                AuthenticationProvider ap = ProviderRegistry.getProvider(id
-                        .getScheme());
-                if (ap != null) {
-                    for (Id authId : ids) {
-                        if (authId.getScheme().equals("super")) {
-                            return;
-                        }
-                        if (authId.getScheme().equals(id.getScheme())
-                                && ap.matches(authId.getId(), id.getId())) {
-                            return;
-                        }
-                    }
-                }
-            }
-        }
-        throw new KeeperException(KeeperException.Code.NoAuth);
-    }
-
-    /**
-     * This method will be called inside the ProcessRequestThread, which is a
-     * singleton, so there will be a single thread calling this code.
-     *
-     * @param request
-     */
-    @SuppressWarnings("unchecked")
-    protected void pRequest(Request request) {
-        // LOG.info("Prep>>> cxid = " + request.cxid + " type = " +
-        // request.type + " id = " + request.sessionId);
-        TxnHeader txnHeader = null;
-        Record txn = null;
-        try {
-            switch (request.type) {
-            case OpCode.create:
-                txnHeader = new TxnHeader(request.sessionId, request.cxid, zks
-                        .getNextZxid(), zks.getTime(), OpCode.create);
-                zks.sessionTracker.checkSession(request.sessionId);
-                CreateRequest createRequest = new CreateRequest();
-                ZooKeeperServer.byteBuffer2Record(request.request,
-                        createRequest);
-                String path = createRequest.getPath();
-                int lastSlash = path.lastIndexOf('/');
-                if (lastSlash == -1 || path.indexOf('\0') != -1) {
-                    throw new KeeperException(Code.BadArguments);
-                }
-                if (!fixupACL(request.authInfo, createRequest.getAcl())) {
-                    throw new KeeperException(Code.InvalidACL);
-                }
-                String parentPath = path.substring(0, lastSlash);
-                ChangeRecord parentRecord = getRecordForPath(parentPath);
-
-                checkACL(zks, parentRecord.acl, ZooDefs.Perms.CREATE,
-                        request.authInfo);
-                int parentCVersion = parentRecord.stat.getCversion();
-                if ((createRequest.getFlags() & CreateFlags.SEQUENCE) != 0) {
-                    path = path + parentCVersion;
-                }
-                try {
-                    if (getRecordForPath(path) != null) {
-                        throw new KeeperException(Code.NodeExists);
-                    }
-                } catch (KeeperException e) {
-                    if (e.getCode() != Code.NoNode) {
-                        throw e;
-                    }
-                }
-                boolean ephemeralParent = parentRecord.stat.getEphemeralOwner() != 0;
-                if (ephemeralParent) {
-                    throw new KeeperException(Code.NoChildrenForEphemerals);
-                }
-                txn = new CreateTxn(path, createRequest.getData(),
-                        createRequest.getAcl(),
-                        (createRequest.getFlags() & CreateFlags.EPHEMERAL) != 0);
-                Stat s = new Stat();
-                if ((createRequest.getFlags() & CreateFlags.EPHEMERAL) != 0) {
-                    s.setEphemeralOwner(request.sessionId);
-                }
-                parentRecord = parentRecord.duplicate(txnHeader.getZxid());
-                parentRecord.childCount++;
-                parentRecord.stat
-                        .setCversion(parentRecord.stat.getCversion() + 1);
-                addChangeRecord(parentRecord);
-                addChangeRecord(new ChangeRecord(txnHeader.getZxid(), path, s,
-                        0, createRequest.getAcl()));
-
-                break;
-            case OpCode.delete:
-                txnHeader = new TxnHeader(request.sessionId, request.cxid, zks
-                        .getNextZxid(), zks.getTime(), OpCode.delete);
-                zks.sessionTracker.checkSession(request.sessionId);
-                DeleteRequest deleteRequest = new DeleteRequest();
-                ZooKeeperServer.byteBuffer2Record(request.request,
-                        deleteRequest);
-                path = deleteRequest.getPath();
-                lastSlash = path.lastIndexOf('/');
-                if (lastSlash == -1 || path.indexOf('\0') != -1
-                        || path.equals("/")) {
-                    throw new KeeperException(Code.BadArguments);
-                }
-                parentPath = path.substring(0, lastSlash);
-                parentRecord = getRecordForPath(parentPath);
-                ChangeRecord nodeRecord = getRecordForPath(path);
-                checkACL(zks, parentRecord.acl, ZooDefs.Perms.DELETE,
-                        request.authInfo);
-                int version = deleteRequest.getVersion();
-                if (version != -1 && nodeRecord.stat.getVersion() != version) {
-                    throw new KeeperException(Code.BadVersion);
-                }
-                if (nodeRecord.childCount > 0) {
-                    throw new KeeperException(Code.NotEmpty);
-                }
-                txn = new DeleteTxn(path);
-                parentRecord = parentRecord.duplicate(txnHeader.getZxid());
-                parentRecord.childCount--;
-                parentRecord.stat
-                        .setCversion(parentRecord.stat.getCversion() + 1);
-                addChangeRecord(parentRecord);
-                addChangeRecord(new ChangeRecord(txnHeader.getZxid(), path,
-                        null, -1, null));
-                break;
-            case OpCode.setData:
-                txnHeader = new TxnHeader(request.sessionId, request.cxid, zks
-                        .getNextZxid(), zks.getTime(), OpCode.setData);
-                zks.sessionTracker.checkSession(request.sessionId);
-                SetDataRequest setDataRequest = new SetDataRequest();
-                ZooKeeperServer.byteBuffer2Record(request.request,
-                        setDataRequest);
-                path = setDataRequest.getPath();
-                nodeRecord = getRecordForPath(path);
-                checkACL(zks, nodeRecord.acl, ZooDefs.Perms.WRITE,
-                        request.authInfo);
-                version = setDataRequest.getVersion();
-                int currentVersion = nodeRecord.stat.getVersion();
-                if (version != -1 && version != currentVersion) {
-                    throw new KeeperException(Code.BadVersion);
-                }
-                version = currentVersion + 1;
-                txn = new SetDataTxn(path, setDataRequest.getData(), version);
-                nodeRecord = nodeRecord.duplicate(txnHeader.getZxid());
-                nodeRecord.stat.setVersion(version);
-                addChangeRecord(nodeRecord);
-                break;
-            case OpCode.setACL:
-                txnHeader = new TxnHeader(request.sessionId, request.cxid, zks
-                        .getNextZxid(), zks.getTime(), OpCode.setACL);
-                zks.sessionTracker.checkSession(request.sessionId);
-                SetACLRequest setAclRequest = new SetACLRequest();
-                if (!fixupACL(request.authInfo, setAclRequest.getAcl())) {
-                    throw new KeeperException(Code.InvalidACL);
-                }
-                ZooKeeperServer.byteBuffer2Record(request.request,
-                        setAclRequest);
-                path = setAclRequest.getPath();
-                nodeRecord = getRecordForPath(path);
-                checkACL(zks, nodeRecord.acl, ZooDefs.Perms.ADMIN,
-                        request.authInfo);
-                version = setAclRequest.getVersion();
-                currentVersion = nodeRecord.stat.getAversion();
-                if (version != -1 && version != currentVersion) {
-                    throw new KeeperException(Code.BadVersion);
-                }
-                version = currentVersion + 1;
-                txn = new SetACLTxn(path, setAclRequest.getAcl(), version);
-                nodeRecord = nodeRecord.duplicate(txnHeader.getZxid());
-                nodeRecord.stat.setAversion(version);
-                addChangeRecord(nodeRecord);
-                break;
-            case OpCode.createSession:
-                txnHeader = new TxnHeader(request.sessionId, request.cxid, zks
-                        .getNextZxid(), zks.getTime(), OpCode.createSession);
-                request.request.rewind();
-                int to = request.request.getInt();
-                txn = new CreateSessionTxn(to);
-                request.request.rewind();
-                zks.sessionTracker.addSession(request.sessionId, to);
-                break;
-            case OpCode.closeSession:
-                txnHeader = new TxnHeader(request.sessionId, request.cxid, zks
-                        .getNextZxid(), zks.getTime(), OpCode.closeSession);
-                HashSet<String> es = zks.dataTree
-                        .getEphemerals(request.sessionId);
-                synchronized (zks.outstandingChanges) {
-                    for (ChangeRecord c : zks.outstandingChanges) {
-                        if (c.stat == null) {
-                            // Doing a delete
-                            es.remove(c.path);
-                        } else if (c.stat.getEphemeralOwner() == request.sessionId) {
-                            es.add(c.path);
-                        }
-                    }
-                    for (String path2Delete : es) {
-                        addChangeRecord(new ChangeRecord(txnHeader.getZxid(),
-                                path2Delete, null, 0, null));
-                    }
-                }
-                LOG.info("Processed session termination request for id: "
-                        + Long.toHexString(request.sessionId));
-                break;
-            case OpCode.sync:
-            case OpCode.exists:
-            case OpCode.getData:
-            case OpCode.getACL:
-            case OpCode.getChildren:
-            case OpCode.ping:
-                break;
-            }
-        } catch (KeeperException e) {
-            if (txnHeader != null) {
-                txnHeader.setType(OpCode.error);
-                txn = new ErrorTxn(e.getCode());
-            }
-        } catch (Exception e) {
-            LOG.error("*********************************" + request);
-            StringBuffer sb = new StringBuffer();
-            ByteBuffer bb = request.request;
-            if(bb!=null){
-                bb.rewind();
-                while (bb.hasRemaining()) {
-                    sb.append(Integer.toHexString(bb.get() & 0xff));
-                }
-            }else
-                sb.append("request buffer is null");
-            LOG.error(sb.toString());
-            LOG.error("Unexpected exception", e);
-            if (txnHeader != null) {
-                txnHeader.setType(OpCode.error);
-                txn = new ErrorTxn(Code.MarshallingError);
-            }
-        }
-        request.hdr = txnHeader;
-        request.txn = txn;
-        if (request.hdr != null) {
-            request.zxid = request.hdr.getZxid();
-        }
-        nextProcessor.processRequest(request);
-    }
-
-    /**
-     *
-     * @param authInfo list of ACL IDs associated with the client connection
-     * @param acl list of ACLs being assigned to the node (create or setACL operation)
-     * @return
-     */
-    private boolean fixupACL(ArrayList<Id> authInfo, ArrayList<ACL> acl) {
-        if (skipACL) {
-            return true;
-        }
-        if (acl == null || acl.size() == 0) {
-            return false;
-        }
-        Iterator<ACL> it = acl.iterator();
-        LinkedList<ACL> toAdd = null;
-        while (it.hasNext()) {
-            ACL a = it.next();
-            Id id = a.getId();
-            if (id.getScheme().equals("world") && id.getId().equals("anyone")) {
-            } else if (id.getScheme().equals("auth")) {
-                it.remove();
-                if (toAdd == null) {
-                    toAdd = new LinkedList<ACL>();
-                }
-                for (Id cid : authInfo) {
-                    AuthenticationProvider ap = ProviderRegistry.getProvider(cid.getScheme());
-                    if (ap == null) {
-                        LOG.error("Missing AuthenticationProvider for "
-                                + cid.getScheme());
-                    } else if (ap.isAuthenticated()) {
-                        toAdd.add(new ACL(a.getPerms(), cid));
-                    }
-                }
-            } else {
-                AuthenticationProvider ap = ProviderRegistry.getProvider(id
-                        .getScheme());
-                if (ap == null) {
-                    return false;
-                }
-                if (!ap.isValid(id.getId())) {
-                    return false;
-                }
-            }
-        }
-        if (toAdd != null) {
-            for (ACL a : toAdd) {
-                acl.add(a);
-            }
-        }
-        return true;
-    }
-
-    public void processRequest(Request request) {
-        // request.addRQRec(">prep="+zks.outstandingChanges.size());
-        submittedRequests.add(request);
-    }
-
-    public void shutdown() {
-        submittedRequests.clear();
-        submittedRequests.add(Request.requestOfDeath);
-        nextProcessor.shutdown();
-    }
-}
+/*
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * 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.
+ */
+
+package com.yahoo.zookeeper.server;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.log4j.Logger;
+
+import com.yahoo.jute.Record;
+import com.yahoo.zookeeper.KeeperException;
+import com.yahoo.zookeeper.ZooDefs;
+import com.yahoo.zookeeper.KeeperException.Code;
+import com.yahoo.zookeeper.ZooDefs.CreateFlags;
+import com.yahoo.zookeeper.ZooDefs.OpCode;
+import com.yahoo.zookeeper.data.ACL;
+import com.yahoo.zookeeper.data.Id;
+import com.yahoo.zookeeper.data.Stat;
+import com.yahoo.zookeeper.proto.CreateRequest;
+import com.yahoo.zookeeper.proto.DeleteRequest;
+import com.yahoo.zookeeper.proto.SetACLRequest;
+import com.yahoo.zookeeper.proto.SetDataRequest;
+import com.yahoo.zookeeper.server.ZooKeeperServer.ChangeRecord;
+import com.yahoo.zookeeper.server.auth.AuthenticationProvider;
+import com.yahoo.zookeeper.server.auth.ProviderRegistry;
+import com.yahoo.zookeeper.txn.CreateSessionTxn;
+import com.yahoo.zookeeper.txn.CreateTxn;
+import com.yahoo.zookeeper.txn.DeleteTxn;
+import com.yahoo.zookeeper.txn.ErrorTxn;
+import com.yahoo.zookeeper.txn.SetACLTxn;
+import com.yahoo.zookeeper.txn.SetDataTxn;
+import com.yahoo.zookeeper.txn.TxnHeader;
+
+/**
+ * This request processor is generally at the start of a RequestProcessor
+ * change. It sets up any transactions associated with requests that change the
+ * state of the system. It counts on ZooKeeperServer to update
+ * outstandingRequests, so that it can take into account transactions that are
+ * in the queue to be applied when generating a transaction.
+ */
+public class PrepRequestProcessor extends Thread implements RequestProcessor {
+    private static final Logger LOG = Logger.getLogger(PrepRequestProcessor.class);
+
+    static boolean skipACL;
+    static {
+        skipACL = System.getProperty("zookeeper.skipACL", "no").equals("yes");
+    }
+
+    LinkedBlockingQueue<Request> submittedRequests = new LinkedBlockingQueue<Request>();
+
+    RequestProcessor nextProcessor;
+
+    ZooKeeperServer zks;
+
+    public PrepRequestProcessor(ZooKeeperServer zks,
+            RequestProcessor nextProcessor) {
+        super("ProcessThread");
+        this.nextProcessor = nextProcessor;
+        this.zks = zks;
+
+        start();
+    }
+
+    public void run() {
+        try {
+            while (true) {
+                Request request = submittedRequests.take();
+                long traceMask = ZooTrace.CLIENT_REQUEST_TRACE_MASK;
+                if (request.type == OpCode.ping) {
+                    traceMask = ZooTrace.CLIENT_PING_TRACE_MASK;
+                }
+                ZooTrace.logRequest(LOG, traceMask, 'P', request, "");
+                if (Request.requestOfDeath == request) {
+                    break;
+                }
+                pRequest(request);
+            }
+        } catch (InterruptedException e) {
+            LOG.error("FIXMSG",e);
+        }
+        ZooTrace.logTraceMessage(LOG, ZooTrace.getTextTraceLevel(),
+                "PrepRequestProcessor exited loop!");
+    }
+
+    ChangeRecord getRecordForPath(String path) throws KeeperException.NoNodeException {
+        ChangeRecord lastChange = null;
+        synchronized (zks.outstandingChanges) {
+            for (int i = 0; i < zks.outstandingChanges.size(); i++) {
+                ChangeRecord c = zks.outstandingChanges.get(i);
+                if (c.path.equals(path)) {
+                    lastChange = c;
+                }
+            }
+            if (lastChange == null) {
+                DataNode n = zks.dataTree.getNode(path);
+                if (n != null) {
+                    lastChange = new ChangeRecord(-1, path, n.stat, n.children
+                            .size(), n.acl);
+                }
+            }
+        }
+        if (lastChange == null || lastChange.stat == null) {
+            throw new KeeperException.NoNodeException();
+        }
+        return lastChange;
+    }
+
+    void addChangeRecord(ChangeRecord c) {
+        synchronized (zks.outstandingChanges) {
+            zks.outstandingChanges.add(c);
+        }
+    }
+
+    static void checkACL(ZooKeeperServer zks, ArrayList<ACL> acl, int perm,
+            ArrayList<Id> ids) throws KeeperException.NoAuthException {
+        if (skipACL) {
+            return;
+        }
+        if (acl == null || acl.size() == 0) {
+            return;
+        }
+        for (ACL a : acl) {
+            Id id = a.getId();
+            if ((a.getPerms() & perm) != 0) {
+                if (id.getScheme().equals("world")
+                        && id.getId().equals("anyone")) {
+                    return;
+                }
+                AuthenticationProvider ap = ProviderRegistry.getProvider(id
+                        .getScheme());
+                if (ap != null) {
+                    for (Id authId : ids) {
+                        if (authId.getScheme().equals("super")) {
+                            return;
+                        }
+                        if (authId.getScheme().equals(id.getScheme())
+                                && ap.matches(authId.getId(), id.getId())) {
+                            return;
+                        }
+                    }
+                }
+            }
+        }
+        throw new KeeperException.NoAuthException();
+    }
+
+    /**
+     * This method will be called inside the ProcessRequestThread, which is a
+     * singleton, so there will be a single thread calling this code.
+     *
+     * @param request
+     */
+    @SuppressWarnings("unchecked")
+    protected void pRequest(Request request) {
+        // LOG.info("Prep>>> cxid = " + request.cxid + " type = " +
+        // request.type + " id = " + request.sessionId);
+        TxnHeader txnHeader = null;
+        Record txn = null;
+        try {
+            switch (request.type) {
+            case OpCode.create:
+                txnHeader = new TxnHeader(request.sessionId, request.cxid, zks
+                        .getNextZxid(), zks.getTime(), OpCode.create);
+                zks.sessionTracker.checkSession(request.sessionId);
+                CreateRequest createRequest = new CreateRequest();
+                ZooKeeperServer.byteBuffer2Record(request.request,
+                        createRequest);
+                String path = createRequest.getPath();
+                int lastSlash = path.lastIndexOf('/');
+                if (lastSlash == -1 || path.indexOf('\0') != -1) {
+                    throw new KeeperException.BadArgumentsException();
+                }
+                if (!fixupACL(request.authInfo, createRequest.getAcl())) {
+                    throw new KeeperException.InvalidACLException();
+                }
+                String parentPath = path.substring(0, lastSlash);
+                ChangeRecord parentRecord = getRecordForPath(parentPath);
+
+                checkACL(zks, parentRecord.acl, ZooDefs.Perms.CREATE,
+                        request.authInfo);
+                int parentCVersion = parentRecord.stat.getCversion();
+                if ((createRequest.getFlags() & CreateFlags.SEQUENCE) != 0) {
+                    path = path + parentCVersion;
+                }
+                try {
+                    if (getRecordForPath(path) != null) {
+                        throw new KeeperException.NodeExistsException();
+                    }
+                } catch (KeeperException.NoNodeException e) {
+                    // ignore this one
+                }
+                boolean ephemeralParent = parentRecord.stat.getEphemeralOwner() != 0;
+                if (ephemeralParent) {
+                    throw new KeeperException.NoChildrenForEphemeralsException();
+                }
+                txn = new CreateTxn(path, createRequest.getData(),
+                        createRequest.getAcl(),
+                        (createRequest.getFlags() & CreateFlags.EPHEMERAL) != 0);
+                Stat s = new Stat();
+                if ((createRequest.getFlags() & CreateFlags.EPHEMERAL) != 0) {
+                    s.setEphemeralOwner(request.sessionId);
+                }
+                parentRecord = parentRecord.duplicate(txnHeader.getZxid());
+                parentRecord.childCount++;
+                parentRecord.stat
+                        .setCversion(parentRecord.stat.getCversion() + 1);
+                addChangeRecord(parentRecord);
+                addChangeRecord(new ChangeRecord(txnHeader.getZxid(), path, s,
+                        0, createRequest.getAcl()));
+
+                break;
+            case OpCode.delete:
+                txnHeader = new TxnHeader(request.sessionId, request.cxid, zks
+                        .getNextZxid(), zks.getTime(), OpCode.delete);
+                zks.sessionTracker.checkSession(request.sessionId);
+                DeleteRequest deleteRequest = new DeleteRequest();
+                ZooKeeperServer.byteBuffer2Record(request.request,
+                        deleteRequest);
+                path = deleteRequest.getPath();
+                lastSlash = path.lastIndexOf('/');
+                if (lastSlash == -1 || path.indexOf('\0') != -1
+                        || path.equals("/")) {
+                    throw new KeeperException.BadArgumentsException();
+                }
+                parentPath = path.substring(0, lastSlash);
+                parentRecord = getRecordForPath(parentPath);
+                ChangeRecord nodeRecord = getRecordForPath(path);
+                checkACL(zks, parentRecord.acl, ZooDefs.Perms.DELETE,
+                        request.authInfo);
+                int version = deleteRequest.getVersion();
+                if (version != -1 && nodeRecord.stat.getVersion() != version) {
+                    throw new KeeperException.BadVersionException();
+                }
+                if (nodeRecord.childCount > 0) {
+                    throw new KeeperException.NotEmptyException();
+                }
+                txn = new DeleteTxn(path);
+                parentRecord = parentRecord.duplicate(txnHeader.getZxid());
+                parentRecord.childCount--;
+                parentRecord.stat
+                        .setCversion(parentRecord.stat.getCversion() + 1);
+                addChangeRecord(parentRecord);
+                addChangeRecord(new ChangeRecord(txnHeader.getZxid(), path,
+                        null, -1, null));
+                break;
+            case OpCode.setData:
+                txnHeader = new TxnHeader(request.sessionId, request.cxid, zks
+                        .getNextZxid(), zks.getTime(), OpCode.setData);
+                zks.sessionTracker.checkSession(request.sessionId);
+                SetDataRequest setDataRequest = new SetDataRequest();
+                ZooKeeperServer.byteBuffer2Record(request.request,
+                        setDataRequest);
+                path = setDataRequest.getPath();
+                nodeRecord = getRecordForPath(path);
+                checkACL(zks, nodeRecord.acl, ZooDefs.Perms.WRITE,
+                        request.authInfo);
+                version = setDataRequest.getVersion();
+                int currentVersion = nodeRecord.stat.getVersion();
+                if (version != -1 && version != currentVersion) {
+                    throw new KeeperException.BadVersionException();
+                }
+                version = currentVersion + 1;
+                txn = new SetDataTxn(path, setDataRequest.getData(), version);
+                nodeRecord = nodeRecord.duplicate(txnHeader.getZxid());
+                nodeRecord.stat.setVersion(version);
+                addChangeRecord(nodeRecord);
+                break;
+            case OpCode.setACL:
+                txnHeader = new TxnHeader(request.sessionId, request.cxid, zks
+                        .getNextZxid(), zks.getTime(), OpCode.setACL);
+                zks.sessionTracker.checkSession(request.sessionId);
+                SetACLRequest setAclRequest = new SetACLRequest();
+                if (!fixupACL(request.authInfo, setAclRequest.getAcl())) {
+                    throw new KeeperException.InvalidACLException();
+                }
+                ZooKeeperServer.byteBuffer2Record(request.request,
+                        setAclRequest);
+                path = setAclRequest.getPath();
+                nodeRecord = getRecordForPath(path);
+                checkACL(zks, nodeRecord.acl, ZooDefs.Perms.ADMIN,
+                        request.authInfo);
+                version = setAclRequest.getVersion();
+                currentVersion = nodeRecord.stat.getAversion();
+                if (version != -1 && version != currentVersion) {
+                    throw new KeeperException.BadVersionException();
+                }
+                version = currentVersion + 1;
+                txn = new SetACLTxn(path, setAclRequest.getAcl(), version);
+                nodeRecord = nodeRecord.duplicate(txnHeader.getZxid());
+                nodeRecord.stat.setAversion(version);
+                addChangeRecord(nodeRecord);
+                break;
+            case OpCode.createSession:
+                txnHeader = new TxnHeader(request.sessionId, request.cxid, zks
+                        .getNextZxid(), zks.getTime(), OpCode.createSession);
+                request.request.rewind();
+                int to = request.request.getInt();
+                txn = new CreateSessionTxn(to);
+                request.request.rewind();
+                zks.sessionTracker.addSession(request.sessionId, to);
+                break;
+            case OpCode.closeSession:
+                txnHeader = new TxnHeader(request.sessionId, request.cxid, zks
+                        .getNextZxid(), zks.getTime(), OpCode.closeSession);
+                HashSet<String> es = zks.dataTree
+                        .getEphemerals(request.sessionId);
+                synchronized (zks.outstandingChanges) {
+                    for (ChangeRecord c : zks.outstandingChanges) {
+                        if (c.stat == null) {
+                            // Doing a delete
+                            es.remove(c.path);
+                        } else if (c.stat.getEphemeralOwner() == request.sessionId) {
+                            es.add(c.path);
+                        }
+                    }
+                    for (String path2Delete : es) {
+                        addChangeRecord(new ChangeRecord(txnHeader.getZxid(),
+                                path2Delete, null, 0, null));
+                    }
+                }
+                LOG.info("Processed session termination request for id: "
+                        + Long.toHexString(request.sessionId));
+                break;
+            case OpCode.sync:
+            case OpCode.exists:
+            case OpCode.getData:
+            case OpCode.getACL:
+            case OpCode.getChildren:
+            case OpCode.ping:
+                break;
+            }
+        } catch (KeeperException e) {
+            if (txnHeader != null) {
+                txnHeader.setType(OpCode.error);
+                txn = new ErrorTxn(e.getCode());
+            }
+        } catch (Exception e) {
+            LOG.error("*********************************" + request);
+            StringBuffer sb = new StringBuffer();
+            ByteBuffer bb = request.request;
+            if(bb!=null){
+                bb.rewind();
+                while (bb.hasRemaining()) {
+                    sb.append(Integer.toHexString(bb.get() & 0xff));
+                }
+            }else
+                sb.append("request buffer is null");
+            LOG.error(sb.toString());
+            LOG.error("Unexpected exception", e);
+            if (txnHeader != null) {
+                txnHeader.setType(OpCode.error);
+                txn = new ErrorTxn(Code.MarshallingError);
+            }
+        }
+        request.hdr = txnHeader;
+        request.txn = txn;
+        if (request.hdr != null) {
+            request.zxid = request.hdr.getZxid();
+        }
+        nextProcessor.processRequest(request);
+    }
+
+    /**
+     *
+     * @param authInfo list of ACL IDs associated with the client connection
+     * @param acl list of ACLs being assigned to the node (create or setACL operation)
+     * @return
+     */
+    private boolean fixupACL(ArrayList<Id> authInfo, ArrayList<ACL> acl) {
+        if (skipACL) {
+            return true;
+        }
+        if (acl == null || acl.size() == 0) {
+            return false;
+        }
+        Iterator<ACL> it = acl.iterator();
+        LinkedList<ACL> toAdd = null;
+        while (it.hasNext()) {
+            ACL a = it.next();
+            Id id = a.getId();
+            if (id.getScheme().equals("world") && id.getId().equals("anyone")) {
+            } else if (id.getScheme().equals("auth")) {
+                it.remove();
+                if (toAdd == null) {
+                    toAdd = new LinkedList<ACL>();
+                }
+                for (Id cid : authInfo) {
+                    AuthenticationProvider ap = ProviderRegistry.getProvider(cid.getScheme());
+                    if (ap == null) {
+                        LOG.error("Missing AuthenticationProvider for "
+                                + cid.getScheme());
+                    } else if (ap.isAuthenticated()) {
+                        toAdd.add(new ACL(a.getPerms(), cid));
+                    }
+                }
+            } else {
+                AuthenticationProvider ap = ProviderRegistry.getProvider(id
+                        .getScheme());
+                if (ap == null) {
+                    return false;
+                }
+                if (!ap.isValid(id.getId())) {
+                    return false;
+                }
+            }
+        }
+        if (toAdd != null) {
+            for (ACL a : toAdd) {
+                acl.add(a);
+            }
+        }
+        return true;
+    }
+
+    public void processRequest(Request request) {
+        // request.addRQRec(">prep="+zks.outstandingChanges.size());
+        submittedRequests.add(request);
+    }
+
+    public void shutdown() {
+        submittedRequests.clear();
+        submittedRequests.add(Request.requestOfDeath);
+        nextProcessor.shutdown();
+    }
+}

+ 1 - 1
zookeeper/java/src/com/yahoo/zookeeper/server/SessionTracker.java

@@ -52,5 +52,5 @@ public interface SessionTracker {
      */
     void removeSession(long sessionId);
 
-    void checkSession(long sessionId) throws KeeperException;
+    void checkSession(long sessionId) throws KeeperException.SessionExpiredException;
 }

+ 3 - 3
zookeeper/java/src/com/yahoo/zookeeper/server/SessionTrackerImpl.java

@@ -190,7 +190,7 @@ public class SessionTrackerImpl extends Thread implements SessionTracker {
                                  "Shutdown SessionTrackerImpl!");
     }
 
-
+   
     synchronized public long createSession(int sessionTimeout) {
         addSession(nextSessionId, sessionTimeout);
         return nextSessionId++;
@@ -212,9 +212,9 @@ public class SessionTrackerImpl extends Thread implements SessionTracker {
         touchSession(id, sessionTimeout);
     }
 
-    public void checkSession(long sessionId) throws KeeperException {
+    public void checkSession(long sessionId) throws KeeperException.SessionExpiredException {
         if (sessionsById.get(sessionId) == null) {
-            throw new KeeperException(KeeperException.Code.SessionExpired);
+            throw new KeeperException.SessionExpiredException();
         }
     }
 }

+ 925 - 927
zookeeper/java/src/com/yahoo/zookeeper/server/ZooKeeperServer.java

@@ -1,927 +1,925 @@
-/*
- * Copyright 2008, Yahoo! Inc.
- *
- * 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.
- */
-
-package com.yahoo.zookeeper.server;
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.EOFException;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Random;
-import java.util.Map.Entry;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.log4j.Logger;
-
-import com.yahoo.jute.BinaryInputArchive;
-import com.yahoo.jute.BinaryOutputArchive;
-import com.yahoo.jute.InputArchive;
-import com.yahoo.jute.Record;
-import com.yahoo.zookeeper.KeeperException;
-import com.yahoo.zookeeper.ZooDefs.OpCode;
-import com.yahoo.zookeeper.data.ACL;
-import com.yahoo.zookeeper.data.Id;
-import com.yahoo.zookeeper.data.Stat;
-import com.yahoo.zookeeper.proto.RequestHeader;
-import com.yahoo.zookeeper.server.SessionTracker.SessionExpirer;
-import com.yahoo.zookeeper.server.quorum.Leader;
-import com.yahoo.zookeeper.server.quorum.QuorumPacket;
-import com.yahoo.zookeeper.server.quorum.Leader.Proposal;
-import com.yahoo.zookeeper.txn.CreateSessionTxn;
-import com.yahoo.zookeeper.txn.CreateTxn;
-import com.yahoo.zookeeper.txn.DeleteTxn;
-import com.yahoo.zookeeper.txn.ErrorTxn;
-import com.yahoo.zookeeper.txn.SetACLTxn;
-import com.yahoo.zookeeper.txn.SetDataTxn;
-import com.yahoo.zookeeper.txn.TxnHeader;
-
-/**
- * This class implements a simple standalone ZooKeeperServer. It sets up the
- * following chain of RequestProcessors to process requests:
- * PrepRequestProcessor -> SyncRequestProcessor -> FinalRequestProcessor
- */
-public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
-    private static final Logger LOG = Logger.getLogger(ZooKeeperServer.class);
-
-    /**
-     * Create an instance of Zookeeper server
-     */
-    public interface Factory {
-        public ZooKeeperServer createServer() throws IOException;
-
-        public NIOServerCnxn.Factory createConnectionFactory()
-                throws IOException;
-    }
-
-    /**
-     * The server delegates loading of the tree to an instance of the interface
-     */
-    public interface DataTreeBuilder {
-        public DataTree build();
-    }
-
-    static public class BasicDataTreeBuilder implements DataTreeBuilder {
-        public DataTree build() {
-            return new DataTree();
-        }
-    }
-
-    private static final int DEFAULT_TICK_TIME = 3000;
-    protected int tickTime = DEFAULT_TICK_TIME;
-
-    public static final int commitLogCount = 500;
-    public int commitLogBuffer = 700;
-    public LinkedList<Proposal> committedLog = new LinkedList<Proposal>();
-    public long minCommittedLog, maxCommittedLog;
-    private DataTreeBuilder treeBuilder;
-    public DataTree dataTree;
-    protected SessionTracker sessionTracker;
-    /**
-     * directory for storing the snapshot
-     */
-    File dataDir;
-    /**
-     * directoy for storing the log tnxns
-     */
-    File dataLogDir;
-    protected ConcurrentHashMap<Long, Integer> sessionsWithTimeouts;
-    protected long hzxid = 0;
-    final public static Exception ok = new Exception("No prob");
-    protected RequestProcessor firstProcessor;
-    LinkedBlockingQueue<Long> sessionsToDie = new LinkedBlockingQueue<Long>();
-    protected boolean running;
-    /**
-     * This is the secret that we use to generate passwords, for the moment it
-     * is more of a sanity check.
-     */
-    final private long superSecret = 0XB3415C00L;
-    int requestsInProcess;
-    ArrayList<ChangeRecord> outstandingChanges = new ArrayList<ChangeRecord>();
-    private NIOServerCnxn.Factory serverCnxnFactory;
-
-    /*
-     * Start up the ZooKeeper server.
-     *
-     * @param args the port and data directory
-     */
-    public static void main(String[] args) {
-        ServerConfig.parse(args);
-        runStandalone(new Factory() {
-            public NIOServerCnxn.Factory createConnectionFactory()
-                    throws IOException {
-                return new NIOServerCnxn.Factory(ServerConfig.getClientPort());
-            }
-
-            public ZooKeeperServer createServer() throws IOException {
-                return new ZooKeeperServer(new BasicDataTreeBuilder());
-            }
-        });
-    }
-
-    public static void runStandalone(Factory factory) {
-        try {
-            // Note that this thread isn't going to be doing anything else,
-            // so rather than spawning another thread, we will just call
-            // run() in this thread.
-            ServerStats.registerAsConcrete();
-            ZooKeeperServer zk = factory.createServer();
-            zk.startup();
-            NIOServerCnxn.Factory t = factory.createConnectionFactory();
-            t.setZooKeeperServer(zk);
-            t.join();
-            if (zk.isRunning())
-                zk.shutdown();
-        } catch (Exception e) {
-            LOG.fatal("Unexpected exception",e);
-        }
-        System.exit(0);
-    }
-
-    void removeCnxn(ServerCnxn cnxn) {
-        dataTree.removeCnxn(cnxn);
-    }
-
-    /**
-     * Creates a ZooKeeperServer instance. It sets everything up, but doesn't
-     * actually start listening for clients until run() is invoked.
-     *
-     * @param dataDir
-     *            the directory to put the data
-     * @throws IOException
-     */
-    public ZooKeeperServer(File dataDir, File dataLogDir, int tickTime,
-            DataTreeBuilder treeBuilder) throws IOException {
-        this.treeBuilder = treeBuilder;
-        this.dataDir = dataDir;
-        this.dataLogDir = dataLogDir;
-        this.tickTime = tickTime;
-        if (!dataDir.isDirectory()) {
-            throw new IOException("data directory does not exist");
-        }
-        ServerStats.getInstance().setStatsProvider(this);
-    }
-
-    /**
-     * This constructor is for backward comaptibility with the existing unit
-     * test code.
-     */
-    public ZooKeeperServer(File dataDir, File dataLogDir, int tickTime)
-            throws IOException {
-        this.treeBuilder = new BasicDataTreeBuilder();
-        this.dataDir = dataDir;
-        this.dataLogDir = dataLogDir;
-        this.tickTime = tickTime;
-        if (!dataDir.isDirectory()) {
-            throw new IOException("data directory does not exist");
-        }
-        ServerStats.getInstance().setStatsProvider(this);
-    }
-
-    /**
-     * Default constructor, relies on the config for its agrument values
-     *
-     * @throws IOException
-     */
-    public ZooKeeperServer(DataTreeBuilder treeBuilder) throws IOException {
-        this(new File(ServerConfig.getDataDir()), new File(ServerConfig
-                .getDataLogDir()), DEFAULT_TICK_TIME, treeBuilder);
-    }
-
-    public static long getZxidFromName(String name, String prefix) {
-        long zxid = -1;
-        String nameParts[] = name.split("\\.");
-        if (nameParts.length == 2 && nameParts[0].equals(prefix)) {
-            try {
-                zxid = Long.parseLong(nameParts[1], 16);
-            } catch (NumberFormatException e) {
-            }
-        }
-        return zxid;
-    }
-
-    static public long isValidSnapshot(File f) throws IOException {
-        long zxid = getZxidFromName(f.getName(), "snapshot");
-        if (zxid == -1)
-            return -1;
-
-        // Check for a valid snapshot
-        RandomAccessFile raf = new RandomAccessFile(f, "r");
-        try {
-            raf.seek(raf.length() - 5);
-            byte bytes[] = new byte[5];
-            raf.read(bytes);
-            ByteBuffer bb = ByteBuffer.wrap(bytes);
-            int len = bb.getInt();
-            byte b = bb.get();
-            if (len != 1 || b != '/') {
-                LOG.warn("Invalid snapshot " + f + " len = " + len
-                        + " byte = " + (b & 0xff));
-                return -1;
-            }
-        } finally {
-            raf.close();
-        }
-
-        return zxid;
-    }
-
-    /**
-     * Compare file file names of form "prefix.version". Sort order result
-     * returned in order of version.
-     */
-    private static class DataDirFileComparator implements Comparator<File> {
-        private String prefix;
-        private boolean ascending;
-        public DataDirFileComparator(String prefix, boolean ascending) {
-            this.prefix = prefix;
-            this.ascending = ascending;
-        }
-
-        public int compare(File o1, File o2) {
-            long z1 = getZxidFromName(o1.getName(), prefix);
-            long z2 = getZxidFromName(o2.getName(), prefix);
-            int result = z1 < z2 ? -1 : (z1 > z2 ? 1 : 0);
-            return ascending ? result : -result;
-        }
-    }
-
-    /**
-     * Sort the list of files. Recency as determined by the version component
-     * of the file name.
-     *
-     * @param files array of files
-     * @param prefix files not matching this prefix are assumed to have a
-     * version = -1)
-     * @param ascending true sorted in ascending order, false results in
-     * descending order
-     * @return sorted input files
-     */
-    static List<File>
-        sortDataDir(File[] files, String prefix, boolean ascending)
-    {
-        List<File> filelist = Arrays.asList(files);
-        Collections.sort(filelist, new DataDirFileComparator(prefix, ascending));
-        return filelist;
-    }
-
-    /**
-     * Find the log file that starts at, or just before, the snapshot. Return
-     * this and all subsequent logs. Results are ordered by zxid of file,
-     * ascending order.
-     *
-     * @param logDirList array of files
-     * @param snapshotZxid return files at, or before this zxid
-     * @return
-     */
-    static File[] getLogFiles(File[] logDirList,long snapshotZxid) {
-        List<File> files = sortDataDir(logDirList, "log", true);
-        long logZxid = 0;
-        // Find the log file that starts before or at the same time as the
-        // zxid of the snapshot
-        for (File f : files) {
-            long fzxid = getZxidFromName(f.getName(), "log");
-            if (fzxid > snapshotZxid) {
-                continue;
-            }
-            if (fzxid > logZxid) {
-                logZxid = fzxid;
-            }
-        }
-        List<File> v=new ArrayList<File>(5);
-        // Apply the logs
-        for (File f : files) {
-            long fzxid = getZxidFromName(f.getName(), "log");
-            if (fzxid < logZxid) {
-                continue;
-            }
-            v.add(f);
-        }
-        return v.toArray(new File[0]);
-    }
-
-    /**
-     *  Restore sessions and data
-     */
-    private void loadSnapshotAndLogs() throws IOException {
-        long zxid = -1;
-
-        // Find the most recent snapshot
-        List<File> files = sortDataDir(dataDir.listFiles(), "snapshot", false);
-        for (File f : files) {
-            zxid = isValidSnapshot(f);
-            if (zxid == -1) {
-                LOG.warn("Skipping " + f);
-                continue;
-            }
-
-            LOG.warn("Processing snapshot: " + f);
-
-            InputStream snapIS =
-                new BufferedInputStream(new FileInputStream(f));
-            loadData(BinaryInputArchive.getArchive(snapIS));
-            snapIS.close();
-
-            dataTree.lastProcessedZxid = zxid;
-
-            // Apply the logs on/after the selected snapshot
-            File[] logfiles = getLogFiles(dataLogDir.listFiles(), zxid);
-            for (File logfile : logfiles) {
-                LOG.warn("Processing log file: " + logfile);
-
-                InputStream logIS =
-                    new BufferedInputStream(new FileInputStream(logfile));
-                zxid = playLog(BinaryInputArchive.getArchive(logIS));
-                logIS.close();
-            }
-            hzxid = zxid;
-
-            break;
-        }
-
-        if (zxid == -1) {
-            sessionsWithTimeouts = new ConcurrentHashMap<Long, Integer>();
-            dataTree = treeBuilder.build();
-        }
-    }
-
-    public void loadData() throws IOException, InterruptedException {
-        loadSnapshotAndLogs();
-
-        // Clean up dead sessions
-        LinkedList<Long> deadSessions = new LinkedList<Long>();
-        for (long session : dataTree.getSessions()) {
-            if (sessionsWithTimeouts.get(session) == null) {
-                deadSessions.add(session);
-            }
-        }
-        dataTree.initialized = true;
-        for (long session : deadSessions) {
-            killSession(session);
-        }
-        // Make a clean snapshot
-        snapshot();
-    }
-
-    public void loadData(InputArchive ia) throws IOException {
-        sessionsWithTimeouts = new ConcurrentHashMap<Long, Integer>();
-        dataTree = treeBuilder.build();
-
-        int count = ia.readInt("count");
-        while (count > 0) {
-            long id = ia.readLong("id");
-            int to = ia.readInt("timeout");
-            sessionsWithTimeouts.put(id, to);
-            ZooTrace.logTraceMessage(LOG, ZooTrace.SESSION_TRACE_MASK,
-                                     "loadData --- session in archive: " + id
-                                     + " with timeout: " + to);
-            count--;
-        }
-        dataTree.deserialize(ia, "tree");
-    }
-
-    public long playLog(InputArchive logStream) throws IOException {
-        long highestZxid = 0;
-        try {
-            while (true) {
-                byte[] bytes = logStream.readBuffer("txnEntry");
-                if (bytes.length == 0) {
-                    // Since we preallocate, we define EOF to be an
-                    // empty transaction
-                    throw new EOFException();
-                }
-                InputArchive ia = BinaryInputArchive
-                        .getArchive(new ByteArrayInputStream(bytes));
-                TxnHeader hdr = new TxnHeader();
-                Record txn = deserializeTxn(ia, hdr);
-                if (logStream.readByte("EOR") != 'B') {
-                    LOG.error("Last transaction was partial.");
-                    throw new EOFException();
-                }
-                if (hdr.getZxid() <= highestZxid && highestZxid != 0) {
-                    LOG.error(highestZxid + "(higestZxid) >= "
-                            + hdr.getZxid() + "(next log) for type "
-                            + hdr.getType());
-                } else {
-                    highestZxid = hdr.getZxid();
-                }
-                switch (hdr.getType()) {
-                case OpCode.createSession:
-                    sessionsWithTimeouts.put(hdr.getClientId(),
-                            ((CreateSessionTxn) txn).getTimeOut());
-                    ZooTrace.logTraceMessage(LOG,
-                                             ZooTrace.SESSION_TRACE_MASK,
-                            "playLog --- create session in log: "
-                                    + Long.toHexString(hdr.getClientId())
-                                    + " with timeout: "
-                                    + ((CreateSessionTxn) txn).getTimeOut());
-                    // give dataTree a chance to sync its lastProcessedZxid
-                    dataTree.processTxn(hdr, txn);
-                    break;
-                case OpCode.closeSession:
-                    sessionsWithTimeouts.remove(hdr.getClientId());
-                    ZooTrace.logTraceMessage(LOG,
-                            ZooTrace.SESSION_TRACE_MASK,
-                            "playLog --- close session in log: "
-                                    + Long.toHexString(hdr.getClientId()));
-                    dataTree.processTxn(hdr, txn);
-                    break;
-                default:
-                    dataTree.processTxn(hdr, txn);
-                }
-                Request r = new Request(null, 0, hdr.getCxid(), hdr.getType(),
-                        null, null);
-                r.txn = txn;
-                r.hdr = hdr;
-                r.zxid = hdr.getZxid();
-                addCommittedProposal(r);
-            }
-        } catch (EOFException e) {
-        }
-        return highestZxid;
-    }
-
-    /**
-     * maintains a list of last 500 or so committed requests. This is used for
-     * fast follower synchronization.
-     *
-     * @param r
-     *            committed request
-     */
-
-    public void addCommittedProposal(Request request) {
-        synchronized (committedLog) {
-            if (committedLog.size() > commitLogCount) {
-                committedLog.removeFirst();
-                minCommittedLog = committedLog.getFirst().packet.getZxid();
-            }
-            if (committedLog.size() == 0) {
-                minCommittedLog = request.zxid;
-                maxCommittedLog = request.zxid;
-            }
-
-            ByteArrayOutputStream baos = new ByteArrayOutputStream();
-            BinaryOutputArchive boa = BinaryOutputArchive.getArchive(baos);
-            try {
-                request.hdr.serialize(boa, "hdr");
-                if (request.txn != null) {
-                    request.txn.serialize(boa, "txn");
-                }
-                baos.close();
-            } catch (IOException e) {
-                // This really should be impossible
-                LOG.error("FIXMSG",e);
-            }
-            QuorumPacket pp = new QuorumPacket(Leader.PROPOSAL, request.zxid,
-                    baos.toByteArray(), null);
-            Proposal p = new Proposal();
-            p.packet = pp;
-            p.request = request;
-            committedLog.add(p);
-            maxCommittedLog = p.packet.getZxid();
-        }
-    }
-
-    static public Record deserializeTxn(InputArchive ia, TxnHeader hdr)
-            throws IOException {
-        hdr.deserialize(ia, "hdr");
-        Record txn = null;
-        switch (hdr.getType()) {
-        case OpCode.createSession:
-            // This isn't really an error txn; it just has the same
-            // format. The error represents the timeout
-            txn = new CreateSessionTxn();
-            break;
-        case OpCode.closeSession:
-            return null;
-        case OpCode.create:
-            txn = new CreateTxn();
-            break;
-        case OpCode.delete:
-            txn = new DeleteTxn();
-            break;
-        case OpCode.setData:
-            txn = new SetDataTxn();
-            break;
-        case OpCode.setACL:
-            txn = new SetACLTxn();
-            break;
-        case OpCode.error:
-            txn = new ErrorTxn();
-            break;
-        }
-        if (txn != null) {
-            txn.deserialize(ia, "txn");
-        }
-        return txn;
-    }
-
-    public void truncateLog(long finalZxid) throws IOException {
-        long highestZxid = 0;
-        for (File f : dataDir.listFiles()) {
-            long zxid = isValidSnapshot(f);
-            if (zxid == -1) {
-                LOG.warn("Skipping " + f);
-                continue;
-            }
-            if (zxid > highestZxid) {
-                highestZxid = zxid;
-            }
-        }
-        File[] files = getLogFiles(dataLogDir.listFiles(), highestZxid);
-        boolean truncated = false;
-        for (File f : files) {
-            FileInputStream fin = new FileInputStream(f);
-            InputArchive ia = BinaryInputArchive.getArchive(fin);
-            FileChannel fchan = fin.getChannel();
-            try {
-                while (true) {
-                    byte[] bytes = ia.readBuffer("txtEntry");
-                    if (bytes.length == 0) {
-                        throw new EOFException();
-                    }
-                    InputArchive iab = BinaryInputArchive
-                            .getArchive(new ByteArrayInputStream(bytes));
-                    TxnHeader hdr = new TxnHeader();
-                    deserializeTxn(iab, hdr);
-                    if (ia.readByte("EOF") != 'B') {
-                        throw new EOFException();
-                    }
-                    if (hdr.getZxid() == finalZxid) {
-                        // this is where we need to truncate
-
-                        long pos = fchan.position();
-                        fin.close();
-                        FileOutputStream fout = new FileOutputStream(f);
-                        FileChannel fchanOut = fout.getChannel();
-                        fchanOut.truncate(pos);
-                        truncated = true;
-                        break;
-                    }
-                }
-            } catch (EOFException eof) {
-            }
-            if (truncated == true) {
-                break;
-            }
-        }
-        if (truncated == false) {
-            // not able to truncate the log
-            LOG.error("Not able to truncate the log "
-                    + Long.toHexString(finalZxid));
-            System.exit(13);
-        }
-
-    }
-
-    public void snapshot(BinaryOutputArchive oa) throws IOException,
-            InterruptedException {
-        HashMap<Long, Integer> sessSnap = new HashMap<Long, Integer>(
-                sessionsWithTimeouts);
-        oa.writeInt(sessSnap.size(), "count");
-        for (Entry<Long, Integer> entry : sessSnap.entrySet()) {
-            oa.writeLong(entry.getKey().longValue(), "id");
-            oa.writeInt(entry.getValue().intValue(), "timeout");
-        }
-        dataTree.serialize(oa, "tree");
-    }
-
-    public void snapshot() throws InterruptedException {
-        long lastZxid = dataTree.lastProcessedZxid;
-        ZooTrace.logTraceMessage(LOG, ZooTrace.getTextTraceLevel(),
-                "Snapshotting: " + Long.toHexString(lastZxid));
-        try {
-            File f =new File(dataDir, "snapshot." + Long.toHexString(lastZxid));
-            OutputStream sessOS = new BufferedOutputStream(new FileOutputStream(f));
-            BinaryOutputArchive oa = BinaryOutputArchive.getArchive(sessOS);
-            snapshot(oa);
-            sessOS.flush();
-            sessOS.close();
-            ZooTrace.logTraceMessage(LOG, ZooTrace.getTextTraceLevel(),
-                    "Snapshotting finished: " + Long.toHexString(lastZxid));
-        } catch (IOException e) {
-            LOG.error("Severe error, exiting",e);
-            // This is a severe error that we cannot recover from,
-            // so we need to exit
-            System.exit(10);
-        }
-    }
-
-    /**
-     * This should be called from a synchronized block on this!
-     */
-    public long getZxid() {
-        return hzxid;
-    }
-
-    synchronized long getNextZxid() {
-        return ++hzxid;
-    }
-
-    long getTime() {
-        return System.currentTimeMillis();
-    }
-
-    static String getLogName(long zxid) {
-        return "log." + Long.toHexString(zxid);
-    }
-
-    public void closeSession(long sessionId) throws KeeperException,
-            InterruptedException {
-        ZooTrace.logTraceMessage(LOG, ZooTrace.SESSION_TRACE_MASK,
-                                 "ZooKeeperServer --- Session to be closed: "
-                + Long.toHexString(sessionId));
-        // we do not want to wait for a session close. send it as soon as we
-        // detect it!
-        submitRequest(null, sessionId, OpCode.closeSession, 0, null, null);
-    }
-
-    protected void killSession(long sessionId) {
-        dataTree.killSession(sessionId);
-        ZooTrace.logTraceMessage(LOG, ZooTrace.SESSION_TRACE_MASK,
-                                     "ZooKeeperServer --- killSession: "
-                + Long.toHexString(sessionId));
-        if (sessionTracker != null) {
-            sessionTracker.removeSession(sessionId);
-        }
-    }
-
-    public void expire(long sessionId) {
-        try {
-            ZooTrace.logTraceMessage(LOG,
-                                     ZooTrace.SESSION_TRACE_MASK,
-                    "ZooKeeperServer --- Session to expire: " + Long.toHexString(sessionId));
-            closeSession(sessionId);
-        } catch (Exception e) {
-            LOG.error("FIXMSG",e);
-        }
-    }
-
-    void touch(ServerCnxn cnxn) throws IOException {
-        if (cnxn == null) {
-            return;
-        }
-        long id = cnxn.getSessionId();
-        int to = cnxn.getSessionTimeout();
-        if (!sessionTracker.touchSession(id, to)) {
-            throw new IOException("Missing session " + Long.toHexString(id));
-        }
-    }
-
-    public void startup() throws IOException, InterruptedException {
-        if (dataTree == null) {
-            loadData();
-        }
-        createSessionTracker();
-        setupRequestProcessors();
-        running = true;
-        synchronized (this) {
-            notifyAll();
-        }
-    }
-
-    protected void setupRequestProcessors() {
-        RequestProcessor finalProcessor = new FinalRequestProcessor(this);
-        RequestProcessor syncProcessor = new SyncRequestProcessor(this,
-                finalProcessor);
-        firstProcessor = new PrepRequestProcessor(this, syncProcessor);
-    }
-
-    protected void createSessionTracker() {
-        sessionTracker = new SessionTrackerImpl(this, sessionsWithTimeouts,
-                tickTime, 1);
-    }
-
-    public boolean isRunning() {
-        return running;
-    }
-
-    public void shutdown() {
-        // new RuntimeException("Calling shutdown").printStackTrace();
-        this.running = false;
-        // Since sessionTracker and syncThreads poll we just have to
-        // set running to false and they will detect it during the poll
-        // interval.
-        if (sessionTracker != null) {
-            sessionTracker.shutdown();
-        }
-        if (firstProcessor != null) {
-            firstProcessor.shutdown();
-        }
-        if (dataTree != null) {
-            dataTree.clear();
-        }
-    }
-
-    synchronized public void incInProcess() {
-        requestsInProcess++;
-    }
-
-    synchronized public void decInProcess() {
-        requestsInProcess--;
-    }
-
-    public int getInProcess() {
-        return requestsInProcess;
-    }
-
-    /**
-     * This structure is used to facilitate information sharing between PrepRP
-     * and FinalRP.
-     */
-    static class ChangeRecord {
-        ChangeRecord(long zxid, String path, Stat stat, int childCount,
-                ArrayList<ACL> acl) {
-            this.zxid = zxid;
-            this.path = path;
-            this.stat = stat;
-            this.childCount = childCount;
-            this.acl = acl;
-        }
-
-        long zxid;
-
-        String path;
-
-        Stat stat; /* Make sure to create a new object when changing */
-
-        int childCount;
-
-        ArrayList<ACL> acl; /* Make sure to create a new object when changing */
-
-        @SuppressWarnings("unchecked")
-        ChangeRecord duplicate(long zxid) {
-            Stat stat = new Stat();
-            if (this.stat != null) {
-                DataTree.copyStat(this.stat, stat);
-            }
-            return new ChangeRecord(zxid, path, stat, childCount,
-                    acl == null ? new ArrayList<ACL>() : (ArrayList<ACL>) acl
-                            .clone());
-        }
-    }
-
-    byte[] generatePasswd(long id) {
-        Random r = new Random(id ^ superSecret);
-        byte p[] = new byte[16];
-        r.nextBytes(p);
-        return p;
-    }
-
-    protected boolean checkPasswd(long sessionId, byte[] passwd) {
-        return sessionId != 0
-                && Arrays.equals(passwd, generatePasswd(sessionId));
-    }
-
-    long createSession(ServerCnxn cnxn, byte passwd[], int timeout)
-            throws InterruptedException {
-        long sessionId = sessionTracker.createSession(timeout);
-        Random r = new Random(sessionId ^ superSecret);
-        r.nextBytes(passwd);
-        ByteBuffer to = ByteBuffer.allocate(4);
-        to.putInt(timeout);
-        cnxn.setSessionId(sessionId);
-        submitRequest(cnxn, sessionId, OpCode.createSession, 0, to, null);
-        return sessionId;
-    }
-
-    protected void revalidateSession(ServerCnxn cnxn, long sessionId,
-            int sessionTimeout) throws IOException, InterruptedException {
-        boolean rc = sessionTracker.touchSession(sessionId, sessionTimeout);
-        ZooTrace.logTraceMessage(LOG,ZooTrace.SESSION_TRACE_MASK,
-                                 "Session " + Long.toHexString(sessionId) +
-                " is valid: " + rc);
-        cnxn.finishSessionInit(rc);
-    }
-
-    public void reopenSession(ServerCnxn cnxn, long sessionId, byte[] passwd,
-            int sessionTimeout) throws IOException, InterruptedException {
-        if (!checkPasswd(sessionId, passwd)) {
-            cnxn.finishSessionInit(false);
-        } else {
-            revalidateSession(cnxn, sessionId, sessionTimeout);
-        }
-    }
-
-    public void closeSession(ServerCnxn cnxn, RequestHeader requestHeader)
-            throws KeeperException, InterruptedException {
-        closeSession(cnxn.getSessionId());
-    }
-
-    public long getServerId() {
-        return 0;
-    }
-
-    /**
-     * @param cnxn
-     * @param sessionId
-     * @param xid
-     * @param bb
-     */
-    public void submitRequest(ServerCnxn cnxn, long sessionId, int type,
-            int xid, ByteBuffer bb, ArrayList<Id> authInfo) {
-        if (firstProcessor == null) {
-            synchronized (this) {
-                try {
-                    while (!running) {
-                        wait(1000);
-                    }
-                } catch (InterruptedException e) {
-                    LOG.error("FIXMSG",e);
-                }
-                if (firstProcessor == null) {
-                    throw new RuntimeException("Not started");
-                }
-            }
-        }
-        try {
-            touch(cnxn);
-            Request si = new Request(cnxn, sessionId, xid, type, bb, authInfo);
-            boolean validpacket = Request.isValid(type);
-            if (validpacket) {
-                firstProcessor.processRequest(si);
-                if (cnxn != null) {
-                    incInProcess();
-                }
-            } else {
-                LOG.warn("Dropping packet at server of type " + type);
-                // if unvalid packet drop the packet.
-            }
-        } catch (IOException e) {
-            LOG.error("FIXMSG",e);
-        }
-    }
-
-    static public void byteBuffer2Record(ByteBuffer bb, Record record)
-            throws IOException {
-        BinaryInputArchive ia;
-        ia = BinaryInputArchive.getArchive(new ByteBufferInputStream(bb));
-        record.deserialize(ia, "request");
-    }
-
-    public static int getSnapCount() {
-        String sc = System.getProperty("zookeeper.snapCount");
-        try {
-            return Integer.parseInt(sc);
-        } catch (Exception e) {
-            return 10000;
-        }
-    }
-
-    public int getGlobalOutstandingLimit() {
-        String sc = System.getProperty("zookeeper.globalOutstandingLimit");
-        int limit;
-        try {
-            limit = Integer.parseInt(sc);
-        } catch (Exception e) {
-            limit = 1000;
-        }
-        return limit;
-    }
-
-    public void setServerCnxnFactory(NIOServerCnxn.Factory factory) {
-        serverCnxnFactory = factory;
-    }
-
-    public NIOServerCnxn.Factory getServerCnxnFactory() {
-        return serverCnxnFactory;
-    }
-
-    public long getLastProcessedZxid() {
-        return dataTree.lastProcessedZxid;
-    }
-
-    public long getOutstandingRequests() {
-        return getInProcess();
-    }
-}
+/*
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * 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.
+ */
+
+package com.yahoo.zookeeper.server;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.log4j.Logger;
+
+import com.yahoo.jute.BinaryInputArchive;
+import com.yahoo.jute.BinaryOutputArchive;
+import com.yahoo.jute.InputArchive;
+import com.yahoo.jute.Record;
+import com.yahoo.zookeeper.ZooDefs.OpCode;
+import com.yahoo.zookeeper.data.ACL;
+import com.yahoo.zookeeper.data.Id;
+import com.yahoo.zookeeper.data.Stat;
+import com.yahoo.zookeeper.proto.RequestHeader;
+import com.yahoo.zookeeper.server.SessionTracker.SessionExpirer;
+import com.yahoo.zookeeper.server.quorum.Leader;
+import com.yahoo.zookeeper.server.quorum.QuorumPacket;
+import com.yahoo.zookeeper.server.quorum.Leader.Proposal;
+import com.yahoo.zookeeper.txn.CreateSessionTxn;
+import com.yahoo.zookeeper.txn.CreateTxn;
+import com.yahoo.zookeeper.txn.DeleteTxn;
+import com.yahoo.zookeeper.txn.ErrorTxn;
+import com.yahoo.zookeeper.txn.SetACLTxn;
+import com.yahoo.zookeeper.txn.SetDataTxn;
+import com.yahoo.zookeeper.txn.TxnHeader;
+
+/**
+ * This class implements a simple standalone ZooKeeperServer. It sets up the
+ * following chain of RequestProcessors to process requests:
+ * PrepRequestProcessor -> SyncRequestProcessor -> FinalRequestProcessor
+ */
+public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
+    private static final Logger LOG = Logger.getLogger(ZooKeeperServer.class);
+
+    /**
+     * Create an instance of Zookeeper server
+     */
+    public interface Factory {
+        public ZooKeeperServer createServer() throws IOException;
+
+        public NIOServerCnxn.Factory createConnectionFactory()
+                throws IOException;
+    }
+
+    /**
+     * The server delegates loading of the tree to an instance of the interface
+     */
+    public interface DataTreeBuilder {
+        public DataTree build();
+    }
+
+    static public class BasicDataTreeBuilder implements DataTreeBuilder {
+        public DataTree build() {
+            return new DataTree();
+        }
+    }
+
+    private static final int DEFAULT_TICK_TIME = 3000;
+    protected int tickTime = DEFAULT_TICK_TIME;
+
+    public static final int commitLogCount = 500;
+    public int commitLogBuffer = 700;
+    public LinkedList<Proposal> committedLog = new LinkedList<Proposal>();
+    public long minCommittedLog, maxCommittedLog;
+    private DataTreeBuilder treeBuilder;
+    public DataTree dataTree;
+    protected SessionTracker sessionTracker;
+    /**
+     * directory for storing the snapshot
+     */
+    File dataDir;
+    /**
+     * directoy for storing the log tnxns
+     */
+    File dataLogDir;
+    protected ConcurrentHashMap<Long, Integer> sessionsWithTimeouts;
+    protected long hzxid = 0;
+    final public static Exception ok = new Exception("No prob");
+    protected RequestProcessor firstProcessor;
+    LinkedBlockingQueue<Long> sessionsToDie = new LinkedBlockingQueue<Long>();
+    protected boolean running;
+    /**
+     * This is the secret that we use to generate passwords, for the moment it
+     * is more of a sanity check.
+     */
+    final private long superSecret = 0XB3415C00L;
+    int requestsInProcess;
+    ArrayList<ChangeRecord> outstandingChanges = new ArrayList<ChangeRecord>();
+    private NIOServerCnxn.Factory serverCnxnFactory;
+
+    /*
+     * Start up the ZooKeeper server.
+     *
+     * @param args the port and data directory
+     */
+    public static void main(String[] args) {
+        ServerConfig.parse(args);
+        runStandalone(new Factory() {
+            public NIOServerCnxn.Factory createConnectionFactory()
+                    throws IOException {
+                return new NIOServerCnxn.Factory(ServerConfig.getClientPort());
+            }
+
+            public ZooKeeperServer createServer() throws IOException {
+                return new ZooKeeperServer(new BasicDataTreeBuilder());
+            }
+        });
+    }
+
+    public static void runStandalone(Factory factory) {
+        try {
+            // Note that this thread isn't going to be doing anything else,
+            // so rather than spawning another thread, we will just call
+            // run() in this thread.
+            ServerStats.registerAsConcrete();
+            ZooKeeperServer zk = factory.createServer();
+            zk.startup();
+            NIOServerCnxn.Factory t = factory.createConnectionFactory();
+            t.setZooKeeperServer(zk);
+            t.join();
+            if (zk.isRunning())
+                zk.shutdown();
+        } catch (Exception e) {
+            LOG.fatal("Unexpected exception",e);
+        }
+        System.exit(0);
+    }
+
+    void removeCnxn(ServerCnxn cnxn) {
+        dataTree.removeCnxn(cnxn);
+    }
+
+    /**
+     * Creates a ZooKeeperServer instance. It sets everything up, but doesn't
+     * actually start listening for clients until run() is invoked.
+     *
+     * @param dataDir
+     *            the directory to put the data
+     * @throws IOException
+     */
+    public ZooKeeperServer(File dataDir, File dataLogDir, int tickTime,
+            DataTreeBuilder treeBuilder) throws IOException {
+        this.treeBuilder = treeBuilder;
+        this.dataDir = dataDir;
+        this.dataLogDir = dataLogDir;
+        this.tickTime = tickTime;
+        if (!dataDir.isDirectory()) {
+            throw new IOException("data directory does not exist");
+        }
+        ServerStats.getInstance().setStatsProvider(this);
+    }
+
+    /**
+     * This constructor is for backward comaptibility with the existing unit
+     * test code.
+     */
+    public ZooKeeperServer(File dataDir, File dataLogDir, int tickTime)
+            throws IOException {
+        this.treeBuilder = new BasicDataTreeBuilder();
+        this.dataDir = dataDir;
+        this.dataLogDir = dataLogDir;
+        this.tickTime = tickTime;
+        if (!dataDir.isDirectory()) {
+            throw new IOException("data directory does not exist");
+        }
+        ServerStats.getInstance().setStatsProvider(this);
+    }
+
+    /**
+     * Default constructor, relies on the config for its agrument values
+     *
+     * @throws IOException
+     */
+    public ZooKeeperServer(DataTreeBuilder treeBuilder) throws IOException {
+        this(new File(ServerConfig.getDataDir()), new File(ServerConfig
+                .getDataLogDir()), DEFAULT_TICK_TIME, treeBuilder);
+    }
+
+    public static long getZxidFromName(String name, String prefix) {
+        long zxid = -1;
+        String nameParts[] = name.split("\\.");
+        if (nameParts.length == 2 && nameParts[0].equals(prefix)) {
+            try {
+                zxid = Long.parseLong(nameParts[1], 16);
+            } catch (NumberFormatException e) {
+            }
+        }
+        return zxid;
+    }
+
+    static public long isValidSnapshot(File f) throws IOException {
+        long zxid = getZxidFromName(f.getName(), "snapshot");
+        if (zxid == -1)
+            return -1;
+
+        // Check for a valid snapshot
+        RandomAccessFile raf = new RandomAccessFile(f, "r");
+        try {
+            raf.seek(raf.length() - 5);
+            byte bytes[] = new byte[5];
+            raf.read(bytes);
+            ByteBuffer bb = ByteBuffer.wrap(bytes);
+            int len = bb.getInt();
+            byte b = bb.get();
+            if (len != 1 || b != '/') {
+                LOG.warn("Invalid snapshot " + f + " len = " + len
+                        + " byte = " + (b & 0xff));
+                return -1;
+            }
+        } finally {
+            raf.close();
+        }
+
+        return zxid;
+    }
+
+    /**
+     * Compare file file names of form "prefix.version". Sort order result
+     * returned in order of version.
+     */
+    private static class DataDirFileComparator implements Comparator<File> {
+        private String prefix;
+        private boolean ascending;
+        public DataDirFileComparator(String prefix, boolean ascending) {
+            this.prefix = prefix;
+            this.ascending = ascending;
+        }
+
+        public int compare(File o1, File o2) {
+            long z1 = getZxidFromName(o1.getName(), prefix);
+            long z2 = getZxidFromName(o2.getName(), prefix);
+            int result = z1 < z2 ? -1 : (z1 > z2 ? 1 : 0);
+            return ascending ? result : -result;
+        }
+    }
+
+    /**
+     * Sort the list of files. Recency as determined by the version component
+     * of the file name.
+     *
+     * @param files array of files
+     * @param prefix files not matching this prefix are assumed to have a
+     * version = -1)
+     * @param ascending true sorted in ascending order, false results in
+     * descending order
+     * @return sorted input files
+     */
+    static List<File>
+        sortDataDir(File[] files, String prefix, boolean ascending)
+    {
+        List<File> filelist = Arrays.asList(files);
+        Collections.sort(filelist, new DataDirFileComparator(prefix, ascending));
+        return filelist;
+    }
+
+    /**
+     * Find the log file that starts at, or just before, the snapshot. Return
+     * this and all subsequent logs. Results are ordered by zxid of file,
+     * ascending order.
+     *
+     * @param logDirList array of files
+     * @param snapshotZxid return files at, or before this zxid
+     * @return
+     */
+    static File[] getLogFiles(File[] logDirList,long snapshotZxid) {
+        List<File> files = sortDataDir(logDirList, "log", true);
+        long logZxid = 0;
+        // Find the log file that starts before or at the same time as the
+        // zxid of the snapshot
+        for (File f : files) {
+            long fzxid = getZxidFromName(f.getName(), "log");
+            if (fzxid > snapshotZxid) {
+                continue;
+            }
+            if (fzxid > logZxid) {
+                logZxid = fzxid;
+            }
+        }
+        List<File> v=new ArrayList<File>(5);
+        // Apply the logs
+        for (File f : files) {
+            long fzxid = getZxidFromName(f.getName(), "log");
+            if (fzxid < logZxid) {
+                continue;
+            }
+            v.add(f);
+        }
+        return v.toArray(new File[0]);
+    }
+
+    /**
+     *  Restore sessions and data
+     */
+    private void loadSnapshotAndLogs() throws IOException {
+        long zxid = -1;
+
+        // Find the most recent snapshot
+        List<File> files = sortDataDir(dataDir.listFiles(), "snapshot", false);
+        for (File f : files) {
+            zxid = isValidSnapshot(f);
+            if (zxid == -1) {
+                LOG.warn("Skipping " + f);
+                continue;
+            }
+
+            LOG.warn("Processing snapshot: " + f);
+
+            InputStream snapIS =
+                new BufferedInputStream(new FileInputStream(f));
+            loadData(BinaryInputArchive.getArchive(snapIS));
+            snapIS.close();
+
+            dataTree.lastProcessedZxid = zxid;
+
+            // Apply the logs on/after the selected snapshot
+            File[] logfiles = getLogFiles(dataLogDir.listFiles(), zxid);
+            for (File logfile : logfiles) {
+                LOG.warn("Processing log file: " + logfile);
+
+                InputStream logIS =
+                    new BufferedInputStream(new FileInputStream(logfile));
+                zxid = playLog(BinaryInputArchive.getArchive(logIS));
+                logIS.close();
+            }
+            hzxid = zxid;
+
+            break;
+        }
+
+        if (zxid == -1) {
+            sessionsWithTimeouts = new ConcurrentHashMap<Long, Integer>();
+            dataTree = treeBuilder.build();
+        }
+    }
+
+    public void loadData() throws IOException, InterruptedException {
+        loadSnapshotAndLogs();
+
+        // Clean up dead sessions
+        LinkedList<Long> deadSessions = new LinkedList<Long>();
+        for (long session : dataTree.getSessions()) {
+            if (sessionsWithTimeouts.get(session) == null) {
+                deadSessions.add(session);
+            }
+        }
+        dataTree.initialized = true;
+        for (long session : deadSessions) {
+            killSession(session);
+        }
+        // Make a clean snapshot
+        snapshot();
+    }
+
+    public void loadData(InputArchive ia) throws IOException {
+        sessionsWithTimeouts = new ConcurrentHashMap<Long, Integer>();
+        dataTree = treeBuilder.build();
+
+        int count = ia.readInt("count");
+        while (count > 0) {
+            long id = ia.readLong("id");
+            int to = ia.readInt("timeout");
+            sessionsWithTimeouts.put(id, to);
+            ZooTrace.logTraceMessage(LOG, ZooTrace.SESSION_TRACE_MASK,
+                                     "loadData --- session in archive: " + id
+                                     + " with timeout: " + to);
+            count--;
+        }
+        dataTree.deserialize(ia, "tree");
+    }
+
+    public long playLog(InputArchive logStream) throws IOException {
+        long highestZxid = 0;
+        try {
+            while (true) {
+                byte[] bytes = logStream.readBuffer("txnEntry");
+                if (bytes.length == 0) {
+                    // Since we preallocate, we define EOF to be an
+                    // empty transaction
+                    throw new EOFException();
+                }
+                InputArchive ia = BinaryInputArchive
+                        .getArchive(new ByteArrayInputStream(bytes));
+                TxnHeader hdr = new TxnHeader();
+                Record txn = deserializeTxn(ia, hdr);
+                if (logStream.readByte("EOR") != 'B') {
+                    LOG.error("Last transaction was partial.");
+                    throw new EOFException();
+                }
+                if (hdr.getZxid() <= highestZxid && highestZxid != 0) {
+                    LOG.error(highestZxid + "(higestZxid) >= "
+                            + hdr.getZxid() + "(next log) for type "
+                            + hdr.getType());
+                } else {
+                    highestZxid = hdr.getZxid();
+                }
+                switch (hdr.getType()) {
+                case OpCode.createSession:
+                    sessionsWithTimeouts.put(hdr.getClientId(),
+                            ((CreateSessionTxn) txn).getTimeOut());
+                    ZooTrace.logTraceMessage(LOG,
+                                             ZooTrace.SESSION_TRACE_MASK,
+                            "playLog --- create session in log: "
+                                    + Long.toHexString(hdr.getClientId())
+                                    + " with timeout: "
+                                    + ((CreateSessionTxn) txn).getTimeOut());
+                    // give dataTree a chance to sync its lastProcessedZxid
+                    dataTree.processTxn(hdr, txn);
+                    break;
+                case OpCode.closeSession:
+                    sessionsWithTimeouts.remove(hdr.getClientId());
+                    ZooTrace.logTraceMessage(LOG,
+                            ZooTrace.SESSION_TRACE_MASK,
+                            "playLog --- close session in log: "
+                                    + Long.toHexString(hdr.getClientId()));
+                    dataTree.processTxn(hdr, txn);
+                    break;
+                default:
+                    dataTree.processTxn(hdr, txn);
+                }
+                Request r = new Request(null, 0, hdr.getCxid(), hdr.getType(),
+                        null, null);
+                r.txn = txn;
+                r.hdr = hdr;
+                r.zxid = hdr.getZxid();
+                addCommittedProposal(r);
+            }
+        } catch (EOFException e) {
+        }
+        return highestZxid;
+    }
+
+    /**
+     * maintains a list of last 500 or so committed requests. This is used for
+     * fast follower synchronization.
+     *
+     * @param request
+     *            committed request
+     */
+
+    public void addCommittedProposal(Request request) {
+        synchronized (committedLog) {
+            if (committedLog.size() > commitLogCount) {
+                committedLog.removeFirst();
+                minCommittedLog = committedLog.getFirst().packet.getZxid();
+            }
+            if (committedLog.size() == 0) {
+                minCommittedLog = request.zxid;
+                maxCommittedLog = request.zxid;
+            }
+
+            ByteArrayOutputStream baos = new ByteArrayOutputStream();
+            BinaryOutputArchive boa = BinaryOutputArchive.getArchive(baos);
+            try {
+                request.hdr.serialize(boa, "hdr");
+                if (request.txn != null) {
+                    request.txn.serialize(boa, "txn");
+                }
+                baos.close();
+            } catch (IOException e) {
+                // This really should be impossible
+                LOG.error("FIXMSG",e);
+            }
+            QuorumPacket pp = new QuorumPacket(Leader.PROPOSAL, request.zxid,
+                    baos.toByteArray(), null);
+            Proposal p = new Proposal();
+            p.packet = pp;
+            p.request = request;
+            committedLog.add(p);
+            maxCommittedLog = p.packet.getZxid();
+        }
+    }
+
+    static public Record deserializeTxn(InputArchive ia, TxnHeader hdr)
+            throws IOException {
+        hdr.deserialize(ia, "hdr");
+        Record txn = null;
+        switch (hdr.getType()) {
+        case OpCode.createSession:
+            // This isn't really an error txn; it just has the same
+            // format. The error represents the timeout
+            txn = new CreateSessionTxn();
+            break;
+        case OpCode.closeSession:
+            return null;
+        case OpCode.create:
+            txn = new CreateTxn();
+            break;
+        case OpCode.delete:
+            txn = new DeleteTxn();
+            break;
+        case OpCode.setData:
+            txn = new SetDataTxn();
+            break;
+        case OpCode.setACL:
+            txn = new SetACLTxn();
+            break;
+        case OpCode.error:
+            txn = new ErrorTxn();
+            break;
+        }
+        if (txn != null) {
+            txn.deserialize(ia, "txn");
+        }
+        return txn;
+    }
+
+    public void truncateLog(long finalZxid) throws IOException {
+        long highestZxid = 0;
+        for (File f : dataDir.listFiles()) {
+            long zxid = isValidSnapshot(f);
+            if (zxid == -1) {
+                LOG.warn("Skipping " + f);
+                continue;
+            }
+            if (zxid > highestZxid) {
+                highestZxid = zxid;
+            }
+        }
+        File[] files = getLogFiles(dataLogDir.listFiles(), highestZxid);
+        boolean truncated = false;
+        for (File f : files) {
+            FileInputStream fin = new FileInputStream(f);
+            InputArchive ia = BinaryInputArchive.getArchive(fin);
+            FileChannel fchan = fin.getChannel();
+            try {
+                while (true) {
+                    byte[] bytes = ia.readBuffer("txtEntry");
+                    if (bytes.length == 0) {
+                        throw new EOFException();
+                    }
+                    InputArchive iab = BinaryInputArchive
+                            .getArchive(new ByteArrayInputStream(bytes));
+                    TxnHeader hdr = new TxnHeader();
+                    deserializeTxn(iab, hdr);
+                    if (ia.readByte("EOF") != 'B') {
+                        throw new EOFException();
+                    }
+                    if (hdr.getZxid() == finalZxid) {
+                        // this is where we need to truncate
+
+                        long pos = fchan.position();
+                        fin.close();
+                        FileOutputStream fout = new FileOutputStream(f);
+                        FileChannel fchanOut = fout.getChannel();
+                        fchanOut.truncate(pos);
+                        truncated = true;
+                        break;
+                    }
+                }
+            } catch (EOFException eof) {
+            }
+            if (truncated == true) {
+                break;
+            }
+        }
+        if (truncated == false) {
+            // not able to truncate the log
+            LOG.error("Not able to truncate the log "
+                    + Long.toHexString(finalZxid));
+            System.exit(13);
+        }
+
+    }
+
+    public void snapshot(BinaryOutputArchive oa) throws IOException,
+            InterruptedException {
+        HashMap<Long, Integer> sessSnap = new HashMap<Long, Integer>(
+                sessionsWithTimeouts);
+        oa.writeInt(sessSnap.size(), "count");
+        for (Entry<Long, Integer> entry : sessSnap.entrySet()) {
+            oa.writeLong(entry.getKey().longValue(), "id");
+            oa.writeInt(entry.getValue().intValue(), "timeout");
+        }
+        dataTree.serialize(oa, "tree");
+    }
+
+    public void snapshot() throws InterruptedException {
+        long lastZxid = dataTree.lastProcessedZxid;
+        ZooTrace.logTraceMessage(LOG, ZooTrace.getTextTraceLevel(),
+                "Snapshotting: " + Long.toHexString(lastZxid));
+        try {
+            File f =new File(dataDir, "snapshot." + Long.toHexString(lastZxid));
+            OutputStream sessOS = new BufferedOutputStream(new FileOutputStream(f));
+            BinaryOutputArchive oa = BinaryOutputArchive.getArchive(sessOS);
+            snapshot(oa);
+            sessOS.flush();
+            sessOS.close();
+            ZooTrace.logTraceMessage(LOG, ZooTrace.getTextTraceLevel(),
+                    "Snapshotting finished: " + Long.toHexString(lastZxid));
+        } catch (IOException e) {
+            LOG.error("Severe error, exiting",e);
+            // This is a severe error that we cannot recover from,
+            // so we need to exit
+            System.exit(10);
+        }
+    }
+
+    /**
+     * This should be called from a synchronized block on this!
+     */
+    public long getZxid() {
+        return hzxid;
+    }
+
+    synchronized long getNextZxid() {
+        return ++hzxid;
+    }
+
+    long getTime() {
+        return System.currentTimeMillis();
+    }
+
+    static String getLogName(long zxid) {
+        return "log." + Long.toHexString(zxid);
+    }
+
+    public void closeSession(long sessionId) throws InterruptedException {
+        ZooTrace.logTraceMessage(LOG, ZooTrace.SESSION_TRACE_MASK,
+                                 "ZooKeeperServer --- Session to be closed: "
+                + Long.toHexString(sessionId));
+        // we do not want to wait for a session close. send it as soon as we
+        // detect it!
+        submitRequest(null, sessionId, OpCode.closeSession, 0, null, null);
+    }
+
+    protected void killSession(long sessionId) {
+        dataTree.killSession(sessionId);
+        ZooTrace.logTraceMessage(LOG, ZooTrace.SESSION_TRACE_MASK,
+                                     "ZooKeeperServer --- killSession: "
+                + Long.toHexString(sessionId));
+        if (sessionTracker != null) {
+            sessionTracker.removeSession(sessionId);
+        }
+    }
+
+    public void expire(long sessionId) {
+        try {
+            ZooTrace.logTraceMessage(LOG,
+                                     ZooTrace.SESSION_TRACE_MASK,
+                    "ZooKeeperServer --- Session to expire: " + Long.toHexString(sessionId));
+            closeSession(sessionId);
+        } catch (Exception e) {
+            LOG.error("FIXMSG",e);
+        }
+    }
+
+    void touch(ServerCnxn cnxn) throws IOException {
+        if (cnxn == null) {
+            return;
+        }
+        long id = cnxn.getSessionId();
+        int to = cnxn.getSessionTimeout();
+        if (!sessionTracker.touchSession(id, to)) {
+            throw new IOException("Missing session " + Long.toHexString(id));
+        }
+    }
+
+    public void startup() throws IOException, InterruptedException {
+        if (dataTree == null) {
+            loadData();
+        }
+        createSessionTracker();
+        setupRequestProcessors();
+        running = true;
+        synchronized (this) {
+            notifyAll();
+        }
+    }
+
+    protected void setupRequestProcessors() {
+        RequestProcessor finalProcessor = new FinalRequestProcessor(this);
+        RequestProcessor syncProcessor = new SyncRequestProcessor(this,
+                finalProcessor);
+        firstProcessor = new PrepRequestProcessor(this, syncProcessor);
+    }
+
+    protected void createSessionTracker() {
+        sessionTracker = new SessionTrackerImpl(this, sessionsWithTimeouts,
+                tickTime, 1);
+    }
+
+    public boolean isRunning() {
+        return running;
+    }
+
+    public void shutdown() {
+        // new RuntimeException("Calling shutdown").printStackTrace();
+        this.running = false;
+        // Since sessionTracker and syncThreads poll we just have to
+        // set running to false and they will detect it during the poll
+        // interval.
+        if (sessionTracker != null) {
+            sessionTracker.shutdown();
+        }
+        if (firstProcessor != null) {
+            firstProcessor.shutdown();
+        }
+        if (dataTree != null) {
+            dataTree.clear();
+        }
+    }
+
+    synchronized public void incInProcess() {
+        requestsInProcess++;
+    }
+
+    synchronized public void decInProcess() {
+        requestsInProcess--;
+    }
+
+    public int getInProcess() {
+        return requestsInProcess;
+    }
+
+    /**
+     * This structure is used to facilitate information sharing between PrepRP
+     * and FinalRP.
+     */
+    static class ChangeRecord {
+        ChangeRecord(long zxid, String path, Stat stat, int childCount,
+                ArrayList<ACL> acl) {
+            this.zxid = zxid;
+            this.path = path;
+            this.stat = stat;
+            this.childCount = childCount;
+            this.acl = acl;
+        }
+
+        long zxid;
+
+        String path;
+
+        Stat stat; /* Make sure to create a new object when changing */
+
+        int childCount;
+
+        ArrayList<ACL> acl; /* Make sure to create a new object when changing */
+
+        @SuppressWarnings("unchecked")
+        ChangeRecord duplicate(long zxid) {
+            Stat stat = new Stat();
+            if (this.stat != null) {
+                DataTree.copyStat(this.stat, stat);
+            }
+            return new ChangeRecord(zxid, path, stat, childCount,
+                    acl == null ? new ArrayList<ACL>() : (ArrayList<ACL>) acl
+                            .clone());
+        }
+    }
+
+    byte[] generatePasswd(long id) {
+        Random r = new Random(id ^ superSecret);
+        byte p[] = new byte[16];
+        r.nextBytes(p);
+        return p;
+    }
+
+    protected boolean checkPasswd(long sessionId, byte[] passwd) {
+        return sessionId != 0
+                && Arrays.equals(passwd, generatePasswd(sessionId));
+    }
+
+    long createSession(ServerCnxn cnxn, byte passwd[], int timeout)
+            throws InterruptedException {
+        long sessionId = sessionTracker.createSession(timeout);
+        Random r = new Random(sessionId ^ superSecret);
+        r.nextBytes(passwd);
+        ByteBuffer to = ByteBuffer.allocate(4);
+        to.putInt(timeout);
+        cnxn.setSessionId(sessionId);
+        submitRequest(cnxn, sessionId, OpCode.createSession, 0, to, null);
+        return sessionId;
+    }
+
+    protected void revalidateSession(ServerCnxn cnxn, long sessionId,
+            int sessionTimeout) throws IOException, InterruptedException {
+        boolean rc = sessionTracker.touchSession(sessionId, sessionTimeout);
+        ZooTrace.logTraceMessage(LOG,ZooTrace.SESSION_TRACE_MASK,
+                                 "Session " + Long.toHexString(sessionId) +
+                " is valid: " + rc);
+        cnxn.finishSessionInit(rc);
+    }
+
+    public void reopenSession(ServerCnxn cnxn, long sessionId, byte[] passwd,
+            int sessionTimeout) throws IOException, InterruptedException {
+        if (!checkPasswd(sessionId, passwd)) {
+            cnxn.finishSessionInit(false);
+        } else {
+            revalidateSession(cnxn, sessionId, sessionTimeout);
+        }
+    }
+
+    public void closeSession(ServerCnxn cnxn, RequestHeader requestHeader)
+            throws InterruptedException {
+        closeSession(cnxn.getSessionId());
+    }
+
+    public long getServerId() {
+        return 0;
+    }
+
+    /**
+     * @param cnxn
+     * @param sessionId
+     * @param xid
+     * @param bb
+     */
+    public void submitRequest(ServerCnxn cnxn, long sessionId, int type,
+            int xid, ByteBuffer bb, ArrayList<Id> authInfo) {
+        if (firstProcessor == null) {
+            synchronized (this) {
+                try {
+                    while (!running) {
+                        wait(1000);
+                    }
+                } catch (InterruptedException e) {
+                    LOG.error("FIXMSG",e);
+                }
+                if (firstProcessor == null) {
+                    throw new RuntimeException("Not started");
+                }
+            }
+        }
+        try {
+            touch(cnxn);
+            Request si = new Request(cnxn, sessionId, xid, type, bb, authInfo);
+            boolean validpacket = Request.isValid(type);
+            if (validpacket) {
+                firstProcessor.processRequest(si);
+                if (cnxn != null) {
+                    incInProcess();
+                }
+            } else {
+                LOG.warn("Dropping packet at server of type " + type);
+                // if unvalid packet drop the packet.
+            }
+        } catch (IOException e) {
+            LOG.error("FIXMSG",e);
+        }
+    }
+
+    static public void byteBuffer2Record(ByteBuffer bb, Record record)
+            throws IOException {
+        BinaryInputArchive ia;
+        ia = BinaryInputArchive.getArchive(new ByteBufferInputStream(bb));
+        record.deserialize(ia, "request");
+    }
+
+    public static int getSnapCount() {
+        String sc = System.getProperty("zookeeper.snapCount");
+        try {
+            return Integer.parseInt(sc);
+        } catch (Exception e) {
+            return 10000;
+        }
+    }
+
+    public int getGlobalOutstandingLimit() {
+        String sc = System.getProperty("zookeeper.globalOutstandingLimit");
+        int limit;
+        try {
+            limit = Integer.parseInt(sc);
+        } catch (Exception e) {
+            limit = 1000;
+        }
+        return limit;
+    }
+
+    public void setServerCnxnFactory(NIOServerCnxn.Factory factory) {
+        serverCnxnFactory = factory;
+    }
+
+    public NIOServerCnxn.Factory getServerCnxnFactory() {
+        return serverCnxnFactory;
+    }
+
+    public long getLastProcessedZxid() {
+        return dataTree.lastProcessedZxid;
+    }
+
+    public long getOutstandingRequests() {
+        return getInProcess();
+    }
+}

+ 1 - 1
zookeeper/java/src/com/yahoo/zookeeper/server/auth/HostAuthenticationProvider.java

@@ -16,9 +16,9 @@
 
 package com.yahoo.zookeeper.server.auth;
 
-import com.yahoo.zookeeper.KeeperException;
 import com.yahoo.zookeeper.data.Id;
 import com.yahoo.zookeeper.server.ServerCnxn;
+import com.yahoo.zookeeper.KeeperException;
 
 public class HostAuthenticationProvider implements AuthenticationProvider {
 

+ 1 - 1
zookeeper/java/src/com/yahoo/zookeeper/server/auth/IPAuthenticationProvider.java

@@ -16,9 +16,9 @@
 
 package com.yahoo.zookeeper.server.auth;
 
-import com.yahoo.zookeeper.KeeperException;
 import com.yahoo.zookeeper.data.Id;
 import com.yahoo.zookeeper.server.ServerCnxn;
+import com.yahoo.zookeeper.KeeperException;
 
 public class IPAuthenticationProvider implements AuthenticationProvider {
 

+ 1 - 2
zookeeper/java/src/com/yahoo/zookeeper/server/quorum/FollowerSessionTracker.java

@@ -19,7 +19,6 @@ package com.yahoo.zookeeper.server.quorum;
 import java.util.HashMap;
 import java.util.concurrent.ConcurrentHashMap;
 
-import com.yahoo.zookeeper.KeeperException;
 import com.yahoo.zookeeper.server.SessionTracker;
 import com.yahoo.zookeeper.server.SessionTrackerImpl;
 
@@ -78,7 +77,7 @@ public class FollowerSessionTracker implements SessionTracker {
         return (nextSessionId++);
     }
 
-    public void checkSession(long sessionId) throws KeeperException {
+    public void checkSession(long sessionId)  {
         // Nothing to do here. Sessions are checked at the Leader
     }
 }

+ 17 - 18
zookeeper/test/com/yahoo/zookeeper/server/DeserializationPerfTest.java

@@ -12,8 +12,7 @@ import com.yahoo.zookeeper.KeeperException;
 
 public class DeserializationPerfTest extends TestCase {
     private static void deserializeTree(int depth, int width, int len)
-            throws KeeperException, InterruptedException, IOException 
-    {
+            throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         BinaryInputArchive ia;
         int count;
         {
@@ -46,43 +45,43 @@ public class DeserializationPerfTest extends TestCase {
                 + width + " datalen=" + len);
     }
 
-    public void testSingleDeserialize() throws KeeperException,
-            InterruptedException, IOException {
+    public void testSingleDeserialize() throws
+            InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         deserializeTree(1, 0, 20);
     }
 
-    public void testWideDeserialize() throws KeeperException,
-            InterruptedException, IOException {
+    public void testWideDeserialize() throws
+            InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         deserializeTree(2, 50000, 20);
     }
 
-    public void testDeepDeserialize() throws KeeperException,
-            InterruptedException, IOException {
+    public void testDeepDeserialize() throws
+            InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         deserializeTree(1000, 1, 20);
     }
 
-    public void test10Wide5DeepDeserialize() throws KeeperException,
-            InterruptedException, IOException {
+    public void test10Wide5DeepDeserialize() throws
+            InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         deserializeTree(5, 10, 20);
     }
 
-    public void test15Wide5DeepDeserialize() throws KeeperException,
-            InterruptedException, IOException {
+    public void test15Wide5DeepDeserialize() throws
+            InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         deserializeTree(5, 15, 20);
     }
 
-    public void test25Wide4DeepDeserialize() throws KeeperException,
-            InterruptedException, IOException {
+    public void test25Wide4DeepDeserialize() throws
+            InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         deserializeTree(4, 25, 20);
     }
 
-    public void test40Wide4DeepDeserialize() throws KeeperException,
-            InterruptedException, IOException {
+    public void test40Wide4DeepDeserialize() throws
+            InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         deserializeTree(4, 40, 20);
     }
 
-    public void test300Wide3DeepDeserialize() throws KeeperException,
-            InterruptedException, IOException {
+    public void test300Wide3DeepDeserialize() throws
+            InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         deserializeTree(3, 300, 20);
     }
 

+ 19 - 30
zookeeper/test/com/yahoo/zookeeper/server/SerializationPerfTest.java

@@ -15,9 +15,7 @@ public class SerializationPerfTest extends TestCase {
     }
 
     static int createNodes(DataTree tree, String path, int depth,
-            int childcount, byte[] data)
-        throws KeeperException 
-    {
+            int childcount, byte[] data) throws KeeperException.NodeExistsException, KeeperException.NoNodeException {
         path += "node" + depth;
         tree.createNode(path, data, null, -1, 1, 1);
 
@@ -35,9 +33,8 @@ public class SerializationPerfTest extends TestCase {
         return count;
     }
 
-    private static void serializeTree(int depth, int width, int len) 
-        throws KeeperException, InterruptedException, IOException
-    {
+    private static void serializeTree(int depth, int width, int len)
+            throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         DataTree tree = new DataTree();
         createNodes(tree, "/", depth, width, new byte[len]);
         int count = tree.getNodeCount();
@@ -55,51 +52,43 @@ public class SerializationPerfTest extends TestCase {
                 + depth + " width=" + width + " datalen=" + len);
     }
 
-    public void testSingleSerialize() 
-        throws KeeperException, InterruptedException, IOException
-    {
+    public void testSingleSerialize()
+            throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         serializeTree(1, 0, 20);
     }
 
-    public void testWideSerialize() 
-        throws KeeperException, InterruptedException, IOException
-    {
+    public void testWideSerialize()
+            throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         serializeTree(2, 50000, 20);
     }
 
-    public void testDeepSerialize() 
-        throws KeeperException, InterruptedException, IOException
-    {
+    public void testDeepSerialize()
+            throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         serializeTree(1000, 1, 20);
     }
     
-    public void test10Wide5DeepSerialize() 
-        throws KeeperException, InterruptedException, IOException
-    {
+    public void test10Wide5DeepSerialize()
+            throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         serializeTree(5, 10, 20);
     }
 
-    public void test15Wide5DeepSerialize() 
-        throws KeeperException, InterruptedException, IOException
-    {
+    public void test15Wide5DeepSerialize()
+            throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         serializeTree(5, 15, 20);
     }
 
-    public void test25Wide4DeepSerialize() 
-        throws KeeperException, InterruptedException, IOException
-    {
+    public void test25Wide4DeepSerialize()
+            throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         serializeTree(4, 25, 20);
     }
 
-    public void test40Wide4DeepSerialize() 
-        throws KeeperException, InterruptedException, IOException
-    {
+    public void test40Wide4DeepSerialize()
+            throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         serializeTree(4, 40, 20);
     }
 
-    public void test300Wide3DeepSerialize() 
-        throws KeeperException, InterruptedException, IOException
-    {
+    public void test300Wide3DeepSerialize()
+            throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         serializeTree(3, 300, 20);
     }
 }

+ 3 - 3
zookeeper/test/com/yahoo/zookeeper/test/AsyncTest.java

@@ -59,7 +59,7 @@ public class AsyncTest extends TestCase implements Watcher, StringCallback, Void
         LOG.error("Client test shutdown finished");
     }
     
-    private ZooKeeper createClient() throws KeeperException, IOException,InterruptedException{
+    private ZooKeeper createClient() throws IOException,InterruptedException{
         clientConnected=new CountDownLatch(1);
 		ZooKeeper zk = new ZooKeeper(hostPort, 30000, this);
 		if(!clientConnected.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS)){
@@ -140,8 +140,8 @@ public class AsyncTest extends TestCase implements Watcher, StringCallback, Void
     
     LinkedList<Integer> results = new LinkedList<Integer>();
     @Test
-    public void testAsync() throws KeeperException, IOException,
-			InterruptedException {
+    public void testAsync() throws IOException,
+            InterruptedException, KeeperException {
 		ZooKeeper zk = null;
 		zk = createClient();
 		zk.addAuthInfo("digest", "ben:passwd".getBytes());

+ 21 - 23
zookeeper/test/com/yahoo/zookeeper/test/ClientTest.java

@@ -17,7 +17,6 @@ import org.junit.Test;
 import com.yahoo.zookeeper.KeeperException;
 import com.yahoo.zookeeper.Watcher;
 import com.yahoo.zookeeper.ZooKeeper;
-import com.yahoo.zookeeper.KeeperException.Code;
 import com.yahoo.zookeeper.ZooDefs.CreateFlags;
 import com.yahoo.zookeeper.ZooDefs.Ids;
 import com.yahoo.zookeeper.data.Stat;
@@ -74,7 +73,7 @@ public class ClientTest extends TestCase implements Watcher {
         d.delete();
     }
 
-    private ZooKeeper createClient() throws KeeperException, IOException,InterruptedException{
+    private ZooKeeper createClient() throws IOException,InterruptedException{
         clientConnected=new CountDownLatch(1);
 		ZooKeeper zk = new ZooKeeper(hostPort, 20000, this);
 		if(!clientConnected.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS)){
@@ -90,15 +89,15 @@ public class ClientTest extends TestCase implements Watcher {
         try {
             zkIdle = createClient();
             zkWatchCreator = createClient();
-            for(int i = 0 ; i < 30; i++) {
+            for (int i = 0; i < 30; i++) {
                 zkWatchCreator.create("/" + i, new byte[0], Ids.OPEN_ACL_UNSAFE, 0);
             }
-            for(int i = 0 ; i < 30; i++) {
+            for (int i = 0; i < 30; i++) {
                 zkIdle.exists("/" + i, true);
             }
-            for(int i = 0; i < 30; i++) {
+            for (int i = 0; i < 30; i++) {
                 Thread.sleep(1000);
-                zkWatchCreator.delete("/"+i, -1);
+                zkWatchCreator.delete("/" + i, -1);
             }
             // The bug will manifest itself here because zkIdle will expire
             zkIdle.exists("/0", false);
@@ -113,8 +112,8 @@ public class ClientTest extends TestCase implements Watcher {
     }
 
     @Test
-    public void testClient() throws KeeperException, IOException,
-            InterruptedException {
+    public void testClient() throws IOException,
+            InterruptedException, KeeperException {
         ZooKeeper zk = null;
         try {
     		zk =createClient();
@@ -125,10 +124,10 @@ public class ClientTest extends TestCase implements Watcher {
             try {
             	zk.setData("/benwashere", "hi".getBytes(), 57);
         		fail("Should have gotten BadVersion exception");
-            } catch(KeeperException e) {
-            	if (e.getCode() != Code.BadVersion) {
-            		fail("Should have gotten BadVersion exception");
-            	}
+            } catch(KeeperException.BadVersionException e) {
+                // expected that
+            } catch (KeeperException e) {
+                fail("Should have gotten BadVersion exception");
             }
             System.out.println("Before delete /benwashere");
             zk.delete("/benwashere", 0);
@@ -143,8 +142,8 @@ public class ClientTest extends TestCase implements Watcher {
             try {
                 zk.delete("/", -1);
                 fail("deleted root!");
-            } catch(KeeperException e) {
-                assertEquals(KeeperException.Code.BadArguments, e.getCode());
+            } catch(KeeperException.BadArgumentsException e) {
+                // good, expected that
             }
             Stat stat = new Stat();
             // Test basic create, ls, and getData
@@ -160,8 +159,8 @@ public class ClientTest extends TestCase implements Watcher {
             try {
                 assertEquals(null, zk.exists("/frog", true));
                 System.out.println("Comment: asseting passed for frog setting /");
-            } catch (KeeperException e) {
-                assertEquals(Code.NoNode, e.getCode());
+            } catch (KeeperException.NoNodeException e) {
+                // OK, expected that
             }
             zk.create("/frog", "hi".getBytes(), Ids.OPEN_ACL_UNSAFE, 0);
             // the first poll is just a sesssion delivery
@@ -215,8 +214,8 @@ public class ClientTest extends TestCase implements Watcher {
             try {
                 zk.create("/duplicate", "".getBytes(), Ids.OPEN_ACL_UNSAFE, 0);
                 fail("duplicate create allowed");
-            } catch(KeeperException e) {
-                assertEquals(Code.NodeExists, e.getCode());
+            } catch(KeeperException.NodeExistsException e) {
+                // OK, expected that
             }
         } finally {
             if (zk != null) {
@@ -225,7 +224,7 @@ public class ClientTest extends TestCase implements Watcher {
         }
     }
 
-    private void notestConnections() throws KeeperException, IOException, InterruptedException {
+    private void notestConnections() throws IOException, InterruptedException, KeeperException {
         ZooKeeper zk;
         for(int i = 0; i < 2000; i++) {
             if (i % 100 == 0) {
@@ -263,8 +262,7 @@ public class ClientTest extends TestCase implements Watcher {
             }
         }
 
-        public void close() throws IOException, InterruptedException,
-                KeeperException {
+        public void close() throws IOException, InterruptedException {
             zk.close();
         }
     }
@@ -289,8 +287,8 @@ public class ClientTest extends TestCase implements Watcher {
     }
 
     @Test
-    public void testHammer() throws KeeperException, IOException,
-            InterruptedException {
+    public void testHammer() throws IOException,
+            InterruptedException, KeeperException {
         File tmpDir = File.createTempFile("test", ".junit", baseTest);
         tmpDir = new File(tmpDir + ".dir");
         tmpDir.mkdirs();

+ 0 - 3
zookeeper/test/com/yahoo/zookeeper/test/GenerateLoad.java

@@ -329,12 +329,9 @@ public class GenerateLoad {
             if (event.getType() == Watcher.Event.EventNone && event.getState() == Watcher.Event.KeeperStateExpired) {
                 try {
                     zk = new ZooKeeper(host, 10000, this);
-                } catch (KeeperException e) {
-                    e.printStackTrace();
                 } catch (IOException e) {
                     e.printStackTrace();
                 }
-                
             }
         }
 

+ 8 - 12
zookeeper/test/com/yahoo/zookeeper/test/IntegrityCheck.java

@@ -19,10 +19,10 @@ import java.util.HashMap;
 
 import org.apache.log4j.Logger;
 
-import com.yahoo.zookeeper.KeeperException;
 import com.yahoo.zookeeper.Watcher;
 import com.yahoo.zookeeper.ZooDefs;
 import com.yahoo.zookeeper.ZooKeeper;
+import com.yahoo.zookeeper.KeeperException;
 import com.yahoo.zookeeper.AsyncCallback.DataCallback;
 import com.yahoo.zookeeper.AsyncCallback.StatCallback;
 import com.yahoo.zookeeper.data.Stat;
@@ -60,7 +60,7 @@ public class IntegrityCheck implements Watcher, StatCallback, DataCallback {
         }
     }
 
-    IntegrityCheck(String hostPort, String path, int count) throws KeeperException,
+    IntegrityCheck(String hostPort, String path, int count) throws
             IOException {
         zk = new ZooKeeper(hostPort, 15000, this);
         this.path = path;
@@ -94,13 +94,12 @@ public class IntegrityCheck implements Watcher, StatCallback, DataCallback {
 
     }
 
-    void doCreate() throws KeeperException, InterruptedException {
+    void doCreate() throws InterruptedException, KeeperException {
         // create top level znode
         try{
             zk.create(path, null, ZooDefs.Ids.OPEN_ACL_UNSAFE, 0);
-        }catch(KeeperException e){
-            if(e.getCode()!=KeeperException.Code.NodeExists)
-                throw e;
+        }catch(KeeperException.NodeExistsException e){
+            // ignore duplicate create
         }
         iteration++;
         byte v[] = ("" + iteration).getBytes();
@@ -111,10 +110,9 @@ public class IntegrityCheck implements Watcher, StatCallback, DataCallback {
                 if(i%10==0)
                     LOG.warn("Creating znode "+cpath);
                 zk.create(cpath, v, ZooDefs.Ids.OPEN_ACL_UNSAFE, 0);
-            }catch(KeeperException e){
-                if(e.getCode()!=KeeperException.Code.NodeExists)
-                    throw e;
-            }               
+            }catch(KeeperException.NodeExistsException e){
+                // ignore duplicate create
+            }
             lastValue.put(cpath, v);
         }
     }
@@ -185,7 +183,6 @@ public class IntegrityCheck implements Watcher, StatCallback, DataCallback {
         }
     }
 
-    @Override
     public void processResult(int rc, String path, Object ctx, Stat stat) {
         if (rc == 0) {
             lastValue.put(path, (byte[]) ctx);
@@ -193,7 +190,6 @@ public class IntegrityCheck implements Watcher, StatCallback, DataCallback {
         decOutstanding();
     }
 
-    @Override
     public void processResult(int rc, String path, Object ctx, byte[] data,
             Stat stat) {
         if (rc == 0) {

+ 7 - 11
zookeeper/test/com/yahoo/zookeeper/test/OOMTest.java

@@ -9,9 +9,9 @@ import java.util.ArrayList;
 
 import junit.framework.TestCase;
 
-import com.yahoo.zookeeper.KeeperException;
 import com.yahoo.zookeeper.Watcher;
 import com.yahoo.zookeeper.ZooKeeper;
+import com.yahoo.zookeeper.KeeperException;
 import com.yahoo.zookeeper.ZooDefs.Ids;
 import com.yahoo.zookeeper.data.Stat;
 import com.yahoo.zookeeper.proto.WatcherEvent;
@@ -23,8 +23,7 @@ import com.yahoo.zookeeper.server.ZooKeeperServer;
  * 
  */
 public class OOMTest extends TestCase implements Watcher {
-    public void testOOM() throws IOException, InterruptedException,
-            KeeperException {
+    public void testOOM() throws IOException, InterruptedException, KeeperException {
         // This test takes too long to run!
         if (true)
             return;
@@ -79,8 +78,7 @@ public class OOMTest extends TestCase implements Watcher {
         f.shutdown();
     }
 
-    private void utestExists() throws IOException, InterruptedException,
-            KeeperException {
+    private void utestExists() throws IOException, InterruptedException, KeeperException {
         ZooKeeper zk = new ZooKeeper("127.0.0.1:33221", 30000, this);
         for (int i = 0; i < 10000; i++) {
             zk.exists("/this/path/doesnt_exist!", true);
@@ -88,8 +86,8 @@ public class OOMTest extends TestCase implements Watcher {
         zk.close();
     }
 
-    private void utestPrep() throws KeeperException, IOException,
-            InterruptedException {
+    private void utestPrep() throws IOException,
+            InterruptedException, KeeperException {
         ZooKeeper zk = new ZooKeeper("127.0.0.1:33221", 30000, this);
         for (int i = 0; i < 10000; i++) {
             zk.create("/" + i, null, Ids.OPEN_ACL_UNSAFE, 0);
@@ -97,8 +95,7 @@ public class OOMTest extends TestCase implements Watcher {
         zk.close();
     }
 
-    private void utestGet() throws IOException, InterruptedException,
-            KeeperException {
+    private void utestGet() throws IOException, InterruptedException, KeeperException {
         ZooKeeper zk = new ZooKeeper("127.0.0.1:33221", 30000, this);
         for (int i = 0; i < 10000; i++) {
             Stat stat = new Stat();
@@ -107,8 +104,7 @@ public class OOMTest extends TestCase implements Watcher {
         zk.close();
     }
 
-    private void utestChildren() throws IOException, InterruptedException,
-            KeeperException {
+    private void utestChildren() throws IOException, InterruptedException, KeeperException {
         ZooKeeper zk = new ZooKeeper("127.0.0.1:33221", 30000, this);
         for (int i = 0; i < 10000; i++) {
             zk.getChildren("/" + i, true);

+ 3 - 6
zookeeper/test/com/yahoo/zookeeper/test/SessionTest.java

@@ -33,8 +33,7 @@ public class SessionTest extends TestCase implements Watcher {
      * @throws InterruptedException
      * @throws KeeperException
      */
-    public void testSessionReuse() throws IOException, InterruptedException,
-            KeeperException {
+    public void testSessionReuse() throws IOException, InterruptedException {
         File tmpDir = File.createTempFile("test", ".junit", baseTest);
         tmpDir = new File(tmpDir + ".dir");
         tmpDir.mkdirs();
@@ -55,8 +54,7 @@ public class SessionTest extends TestCase implements Watcher {
         
     }
     @Test
-    public void testSession() throws IOException, InterruptedException,
-            KeeperException {
+    public void testSession() throws IOException, InterruptedException, KeeperException {
         File tmpDir = File.createTempFile("test", ".junit", baseTest);
         tmpDir = new File(tmpDir + ".dir");
         tmpDir.mkdirs();
@@ -65,8 +63,7 @@ public class SessionTest extends TestCase implements Watcher {
         f.startup(zs);
         Thread.sleep(2000);
         ZooKeeper zk = new ZooKeeper("127.0.0.1:33299", 30000, this);
-        zk
-                .create("/e", new byte[0], Ids.OPEN_ACL_UNSAFE,
+        zk.create("/e", new byte[0], Ids.OPEN_ACL_UNSAFE,
                         CreateFlags.EPHEMERAL);
         System.out.println("zk with session id " + zk.getSessionId()
                 + " was destroyed!");

+ 5 - 5
zookeeper/test/com/yahoo/zookeeper/test/SledgeHammer.java

@@ -20,7 +20,7 @@ public class SledgeHammer extends Thread implements Watcher {
     int readsPerWrite;
 
     public SledgeHammer(String hosts, int count, int readsPerWrite)
-            throws KeeperException, IOException {
+            throws IOException {
         zk = new ZooKeeper(hosts, 10000, this);
         this.count = count;
         this.readsPerWrite = readsPerWrite;
@@ -56,10 +56,10 @@ public class SledgeHammer extends Thread implements Watcher {
                             break;
                         }
                     }
+                } catch (KeeperException.ConnectionLossException e) {
+                    // ignore connection loss
                 } catch (KeeperException e) {
-                    if (e.getCode() != KeeperException.Code.ConnectionLoss) {
-                        e.printStackTrace();
-                    }
+                    e.printStackTrace();
                 }
             }
             System.out.println();
@@ -76,7 +76,7 @@ public class SledgeHammer extends Thread implements Watcher {
      * @throws NumberFormatException
      */
     public static void main(String[] args) throws NumberFormatException,
-            KeeperException, IOException {
+            IOException {
         if (args.length != 3) {
             System.err
                     .println("USAGE: SledgeHammer zookeeper_server reps reads_per_rep");

+ 0 - 4
zookeeper/test/com/yahoo/zookeeper/test/TestHammer.java

@@ -2,7 +2,6 @@ package com.yahoo.zookeeper.test;
 
 import java.io.IOException;
 
-import com.yahoo.zookeeper.KeeperException;
 import com.yahoo.zookeeper.ZooKeeper;
 import com.yahoo.zookeeper.AsyncCallback.VoidCallback;
 import com.yahoo.zookeeper.ZooDefs.CreateFlags;
@@ -19,9 +18,6 @@ public class TestHammer implements VoidCallback {
             ZooKeeper zk = null;
             try {
                 zk = new ZooKeeper(args[0], 10000, null);
-            } catch (KeeperException e1) {
-                // TODO Auto-generated catch block
-                e1.printStackTrace();
             } catch (IOException e1) {
                 // TODO Auto-generated catch block
                 e1.printStackTrace();

+ 14 - 18
zookeeper/test/com/yahoo/zookeeper/test/ZooKeeperTestClient.java

@@ -42,7 +42,7 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
   }
 
   private void deleteZKDir(ZooKeeper zk, String nodeName)
-      throws KeeperException, IOException, InterruptedException {
+          throws IOException, InterruptedException, KeeperException {
 
     Stat stat = zk.exists(nodeName, false);
     if (stat == null) {
@@ -59,37 +59,33 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
     }
   }
 
-  private void checkRoot() throws KeeperException, IOException,
+  private void checkRoot() throws IOException,
       InterruptedException {
     ZooKeeper zk = new ZooKeeper(hostPort, 10000, this);
 
     try {
       zk.create(dirOnZK, null, Ids.OPEN_ACL_UNSAFE, 0);
+    } catch (KeeperException.NodeExistsException ke) {
+        // expected, sort of
     } catch (KeeperException ke) {
-      int code = ke.getCode();
-      boolean valid = code == KeeperException.Code.NodeExists;
-      if (!valid) {
         fail("Unexpected exception code for create " + dirOnZK + ": "
             + ke.getMessage());
-      }
     }
 
     try {
       zk.create(testDirOnZK, null, Ids.OPEN_ACL_UNSAFE, 0);
+    } catch (KeeperException.NodeExistsException ke) {
+        // expected, sort of
     } catch (KeeperException ke) {
-      int code = ke.getCode();
-      boolean valid = code == KeeperException.Code.NodeExists;
-      if (!valid) {
         fail("Unexpected exception code for create " + testDirOnZK + ": "
             + ke.getMessage());
-      }
     }
 
     zk.close();
   }
 
-  private void enode_test_1() throws KeeperException, IOException,
-      InterruptedException {
+  private void enode_test_1() throws IOException,
+          InterruptedException, KeeperException {
     checkRoot();
     String parentName = testDirOnZK;
     String nodeName = parentName + "/enode_abc";
@@ -142,8 +138,8 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
 
   }
 
-  private void enode_test_2() throws KeeperException, IOException,
-      InterruptedException {
+  private void enode_test_2() throws IOException,
+          InterruptedException, KeeperException {
     checkRoot();
     String parentName = testDirOnZK;
     String nodeName = parentName + "/enode_abc";
@@ -286,8 +282,8 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
     zk_1.close();
   }
 
-  private void delete_create_get_set_test_1() throws KeeperException,
-      IOException, InterruptedException {
+  private void delete_create_get_set_test_1() throws
+          IOException, InterruptedException, KeeperException {
     checkRoot();
     ZooKeeper zk = new ZooKeeper(hostPort, 10000, this);
     String parentName = testDirOnZK;
@@ -352,8 +348,8 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
   }
 
   @Test
-  public void my_test_1() throws KeeperException, IOException,
-      InterruptedException {
+  public void my_test_1() throws IOException,
+          InterruptedException, KeeperException {
     enode_test_1();
     enode_test_2();
     delete_create_get_set_test_1();