git-svn-id: https://svn.apache.org/repos/asf/hadoop/zookeeper/trunk@670953 13f79535-47bb-0310-9956-ffa450edef68
@@ -1,844 +1,844 @@
-/*
- * 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 int sessionTimeout;
- private ZooKeeper zooKeeper;
- private long sessionId;
- private byte sessionPasswd[] = new byte[16];
- SendThread sendThread;
- EventThread eventThread;
- 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.error("this should be impossible!", 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() {
- while (true) {
- Object event = waitingEvents.take();
- if (event == eventOfDeath) {
- break;
- if (event instanceof WatcherEvent) {
- zooKeeper.watcher.process((WatcherEvent) event);
- 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.error("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) {
- ((SetDataResponse) p.response)
- } else if (p.response instanceof SetACLResponse) {
- ((SetACLResponse) p.response)
- cb.processResult(rc, path, p.ctx, null);
- } else if (p.response instanceof GetDataResponse) {
- DataCallback cb = (DataCallback) p.cb;
- GetDataResponse rsp = (GetDataResponse) p.response;
- rsp.getData(), rsp.getStat());
- cb.processResult(rc, path, p.ctx, null, null);
- } else if (p.response instanceof GetACLResponse) {
- ACLCallback cb = (ACLCallback) p.cb;
- GetACLResponse rsp = (GetACLResponse) p.response;
- cb.processResult(rc, path, p.ctx, rsp.getAcl(),
- rsp.getStat());
- } else if (p.response instanceof GetChildrenResponse) {
- ChildrenCallback cb = (ChildrenCallback) p.cb;
- GetChildrenResponse rsp = (GetChildrenResponse) p.response;
- cb.processResult(rc, path, p.ctx, rsp
- .getChildren());
- } else if (p.response instanceof CreateResponse) {
- StringCallback cb = (StringCallback) p.cb;
- CreateResponse rsp = (CreateResponse) p.response;
- cb
- .processResult(rc, path, p.ctx, rsp
- .getPath());
- } 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;
- synchronized (p) {
- p.notifyAll();
- 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);
- case CLOSED:
- p.replyHeader.setErr(KeeperException.Code.SessionExpired);
- 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");
- connectTimeout = sessionTimeout / serverAddrs.size();
- sessionId = conRsp.getSessionId();
- sessionPasswd = conRsp.getPasswd();
- Watcher.Event.KeeperStateSyncConnected, null));
- void readResponse() throws IOException {
- ReplyHeader r = new ReplyHeader();
- r.deserialize(bbia, "header");
- if (r.getXid() == -2) {
- // -2 is the xid for pings
- if (r.getXid() == -4) {
- // -2 is the xid for AuthPacket
- // TODO: process AuthPacket here
- 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);
- 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");
- * @return true if a packet was received
- * @param k
- * @throws InterruptedException
- 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;
- readResponse();
- if (sockKey.isWritable()) {
- synchronized (outgoingQueue) {
- 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();
- 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() {
- if ((i & SelectionKey.OP_WRITE) != 0) {
- sockKey.interestOps(i & (~SelectionKey.OP_WRITE));
- synchronized private void enableRead() {
- if ((i & SelectionKey.OP_READ) == 0) {
- sockKey.interestOps(i | SelectionKey.OP_READ);
- synchronized private void disableRead() {
- if ((i & SelectionKey.OP_READ) != 0) {
- sockKey.interestOps(i & (~SelectionKey.OP_READ));
- SendThread() {
- super("SendThread");
- zooKeeper.state = States.CONNECTING;
- private void primeConnection(SelectionKey k) throws IOException {
- LOG.warn("Priming connection to "
- + ((SocketChannel) sockKey.channel()));
- lastConnectIndex = currentConnectIndex;
- ConnectRequest conReq = new ConnectRequest(0, lastZxid,
- sessionTimeout, sessionId, sessionPasswd);
- BinaryOutputArchive boa = BinaryOutputArchive.getArchive(baos);
- boa.writeInt(-1, "len");
- conReq.serialize(boa, "connect");
- ByteBuffer bb = ByteBuffer.wrap(baos.toByteArray());
- bb.putInt(bb.capacity() - 4);
- bb.rewind();
- 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;
- Thread.sleep(r.nextInt(1000));
- } catch (InterruptedException e1) {
- // TODO Auto-generated catch block
- e1.printStackTrace();
- if (nextAddrToTry == lastConnectIndex) {
- // Try not to spin too fast!
- Thread.sleep(1000);
- e.printStackTrace();
- 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.warn("Trying to connect to " + addr);
- sockKey = sock.register(selector, SelectionKey.OP_CONNECT);
- if (sock.connect(addr)) {
- primeConnection(sockKey);
- initialized = false;
- @Override
- long now = System.currentTimeMillis();
- long lastHeard = now;
- long lastSend = now;
- while (zooKeeper.state.isAlive()) {
- 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();
- if (timeToNextPing < to) {
- to = timeToNextPing;
- selector.select(to);
- Set<SelectionKey> selected;
- 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;
- primeConnection(k);
- } else if ((k.readyOps() & (SelectionKey.OP_READ | SelectionKey.OP_WRITE)) != 0) {
- // We have something to send so it's the same
- // as if we do the send now.
- if (doIO()) {
- selected.clear();
- } catch (Exception e) {
- LOG.warn("Closing: ", e);
- cleanup();
- if (zooKeeper.state.isAlive()) {
- waitingEvents.add(new WatcherEvent(Event.EventNone,
- Event.KeeperStateDisconnected, null));
- ZooTrace.logTraceMessage(LOG, ZooTrace.getTextTraceLevel(),
- "SendThread exitedloop.");
- private void cleanup() {
- if (sockKey != null) {
- sockKey.cancel();
- sock.socket().shutdownInput();
- } catch (IOException e2) {
- sock.socket().shutdownOutput();
- sock.socket().close();
- } catch (IOException e1) {
- sock.close();
- Thread.sleep(100);
- sockKey = null;
- for (Packet p : pendingQueue) {
- conLossPacket(p);
- pendingQueue.clear();
- for (Packet p : outgoingQueue) {
- outgoingQueue.clear();
- public void close() {
- selector.wakeup();
- 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 {
- 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;
- 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);
- outgoingQueue.add(packet);
- synchronized (sendThread) {
- return packet;
- public void addAuthInfo(String scheme, byte auth[]) {
- authInfo.add(new AuthData(scheme, auth));
- 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 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() {
+ while (true) {
+ Object event = waitingEvents.take();
+ if (event == eventOfDeath) {
+ break;
+ if (event instanceof WatcherEvent) {
+ zooKeeper.watcher.process((WatcherEvent) event);
+ 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) {
+ ((SetDataResponse) p.response)
+ } else if (p.response instanceof SetACLResponse) {
+ ((SetACLResponse) p.response)
+ cb.processResult(rc, path, p.ctx, null);
+ } else if (p.response instanceof GetDataResponse) {
+ DataCallback cb = (DataCallback) p.cb;
+ GetDataResponse rsp = (GetDataResponse) p.response;
+ rsp.getData(), rsp.getStat());
+ cb.processResult(rc, path, p.ctx, null, null);
+ } else if (p.response instanceof GetACLResponse) {
+ ACLCallback cb = (ACLCallback) p.cb;
+ GetACLResponse rsp = (GetACLResponse) p.response;
+ cb.processResult(rc, path, p.ctx, rsp.getAcl(),
+ rsp.getStat());
+ } else if (p.response instanceof GetChildrenResponse) {
+ ChildrenCallback cb = (ChildrenCallback) p.cb;
+ GetChildrenResponse rsp = (GetChildrenResponse) p.response;
+ cb.processResult(rc, path, p.ctx, rsp
+ .getChildren());
+ } else if (p.response instanceof CreateResponse) {
+ StringCallback cb = (StringCallback) p.cb;
+ CreateResponse rsp = (CreateResponse) p.response;
+ cb
+ .processResult(rc, path, p.ctx, rsp
+ .getPath());
+ } 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;
+ synchronized (p) {
+ p.notifyAll();
+ 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);
+ case CLOSED:
+ p.replyHeader.setErr(KeeperException.Code.SessionExpired);
+ 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");
+ connectTimeout = sessionTimeout / serverAddrs.size();
+ sessionId = conRsp.getSessionId();
+ sessionPasswd = conRsp.getPasswd();
+ Watcher.Event.KeeperStateSyncConnected, null));
+ void readResponse() throws IOException {
+ ReplyHeader r = new ReplyHeader();
+ r.deserialize(bbia, "header");
+ if (r.getXid() == -2) {
+ // -2 is the xid for pings
+ if (r.getXid() == -4) {
+ // -2 is the xid for AuthPacket
+ // TODO: process AuthPacket here
+ 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);
+ 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");
+ * @return true if a packet was received
+ * @param k
+ * @throws InterruptedException
+ 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;
+ readResponse();
+ if (sockKey.isWritable()) {
+ synchronized (outgoingQueue) {
+ 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();
+ 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() {
+ if ((i & SelectionKey.OP_WRITE) != 0) {
+ sockKey.interestOps(i & (~SelectionKey.OP_WRITE));
+ synchronized private void enableRead() {
+ if ((i & SelectionKey.OP_READ) == 0) {
+ sockKey.interestOps(i | SelectionKey.OP_READ);
+ synchronized private void disableRead() {
+ if ((i & SelectionKey.OP_READ) != 0) {
+ sockKey.interestOps(i & (~SelectionKey.OP_READ));
+ SendThread() {
+ super("SendThread");
+ zooKeeper.state = States.CONNECTING;
+ 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);
+ BinaryOutputArchive boa = BinaryOutputArchive.getArchive(baos);
+ boa.writeInt(-1, "len");
+ conReq.serialize(boa, "connect");
+ ByteBuffer bb = ByteBuffer.wrap(baos.toByteArray());
+ bb.putInt(bb.capacity() - 4);
+ bb.rewind();
+ 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;
+ Thread.sleep(r.nextInt(1000));
+ } catch (InterruptedException e1) {
+ LOG.warn("Unexpected exception", e1);
+ if (nextAddrToTry == lastConnectIndex) {
+ // Try not to spin too fast!
+ Thread.sleep(1000);
+ LOG.warn("Unexpected exception", e);
+ 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
+ long now = System.currentTimeMillis();
+ long lastHeard = now;
+ long lastSend = now;
+ while (zooKeeper.state.isAlive()) {
+ 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();
+ if (timeToNextPing < to) {
+ to = timeToNextPing;
+ selector.select(to);
+ Set<SelectionKey> selected;
+ 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;
+ primeConnection(k);
+ LOG.info("Server connection successful");
+ } else if ((k.readyOps() & (SelectionKey.OP_READ | SelectionKey.OP_WRITE)) != 0) {
+ // We have something to send so it's the same
+ // as if we do the send now.
+ if (doIO()) {
+ selected.clear();
+ } catch (Exception e) {
+ LOG.warn("Closing: ", e);
+ cleanup();
+ if (zooKeeper.state.isAlive()) {
+ waitingEvents.add(new WatcherEvent(Event.EventNone,
+ Event.KeeperStateDisconnected, null));
+ ZooTrace.logTraceMessage(LOG, ZooTrace.getTextTraceLevel(),
+ "SendThread exitedloop.");
+ private void cleanup() {
+ if (sockKey != null) {
+ sockKey.cancel();
+ sock.socket().shutdownInput();
+ } catch (IOException e2) {
+ sock.socket().shutdownOutput();
+ sock.socket().close();
+ } catch (IOException e1) {
+ sock.close();
+ Thread.sleep(100);
+ e1.printStackTrace();
+ sockKey = null;
+ for (Packet p : pendingQueue) {
+ conLossPacket(p);
+ pendingQueue.clear();
+ for (Packet p : outgoingQueue) {
+ outgoingQueue.clear();
+ public void close() {
+ selector.wakeup();
+ 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 {
+ 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;
+ 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);
+ outgoingQueue.add(packet);
+ synchronized (sendThread) {
+ return packet;
+ public void addAuthInfo(String scheme, byte auth[]) {
+ authInfo.add(new AuthData(scheme, auth));
+ queuePacket(new RequestHeader(-4, OpCode.auth), null,
+ new AuthPacket(0, scheme, auth), null, null, null, null);
+}
@@ -1,262 +1,262 @@
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.Socket;
-import java.util.StringTokenizer;
-public class ServerAdminClient {
- private static final Logger LOG = Logger.getLogger(ServerAdminClient.class);
- private static long getMask(String mask) {
- long retv = 0;
- if (mask.equalsIgnoreCase("CLIENT_REQUEST_TRACE_MASK")) {
- retv = ZooTrace.CLIENT_REQUEST_TRACE_MASK;
- } else if (mask.equalsIgnoreCase("CLIENT_DATA_PACKET_TRACE_MASK")) {
- retv = ZooTrace.CLIENT_DATA_PACKET_TRACE_MASK;
- } else if (mask.equalsIgnoreCase("CLIENT_PING_TRACE_MASK")) {
- retv = ZooTrace.CLIENT_PING_TRACE_MASK;
- } else if (mask.equalsIgnoreCase("SERVER_PACKET_TRACE_MASK")) {
- retv = ZooTrace.SERVER_PACKET_TRACE_MASK;
- } else if (mask.equalsIgnoreCase("SESSION_TRACE_MASK")) {
- retv = ZooTrace.SESSION_TRACE_MASK;
- } else if (mask.equalsIgnoreCase("EVENT_DELIVERY_TRACE_MASK")) {
- retv = ZooTrace.EVENT_DELIVERY_TRACE_MASK;
- } else if (mask.equalsIgnoreCase("SERVER_PING_TRACE_MASK")) {
- retv = ZooTrace.SERVER_PING_TRACE_MASK;
- } else if (mask.equalsIgnoreCase("WARNING_TRACE_MASK")) {
- retv = ZooTrace.WARNING_TRACE_MASK;
- return retv;
- private static long getMasks(String masks) {
- StringTokenizer st = new StringTokenizer(masks, "|");
- while (st.hasMoreTokens()) {
- String mask = st.nextToken().trim();
- retv = retv | getMask(mask);
- public static void ruok(String host, int port) {
- byte[] reqBytes = new byte[4];
- ByteBuffer req = ByteBuffer.wrap(reqBytes);
- req.putInt(ByteBuffer.wrap("ruok".getBytes()).getInt());
- Socket s = null;
- s = new Socket();
- s.setSoLinger(false, 10);
- s.setSoTimeout(20000);
- s.connect(new InetSocketAddress(host, port));
- InputStream is = s.getInputStream();
- OutputStream os = s.getOutputStream();
- os.write(reqBytes);
- byte[] resBytes = new byte[4];
- int rc = is.read(resBytes);
- String retv = new String(resBytes);
- System.out.println("rc=" + rc + " retv=" + retv);
- } catch (IOException ioe) {
- ioe.printStackTrace();
- public static void dump(String host, int port) {
- req.putInt(ByteBuffer.wrap("dump".getBytes()).getInt());
- byte[] resBytes = new byte[1024];
- public static void stat(String host, int port) {
- req.putInt(ByteBuffer.wrap("stat".getBytes()).getInt());
- public static void kill(String host, int port) {
- req.putInt(ByteBuffer.wrap("kill".getBytes()).getInt());
- public static void setTraceMask(String host, int port, String traceMaskStr) {
- byte[] reqBytes = new byte[12];
- long traceMask = Long.parseLong(traceMaskStr, 8);
- req.putInt(ByteBuffer.wrap("stmk".getBytes()).getInt());
- req.putLong(traceMask);
- byte[] resBytes = new byte[8];
- ByteBuffer res = ByteBuffer.wrap(resBytes);
- long retv = res.getLong();
- System.out.println("rc=" + rc + " retv=0"
- + Long.toOctalString(retv) + " masks=0"
- + Long.toOctalString(traceMask));
- assert (retv == traceMask);
- public static void getTraceMask(String host, int port) {
- req.putInt(ByteBuffer.wrap("gtmk".getBytes()).getInt());
- + Long.toOctalString(retv));
- private static void usage() {
- System.out
- .println("usage: java [-cp CLASSPATH] com.yahoo.zookeeper.ServerAdminClient "
- + "host port op (ruok|stat|dump|kill|gettracemask|settracemask) [arguments]");
- public static void main(String[] args) {
- if (args.length < 3) {
- usage();
- String host = args[0];
- int port = Integer.parseInt(args[1]);
- String op = args[2];
- if (op.equalsIgnoreCase("gettracemask")) {
- getTraceMask(host, port);
- } else if (op.equalsIgnoreCase("settracemask")) {
- setTraceMask(host, port, args[3]);
- } else if (op.equalsIgnoreCase("ruok")) {
- ruok(host, port);
- } else if (op.equalsIgnoreCase("kill")) {
- kill(host, port);
- } else if (op.equalsIgnoreCase("stat")) {
- stat(host, port);
- } else if (op.equalsIgnoreCase("dump")) {
- dump(host, port);
- System.out.println("Unrecognized op: " + op);
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.Socket;
+import java.util.StringTokenizer;
+public class ServerAdminClient {
+ private static final Logger LOG = Logger.getLogger(ServerAdminClient.class);
+ private static long getMask(String mask) {
+ long retv = 0;
+ if (mask.equalsIgnoreCase("CLIENT_REQUEST_TRACE_MASK")) {
+ retv = ZooTrace.CLIENT_REQUEST_TRACE_MASK;
+ } else if (mask.equalsIgnoreCase("CLIENT_DATA_PACKET_TRACE_MASK")) {
+ retv = ZooTrace.CLIENT_DATA_PACKET_TRACE_MASK;
+ } else if (mask.equalsIgnoreCase("CLIENT_PING_TRACE_MASK")) {
+ retv = ZooTrace.CLIENT_PING_TRACE_MASK;
+ } else if (mask.equalsIgnoreCase("SERVER_PACKET_TRACE_MASK")) {
+ retv = ZooTrace.SERVER_PACKET_TRACE_MASK;
+ } else if (mask.equalsIgnoreCase("SESSION_TRACE_MASK")) {
+ retv = ZooTrace.SESSION_TRACE_MASK;
+ } else if (mask.equalsIgnoreCase("EVENT_DELIVERY_TRACE_MASK")) {
+ retv = ZooTrace.EVENT_DELIVERY_TRACE_MASK;
+ } else if (mask.equalsIgnoreCase("SERVER_PING_TRACE_MASK")) {
+ retv = ZooTrace.SERVER_PING_TRACE_MASK;
+ } else if (mask.equalsIgnoreCase("WARNING_TRACE_MASK")) {
+ retv = ZooTrace.WARNING_TRACE_MASK;
+ return retv;
+ private static long getMasks(String masks) {
+ StringTokenizer st = new StringTokenizer(masks, "|");
+ while (st.hasMoreTokens()) {
+ String mask = st.nextToken().trim();
+ retv = retv | getMask(mask);
+ public static void ruok(String host, int port) {
+ byte[] reqBytes = new byte[4];
+ ByteBuffer req = ByteBuffer.wrap(reqBytes);
+ req.putInt(ByteBuffer.wrap("ruok".getBytes()).getInt());
+ Socket s = null;
+ s = new Socket();
+ s.setSoLinger(false, 10);
+ s.setSoTimeout(20000);
+ s.connect(new InetSocketAddress(host, port));
+ InputStream is = s.getInputStream();
+ OutputStream os = s.getOutputStream();
+ os.write(reqBytes);
+ byte[] resBytes = new byte[4];
+ int rc = is.read(resBytes);
+ String retv = new String(resBytes);
+ System.out.println("rc=" + rc + " retv=" + retv);
+ } catch (IOException ioe) {
+ LOG.warn("Unexpected exception", ioe);
+ public static void dump(String host, int port) {
+ req.putInt(ByteBuffer.wrap("dump".getBytes()).getInt());
+ byte[] resBytes = new byte[1024];
+ public static void stat(String host, int port) {
+ req.putInt(ByteBuffer.wrap("stat".getBytes()).getInt());
+ public static void kill(String host, int port) {
+ req.putInt(ByteBuffer.wrap("kill".getBytes()).getInt());
+ public static void setTraceMask(String host, int port, String traceMaskStr) {
+ byte[] reqBytes = new byte[12];
+ long traceMask = Long.parseLong(traceMaskStr, 8);
+ req.putInt(ByteBuffer.wrap("stmk".getBytes()).getInt());
+ req.putLong(traceMask);
+ byte[] resBytes = new byte[8];
+ ByteBuffer res = ByteBuffer.wrap(resBytes);
+ long retv = res.getLong();
+ System.out.println("rc=" + rc + " retv=0"
+ + Long.toOctalString(retv) + " masks=0"
+ + Long.toOctalString(traceMask));
+ assert (retv == traceMask);
+ public static void getTraceMask(String host, int port) {
+ req.putInt(ByteBuffer.wrap("gtmk".getBytes()).getInt());
+ + Long.toOctalString(retv));
+ private static void usage() {
+ System.out
+ .println("usage: java [-cp CLASSPATH] com.yahoo.zookeeper.ServerAdminClient "
+ + "host port op (ruok|stat|dump|kill|gettracemask|settracemask) [arguments]");
+ public static void main(String[] args) {
+ if (args.length < 3) {
+ usage();
+ String host = args[0];
+ int port = Integer.parseInt(args[1]);
+ String op = args[2];
+ if (op.equalsIgnoreCase("gettracemask")) {
+ getTraceMask(host, port);
+ } else if (op.equalsIgnoreCase("settracemask")) {
+ setTraceMask(host, port, args[3]);
+ } else if (op.equalsIgnoreCase("ruok")) {
+ ruok(host, port);
+ } else if (op.equalsIgnoreCase("kill")) {
+ kill(host, port);
+ } else if (op.equalsIgnoreCase("stat")) {
+ stat(host, port);
+ } else if (op.equalsIgnoreCase("dump")) {
+ dump(host, port);
+ System.out.println("Unrecognized op: " + op);
@@ -1,869 +1,874 @@
-import java.io.BufferedReader;
-import java.io.InputStreamReader;
-import java.util.Date;
-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.DeleteRequest;
-import com.yahoo.zookeeper.proto.ExistsRequest;
-import com.yahoo.zookeeper.proto.GetACLRequest;
-import com.yahoo.zookeeper.proto.GetChildrenRequest;
-import com.yahoo.zookeeper.proto.GetDataRequest;
-import com.yahoo.zookeeper.proto.SetACLRequest;
-import com.yahoo.zookeeper.proto.SetDataRequest;
-import com.yahoo.zookeeper.proto.SyncRequest;
-import com.yahoo.zookeeper.proto.SyncResponse;
-import com.yahoo.zookeeper.server.DataTree;
- * 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.
- * 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.
- * Once a connection to a server is established, a session ID is assigned to the
- * client. The client will send heart beats to the server periodically to keep
- * the session valid.
- * The application can call ZooKeeper APIs through a client as long as the
- * session ID of the client remains valid.
- * If for some reason, the client fails to send heart beats to the server for a
- * prolonged period of time (exceeding the sessionTimeout value, for instance),
- * the server will expire the session, and the session ID will become invalid.
- * The client object will no longer be usable. To make ZooKeeper API calls, the
- * application must create a new client object.
- * If the ZooKeeper server the client currently connects to fails or otherwise
- * does not respond, the client will automatically try to connect to another
- * server before its session ID expires. If successful, the application can
- * continue to use the client.
- * Some successful ZooKeeper API calls can leave watches on the "data nodes" in
- * the ZooKeeper server. Other successful ZooKeeper API calls can trigger those
- * watches. Once a watch is triggered, an event will be delivered to the client
- * which left the watch at the first place. Each watch can be triggered only
- * once. Thus, up to one event will be delivered to a client for every watch it
- * leaves.
- * A client needs an object of a class implementing Watcher interface for
- * processing the events delivered to the client.
- * When a client drops current connection and re-connects to a server, all the
- * existing watches are considered as being triggered but the undelivered events
- * are lost. To emulate this, the client will generate a special event to tell
- * the event handler a connection has been dropped. This special event has type
- * EventNone and state sKeeperStateDisconnected.
-public class ZooKeeper {
- 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)
- this.watcher = watcher;
- cnxn = new ClientCnxn(host, sessionTimeout, this);
- public ZooKeeper(String host, int sessionTimeout, Watcher watcher,
- cnxn = new ClientCnxn(host, sessionTimeout, this, sessionId,
- sessionPasswd);
- return cnxn.getSessionId();
- return cnxn.getSessionPasswd();
- cnxn.addAuthInfo(scheme, auth);
- public String describeCNXN() {
- return cnxn.toString();
- public synchronized void register(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.
- public synchronized void close() throws InterruptedException {
- RequestHeader h = new RequestHeader();
- h.setType(ZooDefs.OpCode.closeSession);
- cnxn.submitRequest(h, null, null);
- cnxn.close();
- * Create a node with the given path. The node data will be the given data,
- * and node acl will be the given acl.
- * The flags argument specifies whether the created node will be ephemeral
- * or not.
- * An ephemeral node will be removed by the ZooKeeper automatically when the
- * session associated with the creation of the node expires.
- * The flags argument can also specify to create a sequential node. The
- * actual path name of a sequential node will be the given path plus a
- * suffix "_i" where i is the current sequential number of the node. Once
- * such a node is created, the sequential number will be incremented by one.
- * If a node with the same actual path already exists in the ZooKeeper, a
- * KeeperException with error code KeeperException.NodeExists will be
- * thrown. Note that since a different actual path is used for each
- * invocation of creating sequential node with the same path argument, the
- * call will never throw "file exists" KeeperException.
- * If the parent node does not exist in the ZooKeeper, a KeeperException
- * with error code KeeperException.NoNode will be thrown.
- * An ephemeral node cannot have children. If the parent node of the given
- * path is ephemeral, a KeeperException with error code
- * KeeperException.NoChildrenForEphemerals will be thrown.
- * This operation, if successful, will trigger all the watches left on the
- * node of the given path by exists and getData API calls, and the watches
- * left on the parent node by getChildren API calls.
- * If a node is created successfully, the ZooKeeper server will trigger the
- * watches on the path left by exists calls, and the watches on the parent
- * of the node by getChildren calls.
- * @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
- * an exception with appropriate error code defined in
- * KeeperException class..
- public String create(String path, byte data[], ArrayList<ACL> acl, int flags)
- throws KeeperException, InterruptedException {
- 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) {
- 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).
- * A KeeperException with error code KeeperException.NoNode will be thrown
- * if the nodes does not exist.
- * A KeeperException with error code KeeperException.BadVersion will be
- * thrown if the given version does not match the node's version.
- * A KeeperException with error code KeeperException.NotEmpty will be thrown
- * if the node has children.
- * This operation, if successful, will trigger all the watches on the node
- * of the given path left by exists API calls, and the watches on the parent
- * node left by getChildren API calls.
- * the path of the node to be deleted.
- * @param version
- * the expected node version.
- * an KeeperException with appropriate error code.
- public void delete(String path, int version) throws KeeperException,
- InterruptedException {
- h.setType(ZooDefs.OpCode.delete);
- DeleteRequest request = new DeleteRequest();
- request.setVersion(version);
- ReplyHeader r = cnxn.submitRequest(h, request, null);
- throw new KeeperException(r.getErr());
- * The Asynchronous version of delete. The request doesn't actually until
- * @see #delete(String, int)
- public void delete(String path, int version, VoidCallback cb, Object ctx) {
- 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.
- * 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.
- * 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
- public Stat exists(String path, boolean watch) throws KeeperException,
- h.setType(ZooDefs.OpCode.exists);
- ExistsRequest request = new ExistsRequest();
- request.setWatch(watch);
- SetDataResponse response = new SetDataResponse();
- if (r.getErr() == KeeperException.Code.NoNode) {
- return null;
- return response.getStat().getCzxid() == -1 ? null : response.getStat();
- * The Asynchronous version of exists. The request doesn't actually until
- * @see #exists(String, boolean)
- public void exists(String path, boolean watch, StatCallback cb, Object ctx) {
- cnxn
- .queuePacket(h, new ReplyHeader(), request, response, cb, path,
- ctx);
- * Return the data and the stat of the node of the given path.
- * 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.
- * if no node with the given path exists.
- * the given path
- * @param stat
- * teh stat of the node
- * @return the data of the node
- public byte[] getData(String path, boolean watch, Stat stat)
- h.setType(ZooDefs.OpCode.getData);
- GetDataRequest request = new GetDataRequest();
- GetDataResponse response = new GetDataResponse();
- if (stat != null) {
- DataTree.copyStat(response.getStat(), stat);
- return response.getData();
- * The Asynchronous version of getData. The request doesn't actually until
- * @see #getData(String, boolean, Stat)
- public void getData(String path, boolean watch, DataCallback cb, Object 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.
- * of the given path left by getData calls.
- * the path of the node
- * the data to set
- * the expected matching version
- * @return the state of the node
- public Stat setData(String path, byte data[], int version)
- h.setType(ZooDefs.OpCode.setData);
- SetDataRequest request = new SetDataRequest();
- return response.getStat();
- * The Asynchronous version of setData. The request doesn't actually until
- * @see #setData(String, byte[], int)
- public void setData(String path, byte data[], int version, StatCallback cb,
- Object ctx) {
- * Return the ACL and stat of the node of the given path.
- * the given path for the node
- * the stat of the node will be copied to this parameter.
- * @return the ACL array of the given node.
- public ArrayList<ACL> getACL(String path, Stat stat)
- h.setType(ZooDefs.OpCode.getACL);
- GetACLRequest request = new GetACLRequest();
- GetACLResponse response = new GetACLResponse();
- return response.getAcl();
- * The Asynchronous version of getACL. The request doesn't actually until
- * @see #getACL(String, Stat)
- public void getACL(String path, Stat stat, ACLCallback cb, Object 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.
- * @return the stat of the node.
- public Stat setACL(String path, ArrayList<ACL> acl, int version)
- h.setType(ZooDefs.OpCode.setACL);
- SetACLRequest request = new SetACLRequest();
- SetACLResponse response = new SetACLResponse();
- * The Asynchronous version of setACL. The request doesn't actually until
- * @see #setACL(String, ArrayList, int)
- public void setACL(String path, ArrayList<ACL> acl, int version,
- StatCallback cb, Object ctx) {
- * Return the list of the children of the node of the given path.
- * 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.
- * @return an array of children of the node with the given path
- public ArrayList<String> getChildren(String path, boolean watch)
- h.setType(ZooDefs.OpCode.getChildren);
- GetChildrenRequest request = new GetChildrenRequest();
- GetChildrenResponse response = new GetChildrenResponse();
- 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,
- * Asynchronous sync. Flushes channel between process and leader.
- * @see #sync(String)
- public void sync(String path, VoidCallback cb, Object ctx){
- h.setType(ZooDefs.OpCode.sync);
- SyncRequest request = new SyncRequest();
- SyncResponse response = new SyncResponse();
- 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;
- case 'w':
- perm |= ZooDefs.Perms.WRITE;
- case 'c':
- perm |= ZooDefs.Perms.CREATE;
- case 'd':
- perm |= ZooDefs.Perms.DELETE;
- case 'a':
- perm |= ZooDefs.Perms.ADMIN;
- 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) {
- 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;
- 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);
- } 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));
- } 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]),
- } else if (cmd.equals("stat") && args.length >= 3) {
- stat = zooKeeper.exists(path, watch);
- 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) {
- .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 {
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
+import java.util.Date;
+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.DeleteRequest;
+import com.yahoo.zookeeper.proto.ExistsRequest;
+import com.yahoo.zookeeper.proto.GetACLRequest;
+import com.yahoo.zookeeper.proto.GetChildrenRequest;
+import com.yahoo.zookeeper.proto.GetDataRequest;
+import com.yahoo.zookeeper.proto.SetACLRequest;
+import com.yahoo.zookeeper.proto.SetDataRequest;
+import com.yahoo.zookeeper.proto.SyncRequest;
+import com.yahoo.zookeeper.proto.SyncResponse;
+import com.yahoo.zookeeper.server.DataTree;
+ * 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.
+ * 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.
+ * Once a connection to a server is established, a session ID is assigned to the
+ * client. The client will send heart beats to the server periodically to keep
+ * the session valid.
+ * The application can call ZooKeeper APIs through a client as long as the
+ * session ID of the client remains valid.
+ * If for some reason, the client fails to send heart beats to the server for a
+ * prolonged period of time (exceeding the sessionTimeout value, for instance),
+ * the server will expire the session, and the session ID will become invalid.
+ * The client object will no longer be usable. To make ZooKeeper API calls, the
+ * application must create a new client object.
+ * If the ZooKeeper server the client currently connects to fails or otherwise
+ * does not respond, the client will automatically try to connect to another
+ * server before its session ID expires. If successful, the application can
+ * continue to use the client.
+ * Some successful ZooKeeper API calls can leave watches on the "data nodes" in
+ * the ZooKeeper server. Other successful ZooKeeper API calls can trigger those
+ * watches. Once a watch is triggered, an event will be delivered to the client
+ * which left the watch at the first place. Each watch can be triggered only
+ * once. Thus, up to one event will be delivered to a client for every watch it
+ * leaves.
+ * A client needs an object of a class implementing Watcher interface for
+ * processing the events delivered to the client.
+ * When a client drops current connection and re-connects to a server, all the
+ * existing watches are considered as being triggered but the undelivered events
+ * are lost. To emulate this, the client will generate a special event to tell
+ * the event handler a connection has been dropped. This special event has type
+ * EventNone and state sKeeperStateDisconnected.
+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)
+ this.watcher = watcher;
+ cnxn = new ClientCnxn(host, sessionTimeout, this);
+ public ZooKeeper(String host, int sessionTimeout, Watcher watcher,
+ cnxn = new ClientCnxn(host, sessionTimeout, this, sessionId,
+ sessionPasswd);
+ return cnxn.getSessionId();
+ return cnxn.getSessionPasswd();
+ cnxn.addAuthInfo(scheme, auth);
+ public String describeCNXN() {
+ return cnxn.toString();
+ public synchronized void register(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.
+ public synchronized void close() throws InterruptedException {
+ RequestHeader h = new RequestHeader();
+ h.setType(ZooDefs.OpCode.closeSession);
+ cnxn.submitRequest(h, null, null);
+ cnxn.close();
+ * Create a node with the given path. The node data will be the given data,
+ * and node acl will be the given acl.
+ * The flags argument specifies whether the created node will be ephemeral
+ * or not.
+ * An ephemeral node will be removed by the ZooKeeper automatically when the
+ * session associated with the creation of the node expires.
+ * The flags argument can also specify to create a sequential node. The
+ * actual path name of a sequential node will be the given path plus a
+ * suffix "_i" where i is the current sequential number of the node. Once
+ * such a node is created, the sequential number will be incremented by one.
+ * If a node with the same actual path already exists in the ZooKeeper, a
+ * KeeperException with error code KeeperException.NodeExists will be
+ * thrown. Note that since a different actual path is used for each
+ * invocation of creating sequential node with the same path argument, the
+ * call will never throw "file exists" KeeperException.
+ * If the parent node does not exist in the ZooKeeper, a KeeperException
+ * with error code KeeperException.NoNode will be thrown.
+ * An ephemeral node cannot have children. If the parent node of the given
+ * path is ephemeral, a KeeperException with error code
+ * KeeperException.NoChildrenForEphemerals will be thrown.
+ * This operation, if successful, will trigger all the watches left on the
+ * node of the given path by exists and getData API calls, and the watches
+ * left on the parent node by getChildren API calls.
+ * If a node is created successfully, the ZooKeeper server will trigger the
+ * watches on the path left by exists calls, and the watches on the parent
+ * of the node by getChildren calls.
+ * @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
+ * an exception with appropriate error code defined in
+ * KeeperException class..
+ public String create(String path, byte data[], ArrayList<ACL> acl, int flags)
+ throws KeeperException, InterruptedException {
+ 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) {
+ 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).
+ * A KeeperException with error code KeeperException.NoNode will be thrown
+ * if the nodes does not exist.
+ * A KeeperException with error code KeeperException.BadVersion will be
+ * thrown if the given version does not match the node's version.
+ * A KeeperException with error code KeeperException.NotEmpty will be thrown
+ * if the node has children.
+ * This operation, if successful, will trigger all the watches on the node
+ * of the given path left by exists API calls, and the watches on the parent
+ * node left by getChildren API calls.
+ * the path of the node to be deleted.
+ * @param version
+ * the expected node version.
+ * an KeeperException with appropriate error code.
+ public void delete(String path, int version) throws KeeperException,
+ InterruptedException {
+ h.setType(ZooDefs.OpCode.delete);
+ DeleteRequest request = new DeleteRequest();
+ request.setVersion(version);
+ ReplyHeader r = cnxn.submitRequest(h, request, null);
+ throw new KeeperException(r.getErr());
+ * The Asynchronous version of delete. The request doesn't actually until
+ * @see #delete(String, int)
+ public void delete(String path, int version, VoidCallback cb, Object ctx) {
+ 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.
+ * 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.
+ * 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
+ public Stat exists(String path, boolean watch) throws KeeperException,
+ h.setType(ZooDefs.OpCode.exists);
+ ExistsRequest request = new ExistsRequest();
+ request.setWatch(watch);
+ SetDataResponse response = new SetDataResponse();
+ if (r.getErr() == KeeperException.Code.NoNode) {
+ return null;
+ return response.getStat().getCzxid() == -1 ? null : response.getStat();
+ * The Asynchronous version of exists. The request doesn't actually until
+ * @see #exists(String, boolean)
+ public void exists(String path, boolean watch, StatCallback cb, Object ctx) {
+ cnxn
+ .queuePacket(h, new ReplyHeader(), request, response, cb, path,
+ ctx);
+ * Return the data and the stat of the node of the given path.
+ * 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.
+ * if no node with the given path exists.
+ * the given path
+ * @param stat
+ * teh stat of the node
+ * @return the data of the node
+ public byte[] getData(String path, boolean watch, Stat stat)
+ h.setType(ZooDefs.OpCode.getData);
+ GetDataRequest request = new GetDataRequest();
+ GetDataResponse response = new GetDataResponse();
+ if (stat != null) {
+ DataTree.copyStat(response.getStat(), stat);
+ return response.getData();
+ * The Asynchronous version of getData. The request doesn't actually until
+ * @see #getData(String, boolean, Stat)
+ public void getData(String path, boolean watch, DataCallback cb, Object 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.
+ * of the given path left by getData calls.
+ * the path of the node
+ * the data to set
+ * the expected matching version
+ * @return the state of the node
+ public Stat setData(String path, byte data[], int version)
+ h.setType(ZooDefs.OpCode.setData);
+ SetDataRequest request = new SetDataRequest();
+ return response.getStat();
+ * The Asynchronous version of setData. The request doesn't actually until
+ * @see #setData(String, byte[], int)
+ public void setData(String path, byte data[], int version, StatCallback cb,
+ Object ctx) {
+ * Return the ACL and stat of the node of the given path.
+ * the given path for the node
+ * the stat of the node will be copied to this parameter.
+ * @return the ACL array of the given node.
+ public ArrayList<ACL> getACL(String path, Stat stat)
+ h.setType(ZooDefs.OpCode.getACL);
+ GetACLRequest request = new GetACLRequest();
+ GetACLResponse response = new GetACLResponse();
+ return response.getAcl();
+ * The Asynchronous version of getACL. The request doesn't actually until
+ * @see #getACL(String, Stat)
+ public void getACL(String path, Stat stat, ACLCallback cb, Object 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.
+ * @return the stat of the node.
+ public Stat setACL(String path, ArrayList<ACL> acl, int version)
+ h.setType(ZooDefs.OpCode.setACL);
+ SetACLRequest request = new SetACLRequest();
+ SetACLResponse response = new SetACLResponse();
+ * The Asynchronous version of setACL. The request doesn't actually until
+ * @see #setACL(String, ArrayList, int)
+ public void setACL(String path, ArrayList<ACL> acl, int version,
+ StatCallback cb, Object ctx) {
+ * Return the list of the children of the node of the given path.
+ * 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.
+ * @return an array of children of the node with the given path
+ public ArrayList<String> getChildren(String path, boolean watch)
+ h.setType(ZooDefs.OpCode.getChildren);
+ GetChildrenRequest request = new GetChildrenRequest();
+ GetChildrenResponse response = new GetChildrenResponse();
+ 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,
+ * Asynchronous sync. Flushes channel between process and leader.
+ * @see #sync(String)
+ public void sync(String path, VoidCallback cb, Object ctx){
+ h.setType(ZooDefs.OpCode.sync);
+ SyncRequest request = new SyncRequest();
+ SyncResponse response = new SyncResponse();
+ 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;
+ case 'w':
+ perm |= ZooDefs.Perms.WRITE;
+ case 'c':
+ perm |= ZooDefs.Perms.CREATE;
+ case 'd':
+ perm |= ZooDefs.Perms.DELETE;
+ case 'a':
+ perm |= ZooDefs.Perms.ADMIN;
+ 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) {
+ 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;
+ 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);
+ } 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));
+ } 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]),
+ } else if (cmd.equals("stat") && args.length >= 3) {
+ stat = zooKeeper.exists(path, watch);
+ 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) {
+ .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 {
@@ -1,243 +1,243 @@
-package com.yahoo.zookeeper.server;
-import com.yahoo.zookeeper.KeeperException;
-import com.yahoo.zookeeper.ZooDefs;
-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.warn("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.error("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());
- 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) {
- zks.decInProcess();
- int err = 0;
- Record rsp = null;
- 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");
- case OpCode.createSession:
- request.cnxn.finishSessionInit(true);
- case OpCode.create:
- rsp = new CreateResponse(rc.path);
- err = rc.err;
- case OpCode.delete:
- case OpCode.setData:
- rsp = new SetDataResponse(rc.stat);
- case OpCode.setACL:
- rsp = new SetACLResponse(rc.stat);
- case OpCode.closeSession:
- case OpCode.sync:
- SyncRequest syncRequest = new SyncRequest();
- ZooKeeperServer.byteBuffer2Record(request.request,
- syncRequest);
- rsp = new SyncResponse(syncRequest.getPath());
- case OpCode.exists:
- // TODO we need to figure out the security requirement for this!
- ExistsRequest existsRequest = new ExistsRequest();
- 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);
- case OpCode.getData:
- GetDataRequest getDataRequest = new GetDataRequest();
- 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);
- case OpCode.getACL:
- GetACLRequest getACLRequest = new GetACLRequest();
- getACLRequest);
- ArrayList<ACL> acl = zks.dataTree.getACL(getACLRequest
- .getPath(), stat);
- rsp = new GetACLResponse(acl, stat);
- case OpCode.getChildren:
- GetChildrenRequest getChildrenRequest = new GetChildrenRequest();
- getChildrenRequest);
- n = zks.dataTree.getNode(getChildrenRequest.getPath());
- ArrayList<String> children = zks.dataTree.getChildren(
- getChildrenRequest.getPath(), stat, getChildrenRequest
- rsp = new GetChildrenResponse(children);
- } catch (KeeperException e) {
- err = e.getCode();
- LOG.warn("****************************** " + request);
- ByteBuffer bb = request.request;
- 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);
- request.cnxn.sendResponse(hdr, rsp, "response");
- public void shutdown() {
+package com.yahoo.zookeeper.server;
+import com.yahoo.zookeeper.KeeperException;
+import com.yahoo.zookeeper.ZooDefs;
+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());
+ 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) {
+ zks.decInProcess();
+ int err = 0;
+ Record rsp = null;
+ 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");
+ case OpCode.createSession:
+ request.cnxn.finishSessionInit(true);
+ case OpCode.create:
+ rsp = new CreateResponse(rc.path);
+ err = rc.err;
+ case OpCode.delete:
+ case OpCode.setData:
+ rsp = new SetDataResponse(rc.stat);
+ case OpCode.setACL:
+ rsp = new SetACLResponse(rc.stat);
+ case OpCode.closeSession:
+ case OpCode.sync:
+ SyncRequest syncRequest = new SyncRequest();
+ ZooKeeperServer.byteBuffer2Record(request.request,
+ syncRequest);
+ rsp = new SyncResponse(syncRequest.getPath());
+ case OpCode.exists:
+ // TODO we need to figure out the security requirement for this!
+ ExistsRequest existsRequest = new ExistsRequest();
+ 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);
+ case OpCode.getData:
+ GetDataRequest getDataRequest = new GetDataRequest();
+ 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);
+ case OpCode.getACL:
+ GetACLRequest getACLRequest = new GetACLRequest();
+ getACLRequest);
+ ArrayList<ACL> acl = zks.dataTree.getACL(getACLRequest
+ .getPath(), stat);
+ rsp = new GetACLResponse(acl, stat);
+ case OpCode.getChildren:
+ GetChildrenRequest getChildrenRequest = new GetChildrenRequest();
+ getChildrenRequest);
+ n = zks.dataTree.getNode(getChildrenRequest.getPath());
+ ArrayList<String> children = zks.dataTree.getChildren(
+ getChildrenRequest.getPath(), stat, getChildrenRequest
+ rsp = new GetChildrenResponse(children);
+ } catch (KeeperException e) {
+ err = e.getCode();
+ LOG.error("****************************** " + request);
+ ByteBuffer bb = request.request;
+ 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);
+ request.cnxn.sendResponse(hdr, rsp, "response");
+ public void shutdown() {
@@ -1,847 +1,848 @@
-import java.nio.channels.CancelledKeyException;
-import java.nio.channels.Channel;
-import java.nio.channels.ServerSocketChannel;
-import java.util.HashSet;
-import java.util.Iterator;
-import com.yahoo.zookeeper.Version;
-import com.yahoo.zookeeper.Watcher;
-import com.yahoo.zookeeper.server.auth.AuthenticationProvider;
-import com.yahoo.zookeeper.server.auth.ProviderRegistry;
- * This class handles communication with clients using NIO. There is one per
- * client, but only one thread doing the communication.
-public class NIOServerCnxn implements Watcher, ServerCnxn {
- private static final Logger LOG = Logger.getLogger(NIOServerCnxn.class);
- static public class Factory extends Thread {
- ServerSocketChannel ss;
- * We use this buffer to do efficient socket I/O. Since there is a single
- * sender thread per NIOServerCnxn instance, we can use a member variable to
- * only allocate it once.
- ByteBuffer directBuffer = ByteBuffer.allocateDirect(64 * 1024);
- HashSet<NIOServerCnxn> cnxns = new HashSet<NIOServerCnxn>();
- int outstandingLimit = 1;
- public Factory(int port) throws IOException {
- super("NIOServerCxn.Factory");
- this.ss = ServerSocketChannel.open();
- ss.socket().bind(new InetSocketAddress(port));
- ss.configureBlocking(false);
- ss.register(selector, SelectionKey.OP_ACCEPT);
- start();
- public void startup(ZooKeeperServer zks) throws IOException,
- zks.startup();
- setZooKeeperServer(zks);
- public void setZooKeeperServer(ZooKeeperServer zks) {
- if (zks != null) {
- this.outstandingLimit = zks.getGlobalOutstandingLimit();
- zks.setServerCnxnFactory(this);
- this.outstandingLimit = 1;
- public InetSocketAddress getLocalAddress(){
- return (InetSocketAddress)ss.socket().getLocalSocketAddress();
- private void addCnxn(NIOServerCnxn cnxn) {
- synchronized (cnxns) {
- cnxns.add(cnxn);
- protected NIOServerCnxn createConnection(SocketChannel sock,
- SelectionKey sk) throws IOException {
- return new NIOServerCnxn(zks, sock, sk, this);
- while (!ss.socket().isClosed()) {
- selector.select(1000);
- ArrayList<SelectionKey> selectedList = new ArrayList<SelectionKey>(
- selected);
- Collections.shuffle(selectedList);
- for (SelectionKey k : selectedList) {
- if ((k.readyOps() & SelectionKey.OP_ACCEPT) != 0) {
- SocketChannel sc = ((ServerSocketChannel) k
- .channel()).accept();
- sc.configureBlocking(false);
- SelectionKey sk = sc.register(selector,
- SelectionKey.OP_READ);
- NIOServerCnxn cnxn = createConnection(sc, sk);
- sk.attach(cnxn);
- addCnxn(cnxn);
- NIOServerCnxn c = (NIOServerCnxn) k.attachment();
- c.doIO(k);
- "NIOServerCnxn factory exitedloop.");
- clear();
- LOG.error("=====> Goodbye cruel world <======");
- // System.exit(0);
- * clear all the connections in the selector
- synchronized public void clear() {
- // got to clear all the connections that we have in the selector
- for (Iterator<NIOServerCnxn> it = cnxns.iterator(); it
- .hasNext();) {
- NIOServerCnxn cnxn = it.next();
- it.remove();
- // Do nothing.
- ss.close();
- this.interrupt();
- this.join();
- zks.shutdown();
- synchronized void closeSession(long sessionId) {
- if (cnxn.sessionId == sessionId) {
- * The buffer will cause the connection to be close when we do a send.
- static final ByteBuffer closeConn = ByteBuffer.allocate(0);
- Factory factory;
- ZooKeeperServer zk;
- private SocketChannel sock;
- private SelectionKey sk;
- ByteBuffer lenBuffer = ByteBuffer.allocate(4);
- LinkedBlockingQueue<ByteBuffer> outgoingBuffers = new LinkedBlockingQueue<ByteBuffer>();
- int sessionTimeout;
- ArrayList<Id> authInfo = new ArrayList<Id>();
- LinkedList<Request> outstanding = new LinkedList<Request>();
- void sendBuffer(ByteBuffer bb) {
- synchronized (factory) {
- sk.selector().wakeup();
- // ZooLog.logTraceMessage(LOG,
- // ZooLog.CLIENT_DATA_PACKET_TRACE_MASK,
- // "Add a buffer to outgoingBuffers");
- //"sk " + sk + " is valid: " +
- // sk.isValid(), );
- outgoingBuffers.add(bb);
- if (sk.isValid()) {
- sk.interestOps(sk.interestOps() | SelectionKey.OP_WRITE);
- } catch (RuntimeException e) {
- throw e;
- void doIO(SelectionKey k) throws InterruptedException {
- if (k.isReadable()) {
- throw new IOException("Read error");
- readLength(k);
- stats.packetsReceived++;
- ServerStats.getInstance().incrementPacketsReceived();
- readConnectRequest();
- readRequest();
- if (k.isWritable()) {
- // ZooLog.CLIENT_DATA_PACKET_TRACE_MASK
- // "outgoingBuffers.size() = " +
- // outgoingBuffers.size());
- if (outgoingBuffers.size() > 0) {
- // "sk " + k + " is valid: " +
- // k.isValid());
- * This is going to reset the buffer position to 0 and the
- * limit to the size of the buffer, so that we can fill it
- * with data from the non-direct buffers that we need to
- * send.
- ByteBuffer directBuffer = factory.directBuffer;
- directBuffer.clear();
- for (ByteBuffer b : outgoingBuffers) {
- if (directBuffer.remaining() < b.remaining()) {
- * When we call put later, if the directBuffer is to
- * small to hold everything, nothing will be copied,
- * so we've got to slice the buffer if it's too big.
- b = (ByteBuffer) b.slice().limit(
- directBuffer.remaining());
- * put() is going to modify the positions of both
- * buffers, put we don't want to change the position of
- * the source buffers (we'll do that after the send, if
- * needed), so we save and reset the position after the
- * copy
- int p = b.position();
- directBuffer.put(b);
- b.position(p);
- if (directBuffer.remaining() == 0) {
- * Do the flip: limit becomes position, position gets set to
- * 0. This sets us up for the write.
- directBuffer.flip();
- int sent = sock.write(directBuffer);
- // Remove the buffers that we have sent
- while (outgoingBuffers.size() > 0) {
- bb = outgoingBuffers.peek();
- if (bb == closeConn) {
- throw new IOException("closing");
- int left = bb.remaining() - sent;
- if (left > 0) {
- * We only partially sent this buffer, so we update
- * the position and exit the loop.
- bb.position(bb.position() + sent);
- stats.packetsSent++;
- /* We've sent the whole buffer, so drop the buffer */
- sent -= bb.remaining();
- ServerStats.getInstance().incrementPacketsSent();
- outgoingBuffers.remove();
- // ZooLog.CLIENT_DATA_PACKET_TRACE_MASK, "after send,
- // outgoingBuffers.size() = " + outgoingBuffers.size());
- if (outgoingBuffers.size() == 0) {
- if (!initialized
- && (sk.interestOps() & SelectionKey.OP_READ) == 0) {
- throw new IOException("Responded to info probe");
- sk.interestOps(sk.interestOps()
- & (~SelectionKey.OP_WRITE));
- | SelectionKey.OP_WRITE);
- } catch (CancelledKeyException e) {
- close();
- // LOG.error("FIXMSG",e);
- private void readRequest() throws IOException {
- // We have the request, now process and setup for next
- InputStream bais = new ByteBufferInputStream(incomingBuffer);
- BinaryInputArchive bia = BinaryInputArchive.getArchive(bais);
- h.deserialize(bia, "header");
- // Through the magic of byte buffers, txn will not be
- // pointing
- // to the start of the txn
- incomingBuffer = incomingBuffer.slice();
- if (h.getType() == OpCode.auth) {
- AuthPacket authPacket = new AuthPacket();
- ZooKeeperServer.byteBuffer2Record(incomingBuffer, authPacket);
- String scheme = authPacket.getScheme();
- AuthenticationProvider ap = ProviderRegistry.getProvider(scheme);
- if (ap == null
- || ap.handleAuthentication(this, authPacket.getAuth()) != KeeperException.Code.Ok) {
- if (ap == null)
- LOG.error("No authentication provider for scheme: "
- + scheme);
- else
- LOG.error("Authentication failed for scheme: "
- // send a response...
- ReplyHeader rh = new ReplyHeader(h.getXid(), 0,
- KeeperException.Code.AuthFailed);
- sendResponse(rh, null, null);
- // ... and close connection
- sendBuffer(NIOServerCnxn.closeConn);
- disableRecv();
- LOG.error("Authentication succeeded for scheme: "
- KeeperException.Code.Ok);
- zk.submitRequest(this, sessionId, h.getType(), h.getXid(),
- incomingBuffer, authInfo);
- if (h.getXid() >= 0) {
- outstandingRequests++;
- // check throttling
- if (zk.getInProcess() > factory.outstandingLimit) {
- // following lines should not be needed since we are already
- // reading
- // } else {
- // enableRecv();
- public void disableRecv() {
- sk.interestOps(sk.interestOps() & (~SelectionKey.OP_READ));
- public void enableRecv() {
- int interest = sk.interestOps();
- if ((interest & SelectionKey.OP_READ) == 0) {
- sk.interestOps(interest | SelectionKey.OP_READ);
- private void readConnectRequest() throws IOException, InterruptedException {
- BinaryInputArchive bia = BinaryInputArchive
- .getArchive(new ByteBufferInputStream(incomingBuffer));
- ConnectRequest connReq = new ConnectRequest();
- connReq.deserialize(bia, "connect");
- LOG.warn("Connected to " + sock.socket().getRemoteSocketAddress()
- + " lastZxid " + connReq.getLastZxidSeen());
- if (zk == null) {
- throw new IOException("ZooKeeperServer not running");
- if (connReq.getLastZxidSeen() > zk.dataTree.lastProcessedZxid) {
- LOG.error("Client has seen "
- + Long.toHexString(connReq.getLastZxidSeen())
- + " our last zxid is "
- + Long.toHexString(zk.dataTree.lastProcessedZxid));
- throw new IOException("We are out of date");
- sessionTimeout = connReq.getTimeOut();
- byte passwd[] = connReq.getPasswd();
- if (sessionTimeout < zk.tickTime * 2) {
- sessionTimeout = zk.tickTime * 2;
- if (sessionTimeout > zk.tickTime * 20) {
- sessionTimeout = zk.tickTime * 20;
- // We don't want to receive any packets until we are sure that the
- // session is setup
- if (connReq.getSessionId() != 0) {
- setSessionId(connReq.getSessionId());
- zk.reopenSession(this, sessionId, passwd, sessionTimeout);
- LOG.warn("Renewing session " + Long.toHexString(sessionId));
- zk.createSession(this, passwd, sessionTimeout);
- LOG.warn("Creating new session "
- + Long.toHexString(sessionId));
- private void readLength(SelectionKey k) throws IOException {
- // Read the length, now get the buffer
- int len = lenBuffer.getInt();
- if (!initialized) {
- // We take advantage of the limited size of the length to look
- // for cmds. They are all 4-bytes which fits inside of an int
- if (len == ruokCmd) {
- sendBuffer(imok.duplicate());
- k.interestOps(SelectionKey.OP_WRITE);
- } else if (len == killCmd) {
- } else if (len == getTraceMaskCmd) {
- long traceMask = ZooTrace.getTextTraceLevel();
- ByteBuffer resp = ByteBuffer.allocate(8);
- resp.putLong(traceMask);
- resp.flip();
- sendBuffer(resp);
- } else if (len == setTraceMaskCmd) {
- incomingBuffer = ByteBuffer.allocate(8);
- System.out.println("rc=" + rc);
- long traceMask = incomingBuffer.getLong();
- ZooTrace.setTextTraceLevel(traceMask);
- } else if (len == dumpCmd) {
- sendBuffer(ByteBuffer.wrap("ZooKeeper not active \n"
- .getBytes()));
- sb.append("SessionTracker dump: \n");
- sb.append(zk.sessionTracker.toString()).append("\n");
- sb.append("ephemeral nodes dump:\n");
- sb.append(zk.dataTree.dumpEphemerals()).append("\n");
- sendBuffer(ByteBuffer.wrap(sb.toString().getBytes()));
- } else if (len == reqsCmd) {
- sb.append("Requests:\n");
- synchronized (outstanding) {
- for (Request r : outstanding) {
- sb.append(r.toString());
- sb.append('\n');
- } else if (len == statCmd) {
- if(zk!=null){
- sb.append("Zookeeper version: ").append(Version.getFullVersion())
- sb.append("Clients:\n");
- synchronized(factory.cnxns){
- for(NIOServerCnxn c : factory.cnxns){
- sb.append(c.getStats().toString());
- sb.append("\n");
- sb.append(ServerStats.getInstance().toString());
- sb.append("Node count: ").append(zk.dataTree.getNodeCount()).
- append("\n");
- }else
- sb.append("ZooKeeperServer not running\n");
- if (len < 0 || len > BinaryInputArchive.maxBuffer) {
- throw new IOException("Len error " + len);
- * The number of requests that have been submitted but not yet responded to.
- int outstandingRequests;
- * (non-Javadoc)
- * @see com.yahoo.zookeeper.server.ServerCnxnIface#getSessionTimeout()
- public int getSessionTimeout() {
- return sessionTimeout;
- * This is the id that uniquely identifies the session of a client. Once
- * this session is no longer active, the ephemeral nodes will go away.
- long sessionId;
- static long nextSessionId = 1;
- public NIOServerCnxn(ZooKeeperServer zk, SocketChannel sock,
- SelectionKey sk, Factory factory) throws IOException {
- this.zk = zk;
- this.sock = sock;
- this.sk = sk;
- this.factory = factory;
- sock.socket().setSoLinger(true, 2);
- InetAddress addr = ((InetSocketAddress) sock.socket()
- .getRemoteSocketAddress()).getAddress();
- authInfo.add(new Id("ip", addr.getHostAddress()));
- authInfo.add(new Id("host", addr.getCanonicalHostName()));
- sk.interestOps(SelectionKey.OP_READ);
- return "NIOServerCnxn object with sock = " + sock + " and sk = " + sk;
- boolean closed;
- * @see com.yahoo.zookeeper.server.ServerCnxnIface#close()
- if (closed) {
- closed = true;
- synchronized (factory.cnxns) {
- factory.cnxns.remove(this);
- if (zk != null) {
- zk.removeCnxn(this);
- ZooTrace.logTraceMessage(LOG, ZooTrace.SESSION_TRACE_MASK,
- "close NIOServerCnxn: " + sock);
- * The following sequence of code is stupid! You would think that
- * only sock.close() is needed, but alas, it doesn't work that way.
- * If you just do sock.close() there are cases where the socket
- * doesn't actually close...
- // This is a relatively common exception that we can't avoid
- // XXX The next line doesn't seem to be needed, but some posts
- // to forums suggest that it is needed. Keep in mind if errors in
- // this section arise.
- // factory.selector.wakeup();
- sock = null;
- if (sk != null) {
- // need to cancel this selection key from the selector
- sk.cancel();
- private final static byte fourBytes[] = new byte[4];
- * @see com.yahoo.zookeeper.server.ServerCnxnIface#sendResponse(com.yahoo.zookeeper.proto.ReplyHeader,
- * com.yahoo.jute.Record, java.lang.String)
- synchronized public void sendResponse(ReplyHeader h, Record r, String tag) {
- // Make space for length
- BinaryOutputArchive bos = BinaryOutputArchive.getArchive(baos);
- baos.write(fourBytes);
- bos.writeRecord(h, "header");
- if (r != null) {
- bos.writeRecord(r, tag);
- LOG.error("Error serializing response");
- byte b[] = baos.toByteArray();
- ByteBuffer bb = ByteBuffer.wrap(b);
- bb.putInt(b.length - 4).rewind();
- sendBuffer(bb);
- if (h.getXid() > 0) {
- synchronized (this.factory) {
- outstandingRequests--;
- if (zk.getInProcess() < factory.outstandingLimit
- || outstandingRequests < 1) {
- enableRecv();
- * @see com.yahoo.zookeeper.server.ServerCnxnIface#process(com.yahoo.zookeeper.proto.WatcherEvent)
- synchronized public void process(WatcherEvent event) {
- ReplyHeader h = new ReplyHeader(-1, -1L, 0);
- ZooTrace.logTraceMessage(LOG, ZooTrace.EVENT_DELIVERY_TRACE_MASK,
- "Deliver event " + event + " to "
- + this.sessionId + " through " + this);
- sendResponse(h, event, "notification");
- public void finishSessionInit(boolean valid) {
- ConnectResponse rsp = new ConnectResponse(0, valid ? sessionTimeout
- : 0, valid ? sessionId : 0, // send 0 if session is no
- // longer valid
- valid ? zk.generatePasswd(sessionId) : new byte[16]);
- bos.writeInt(-1, "len");
- rsp.serialize(bos, "connect");
- bb.putInt(bb.remaining() - 4).rewind();
- LOG.warn("Finished init of " + Long.toHexString(sessionId)
- + ": " + valid);
- if (!valid) {
- sendBuffer(closeConn);
- // Now that the session is ready we can start receiving packets
- * @see com.yahoo.zookeeper.server.ServerCnxnIface#getSessionId()
- public void setSessionId(long sessionId) {
- public ArrayList<Id> getAuthInfo() {
- return authInfo;
- public InetSocketAddress getRemoteAddress() {
- return (InetSocketAddress) sock.socket().getRemoteSocketAddress();
- private class CnxnStats implements ServerCnxn.Stats{
- long packetsReceived;
- long packetsSent;
- public long getOutstandingRequests() {
- return outstandingRequests;
- public long getPacketsReceived() {
- return packetsReceived;
- public long getPacketsSent() {
- return packetsSent;
- public String toString(){
- StringBuilder sb=new StringBuilder();
- Channel channel = sk.channel();
- if (channel instanceof SocketChannel) {
- sb.append(" ").append(((SocketChannel)channel).socket()
- .getRemoteSocketAddress())
- .append("[").append(Integer.toHexString(sk.interestOps()))
- .append("](queued=").append(getOutstandingRequests())
- .append(",recved=").append(getPacketsReceived())
- .append(",sent=").append(getPacketsSent()).append(")\n");
- private CnxnStats stats=new CnxnStats();
- public Stats getStats() {
- return stats;
+import java.nio.channels.CancelledKeyException;
+import java.nio.channels.Channel;
+import java.nio.channels.ServerSocketChannel;
+import java.util.HashSet;
+import java.util.Iterator;
+import com.yahoo.zookeeper.Version;
+import com.yahoo.zookeeper.Watcher;
+import com.yahoo.zookeeper.server.auth.AuthenticationProvider;
+import com.yahoo.zookeeper.server.auth.ProviderRegistry;
+ * This class handles communication with clients using NIO. There is one per
+ * client, but only one thread doing the communication.
+public class NIOServerCnxn implements Watcher, ServerCnxn {
+ private static final Logger LOG = Logger.getLogger(NIOServerCnxn.class);
+ static public class Factory extends Thread {
+ ServerSocketChannel ss;
+ Selector selector = Selector.open();
+ * We use this buffer to do efficient socket I/O. Since there is a single
+ * sender thread per NIOServerCnxn instance, we can use a member variable to
+ * only allocate it once.
+ ByteBuffer directBuffer = ByteBuffer.allocateDirect(64 * 1024);
+ HashSet<NIOServerCnxn> cnxns = new HashSet<NIOServerCnxn>();
+ int outstandingLimit = 1;
+ public Factory(int port) throws IOException {
+ super("NIOServerCxn.Factory");
+ this.ss = ServerSocketChannel.open();
+ ss.socket().bind(new InetSocketAddress(port));
+ ss.configureBlocking(false);
+ ss.register(selector, SelectionKey.OP_ACCEPT);
+ start();
+ public void startup(ZooKeeperServer zks) throws IOException,
+ zks.startup();
+ setZooKeeperServer(zks);
+ public void setZooKeeperServer(ZooKeeperServer zks) {
+ if (zks != null) {
+ this.outstandingLimit = zks.getGlobalOutstandingLimit();
+ zks.setServerCnxnFactory(this);
+ this.outstandingLimit = 1;
+ public InetSocketAddress getLocalAddress(){
+ return (InetSocketAddress)ss.socket().getLocalSocketAddress();
+ private void addCnxn(NIOServerCnxn cnxn) {
+ synchronized (cnxns) {
+ cnxns.add(cnxn);
+ protected NIOServerCnxn createConnection(SocketChannel sock,
+ SelectionKey sk) throws IOException {
+ return new NIOServerCnxn(zks, sock, sk, this);
+ while (!ss.socket().isClosed()) {
+ selector.select(1000);
+ ArrayList<SelectionKey> selectedList = new ArrayList<SelectionKey>(
+ selected);
+ Collections.shuffle(selectedList);
+ for (SelectionKey k : selectedList) {
+ if ((k.readyOps() & SelectionKey.OP_ACCEPT) != 0) {
+ SocketChannel sc = ((ServerSocketChannel) k
+ .channel()).accept();
+ sc.configureBlocking(false);
+ SelectionKey sk = sc.register(selector,
+ SelectionKey.OP_READ);
+ NIOServerCnxn cnxn = createConnection(sc, sk);
+ sk.attach(cnxn);
+ addCnxn(cnxn);
+ NIOServerCnxn c = (NIOServerCnxn) k.attachment();
+ c.doIO(k);
+ LOG.error("FIXMSG",e);
+ "NIOServerCnxn factory exitedloop.");
+ clear();
+ LOG.error("=====> Goodbye cruel world <======");
+ // System.exit(0);
+ * clear all the connections in the selector
+ synchronized public void clear() {
+ // got to clear all the connections that we have in the selector
+ for (Iterator<NIOServerCnxn> it = cnxns.iterator(); it
+ .hasNext();) {
+ NIOServerCnxn cnxn = it.next();
+ it.remove();
+ // Do nothing.
+ ss.close();
+ this.interrupt();
+ this.join();
+ LOG.warn("Interrupted",e);
+ LOG.error("Unexpected exception", e);
+ zks.shutdown();
+ synchronized void closeSession(long sessionId) {
+ if (cnxn.sessionId == sessionId) {
+ * The buffer will cause the connection to be close when we do a send.
+ static final ByteBuffer closeConn = ByteBuffer.allocate(0);
+ Factory factory;
+ ZooKeeperServer zk;
+ private SocketChannel sock;
+ private SelectionKey sk;
+ ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+ LinkedBlockingQueue<ByteBuffer> outgoingBuffers = new LinkedBlockingQueue<ByteBuffer>();
+ int sessionTimeout;
+ ArrayList<Id> authInfo = new ArrayList<Id>();
+ LinkedList<Request> outstanding = new LinkedList<Request>();
+ void sendBuffer(ByteBuffer bb) {
+ synchronized (factory) {
+ sk.selector().wakeup();
+ // ZooLog.logTraceMessage(LOG,
+ // ZooLog.CLIENT_DATA_PACKET_TRACE_MASK,
+ // "Add a buffer to outgoingBuffers");
+ //"sk " + sk + " is valid: " +
+ // sk.isValid(), );
+ outgoingBuffers.add(bb);
+ if (sk.isValid()) {
+ sk.interestOps(sk.interestOps() | SelectionKey.OP_WRITE);
+ } catch (RuntimeException e) {
+ throw e;
+ void doIO(SelectionKey k) throws InterruptedException {
+ if (k.isReadable()) {
+ throw new IOException("Read error");
+ readLength(k);
+ stats.packetsReceived++;
+ ServerStats.getInstance().incrementPacketsReceived();
+ readConnectRequest();
+ readRequest();
+ if (k.isWritable()) {
+ // ZooLog.CLIENT_DATA_PACKET_TRACE_MASK
+ // "outgoingBuffers.size() = " +
+ // outgoingBuffers.size());
+ if (outgoingBuffers.size() > 0) {
+ // "sk " + k + " is valid: " +
+ // k.isValid());
+ * This is going to reset the buffer position to 0 and the
+ * limit to the size of the buffer, so that we can fill it
+ * with data from the non-direct buffers that we need to
+ * send.
+ ByteBuffer directBuffer = factory.directBuffer;
+ directBuffer.clear();
+ for (ByteBuffer b : outgoingBuffers) {
+ if (directBuffer.remaining() < b.remaining()) {
+ * When we call put later, if the directBuffer is to
+ * small to hold everything, nothing will be copied,
+ * so we've got to slice the buffer if it's too big.
+ b = (ByteBuffer) b.slice().limit(
+ directBuffer.remaining());
+ * put() is going to modify the positions of both
+ * buffers, put we don't want to change the position of
+ * the source buffers (we'll do that after the send, if
+ * needed), so we save and reset the position after the
+ * copy
+ int p = b.position();
+ directBuffer.put(b);
+ b.position(p);
+ if (directBuffer.remaining() == 0) {
+ * Do the flip: limit becomes position, position gets set to
+ * 0. This sets us up for the write.
+ directBuffer.flip();
+ int sent = sock.write(directBuffer);
+ // Remove the buffers that we have sent
+ while (outgoingBuffers.size() > 0) {
+ bb = outgoingBuffers.peek();
+ if (bb == closeConn) {
+ throw new IOException("closing");
+ int left = bb.remaining() - sent;
+ if (left > 0) {
+ * We only partially sent this buffer, so we update
+ * the position and exit the loop.
+ bb.position(bb.position() + sent);
+ stats.packetsSent++;
+ /* We've sent the whole buffer, so drop the buffer */
+ sent -= bb.remaining();
+ ServerStats.getInstance().incrementPacketsSent();
+ outgoingBuffers.remove();
+ // ZooLog.CLIENT_DATA_PACKET_TRACE_MASK, "after send,
+ // outgoingBuffers.size() = " + outgoingBuffers.size());
+ if (outgoingBuffers.size() == 0) {
+ if (!initialized
+ && (sk.interestOps() & SelectionKey.OP_READ) == 0) {
+ throw new IOException("Responded to info probe");
+ sk.interestOps(sk.interestOps()
+ & (~SelectionKey.OP_WRITE));
+ | SelectionKey.OP_WRITE);
+ } catch (CancelledKeyException e) {
+ close();
+ // LOG.error("FIXMSG",e);
+ private void readRequest() throws IOException {
+ // We have the request, now process and setup for next
+ InputStream bais = new ByteBufferInputStream(incomingBuffer);
+ BinaryInputArchive bia = BinaryInputArchive.getArchive(bais);
+ h.deserialize(bia, "header");
+ // Through the magic of byte buffers, txn will not be
+ // pointing
+ // to the start of the txn
+ incomingBuffer = incomingBuffer.slice();
+ if (h.getType() == OpCode.auth) {
+ AuthPacket authPacket = new AuthPacket();
+ ZooKeeperServer.byteBuffer2Record(incomingBuffer, authPacket);
+ String scheme = authPacket.getScheme();
+ AuthenticationProvider ap = ProviderRegistry.getProvider(scheme);
+ if (ap == null
+ || ap.handleAuthentication(this, authPacket.getAuth()) != KeeperException.Code.Ok) {
+ if (ap == null)
+ LOG.error("No authentication provider for scheme: "
+ + scheme);
+ else
+ LOG.error("Authentication failed for scheme: "
+ // send a response...
+ ReplyHeader rh = new ReplyHeader(h.getXid(), 0,
+ KeeperException.Code.AuthFailed);
+ sendResponse(rh, null, null);
+ // ... and close connection
+ sendBuffer(NIOServerCnxn.closeConn);
+ disableRecv();
+ LOG.error("Authentication succeeded for scheme: "
+ KeeperException.Code.Ok);
+ zk.submitRequest(this, sessionId, h.getType(), h.getXid(),
+ incomingBuffer, authInfo);
+ if (h.getXid() >= 0) {
+ outstandingRequests++;
+ // check throttling
+ if (zk.getInProcess() > factory.outstandingLimit) {
+ // following lines should not be needed since we are already
+ // reading
+ // } else {
+ // enableRecv();
+ public void disableRecv() {
+ sk.interestOps(sk.interestOps() & (~SelectionKey.OP_READ));
+ public void enableRecv() {
+ int interest = sk.interestOps();
+ if ((interest & SelectionKey.OP_READ) == 0) {
+ sk.interestOps(interest | SelectionKey.OP_READ);
+ private void readConnectRequest() throws IOException, InterruptedException {
+ BinaryInputArchive bia = BinaryInputArchive
+ .getArchive(new ByteBufferInputStream(incomingBuffer));
+ ConnectRequest connReq = new ConnectRequest();
+ connReq.deserialize(bia, "connect");
+ LOG.warn("Connected to " + sock.socket().getRemoteSocketAddress()
+ + " lastZxid " + connReq.getLastZxidSeen());
+ if (zk == null) {
+ throw new IOException("ZooKeeperServer not running");
+ if (connReq.getLastZxidSeen() > zk.dataTree.lastProcessedZxid) {
+ LOG.error("Client has seen "
+ + Long.toHexString(connReq.getLastZxidSeen())
+ + " our last zxid is "
+ + Long.toHexString(zk.dataTree.lastProcessedZxid));
+ throw new IOException("We are out of date");
+ sessionTimeout = connReq.getTimeOut();
+ byte passwd[] = connReq.getPasswd();
+ if (sessionTimeout < zk.tickTime * 2) {
+ sessionTimeout = zk.tickTime * 2;
+ if (sessionTimeout > zk.tickTime * 20) {
+ sessionTimeout = zk.tickTime * 20;
+ // We don't want to receive any packets until we are sure that the
+ // session is setup
+ if (connReq.getSessionId() != 0) {
+ setSessionId(connReq.getSessionId());
+ zk.reopenSession(this, sessionId, passwd, sessionTimeout);
+ LOG.warn("Renewing session " + Long.toHexString(sessionId));
+ zk.createSession(this, passwd, sessionTimeout);
+ LOG.warn("Creating new session "
+ + Long.toHexString(sessionId));
+ private void readLength(SelectionKey k) throws IOException {
+ // Read the length, now get the buffer
+ int len = lenBuffer.getInt();
+ if (!initialized) {
+ // We take advantage of the limited size of the length to look
+ // for cmds. They are all 4-bytes which fits inside of an int
+ if (len == ruokCmd) {
+ sendBuffer(imok.duplicate());
+ k.interestOps(SelectionKey.OP_WRITE);
+ } else if (len == killCmd) {
+ } else if (len == getTraceMaskCmd) {
+ long traceMask = ZooTrace.getTextTraceLevel();
+ ByteBuffer resp = ByteBuffer.allocate(8);
+ resp.putLong(traceMask);
+ resp.flip();
+ sendBuffer(resp);
+ } else if (len == setTraceMaskCmd) {
+ incomingBuffer = ByteBuffer.allocate(8);
+ System.out.println("rc=" + rc);
+ long traceMask = incomingBuffer.getLong();
+ ZooTrace.setTextTraceLevel(traceMask);
+ } else if (len == dumpCmd) {
+ sendBuffer(ByteBuffer.wrap("ZooKeeper not active \n"
+ .getBytes()));
+ sb.append("SessionTracker dump: \n");
+ sb.append(zk.sessionTracker.toString()).append("\n");
+ sb.append("ephemeral nodes dump:\n");
+ sb.append(zk.dataTree.dumpEphemerals()).append("\n");
+ sendBuffer(ByteBuffer.wrap(sb.toString().getBytes()));
+ } else if (len == reqsCmd) {
+ sb.append("Requests:\n");
+ synchronized (outstanding) {
+ for (Request r : outstanding) {
+ sb.append(r.toString());
+ sb.append('\n');
+ } else if (len == statCmd) {
+ if(zk!=null){
+ sb.append("Zookeeper version: ").append(Version.getFullVersion())
+ sb.append("Clients:\n");
+ synchronized(factory.cnxns){
+ for(NIOServerCnxn c : factory.cnxns){
+ sb.append(c.getStats().toString());
+ sb.append("\n");
+ sb.append(ServerStats.getInstance().toString());
+ sb.append("Node count: ").append(zk.dataTree.getNodeCount()).
+ append("\n");
+ }else
+ sb.append("ZooKeeperServer not running\n");
+ if (len < 0 || len > BinaryInputArchive.maxBuffer) {
+ throw new IOException("Len error " + len);
+ * The number of requests that have been submitted but not yet responded to.
+ int outstandingRequests;
+ * (non-Javadoc)
+ * @see com.yahoo.zookeeper.server.ServerCnxnIface#getSessionTimeout()
+ public int getSessionTimeout() {
+ return sessionTimeout;
+ * This is the id that uniquely identifies the session of a client. Once
+ * this session is no longer active, the ephemeral nodes will go away.
+ long sessionId;
+ static long nextSessionId = 1;
+ public NIOServerCnxn(ZooKeeperServer zk, SocketChannel sock,
+ SelectionKey sk, Factory factory) throws IOException {
+ this.zk = zk;
+ this.sock = sock;
+ this.sk = sk;
+ this.factory = factory;
+ sock.socket().setSoLinger(true, 2);
+ InetAddress addr = ((InetSocketAddress) sock.socket()
+ .getRemoteSocketAddress()).getAddress();
+ authInfo.add(new Id("ip", addr.getHostAddress()));
+ authInfo.add(new Id("host", addr.getCanonicalHostName()));
+ sk.interestOps(SelectionKey.OP_READ);
+ return "NIOServerCnxn object with sock = " + sock + " and sk = " + sk;
+ boolean closed;
+ * @see com.yahoo.zookeeper.server.ServerCnxnIface#close()
+ if (closed) {
+ closed = true;
+ synchronized (factory.cnxns) {
+ factory.cnxns.remove(this);
+ if (zk != null) {
+ zk.removeCnxn(this);
+ ZooTrace.logTraceMessage(LOG, ZooTrace.SESSION_TRACE_MASK,
+ "close NIOServerCnxn: " + sock);
+ * The following sequence of code is stupid! You would think that
+ * only sock.close() is needed, but alas, it doesn't work that way.
+ * If you just do sock.close() there are cases where the socket
+ * doesn't actually close...
+ // This is a relatively common exception that we can't avoid
+ // XXX The next line doesn't seem to be needed, but some posts
+ // to forums suggest that it is needed. Keep in mind if errors in
+ // this section arise.
+ // factory.selector.wakeup();
+ sock = null;
+ if (sk != null) {
+ // need to cancel this selection key from the selector
+ sk.cancel();
+ private final static byte fourBytes[] = new byte[4];
+ * @see com.yahoo.zookeeper.server.ServerCnxnIface#sendResponse(com.yahoo.zookeeper.proto.ReplyHeader,
+ * com.yahoo.jute.Record, java.lang.String)
+ synchronized public void sendResponse(ReplyHeader h, Record r, String tag) {
+ // Make space for length
+ BinaryOutputArchive bos = BinaryOutputArchive.getArchive(baos);
+ baos.write(fourBytes);
+ bos.writeRecord(h, "header");
+ if (r != null) {
+ bos.writeRecord(r, tag);
+ LOG.error("Error serializing response");
+ byte b[] = baos.toByteArray();
+ ByteBuffer bb = ByteBuffer.wrap(b);
+ bb.putInt(b.length - 4).rewind();
+ sendBuffer(bb);
+ if (h.getXid() > 0) {
+ synchronized (this.factory) {
+ outstandingRequests--;
+ if (zk.getInProcess() < factory.outstandingLimit
+ || outstandingRequests < 1) {
+ enableRecv();
+ * @see com.yahoo.zookeeper.server.ServerCnxnIface#process(com.yahoo.zookeeper.proto.WatcherEvent)
+ synchronized public void process(WatcherEvent event) {
+ ReplyHeader h = new ReplyHeader(-1, -1L, 0);
+ ZooTrace.logTraceMessage(LOG, ZooTrace.EVENT_DELIVERY_TRACE_MASK,
+ "Deliver event " + event + " to "
+ + this.sessionId + " through " + this);
+ sendResponse(h, event, "notification");
+ public void finishSessionInit(boolean valid) {
+ ConnectResponse rsp = new ConnectResponse(0, valid ? sessionTimeout
+ : 0, valid ? sessionId : 0, // send 0 if session is no
+ // longer valid
+ valid ? zk.generatePasswd(sessionId) : new byte[16]);
+ bos.writeInt(-1, "len");
+ rsp.serialize(bos, "connect");
+ bb.putInt(bb.remaining() - 4).rewind();
+ LOG.warn("Finished init of " + Long.toHexString(sessionId)
+ + ": " + valid);
+ if (!valid) {
+ sendBuffer(closeConn);
+ // Now that the session is ready we can start receiving packets
+ * @see com.yahoo.zookeeper.server.ServerCnxnIface#getSessionId()
+ public void setSessionId(long sessionId) {
+ public ArrayList<Id> getAuthInfo() {
+ return authInfo;
+ public InetSocketAddress getRemoteAddress() {
+ return (InetSocketAddress) sock.socket().getRemoteSocketAddress();
+ private class CnxnStats implements ServerCnxn.Stats{
+ long packetsReceived;
+ long packetsSent;
+ public long getOutstandingRequests() {
+ return outstandingRequests;
+ public long getPacketsReceived() {
+ return packetsReceived;
+ public long getPacketsSent() {
+ return packetsSent;
+ public String toString(){
+ StringBuilder sb=new StringBuilder();
+ Channel channel = sk.channel();
+ if (channel instanceof SocketChannel) {
+ sb.append(" ").append(((SocketChannel)channel).socket()
+ .getRemoteSocketAddress())
+ .append("[").append(Integer.toHexString(sk.interestOps()))
+ .append("](queued=").append(getOutstandingRequests())
+ .append(",recved=").append(getPacketsReceived())
+ .append(",sent=").append(getPacketsSent()).append(")\n");
+ private CnxnStats stats=new CnxnStats();
+ public Stats getStats() {
+ return stats;
@@ -1,445 +1,445 @@
-import com.yahoo.zookeeper.ZooDefs.CreateFlags;
-import com.yahoo.zookeeper.server.ZooKeeperServer.ChangeRecord;
-import com.yahoo.zookeeper.txn.CreateTxn;
-import com.yahoo.zookeeper.txn.DeleteTxn;
-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;
- public PrepRequestProcessor(ZooKeeperServer zks,
- RequestProcessor nextProcessor) {
- super("ProcessThread");
- this.nextProcessor = nextProcessor;
- Request request = submittedRequests.take();
- traceMask = ZooTrace.CLIENT_PING_TRACE_MASK;
- ZooTrace.logRequest(LOG, traceMask, 'P', request, "");
- if (Request.requestOfDeath == request) {
- pRequest(request);
- "PrepRequestProcessor exited loop!");
- ChangeRecord getRecordForPath(String path) throws KeeperException {
- ChangeRecord lastChange = null;
- 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) {
- zks.outstandingChanges.add(c);
- static void checkACL(ZooKeeperServer zks, ArrayList<ACL> acl, int perm,
- ArrayList<Id> ids) throws KeeperException {
- if (skipACL) {
- if (acl == null || acl.size() == 0) {
- Id id = a.getId();
- if ((a.getPerms() & perm) != 0) {
- if (id.getScheme().equals("world")
- && id.getId().equals("anyone")) {
- AuthenticationProvider ap = ProviderRegistry.getProvider(id
- .getScheme());
- if (ap != null) {
- for (Id authId : ids) {
- if (authId.getScheme().equals("super")) {
- if (authId.getScheme().equals(id.getScheme())
- && ap.matches(authId.getId(), id.getId())) {
- 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
- protected void pRequest(Request request) {
- // LOG.warn("Prep>>> cxid = " + request.cxid + " type = " +
- // request.type + " id = " + request.sessionId);
- TxnHeader txnHeader = null;
- Record txn = null;
- txnHeader = new TxnHeader(request.sessionId, request.cxid, zks
- .getNextZxid(), zks.getTime(), OpCode.create);
- zks.sessionTracker.checkSession(request.sessionId);
- CreateRequest createRequest = new CreateRequest();
- createRequest);
- String path = createRequest.getPath();
- int lastSlash = path.lastIndexOf('/');
- if (lastSlash == -1 || path.indexOf('\0') != -1) {
- if (!fixupACL(request.authInfo, createRequest.getAcl())) {
- String parentPath = path.substring(0, lastSlash);
- ChangeRecord parentRecord = getRecordForPath(parentPath);
- checkACL(zks, parentRecord.acl, ZooDefs.Perms.CREATE,
- int parentCVersion = parentRecord.stat.getCversion();
- if ((createRequest.getFlags() & CreateFlags.SEQUENCE) != 0) {
- path = path + parentCVersion;
- if (getRecordForPath(path) != null) {
- throw new KeeperException(Code.NodeExists);
- if (e.getCode() != Code.NoNode) {
- 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()));
- .getNextZxid(), zks.getTime(), OpCode.delete);
- DeleteRequest deleteRequest = new DeleteRequest();
- deleteRequest);
- path = deleteRequest.getPath();
- lastSlash = path.lastIndexOf('/');
- if (lastSlash == -1 || path.indexOf('\0') != -1
- || path.equals("/")) {
- parentPath = path.substring(0, lastSlash);
- parentRecord = getRecordForPath(parentPath);
- ChangeRecord nodeRecord = getRecordForPath(path);
- checkACL(zks, parentRecord.acl, ZooDefs.Perms.DELETE,
- 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.childCount--;
- addChangeRecord(new ChangeRecord(txnHeader.getZxid(), path,
- null, -1, null));
- .getNextZxid(), zks.getTime(), OpCode.setData);
- SetDataRequest setDataRequest = new SetDataRequest();
- setDataRequest);
- path = setDataRequest.getPath();
- nodeRecord = getRecordForPath(path);
- checkACL(zks, nodeRecord.acl, ZooDefs.Perms.WRITE,
- version = setDataRequest.getVersion();
- int currentVersion = nodeRecord.stat.getVersion();
- if (version != -1 && version != currentVersion) {
- version = currentVersion + 1;
- txn = new SetDataTxn(path, setDataRequest.getData(), version);
- nodeRecord = nodeRecord.duplicate(txnHeader.getZxid());
- nodeRecord.stat.setVersion(version);
- addChangeRecord(nodeRecord);
- .getNextZxid(), zks.getTime(), OpCode.setACL);
- SetACLRequest setAclRequest = new SetACLRequest();
- if (!fixupACL(request.authInfo, setAclRequest.getAcl())) {
- setAclRequest);
- path = setAclRequest.getPath();
- checkACL(zks, nodeRecord.acl, ZooDefs.Perms.ADMIN,
- version = setAclRequest.getVersion();
- currentVersion = nodeRecord.stat.getAversion();
- txn = new SetACLTxn(path, setAclRequest.getAcl(), version);
- nodeRecord.stat.setAversion(version);
- .getNextZxid(), zks.getTime(), OpCode.createSession);
- request.request.rewind();
- int to = request.request.getInt();
- txn = new CreateSessionTxn(to);
- zks.sessionTracker.addSession(request.sessionId, to);
- .getNextZxid(), zks.getTime(), OpCode.closeSession);
- HashSet<String> es = zks.dataTree
- .getEphemerals(request.sessionId);
- 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.warn("Processed session termination request for id: "
- + Long.toHexString(request.sessionId));
- if (txnHeader != null) {
- txnHeader.setType(OpCode.error);
- txn = new ErrorTxn(e.getCode());
- LOG.warn("*********************************" + request);
- if(bb!=null){
- sb.append("request buffer is null");
- txn = new ErrorTxn(Code.MarshallingError);
- request.hdr = txnHeader;
- request.txn = txn;
- 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) {
- return true;
- Iterator<ACL> it = acl.iterator();
- LinkedList<ACL> toAdd = null;
- while (it.hasNext()) {
- ACL a = it.next();
- if (id.getScheme().equals("world") && id.getId().equals("anyone")) {
- } else if (id.getScheme().equals("auth")) {
- 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));
- if (!ap.isValid(id.getId())) {
- if (toAdd != null) {
- for (ACL a : toAdd) {
- acl.add(a);
- // request.addRQRec(">prep="+zks.outstandingChanges.size());
- submittedRequests.add(request);
- submittedRequests.clear();
- submittedRequests.add(Request.requestOfDeath);
- nextProcessor.shutdown();
+import com.yahoo.zookeeper.ZooDefs.CreateFlags;
+import com.yahoo.zookeeper.server.ZooKeeperServer.ChangeRecord;
+import com.yahoo.zookeeper.txn.CreateTxn;
+import com.yahoo.zookeeper.txn.DeleteTxn;
+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;
+ public PrepRequestProcessor(ZooKeeperServer zks,
+ RequestProcessor nextProcessor) {
+ super("ProcessThread");
+ this.nextProcessor = nextProcessor;
+ Request request = submittedRequests.take();
+ traceMask = ZooTrace.CLIENT_PING_TRACE_MASK;
+ ZooTrace.logRequest(LOG, traceMask, 'P', request, "");
+ if (Request.requestOfDeath == request) {
+ pRequest(request);
+ "PrepRequestProcessor exited loop!");
+ ChangeRecord getRecordForPath(String path) throws KeeperException {
+ ChangeRecord lastChange = null;
+ 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) {
+ zks.outstandingChanges.add(c);
+ static void checkACL(ZooKeeperServer zks, ArrayList<ACL> acl, int perm,
+ ArrayList<Id> ids) throws KeeperException {
+ if (skipACL) {
+ if (acl == null || acl.size() == 0) {
+ Id id = a.getId();
+ if ((a.getPerms() & perm) != 0) {
+ if (id.getScheme().equals("world")
+ && id.getId().equals("anyone")) {
+ AuthenticationProvider ap = ProviderRegistry.getProvider(id
+ .getScheme());
+ if (ap != null) {
+ for (Id authId : ids) {
+ if (authId.getScheme().equals("super")) {
+ if (authId.getScheme().equals(id.getScheme())
+ && ap.matches(authId.getId(), id.getId())) {
+ 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
+ protected void pRequest(Request request) {
+ // LOG.info("Prep>>> cxid = " + request.cxid + " type = " +
+ // request.type + " id = " + request.sessionId);
+ TxnHeader txnHeader = null;
+ Record txn = null;
+ txnHeader = new TxnHeader(request.sessionId, request.cxid, zks
+ .getNextZxid(), zks.getTime(), OpCode.create);
+ zks.sessionTracker.checkSession(request.sessionId);
+ CreateRequest createRequest = new CreateRequest();
+ createRequest);
+ String path = createRequest.getPath();
+ int lastSlash = path.lastIndexOf('/');
+ if (lastSlash == -1 || path.indexOf('\0') != -1) {
+ if (!fixupACL(request.authInfo, createRequest.getAcl())) {
+ String parentPath = path.substring(0, lastSlash);
+ ChangeRecord parentRecord = getRecordForPath(parentPath);
+ checkACL(zks, parentRecord.acl, ZooDefs.Perms.CREATE,
+ int parentCVersion = parentRecord.stat.getCversion();
+ if ((createRequest.getFlags() & CreateFlags.SEQUENCE) != 0) {
+ path = path + parentCVersion;
+ if (getRecordForPath(path) != null) {
+ throw new KeeperException(Code.NodeExists);
+ if (e.getCode() != Code.NoNode) {
+ 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()));
+ .getNextZxid(), zks.getTime(), OpCode.delete);
+ DeleteRequest deleteRequest = new DeleteRequest();
+ deleteRequest);
+ path = deleteRequest.getPath();
+ lastSlash = path.lastIndexOf('/');
+ if (lastSlash == -1 || path.indexOf('\0') != -1
+ || path.equals("/")) {
+ parentPath = path.substring(0, lastSlash);
+ parentRecord = getRecordForPath(parentPath);
+ ChangeRecord nodeRecord = getRecordForPath(path);
+ checkACL(zks, parentRecord.acl, ZooDefs.Perms.DELETE,
+ 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.childCount--;
+ addChangeRecord(new ChangeRecord(txnHeader.getZxid(), path,
+ null, -1, null));
+ .getNextZxid(), zks.getTime(), OpCode.setData);
+ SetDataRequest setDataRequest = new SetDataRequest();
+ setDataRequest);
+ path = setDataRequest.getPath();
+ nodeRecord = getRecordForPath(path);
+ checkACL(zks, nodeRecord.acl, ZooDefs.Perms.WRITE,
+ version = setDataRequest.getVersion();
+ int currentVersion = nodeRecord.stat.getVersion();
+ if (version != -1 && version != currentVersion) {
+ version = currentVersion + 1;
+ txn = new SetDataTxn(path, setDataRequest.getData(), version);
+ nodeRecord = nodeRecord.duplicate(txnHeader.getZxid());
+ nodeRecord.stat.setVersion(version);
+ addChangeRecord(nodeRecord);
+ .getNextZxid(), zks.getTime(), OpCode.setACL);
+ SetACLRequest setAclRequest = new SetACLRequest();
+ if (!fixupACL(request.authInfo, setAclRequest.getAcl())) {
+ setAclRequest);
+ path = setAclRequest.getPath();
+ checkACL(zks, nodeRecord.acl, ZooDefs.Perms.ADMIN,
+ version = setAclRequest.getVersion();
+ currentVersion = nodeRecord.stat.getAversion();
+ txn = new SetACLTxn(path, setAclRequest.getAcl(), version);
+ nodeRecord.stat.setAversion(version);
+ .getNextZxid(), zks.getTime(), OpCode.createSession);
+ request.request.rewind();
+ int to = request.request.getInt();
+ txn = new CreateSessionTxn(to);
+ zks.sessionTracker.addSession(request.sessionId, to);
+ .getNextZxid(), zks.getTime(), OpCode.closeSession);
+ HashSet<String> es = zks.dataTree
+ .getEphemerals(request.sessionId);
+ 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));
+ if (txnHeader != null) {
+ txnHeader.setType(OpCode.error);
+ txn = new ErrorTxn(e.getCode());
+ LOG.error("*********************************" + request);
+ if(bb!=null){
+ sb.append("request buffer is null");
+ txn = new ErrorTxn(Code.MarshallingError);
+ request.hdr = txnHeader;
+ request.txn = txn;
+ 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) {
+ return true;
+ Iterator<ACL> it = acl.iterator();
+ LinkedList<ACL> toAdd = null;
+ while (it.hasNext()) {
+ ACL a = it.next();
+ if (id.getScheme().equals("world") && id.getId().equals("anyone")) {
+ } else if (id.getScheme().equals("auth")) {
+ 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));
+ if (!ap.isValid(id.getId())) {
+ if (toAdd != null) {
+ for (ACL a : toAdd) {
+ acl.add(a);
+ // request.addRQRec(">prep="+zks.outstandingChanges.size());
+ submittedRequests.add(request);
+ submittedRequests.clear();
+ submittedRequests.add(Request.requestOfDeath);
+ nextProcessor.shutdown();
-import java.io.File;
-import java.io.FileOutputStream;
-import java.nio.channels.FileChannel;
-import com.yahoo.zookeeper.server.util.Profiler;
- * This RequestProcessor logs requests to disk. It batches the requests to do
- * the io efficiently. The request is not passed to the next RequestProcessor
- * until its log has been synced to disk.
-public class SyncRequestProcessor extends Thread implements RequestProcessor {
- private static final Logger LOG = Logger.getLogger(SyncRequestProcessor.class);
- static final int PADDING_TIMEOUT=1000;
- LinkedBlockingQueue<Request> queuedRequests = new LinkedBlockingQueue<Request>();
- static boolean forceSync;
- forceSync = !System.getProperty("zookeeper.forceSync", "yes").equals(
- "no");
- static long preAllocSize = 65536 * 1024;
- String size = System.getProperty("zookeeper.preAllocSize");
- if (size != null) {
- preAllocSize = Long.parseLong(size) * 1024;
- } catch (NumberFormatException e) {
- LOG.warn(size + " is not a valid value for preAllocSize");
- * The number of log entries to log before starting a snapshot
- static public int snapCount = ZooKeeperServer.getSnapCount();
- Thread snapInProcess;
- boolean timeToDie = false;
- public SyncRequestProcessor(ZooKeeperServer zks,
- super("SyncThread");
- * Transactions that have been written and are waiting to be flushed to
- * disk. Basically this is the list of SyncItems whose callbacks will be
- * invoked after flush returns successfully.
- LinkedList<Request> toFlush = new LinkedList<Request>();
- FileOutputStream logStream;
- BinaryOutputArchive logArchive;
- int logCount = 0;
- Request requestOfDeath = Request.requestOfDeath;
- private static ByteBuffer fill = ByteBuffer.allocateDirect(1024);
- LinkedList<FileOutputStream> streamsToFlush = new LinkedList<FileOutputStream>();
- private long padLogFile(FileChannel fc,long fileSize) throws IOException{
- long position = fc.position();
- // We pad the file in 1M chunks to avoid syncing to
- // write the new filesize.
- if (position + 4096 >= fileSize) {
- fileSize = fileSize + preAllocSize;
- fill.position(0);
- fc.write(fill, fileSize);
- return fileSize;
- long fileSize = 0;
- long lastZxidSeen = -1;
- FileChannel fc = null;
- Request si = null;
- if (toFlush.isEmpty()) {
- si = queuedRequests.take();
- si = queuedRequests.poll();
- if (si == null) {
- flush(toFlush);
- if (si == requestOfDeath) {
- if (si != null) {
- // LOG.warn("Sync>>> cxid = " + si.cxid + " type = " +
- // si.type + " id = " + si.sessionId + " zxid = " +
- // Long.toHexString(si.zxid));
- ZooTrace.logRequest(LOG, ZooTrace.CLIENT_REQUEST_TRACE_MASK,
- 'S', si, "");
- TxnHeader hdr = si.hdr;
- if (hdr != null) {
- if (hdr.getZxid() <= lastZxidSeen) {
- LOG.error("Current zxid " + hdr.getZxid()
- + " is <= " + lastZxidSeen + " for "
- + hdr.getType());
- Record txn = si.txn;
- if (logStream == null) {
- fileSize = 0;
- logStream = new FileOutputStream(new File(
- zks.dataLogDir, ZooKeeperServer
- .getLogName(hdr.getZxid())));
- synchronized (streamsToFlush) {
- streamsToFlush.add(logStream);
- fc = logStream.getChannel();
- logArchive = BinaryOutputArchive
- .getArchive(logStream);
- final long fsize=fileSize;
- final FileChannel ffc=fc;
- fileSize = Profiler.profile(
- new Profiler.Operation<Long>() {
- public Long execute() throws Exception {
- return SyncRequestProcessor.this
- .padLogFile(ffc, fsize);
- }, PADDING_TIMEOUT,
- "Logfile padding exceeded time threshold"
- );
- hdr.serialize(boa, "hdr");
- if (txn != null) {
- txn.serialize(boa, "txn");
- logArchive.writeBuffer(baos.toByteArray(), "txnEntry");
- logArchive.writeByte((byte) 0x42, "EOR");
- logCount++;
- if (logCount > snapCount / 2
- && r.nextInt(snapCount / 2) == 0) {
- // We just want one snapshot going at a time
- if (snapInProcess != null
- && snapInProcess.isAlive()) {
- LOG.warn("Too busy to snap, skipping");
- logStream = null;
- logArchive = null;
- snapInProcess = new Thread() {
- zks.snapshot();
- snapInProcess.start();
- logCount = 0;
- toFlush.add(si);
- if (toFlush.size() > 1000) {
- LOG.error("Severe error, exiting",e);
- System.exit(11);
- "SyncRequestProcessor exiyed!");
- private void flush(LinkedList<Request> toFlush) throws IOException {
- if (toFlush.size() == 0) {
- LinkedList<FileOutputStream> streamsToFlushNow;
- streamsToFlushNow = (LinkedList<FileOutputStream>) streamsToFlush
- .clone();
- for (FileOutputStream fos : streamsToFlushNow) {
- fos.flush();
- if (forceSync) {
- ((FileChannel) fos.getChannel()).force(false);
- while (streamsToFlushNow.size() > 1) {
- FileOutputStream fos = streamsToFlushNow.removeFirst();
- fos.close();
- streamsToFlush.remove(fos);
- while (toFlush.size() > 0) {
- Request i = toFlush.remove();
- nextProcessor.processRequest(i);
- timeToDie = true;
- queuedRequests.add(requestOfDeath);
- // request.addRQRec(">sync");
- queuedRequests.add(request);
+import java.io.File;
+import java.io.FileOutputStream;
+import java.nio.channels.FileChannel;
+import com.yahoo.zookeeper.server.util.Profiler;
+ * This RequestProcessor logs requests to disk. It batches the requests to do
+ * the io efficiently. The request is not passed to the next RequestProcessor
+ * until its log has been synced to disk.
+public class SyncRequestProcessor extends Thread implements RequestProcessor {
+ private static final Logger LOG = Logger.getLogger(SyncRequestProcessor.class);
+ static final int PADDING_TIMEOUT=1000;
+ LinkedBlockingQueue<Request> queuedRequests = new LinkedBlockingQueue<Request>();
+ static boolean forceSync;
+ forceSync = !System.getProperty("zookeeper.forceSync", "yes").equals(
+ "no");
+ static long preAllocSize = 65536 * 1024;
+ String size = System.getProperty("zookeeper.preAllocSize");
+ if (size != null) {
+ preAllocSize = Long.parseLong(size) * 1024;
+ } catch (NumberFormatException e) {
+ LOG.warn(size + " is not a valid value for preAllocSize");
+ * The number of log entries to log before starting a snapshot
+ static public int snapCount = ZooKeeperServer.getSnapCount();
+ Thread snapInProcess;
+ boolean timeToDie = false;
+ public SyncRequestProcessor(ZooKeeperServer zks,
+ super("SyncThread");
+ * Transactions that have been written and are waiting to be flushed to
+ * disk. Basically this is the list of SyncItems whose callbacks will be
+ * invoked after flush returns successfully.
+ LinkedList<Request> toFlush = new LinkedList<Request>();
+ FileOutputStream logStream;
+ BinaryOutputArchive logArchive;
+ int logCount = 0;
+ Request requestOfDeath = Request.requestOfDeath;
+ private static ByteBuffer fill = ByteBuffer.allocateDirect(1024);
+ LinkedList<FileOutputStream> streamsToFlush = new LinkedList<FileOutputStream>();
+ private long padLogFile(FileChannel fc,long fileSize) throws IOException{
+ long position = fc.position();
+ // We pad the file in 1M chunks to avoid syncing to
+ // write the new filesize.
+ if (position + 4096 >= fileSize) {
+ fileSize = fileSize + preAllocSize;
+ fill.position(0);
+ fc.write(fill, fileSize);
+ return fileSize;
+ long fileSize = 0;
+ long lastZxidSeen = -1;
+ FileChannel fc = null;
+ Request si = null;
+ if (toFlush.isEmpty()) {
+ si = queuedRequests.take();
+ si = queuedRequests.poll();
+ if (si == null) {
+ flush(toFlush);
+ if (si == requestOfDeath) {
+ if (si != null) {
+ // LOG.warn("Sync>>> cxid = " + si.cxid + " type = " +
+ // si.type + " id = " + si.sessionId + " zxid = " +
+ // Long.toHexString(si.zxid));
+ ZooTrace.logRequest(LOG, ZooTrace.CLIENT_REQUEST_TRACE_MASK,
+ 'S', si, "");
+ TxnHeader hdr = si.hdr;
+ if (hdr != null) {
+ if (hdr.getZxid() <= lastZxidSeen) {
+ LOG.warn("Current zxid " + hdr.getZxid()
+ + " is <= " + lastZxidSeen + " for "
+ + hdr.getType());
+ Record txn = si.txn;
+ if (logStream == null) {
+ fileSize = 0;
+ logStream = new FileOutputStream(new File(
+ zks.dataLogDir, ZooKeeperServer
+ .getLogName(hdr.getZxid())));
+ synchronized (streamsToFlush) {
+ streamsToFlush.add(logStream);
+ fc = logStream.getChannel();
+ logArchive = BinaryOutputArchive
+ .getArchive(logStream);
+ final long fsize=fileSize;
+ final FileChannel ffc=fc;
+ fileSize = Profiler.profile(
+ new Profiler.Operation<Long>() {
+ public Long execute() throws Exception {
+ return SyncRequestProcessor.this
+ .padLogFile(ffc, fsize);
+ }, PADDING_TIMEOUT,
+ "Logfile padding exceeded time threshold"
+ );
+ hdr.serialize(boa, "hdr");
+ if (txn != null) {
+ txn.serialize(boa, "txn");
+ logArchive.writeBuffer(baos.toByteArray(), "txnEntry");
+ logArchive.writeByte((byte) 0x42, "EOR");
+ logCount++;
+ if (logCount > snapCount / 2
+ && r.nextInt(snapCount / 2) == 0) {
+ // We just want one snapshot going at a time
+ if (snapInProcess != null
+ && snapInProcess.isAlive()) {
+ LOG.warn("Too busy to snap, skipping");
+ logStream = null;
+ logArchive = null;
+ snapInProcess = new Thread() {
+ zks.snapshot();
+ snapInProcess.start();
+ logCount = 0;
+ toFlush.add(si);
+ if (toFlush.size() > 1000) {
+ LOG.error("Severe error, exiting",e);
+ System.exit(11);
+ "SyncRequestProcessor exiyed!");
+ private void flush(LinkedList<Request> toFlush) throws IOException {
+ if (toFlush.size() == 0) {
+ LinkedList<FileOutputStream> streamsToFlushNow;
+ streamsToFlushNow = (LinkedList<FileOutputStream>) streamsToFlush
+ .clone();
+ for (FileOutputStream fos : streamsToFlushNow) {
+ fos.flush();
+ if (forceSync) {
+ ((FileChannel) fos.getChannel()).force(false);
+ while (streamsToFlushNow.size() > 1) {
+ FileOutputStream fos = streamsToFlushNow.removeFirst();
+ fos.close();
+ streamsToFlush.remove(fos);
+ while (toFlush.size() > 0) {
+ Request i = toFlush.remove();
+ nextProcessor.processRequest(i);
+ timeToDie = true;
+ queuedRequests.add(requestOfDeath);
+ // request.addRQRec(">sync");
+ queuedRequests.add(request);
@@ -1,927 +1,927 @@
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.ByteArrayInputStream;
-import java.io.EOFException;
-import java.io.FileInputStream;
-import java.io.RandomAccessFile;
-import java.util.Arrays;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.concurrent.ConcurrentHashMap;
-import com.yahoo.jute.InputArchive;
-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;
- * 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 {
- * 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
- ServerConfig.parse(args);
- runStandalone(new Factory() {
- throws IOException {
- return new NIOServerCnxn.Factory(ServerConfig.getClientPort());
- public ZooKeeperServer createServer() throws IOException {
- return new ZooKeeperServer(new BasicDataTreeBuilder());
- });
- public static void runStandalone(Factory factory) {
- // 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();
- 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
- 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)
- this.treeBuilder = new BasicDataTreeBuilder();
- * Default constructor, relies on the config for its agrument values
- 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)) {
- zxid = Long.parseLong(nameParts[1], 16);
- 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");
- 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));
- } finally {
- raf.close();
- * 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
- 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) {
- if (fzxid > logZxid) {
- logZxid = fzxid;
- List<File> v=new ArrayList<File>(5);
- // Apply the logs
- if (fzxid < logZxid) {
- v.add(f);
- return v.toArray(new File[0]);
- * Restore sessions and data
- private void loadSnapshotAndLogs() throws IOException {
- // Find the most recent snapshot
- List<File> files = sortDataDir(dataDir.listFiles(), "snapshot", false);
- zxid = isValidSnapshot(f);
- if (zxid == -1) {
- LOG.warn("Skipping " + f);
- 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;
- 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 {
- int count = ia.readInt("count");
- while (count > 0) {
- long id = ia.readLong("id");
- int to = ia.readInt("timeout");
- sessionsWithTimeouts.put(id, to);
- "loadData --- session in archive: " + id
- + " with timeout: " + to);
- count--;
- dataTree.deserialize(ia, "tree");
- public long playLog(InputArchive logStream) throws IOException {
- long highestZxid = 0;
- 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.");
- if (hdr.getZxid() <= highestZxid && highestZxid != 0) {
- LOG.error(highestZxid + "(higestZxid) >= "
- + hdr.getZxid() + "(next log) for type "
- highestZxid = hdr.getZxid();
- switch (hdr.getType()) {
- 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);
- sessionsWithTimeouts.remove(hdr.getClientId());
- "playLog --- close session in log: "
- + Long.toHexString(hdr.getClientId()));
- 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;
- request.hdr.serialize(boa, "hdr");
- if (request.txn != null) {
- request.txn.serialize(boa, "txn");
- // This really should be impossible
- 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)
- hdr.deserialize(ia, "hdr");
- // This isn't really an error txn; it just has the same
- // format. The error represents the timeout
- txn = new CreateSessionTxn();
- txn = new CreateTxn();
- txn = new DeleteTxn();
- txn = new SetDataTxn();
- txn = new SetACLTxn();
- case OpCode.error:
- txn = new ErrorTxn();
- txn.deserialize(ia, "txn");
- return txn;
- public void truncateLog(long finalZxid) throws IOException {
- for (File f : dataDir.listFiles()) {
- long zxid = isValidSnapshot(f);
- if (zxid > highestZxid) {
- highestZxid = zxid;
- File[] files = getLogFiles(dataLogDir.listFiles(), highestZxid);
- boolean truncated = false;
- FileInputStream fin = new FileInputStream(f);
- InputArchive ia = BinaryInputArchive.getArchive(fin);
- FileChannel fchan = fin.getChannel();
- byte[] bytes = ia.readBuffer("txtEntry");
- InputArchive iab = BinaryInputArchive
- deserializeTxn(iab, hdr);
- if (ia.readByte("EOF") != 'B') {
- 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;
- } catch (EOFException eof) {
- if (truncated == true) {
- 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,
- 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;
- "Snapshotting: " + Long.toHexString(lastZxid));
- 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();
- "Snapshotting finished: " + Long.toHexString(lastZxid));
- // 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,
- "ZooKeeperServer --- Session to be closed: "
- // 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);
- "ZooKeeperServer --- killSession: "
- if (sessionTracker != null) {
- sessionTracker.removeSession(sessionId);
- public void expire(long sessionId) {
- "ZooKeeperServer --- Session to expire: " + Long.toHexString(sessionId));
- closeSession(sessionId);
- void touch(ServerCnxn cnxn) throws IOException {
- if (cnxn == null) {
- 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;
- 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;
- // 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.
- 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;
- 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 */
- ChangeRecord duplicate(long zxid) {
- 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);
- 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,
- if (!checkPasswd(sessionId, passwd)) {
- cnxn.finishSessionInit(false);
- revalidateSession(cnxn, sessionId, sessionTimeout);
- public void closeSession(ServerCnxn cnxn, RequestHeader requestHeader)
- 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) {
- while (!running) {
- wait(1000);
- throw new RuntimeException("Not started");
- 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();
- LOG.warn("Dropping packet at server of type " + type);
- // if unvalid packet drop the packet.
- static public void byteBuffer2Record(ByteBuffer bb, Record record)
- BinaryInputArchive ia;
- ia = BinaryInputArchive.getArchive(new ByteBufferInputStream(bb));
- record.deserialize(ia, "request");
- public static int getSnapCount() {
- String sc = System.getProperty("zookeeper.snapCount");
- return Integer.parseInt(sc);
- return 10000;
- public int getGlobalOutstandingLimit() {
- String sc = System.getProperty("zookeeper.globalOutstandingLimit");
- int limit;
- limit = Integer.parseInt(sc);
- limit = 1000;
- return limit;
- public void setServerCnxnFactory(NIOServerCnxn.Factory factory) {
- serverCnxnFactory = factory;
- public NIOServerCnxn.Factory getServerCnxnFactory() {
- return serverCnxnFactory;
- public long getLastProcessedZxid() {
- return dataTree.lastProcessedZxid;
- return getInProcess();
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayInputStream;
+import java.io.EOFException;
+import java.io.FileInputStream;
+import java.io.RandomAccessFile;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+import com.yahoo.jute.InputArchive;
+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;
+ * 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 {
+ * 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
+ ServerConfig.parse(args);
+ runStandalone(new Factory() {
+ throws IOException {
+ return new NIOServerCnxn.Factory(ServerConfig.getClientPort());
+ public ZooKeeperServer createServer() throws IOException {
+ return new ZooKeeperServer(new BasicDataTreeBuilder());
+ });
+ public static void runStandalone(Factory factory) {
+ // 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();
+ LOG.fatal("Unexpected exception",e);
+ 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
+ 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)
+ this.treeBuilder = new BasicDataTreeBuilder();
+ * Default constructor, relies on the config for its agrument values
+ 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)) {
+ zxid = Long.parseLong(nameParts[1], 16);
+ 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");
+ 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));
+ } finally {
+ raf.close();
+ * 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
+ 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) {
+ if (fzxid > logZxid) {
+ logZxid = fzxid;
+ List<File> v=new ArrayList<File>(5);
+ // Apply the logs
+ if (fzxid < logZxid) {
+ v.add(f);
+ return v.toArray(new File[0]);
+ * Restore sessions and data
+ private void loadSnapshotAndLogs() throws IOException {
+ // Find the most recent snapshot
+ List<File> files = sortDataDir(dataDir.listFiles(), "snapshot", false);
+ zxid = isValidSnapshot(f);
+ if (zxid == -1) {
+ LOG.warn("Skipping " + f);
+ 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;
+ 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 {
+ int count = ia.readInt("count");
+ while (count > 0) {
+ long id = ia.readLong("id");
+ int to = ia.readInt("timeout");
+ sessionsWithTimeouts.put(id, to);
+ "loadData --- session in archive: " + id
+ + " with timeout: " + to);
+ count--;
+ dataTree.deserialize(ia, "tree");
+ public long playLog(InputArchive logStream) throws IOException {
+ long highestZxid = 0;
+ 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.");
+ if (hdr.getZxid() <= highestZxid && highestZxid != 0) {
+ LOG.error(highestZxid + "(higestZxid) >= "
+ + hdr.getZxid() + "(next log) for type "
+ highestZxid = hdr.getZxid();
+ switch (hdr.getType()) {
+ 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);
+ sessionsWithTimeouts.remove(hdr.getClientId());
+ "playLog --- close session in log: "
+ + Long.toHexString(hdr.getClientId()));
+ 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;
+ request.hdr.serialize(boa, "hdr");
+ if (request.txn != null) {
+ request.txn.serialize(boa, "txn");
+ // This really should be impossible
+ 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)
+ hdr.deserialize(ia, "hdr");
+ // This isn't really an error txn; it just has the same
+ // format. The error represents the timeout
+ txn = new CreateSessionTxn();
+ txn = new CreateTxn();
+ txn = new DeleteTxn();
+ txn = new SetDataTxn();
+ txn = new SetACLTxn();
+ case OpCode.error:
+ txn = new ErrorTxn();
+ txn.deserialize(ia, "txn");
+ return txn;
+ public void truncateLog(long finalZxid) throws IOException {
+ for (File f : dataDir.listFiles()) {
+ long zxid = isValidSnapshot(f);
+ if (zxid > highestZxid) {
+ highestZxid = zxid;
+ File[] files = getLogFiles(dataLogDir.listFiles(), highestZxid);
+ boolean truncated = false;
+ FileInputStream fin = new FileInputStream(f);
+ InputArchive ia = BinaryInputArchive.getArchive(fin);
+ FileChannel fchan = fin.getChannel();
+ byte[] bytes = ia.readBuffer("txtEntry");
+ InputArchive iab = BinaryInputArchive
+ deserializeTxn(iab, hdr);
+ if (ia.readByte("EOF") != 'B') {
+ 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;
+ } catch (EOFException eof) {
+ if (truncated == true) {
+ 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,
+ 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;
+ "Snapshotting: " + Long.toHexString(lastZxid));
+ 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();
+ "Snapshotting finished: " + Long.toHexString(lastZxid));
+ // 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,
+ "ZooKeeperServer --- Session to be closed: "
+ // 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);
+ "ZooKeeperServer --- killSession: "
+ if (sessionTracker != null) {
+ sessionTracker.removeSession(sessionId);
+ public void expire(long sessionId) {
+ "ZooKeeperServer --- Session to expire: " + Long.toHexString(sessionId));
+ closeSession(sessionId);
+ void touch(ServerCnxn cnxn) throws IOException {
+ if (cnxn == null) {
+ 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;
+ 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;
+ // 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.
+ 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;
+ 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 */
+ ChangeRecord duplicate(long zxid) {
+ 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);
+ 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,
+ if (!checkPasswd(sessionId, passwd)) {
+ cnxn.finishSessionInit(false);
+ revalidateSession(cnxn, sessionId, sessionTimeout);
+ public void closeSession(ServerCnxn cnxn, RequestHeader requestHeader)
+ 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) {
+ while (!running) {
+ wait(1000);
+ throw new RuntimeException("Not started");
+ 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();
+ LOG.warn("Dropping packet at server of type " + type);
+ // if unvalid packet drop the packet.
+ static public void byteBuffer2Record(ByteBuffer bb, Record record)
+ BinaryInputArchive ia;
+ ia = BinaryInputArchive.getArchive(new ByteBufferInputStream(bb));
+ record.deserialize(ia, "request");
+ public static int getSnapCount() {
+ String sc = System.getProperty("zookeeper.snapCount");
+ return Integer.parseInt(sc);
+ return 10000;
+ public int getGlobalOutstandingLimit() {
+ String sc = System.getProperty("zookeeper.globalOutstandingLimit");
+ int limit;
+ limit = Integer.parseInt(sc);
+ limit = 1000;
+ return limit;
+ public void setServerCnxnFactory(NIOServerCnxn.Factory factory) {
+ serverCnxnFactory = factory;
+ public NIOServerCnxn.Factory getServerCnxnFactory() {
+ return serverCnxnFactory;
+ public long getLastProcessedZxid() {
+ return dataTree.lastProcessedZxid;
+ return getInProcess();
@@ -1,125 +1,125 @@
-package com.yahoo.zookeeper.server.auth;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import com.yahoo.zookeeper.server.ServerCnxn;
-public class DigestAuthenticationProvider implements AuthenticationProvider {
- private static final Logger LOG = Logger.getLogger(DigestAuthenticationProvider.class);
- public static String superDigest = "super:1wZ8qIvQBMTq0KPxMc6RQ/PCXKM=";
- public String getScheme() {
- return "digest";
- static final private String base64Encode(byte b[]) {
- for (int i = 0; i < b.length;) {
- int pad = 0;
- int v = (b[i++] & 0xff) << 16;
- if (i < b.length) {
- v |= (b[i++] & 0xff) << 8;
- pad++;
- v |= (b[i++] & 0xff);
- sb.append(encode(v >> 18));
- sb.append(encode(v >> 12));
- if (pad < 2) {
- sb.append(encode(v >> 6));
- sb.append('=');
- if (pad < 1) {
- sb.append(encode(v));
- static final private char encode(int i) {
- i &= 0x3f;
- if (i < 26) {
- return (char) ('A' + i);
- if (i < 52) {
- return (char) ('a' + i - 26);
- if (i < 62) {
- return (char) ('0' + i - 52);
- return i == 62 ? '+' : '/';
- static public String generateDigest(String idPassword)
- throws NoSuchAlgorithmException {
- String parts[] = idPassword.split(":", 2);
- byte digest[] = MessageDigest.getInstance("SHA1").digest(
- idPassword.getBytes());
- return parts[0] + ":" + base64Encode(digest);
- public int handleAuthentication(ServerCnxn cnxn, byte[] authData) {
- String id = new String(authData);
- String digest = generateDigest(id);
- if (digest.equals(superDigest)) {
- cnxn.getAuthInfo().add(new Id("super", ""));
- cnxn.getAuthInfo().add(new Id(getScheme(), digest));
- return KeeperException.Code.Ok;
- } catch (NoSuchAlgorithmException e) {
- return KeeperException.Code.AuthFailed;
- public boolean isAuthenticated() {
- public boolean isValid(String id) {
- String parts[] = id.split(":");
- return parts.length == 2;
- public boolean matches(String id, String aclExpr) {
- return id.equals(aclExpr);
- public static void main(String args[]) throws IOException,
- NoSuchAlgorithmException {
- for (int i = 0; i < args.length; i++) {
- System.out.println(args[i] + "->" + generateDigest(args[i]));
+package com.yahoo.zookeeper.server.auth;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import com.yahoo.zookeeper.server.ServerCnxn;
+public class DigestAuthenticationProvider implements AuthenticationProvider {
+ private static final Logger LOG = Logger.getLogger(DigestAuthenticationProvider.class);
+ public final static String superDigest = "super:1wZ8qIvQBMTq0KPxMc6RQ/PCXKM=";
+ public String getScheme() {
+ return "digest";
+ static final private String base64Encode(byte b[]) {
+ for (int i = 0; i < b.length;) {
+ int pad = 0;
+ int v = (b[i++] & 0xff) << 16;
+ if (i < b.length) {
+ v |= (b[i++] & 0xff) << 8;
+ pad++;
+ v |= (b[i++] & 0xff);
+ sb.append(encode(v >> 18));
+ sb.append(encode(v >> 12));
+ if (pad < 2) {
+ sb.append(encode(v >> 6));
+ sb.append('=');
+ if (pad < 1) {
+ sb.append(encode(v));
+ static final private char encode(int i) {
+ i &= 0x3f;
+ if (i < 26) {
+ return (char) ('A' + i);
+ if (i < 52) {
+ return (char) ('a' + i - 26);
+ if (i < 62) {
+ return (char) ('0' + i - 52);
+ return i == 62 ? '+' : '/';
+ static public String generateDigest(String idPassword)
+ throws NoSuchAlgorithmException {
+ String parts[] = idPassword.split(":", 2);
+ byte digest[] = MessageDigest.getInstance("SHA1").digest(
+ idPassword.getBytes());
+ return parts[0] + ":" + base64Encode(digest);
+ public int handleAuthentication(ServerCnxn cnxn, byte[] authData) {
+ String id = new String(authData);
+ String digest = generateDigest(id);
+ if (digest.equals(superDigest)) {
+ cnxn.getAuthInfo().add(new Id("super", ""));
+ cnxn.getAuthInfo().add(new Id(getScheme(), digest));
+ return KeeperException.Code.Ok;
+ } catch (NoSuchAlgorithmException e) {
+ LOG.error("Missing algorithm",e);
+ return KeeperException.Code.AuthFailed;
+ public boolean isAuthenticated() {
+ public boolean isValid(String id) {
+ String parts[] = id.split(":");
+ return parts.length == 2;
+ public boolean matches(String id, String aclExpr) {
+ return id.equals(aclExpr);
+ public static void main(String args[]) throws IOException,
+ NoSuchAlgorithmException {
+ for (int i = 0; i < args.length; i++) {
+ System.out.println(args[i] + "->" + generateDigest(args[i]));
@@ -1,68 +1,68 @@
-import java.util.Enumeration;
-public class ProviderRegistry {
- private static final Logger LOG = Logger.getLogger(ProviderRegistry.class);
- private static boolean initialized = false;
- private static HashMap<String, AuthenticationProvider> authenticationProviders =
- new HashMap<String, AuthenticationProvider>();
- public static void initialize() {
- synchronized (ProviderRegistry.class) {
- if (initialized)
- IPAuthenticationProvider ipp = new IPAuthenticationProvider();
- HostAuthenticationProvider hostp = new HostAuthenticationProvider();
- DigestAuthenticationProvider digp = new DigestAuthenticationProvider();
- authenticationProviders.put(ipp.getScheme(), ipp);
- authenticationProviders.put(hostp.getScheme(), hostp);
- authenticationProviders.put(digp.getScheme(), digp);
- Enumeration<Object> en = System.getProperties().keys();
- while (en.hasMoreElements()) {
- String k = (String) en.nextElement();
- if (k.startsWith("zookeeper.authProvider.")) {
- String className = System.getProperty(k);
- Class<?> c = ZooKeeperServer.class.getClassLoader()
- .loadClass(className);
- AuthenticationProvider ap = (AuthenticationProvider) c
- .newInstance();
- authenticationProviders.put(ap.getScheme(), ap);
- LOG.error("Problems loading " + className,e);
- public static AuthenticationProvider getProvider(String scheme) {
- if(!initialized)
- initialize();
- return authenticationProviders.get(scheme);
+import java.util.Enumeration;
+public class ProviderRegistry {
+ private static final Logger LOG = Logger.getLogger(ProviderRegistry.class);
+ private static boolean initialized = false;
+ private static HashMap<String, AuthenticationProvider> authenticationProviders =
+ new HashMap<String, AuthenticationProvider>();
+ public static void initialize() {
+ synchronized (ProviderRegistry.class) {
+ if (initialized)
+ IPAuthenticationProvider ipp = new IPAuthenticationProvider();
+ HostAuthenticationProvider hostp = new HostAuthenticationProvider();
+ DigestAuthenticationProvider digp = new DigestAuthenticationProvider();
+ authenticationProviders.put(ipp.getScheme(), ipp);
+ authenticationProviders.put(hostp.getScheme(), hostp);
+ authenticationProviders.put(digp.getScheme(), digp);
+ Enumeration<Object> en = System.getProperties().keys();
+ while (en.hasMoreElements()) {
+ String k = (String) en.nextElement();
+ if (k.startsWith("zookeeper.authProvider.")) {
+ String className = System.getProperty(k);
+ Class<?> c = ZooKeeperServer.class.getClassLoader()
+ .loadClass(className);
+ AuthenticationProvider ap = (AuthenticationProvider) c
+ .newInstance();
+ authenticationProviders.put(ap.getScheme(), ap);
+ LOG.warn("Problems loading " + className,e);
+ public static AuthenticationProvider getProvider(String scheme) {
+ if(!initialized)
+ initialize();
+ return authenticationProviders.get(scheme);
@@ -1,906 +1,906 @@
-package com.yahoo.zookeeper.server.quorum;
-import java.net.DatagramPacket;
-import java.net.DatagramSocket;
-import java.net.SocketException;
-import java.util.Collection;
-import java.util.concurrent.TimeUnit;
-import com.yahoo.zookeeper.server.quorum.Election;
-import com.yahoo.zookeeper.server.quorum.Vote;
-import com.yahoo.zookeeper.server.quorum.QuorumPeer.QuorumServer;
-import com.yahoo.zookeeper.server.quorum.QuorumPeer.ServerState;
-public class AuthFastLeaderElection implements Election {
- private static final Logger LOG = Logger.getLogger(AuthFastLeaderElection.class);
- /* Sequence numbers for messages */
- static int sequencer = 0;
- static int maxTag = 0;
- * Determine how much time a process has to wait once it believes that it
- * has reached the end of leader election.
- static int finalizeWait = 100;
- * Challenge counter to avoid replay attacks
- static int challengeCounter = 0;
- * Flag to determine whether to authenticate or not
- private boolean authEnabled = false;
- static public class Notification {
- * Proposed leader
- long leader;
- * zxid of the proposed leader
- * Epoch
- long epoch;
- * current state of sender
- QuorumPeer.ServerState state;
- * Address of the sender
- InetSocketAddress addr;
- * Messages to send, both Notifications and Acks
- static public class ToSend {
- static enum mType {
- crequest, challenge, notification, ack
- ToSend(mType type, long tag, long leader, long zxid, long epoch,
- ServerState state, InetSocketAddress addr) {
- switch (type) {
- case crequest:
- this.type = 0;
- this.tag = tag;
- this.leader = leader;
- this.epoch = epoch;
- this.state = state;
- this.addr = addr;
- case challenge:
- this.type = 1;
- case notification:
- this.type = 2;
- this.state = QuorumPeer.ServerState.LOOKING;
- case ack:
- this.type = 3;
- * Message type: 0 notification, 1 acknowledgement
- int type;
- * Proposed leader in the case of notification
- * id contains the tag for acks, and zxid for notifications
- * Current state;
- * Message tag
- long tag;
- LinkedBlockingQueue<ToSend> sendqueue;
- LinkedBlockingQueue<Notification> recvqueue;
- private class Messenger {
- DatagramSocket mySocket;
- long lastProposedLeader;
- long lastProposedZxid;
- long lastEpoch;
- LinkedBlockingQueue<Long> acksqueue;
- HashMap<Long, Long> challengeMap;
- HashMap<Long, Long> challengeMutex;
- HashMap<Long, Long> ackMutex;
- HashMap<InetSocketAddress, HashMap<Long, Long>> addrChallengeMap;
- class WorkerReceiver implements Runnable {
- Messenger myMsg;
- WorkerReceiver(DatagramSocket s, Messenger msg) {
- mySocket = s;
- myMsg = msg;
- boolean saveChallenge(long tag, long challenge) {
- Long l = challengeMutex.get(tag);
- synchronized (challengeMap) {
- challengeMap.put(tag, challenge);
- challengeMutex.remove(tag);
- if (l != null) {
- synchronized(l){
- l.notify();
- byte responseBytes[] = new byte[48];
- ByteBuffer responseBuffer = ByteBuffer.wrap(responseBytes);
- DatagramPacket responsePacket = new DatagramPacket(
- responseBytes, responseBytes.length);
- // Sleeps on receive
- responseBuffer.clear();
- mySocket.receive(responsePacket);
- LOG.warn("Exception receiving: " + e.toString());
- // Receive new message
- if (responsePacket.getLength() != responseBytes.length) {
- LOG.error("Got a short response: "
- + responsePacket.getLength() + " "
- + responsePacket.toString());
- int type = responseBuffer.getInt();
- if ((type > 3) || (type < 0)) {
- LOG.error("Got bad Msg type: " + type);
- long tag = responseBuffer.getLong();
- QuorumPeer.ServerState ackstate = QuorumPeer.ServerState.LOOKING;
- switch (responseBuffer.getInt()) {
- case 0:
- ackstate = QuorumPeer.ServerState.LOOKING;
- case 1:
- ackstate = QuorumPeer.ServerState.LEADING;
- case 2:
- ackstate = QuorumPeer.ServerState.FOLLOWING;
- // Receive challenge request
- ToSend c = new ToSend(ToSend.mType.challenge, tag,
- self.currentVote.id, self.currentVote.zxid,
- logicalclock, self.getPeerState(),
- (InetSocketAddress) responsePacket
- .getSocketAddress());
- sendqueue.offer(c);
- // Receive challenge and store somewhere else
- long challenge = responseBuffer.getLong();
- saveChallenge(tag, challenge);
- Notification n = new Notification();
- n.leader = responseBuffer.getLong();
- n.zxid = responseBuffer.getLong();
- n.epoch = responseBuffer.getLong();
- n.state = ackstate;
- n.addr = (InetSocketAddress) responsePacket
- .getSocketAddress();
- if ((myMsg.lastEpoch <= n.epoch)
- && ((n.zxid > myMsg.lastProposedZxid)
- || ((n.zxid == myMsg.lastProposedZxid)
- && (n.leader > myMsg.lastProposedLeader)))) {
- myMsg.lastProposedZxid = n.zxid;
- myMsg.lastProposedLeader = n.leader;
- myMsg.lastEpoch = n.epoch;
- long recChallenge;
- InetSocketAddress addr = (InetSocketAddress) responsePacket
- if (authEnabled) {
- if (addrChallengeMap.get(addr).get(tag) != null) {
- recChallenge = responseBuffer.getLong();
- if (addrChallengeMap.get(addr).get(tag) == recChallenge) {
- recvqueue.offer(n);
- ToSend a = new ToSend(ToSend.mType.ack,
- tag, self.currentVote.id,
- self.currentVote.zxid,
- (InetSocketAddress) addr);
- sendqueue.offer(a);
- LOG.warn("Incorrect challenge: "
- + recChallenge + ", "
- + addrChallengeMap.toString());
- LOG.warn("No challenge for host: " + addr
- + " " + tag);
- ToSend a = new ToSend(ToSend.mType.ack, tag,
- // Upon reception of an ack message, remove it from the
- // queue
- case 3:
- Long l = ackMutex.get(tag);
- acksqueue.offer(tag);
- addrChallengeMap.get(
- .getSocketAddress()).remove(tag);
- if (ackstate != QuorumPeer.ServerState.LOOKING) {
- Notification outofsync = new Notification();
- outofsync.leader = responseBuffer.getLong();
- outofsync.zxid = responseBuffer.getLong();
- outofsync.epoch = responseBuffer.getLong();
- outofsync.state = ackstate;
- outofsync.addr = (InetSocketAddress) responsePacket
- recvqueue.offer(outofsync);
- // Default case
- LOG.warn("Received message of incorrect type");
- class WorkerSender implements Runnable {
- Random rand;
- boolean processing;
- int maxAttempts;
- int ackWait = finalizeWait;
- * Receives a socket and max number of attempts as input
- WorkerSender(int attempts) {
- maxAttempts = attempts;
- rand = new Random(java.lang.Thread.currentThread().getId()
- + System.currentTimeMillis());
- long genChallenge() {
- byte buf[] = new byte[8];
- buf[0] = (byte) ((challengeCounter & 0xff000000) >>> 24);
- buf[1] = (byte) ((challengeCounter & 0x00ff0000) >>> 16);
- buf[2] = (byte) ((challengeCounter & 0x0000ff00) >>> 8);
- buf[3] = (byte) ((challengeCounter & 0x000000ff));
- challengeCounter++;
- int secret = rand.nextInt(java.lang.Integer.MAX_VALUE);
- buf[4] = (byte) ((secret & 0xff000000) >>> 24);
- buf[5] = (byte) ((secret & 0x00ff0000) >>> 16);
- buf[6] = (byte) ((secret & 0x0000ff00) >>> 8);
- buf[7] = (byte) ((secret & 0x000000ff));
- return (((long)(buf[0] & 0xFF)) << 56)
- + (((long)(buf[1] & 0xFF)) << 48)
- + (((long)(buf[2] & 0xFF)) << 40)
- + (((long)(buf[3] & 0xFF)) << 32)
- + (((long)(buf[4] & 0xFF)) << 24)
- + (((long)(buf[5] & 0xFF)) << 16)
- + (((long)(buf[6] & 0xFF)) << 8)
- + ((long)(buf[7] & 0xFF));
- ToSend m = sendqueue.take();
- process(m);
- private void process(ToSend m) {
- int attempts = 0;
- byte zeroes[];
- byte requestBytes[] = new byte[48];
- DatagramPacket requestPacket = new DatagramPacket(requestBytes,
- requestBytes.length);
- ByteBuffer requestBuffer = ByteBuffer.wrap(requestBytes);
- switch (m.type) {
- * Building challenge request packet to send
- requestBuffer.clear();
- requestBuffer.putInt(ToSend.mType.crequest.ordinal());
- requestBuffer.putLong(m.tag);
- requestBuffer.putInt(m.state.ordinal());
- zeroes = new byte[32];
- requestBuffer.put(zeroes);
- requestPacket.setLength(48);
- requestPacket.setSocketAddress(m.addr);
- if (challengeMap.get(m.tag) == null) {
- mySocket.send(requestPacket);
- LOG.warn("Exception while sending challenge: "
- + e.toString());
- * Building challenge packet to send
- long newChallenge;
- if (addrChallengeMap.get(m.addr).containsKey(m.tag)) {
- newChallenge = addrChallengeMap.get(m.addr).get(m.tag);
- newChallenge = genChallenge();
- addrChallengeMap.get(m.addr).put(m.tag, newChallenge);
- requestBuffer.putInt(ToSend.mType.challenge.ordinal());
- requestBuffer.putLong(newChallenge);
- zeroes = new byte[24];
- * Building notification packet to send
- requestBuffer.putInt(m.type);
- requestBuffer.putLong(m.leader);
- requestBuffer.putLong(m.zxid);
- requestBuffer.putLong(m.epoch);
- zeroes = new byte[8];
- boolean myChallenge = false;
- boolean myAck = false;
- while (attempts < maxAttempts) {
- * Try to obtain a challenge only if does not have
- * one yet
- if (!myChallenge && authEnabled) {
- ToSend crequest = new ToSend(
- ToSend.mType.crequest, m.tag, m.leader,
- m.zxid, m.epoch,
- QuorumPeer.ServerState.LOOKING, m.addr);
- sendqueue.offer(crequest);
- double timeout = ackWait
- * java.lang.Math.pow(2, attempts);
- Long l = Long.valueOf(m.tag);
- synchronized (l) {
- challengeMutex.put(m.tag, l);
- l.wait((long) timeout);
- myChallenge = challengeMap
- .containsKey(m.tag);
- LOG.warn("Challenge request exception: "
- * If don't have challenge yet, skip sending
- * notification
- if (authEnabled && !myChallenge) {
- attempts++;
- requestBuffer.position(40);
- requestBuffer.putLong(challengeMap.get(m.tag));
- * java.lang.Math.pow(10, attempts);
- ackMutex.put(m.tag, l);
- l.wait((int) timeout);
- LOG.warn("Ack exception: "
- synchronized (acksqueue) {
- for (int i = 0; i < acksqueue.size(); ++i) {
- Long newack = acksqueue.poll();
- * Under highly concurrent load, a thread
- * may get into this loop but by the time it
- * tries to read from the queue, the queue
- * is empty. There are two alternatives:
- * synchronize this block, or test if newack
- * is null.
- if (newack == m.tag) {
- myAck = true;
- } else
- acksqueue.offer(newack);
- LOG.warn("Sending exception: "
- * Do nothing, just try again
- if (myAck) {
- * Received ack successfully, so return
- if (challengeMap.get(m.tag) != null)
- challengeMap.remove(m.tag);
- * Return message to queue for another attempt later if
- * epoch hasn't changed.
- if (m.epoch == logicalclock) {
- sendqueue.offer(m);
- LOG.warn("Exception while sending ack: "
- public boolean queueEmpty() {
- return (sendqueue.isEmpty() || acksqueue.isEmpty() || recvqueue
- .isEmpty());
- Messenger(int threads, DatagramSocket s) {
- acksqueue = new LinkedBlockingQueue<Long>();
- challengeMap = new HashMap<Long, Long>();
- challengeMutex = new HashMap<Long, Long>();
- ackMutex = new HashMap<Long, Long>();
- addrChallengeMap = new HashMap<InetSocketAddress, HashMap<Long, Long>>();
- lastProposedLeader = 0;
- lastProposedZxid = 0;
- lastEpoch = 0;
- for (int i = 0; i < threads; ++i) {
- Thread t = new Thread(new WorkerSender(3),
- "WorkerSender Thread: " + (i + 1));
- t.setDaemon(true);
- t.start();
- for (QuorumServer server : self.quorumPeers) {
- InetSocketAddress saddr = new InetSocketAddress(server.addr
- .getAddress(), port);
- addrChallengeMap.put(saddr, new HashMap<Long, Long>());
- Thread t = new Thread(new WorkerReceiver(s, this),
- "WorkerReceiver Thread");
- QuorumPeer self;
- int port;
- long logicalclock; /* Election instance */
- long proposedLeader;
- long proposedZxid;
- public AuthFastLeaderElection(QuorumPeer self, int electionPort,
- boolean auth) {
- this.authEnabled = auth;
- starter(self, electionPort);
- public AuthFastLeaderElection(QuorumPeer self, int electionPort) {
- private void starter(QuorumPeer self, int electionPort) {
- this.self = self;
- port = electionPort;
- proposedLeader = -1;
- proposedZxid = -1;
- mySocket = new DatagramSocket(port);
- // mySocket.setSoTimeout(20000);
- } catch (SocketException e1) {
- throw new RuntimeException();
- sendqueue = new LinkedBlockingQueue<ToSend>(2 * self.quorumPeers.size());
- recvqueue = new LinkedBlockingQueue<Notification>(2 * self.quorumPeers
- .size());
- new Messenger(self.quorumPeers.size() * 2, mySocket);
- private void leaveInstance() {
- logicalclock++;
- // sendqueue.clear();
- // recvqueue.clear();
- public static class ElectionResult {
- public Vote vote;
- public int count;
- public Vote winner;
- public int winningCount;
- private void sendNotifications() {
- ToSend notmsg = new ToSend(ToSend.mType.notification,
- AuthFastLeaderElection.sequencer++, proposedLeader,
- proposedZxid, logicalclock, QuorumPeer.ServerState.LOOKING,
- saddr);
- sendqueue.offer(notmsg);
- private boolean totalOrderPredicate(long id, long zxid) {
- if ((zxid > proposedZxid)
- || ((zxid == proposedZxid) && (id > proposedLeader)))
- private boolean termPredicate(HashMap<InetSocketAddress, Vote> votes,
- long l, long zxid) {
- int count = 0;
- Collection<Vote> votesCast = votes.values();
- * First make the views consistent. Sometimes peers will have different
- * zxids for a server depending on timing.
- for (Vote v : votesCast) {
- if ((v.id == l) && (v.zxid == zxid))
- count++;
- if (count > (self.quorumPeers.size() / 2))
- public Vote lookForLeader() throws InterruptedException {
- HashMap<InetSocketAddress, Vote> recvset =
- new HashMap<InetSocketAddress, Vote>();
- HashMap<InetSocketAddress, Vote> outofelection =
- proposedLeader = self.getId();
- proposedZxid = self.getLastLoggedZxid();
- LOG.warn("Election tally");
- sendNotifications();
- * Loop in which we exchange notifications until we find a leader
- while (self.getPeerState() == ServerState.LOOKING) {
- * Remove next notification from queue, times out after 2 times the
- * termination time
- Notification n = recvqueue.poll(2 * finalizeWait,
- TimeUnit.MILLISECONDS);
- * Sends more notifications if haven't received enough. Otherwise
- * processes new notification.
- if (((!outofelection.isEmpty()) || (recvset.size() > 1)))
- switch (n.state) {
- case LOOKING:
- if (n.epoch > logicalclock) {
- logicalclock = n.epoch;
- recvset.clear();
- if (totalOrderPredicate(n.leader, n.zxid)) {
- proposedLeader = n.leader;
- proposedZxid = n.zxid;
- } else if (n.epoch < logicalclock) {
- } else if (totalOrderPredicate(n.leader, n.zxid)) {
- recvset.put(n.addr, new Vote(n.leader, n.zxid));
- // If have received from all nodes, then terminate
- if (self.quorumPeers.size() == recvset.size()) {
- self.setPeerState((proposedLeader == self.getId()) ?
- ServerState.LEADING: ServerState.FOLLOWING);
- // if (self.state == ServerState.FOLLOWING) {
- // Thread.sleep(100);
- // }
- leaveInstance();
- return new Vote(proposedLeader, proposedZxid);
- } else if (termPredicate(recvset, proposedLeader,
- proposedZxid)) {
- // Otherwise, wait for a fixed amount of time
- LOG.warn("Passed predicate");
- Thread.sleep(finalizeWait);
- // Notification probe = recvqueue.peek();
- // Verify if there is any change in the proposed leader
- while ((!recvqueue.isEmpty())
- && !totalOrderPredicate(
- recvqueue.peek().leader, recvqueue
- .peek().zxid)) {
- recvqueue.poll();
- if (recvqueue.isEmpty()) {
- // LOG.warn("Proposed leader: " +
- // proposedLeader);
- case LEADING:
- outofelection.put(n.addr, new Vote(n.leader, n.zxid));
- if (termPredicate(outofelection, n.leader, n.zxid)) {
- self.setPeerState((n.leader == self.getId()) ?
- return new Vote(n.leader, n.zxid);
- case FOLLOWING:
+package com.yahoo.zookeeper.server.quorum;
+import java.net.DatagramPacket;
+import java.net.DatagramSocket;
+import java.net.SocketException;
+import java.util.Collection;
+import java.util.concurrent.TimeUnit;
+import com.yahoo.zookeeper.server.quorum.Election;
+import com.yahoo.zookeeper.server.quorum.Vote;
+import com.yahoo.zookeeper.server.quorum.QuorumPeer.QuorumServer;
+import com.yahoo.zookeeper.server.quorum.QuorumPeer.ServerState;
+public class AuthFastLeaderElection implements Election {
+ private static final Logger LOG = Logger.getLogger(AuthFastLeaderElection.class);
+ /* Sequence numbers for messages */
+ static int sequencer = 0;
+ static int maxTag = 0;
+ * Determine how much time a process has to wait once it believes that it
+ * has reached the end of leader election.
+ static int finalizeWait = 100;
+ * Challenge counter to avoid replay attacks
+ static int challengeCounter = 0;
+ * Flag to determine whether to authenticate or not
+ private boolean authEnabled = false;
+ static public class Notification {
+ * Proposed leader
+ long leader;
+ * zxid of the proposed leader
+ * Epoch
+ long epoch;
+ * current state of sender
+ QuorumPeer.ServerState state;
+ * Address of the sender
+ InetSocketAddress addr;
+ * Messages to send, both Notifications and Acks
+ static public class ToSend {
+ static enum mType {
+ crequest, challenge, notification, ack
+ ToSend(mType type, long tag, long leader, long zxid, long epoch,
+ ServerState state, InetSocketAddress addr) {
+ switch (type) {
+ case crequest:
+ this.type = 0;
+ this.tag = tag;
+ this.leader = leader;
+ this.epoch = epoch;
+ this.state = state;
+ this.addr = addr;
+ case challenge:
+ this.type = 1;
+ case notification:
+ this.type = 2;
+ this.state = QuorumPeer.ServerState.LOOKING;
+ case ack:
+ this.type = 3;
+ * Message type: 0 notification, 1 acknowledgement
+ int type;
+ * Proposed leader in the case of notification
+ * id contains the tag for acks, and zxid for notifications
+ * Current state;
+ * Message tag
+ long tag;
+ LinkedBlockingQueue<ToSend> sendqueue;
+ LinkedBlockingQueue<Notification> recvqueue;
+ private class Messenger {
+ DatagramSocket mySocket;
+ long lastProposedLeader;
+ long lastProposedZxid;
+ long lastEpoch;
+ LinkedBlockingQueue<Long> acksqueue;
+ HashMap<Long, Long> challengeMap;
+ HashMap<Long, Long> challengeMutex;
+ HashMap<Long, Long> ackMutex;
+ HashMap<InetSocketAddress, HashMap<Long, Long>> addrChallengeMap;
+ class WorkerReceiver implements Runnable {
+ Messenger myMsg;
+ WorkerReceiver(DatagramSocket s, Messenger msg) {
+ mySocket = s;
+ myMsg = msg;
+ boolean saveChallenge(long tag, long challenge) {
+ Long l = challengeMutex.get(tag);
+ synchronized (challengeMap) {
+ challengeMap.put(tag, challenge);
+ challengeMutex.remove(tag);
+ if (l != null) {
+ synchronized(l){
+ l.notify();
+ byte responseBytes[] = new byte[48];
+ ByteBuffer responseBuffer = ByteBuffer.wrap(responseBytes);
+ DatagramPacket responsePacket = new DatagramPacket(
+ responseBytes, responseBytes.length);
+ // Sleeps on receive
+ responseBuffer.clear();
+ mySocket.receive(responsePacket);
+ LOG.warn("Exception receiving: " + e.toString());
+ // Receive new message
+ if (responsePacket.getLength() != responseBytes.length) {
+ LOG.error("Got a short response: "
+ + responsePacket.getLength() + " "
+ + responsePacket.toString());
+ int type = responseBuffer.getInt();
+ if ((type > 3) || (type < 0)) {
+ LOG.error("Got bad Msg type: " + type);
+ long tag = responseBuffer.getLong();
+ QuorumPeer.ServerState ackstate = QuorumPeer.ServerState.LOOKING;
+ switch (responseBuffer.getInt()) {
+ case 0:
+ ackstate = QuorumPeer.ServerState.LOOKING;
+ case 1:
+ ackstate = QuorumPeer.ServerState.LEADING;
+ case 2:
+ ackstate = QuorumPeer.ServerState.FOLLOWING;
+ // Receive challenge request
+ ToSend c = new ToSend(ToSend.mType.challenge, tag,
+ self.currentVote.id, self.currentVote.zxid,
+ logicalclock, self.getPeerState(),
+ (InetSocketAddress) responsePacket
+ .getSocketAddress());
+ sendqueue.offer(c);
+ // Receive challenge and store somewhere else
+ long challenge = responseBuffer.getLong();
+ saveChallenge(tag, challenge);
+ Notification n = new Notification();
+ n.leader = responseBuffer.getLong();
+ n.zxid = responseBuffer.getLong();
+ n.epoch = responseBuffer.getLong();
+ n.state = ackstate;
+ n.addr = (InetSocketAddress) responsePacket
+ .getSocketAddress();
+ if ((myMsg.lastEpoch <= n.epoch)
+ && ((n.zxid > myMsg.lastProposedZxid)
+ || ((n.zxid == myMsg.lastProposedZxid)
+ && (n.leader > myMsg.lastProposedLeader)))) {
+ myMsg.lastProposedZxid = n.zxid;
+ myMsg.lastProposedLeader = n.leader;
+ myMsg.lastEpoch = n.epoch;
+ long recChallenge;
+ InetSocketAddress addr = (InetSocketAddress) responsePacket
+ if (authEnabled) {
+ if (addrChallengeMap.get(addr).get(tag) != null) {
+ recChallenge = responseBuffer.getLong();
+ if (addrChallengeMap.get(addr).get(tag) == recChallenge) {
+ recvqueue.offer(n);
+ ToSend a = new ToSend(ToSend.mType.ack,
+ tag, self.currentVote.id,
+ self.currentVote.zxid,
+ (InetSocketAddress) addr);
+ sendqueue.offer(a);
+ LOG.warn("Incorrect challenge: "
+ + recChallenge + ", "
+ + addrChallengeMap.toString());
+ LOG.warn("No challenge for host: " + addr
+ + " " + tag);
+ ToSend a = new ToSend(ToSend.mType.ack, tag,
+ // Upon reception of an ack message, remove it from the
+ // queue
+ case 3:
+ Long l = ackMutex.get(tag);
+ acksqueue.offer(tag);
+ addrChallengeMap.get(
+ .getSocketAddress()).remove(tag);
+ if (ackstate != QuorumPeer.ServerState.LOOKING) {
+ Notification outofsync = new Notification();
+ outofsync.leader = responseBuffer.getLong();
+ outofsync.zxid = responseBuffer.getLong();
+ outofsync.epoch = responseBuffer.getLong();
+ outofsync.state = ackstate;
+ outofsync.addr = (InetSocketAddress) responsePacket
+ recvqueue.offer(outofsync);
+ // Default case
+ LOG.warn("Received message of incorrect type");
+ class WorkerSender implements Runnable {
+ Random rand;
+ boolean processing;
+ int maxAttempts;
+ int ackWait = finalizeWait;
+ * Receives a socket and max number of attempts as input
+ WorkerSender(int attempts) {
+ maxAttempts = attempts;
+ rand = new Random(java.lang.Thread.currentThread().getId()
+ + System.currentTimeMillis());
+ long genChallenge() {
+ byte buf[] = new byte[8];
+ buf[0] = (byte) ((challengeCounter & 0xff000000) >>> 24);
+ buf[1] = (byte) ((challengeCounter & 0x00ff0000) >>> 16);
+ buf[2] = (byte) ((challengeCounter & 0x0000ff00) >>> 8);
+ buf[3] = (byte) ((challengeCounter & 0x000000ff));
+ challengeCounter++;
+ int secret = rand.nextInt(java.lang.Integer.MAX_VALUE);
+ buf[4] = (byte) ((secret & 0xff000000) >>> 24);
+ buf[5] = (byte) ((secret & 0x00ff0000) >>> 16);
+ buf[6] = (byte) ((secret & 0x0000ff00) >>> 8);
+ buf[7] = (byte) ((secret & 0x000000ff));
+ return (((long)(buf[0] & 0xFF)) << 56)
+ + (((long)(buf[1] & 0xFF)) << 48)
+ + (((long)(buf[2] & 0xFF)) << 40)
+ + (((long)(buf[3] & 0xFF)) << 32)
+ + (((long)(buf[4] & 0xFF)) << 24)
+ + (((long)(buf[5] & 0xFF)) << 16)
+ + (((long)(buf[6] & 0xFF)) << 8)
+ + ((long)(buf[7] & 0xFF));
+ ToSend m = sendqueue.take();
+ process(m);
+ private void process(ToSend m) {
+ int attempts = 0;
+ byte zeroes[];
+ byte requestBytes[] = new byte[48];
+ DatagramPacket requestPacket = new DatagramPacket(requestBytes,
+ requestBytes.length);
+ ByteBuffer requestBuffer = ByteBuffer.wrap(requestBytes);
+ switch (m.type) {
+ * Building challenge request packet to send
+ requestBuffer.clear();
+ requestBuffer.putInt(ToSend.mType.crequest.ordinal());
+ requestBuffer.putLong(m.tag);
+ requestBuffer.putInt(m.state.ordinal());
+ zeroes = new byte[32];
+ requestBuffer.put(zeroes);
+ requestPacket.setLength(48);
+ requestPacket.setSocketAddress(m.addr);
+ if (challengeMap.get(m.tag) == null) {
+ mySocket.send(requestPacket);
+ LOG.warn("Exception while sending challenge: "
+ + e.toString());
+ * Building challenge packet to send
+ long newChallenge;
+ if (addrChallengeMap.get(m.addr).containsKey(m.tag)) {
+ newChallenge = addrChallengeMap.get(m.addr).get(m.tag);
+ newChallenge = genChallenge();
+ addrChallengeMap.get(m.addr).put(m.tag, newChallenge);
+ requestBuffer.putInt(ToSend.mType.challenge.ordinal());
+ requestBuffer.putLong(newChallenge);
+ zeroes = new byte[24];
+ * Building notification packet to send
+ requestBuffer.putInt(m.type);
+ requestBuffer.putLong(m.leader);
+ requestBuffer.putLong(m.zxid);
+ requestBuffer.putLong(m.epoch);
+ zeroes = new byte[8];
+ boolean myChallenge = false;
+ boolean myAck = false;
+ while (attempts < maxAttempts) {
+ * Try to obtain a challenge only if does not have
+ * one yet
+ if (!myChallenge && authEnabled) {
+ ToSend crequest = new ToSend(
+ ToSend.mType.crequest, m.tag, m.leader,
+ m.zxid, m.epoch,
+ QuorumPeer.ServerState.LOOKING, m.addr);
+ sendqueue.offer(crequest);
+ double timeout = ackWait
+ * java.lang.Math.pow(2, attempts);
+ Long l = Long.valueOf(m.tag);
+ synchronized (l) {
+ challengeMutex.put(m.tag, l);
+ l.wait((long) timeout);
+ myChallenge = challengeMap
+ .containsKey(m.tag);
+ LOG.warn("Challenge request exception: "
+ * If don't have challenge yet, skip sending
+ * notification
+ if (authEnabled && !myChallenge) {
+ attempts++;
+ requestBuffer.position(40);
+ requestBuffer.putLong(challengeMap.get(m.tag));
+ * java.lang.Math.pow(10, attempts);
+ ackMutex.put(m.tag, l);
+ l.wait((int) timeout);
+ LOG.warn("Ack exception: "
+ synchronized (acksqueue) {
+ for (int i = 0; i < acksqueue.size(); ++i) {
+ Long newack = acksqueue.poll();
+ * Under highly concurrent load, a thread
+ * may get into this loop but by the time it
+ * tries to read from the queue, the queue
+ * is empty. There are two alternatives:
+ * synchronize this block, or test if newack
+ * is null.
+ if (newack == m.tag) {
+ myAck = true;
+ } else
+ acksqueue.offer(newack);
+ LOG.warn("Sending exception: "
+ * Do nothing, just try again
+ if (myAck) {
+ * Received ack successfully, so return
+ if (challengeMap.get(m.tag) != null)
+ challengeMap.remove(m.tag);
+ * Return message to queue for another attempt later if
+ * epoch hasn't changed.
+ if (m.epoch == logicalclock) {
+ sendqueue.offer(m);
+ LOG.warn("Exception while sending ack: "
+ public boolean queueEmpty() {
+ return (sendqueue.isEmpty() || acksqueue.isEmpty() || recvqueue
+ .isEmpty());
+ Messenger(int threads, DatagramSocket s) {
+ acksqueue = new LinkedBlockingQueue<Long>();
+ challengeMap = new HashMap<Long, Long>();
+ challengeMutex = new HashMap<Long, Long>();
+ ackMutex = new HashMap<Long, Long>();
+ addrChallengeMap = new HashMap<InetSocketAddress, HashMap<Long, Long>>();
+ lastProposedLeader = 0;
+ lastProposedZxid = 0;
+ lastEpoch = 0;
+ for (int i = 0; i < threads; ++i) {
+ Thread t = new Thread(new WorkerSender(3),
+ "WorkerSender Thread: " + (i + 1));
+ t.setDaemon(true);
+ t.start();
+ for (QuorumServer server : self.quorumPeers) {
+ InetSocketAddress saddr = new InetSocketAddress(server.addr
+ .getAddress(), port);
+ addrChallengeMap.put(saddr, new HashMap<Long, Long>());
+ Thread t = new Thread(new WorkerReceiver(s, this),
+ "WorkerReceiver Thread");
+ QuorumPeer self;
+ int port;
+ long logicalclock; /* Election instance */
+ long proposedLeader;
+ long proposedZxid;
+ public AuthFastLeaderElection(QuorumPeer self, int electionPort,
+ boolean auth) {
+ this.authEnabled = auth;
+ starter(self, electionPort);
+ public AuthFastLeaderElection(QuorumPeer self, int electionPort) {
+ private void starter(QuorumPeer self, int electionPort) {
+ this.self = self;
+ port = electionPort;
+ proposedLeader = -1;
+ proposedZxid = -1;
+ mySocket = new DatagramSocket(port);
+ // mySocket.setSoTimeout(20000);
+ } catch (SocketException e1) {
+ throw new RuntimeException();
+ sendqueue = new LinkedBlockingQueue<ToSend>(2 * self.quorumPeers.size());
+ recvqueue = new LinkedBlockingQueue<Notification>(2 * self.quorumPeers
+ .size());
+ new Messenger(self.quorumPeers.size() * 2, mySocket);
+ private void leaveInstance() {
+ logicalclock++;
+ // sendqueue.clear();
+ // recvqueue.clear();
+ public static class ElectionResult {
+ public Vote vote;
+ public int count;
+ public Vote winner;
+ public int winningCount;
+ private void sendNotifications() {
+ ToSend notmsg = new ToSend(ToSend.mType.notification,
+ AuthFastLeaderElection.sequencer++, proposedLeader,
+ proposedZxid, logicalclock, QuorumPeer.ServerState.LOOKING,
+ saddr);
+ sendqueue.offer(notmsg);
+ private boolean totalOrderPredicate(long id, long zxid) {
+ if ((zxid > proposedZxid)
+ || ((zxid == proposedZxid) && (id > proposedLeader)))
+ private boolean termPredicate(HashMap<InetSocketAddress, Vote> votes,
+ long l, long zxid) {
+ int count = 0;
+ Collection<Vote> votesCast = votes.values();
+ * First make the views consistent. Sometimes peers will have different
+ * zxids for a server depending on timing.
+ for (Vote v : votesCast) {
+ if ((v.id == l) && (v.zxid == zxid))
+ count++;
+ if (count > (self.quorumPeers.size() / 2))
+ public Vote lookForLeader() throws InterruptedException {
+ HashMap<InetSocketAddress, Vote> recvset =
+ new HashMap<InetSocketAddress, Vote>();
+ HashMap<InetSocketAddress, Vote> outofelection =
+ proposedLeader = self.getId();
+ proposedZxid = self.getLastLoggedZxid();
+ LOG.warn("Election tally");
+ sendNotifications();
+ * Loop in which we exchange notifications until we find a leader
+ while (self.getPeerState() == ServerState.LOOKING) {
+ * Remove next notification from queue, times out after 2 times the
+ * termination time
+ Notification n = recvqueue.poll(2 * finalizeWait,
+ TimeUnit.MILLISECONDS);
+ * Sends more notifications if haven't received enough. Otherwise
+ * processes new notification.
+ if (((!outofelection.isEmpty()) || (recvset.size() > 1)))
+ switch (n.state) {
+ case LOOKING:
+ if (n.epoch > logicalclock) {
+ logicalclock = n.epoch;
+ recvset.clear();
+ if (totalOrderPredicate(n.leader, n.zxid)) {
+ proposedLeader = n.leader;
+ proposedZxid = n.zxid;
+ } else if (n.epoch < logicalclock) {
+ } else if (totalOrderPredicate(n.leader, n.zxid)) {
+ recvset.put(n.addr, new Vote(n.leader, n.zxid));
+ // If have received from all nodes, then terminate
+ if (self.quorumPeers.size() == recvset.size()) {
+ self.setPeerState((proposedLeader == self.getId()) ?
+ ServerState.LEADING: ServerState.FOLLOWING);
+ // if (self.state == ServerState.FOLLOWING) {
+ // Thread.sleep(100);
+ // }
+ leaveInstance();
+ return new Vote(proposedLeader, proposedZxid);
+ } else if (termPredicate(recvset, proposedLeader,
+ proposedZxid)) {
+ // Otherwise, wait for a fixed amount of time
+ LOG.warn("Passed predicate");
+ Thread.sleep(finalizeWait);
+ // Notification probe = recvqueue.peek();
+ // Verify if there is any change in the proposed leader
+ while ((!recvqueue.isEmpty())
+ && !totalOrderPredicate(
+ recvqueue.peek().leader, recvqueue
+ .peek().zxid)) {
+ recvqueue.poll();
+ if (recvqueue.isEmpty()) {
+ // LOG.warn("Proposed leader: " +
+ // proposedLeader);
+ case LEADING:
+ outofelection.put(n.addr, new Vote(n.leader, n.zxid));
+ if (termPredicate(outofelection, n.leader, n.zxid)) {
+ self.setPeerState((n.leader == self.getId()) ?
+ return new Vote(n.leader, n.zxid);
+ case FOLLOWING:
@@ -1,174 +1,175 @@
-import com.yahoo.zookeeper.server.Request;
-import com.yahoo.zookeeper.server.RequestProcessor;
- * This RequestProcessor matches the incoming committed requests with the
- * locally submitted requests. The trick is that locally submitted requests that
- * change the state of the system will come back as incoming committed requests,
- * so we need to match them up.
-public class CommitProcessor extends Thread implements RequestProcessor {
- private static final Logger LOG = Logger.getLogger(CommitProcessor.class);
- * Requests that we are holding until the commit comes in.
- LinkedList<Request> queuedRequests = new LinkedList<Request>();
- * Requests that have been committed.
- LinkedList<Request> committedRequests = new LinkedList<Request>();
- * Pending sync requests
- LinkedList<Request> pendingSyncs = new LinkedList<Request>();
- public CommitProcessor(RequestProcessor nextProcessor) {
- boolean finished = false;
- Request nextPending = null;
- ArrayList<Request> toProcess = new ArrayList<Request>();
- while (!finished) {
- int len = toProcess.size();
- for (int i = 0; i < len; i++) {
- nextProcessor.processRequest(toProcess.get(i));
- toProcess.clear();
- if ((queuedRequests.size() == 0 || nextPending != null)
- && committedRequests.size() == 0) {
- wait();
- // First check and see if the commit came in for the pending
- // request
- && committedRequests.size() > 0) {
- Request r = committedRequests.remove();
- * We match with nextPending so that we can move to the
- * next request when it is committed. We also want to
- * use nextPending because it has the cnxn member set
- * properly.
- if (nextPending != null
- && nextPending.sessionId == r.sessionId
- && nextPending.cxid == r.cxid) {
- // we want to send our version of the request.
- // the pointer to the connection in the request
- nextPending.hdr = r.hdr;
- nextPending.txn = r.txn;
- nextPending.zxid = r.zxid;
- toProcess.add(nextPending);
- nextPending = null;
- // this request came from someone else so just
- // send the commit packet
- toProcess.add(r);
- // We haven't matched the pending requests, so go back to
- // waiting
- if (nextPending != null) {
- // Process the next requests in the queuedRequests
- while (nextPending == null && queuedRequests.size() > 0) {
- Request request = queuedRequests.remove();
- nextPending = request;
- pendingSyncs.add(request);
- toProcess.add(request);
- "CommitProcessor exited loop!");
- synchronized public void commit(Request request) {
- if (!finished) {
- if (request == null) {
- LOG.error("FIXMSG",new Exception("committing a null! "));
- committedRequests.add(request);
- synchronized public void processRequest(Request request) {
- // request.addRQRec(">commit");
- // LOG.warn("Zoo processReq>>> cxid = " + request.cxid + " type =
- // " + request.type + " id = " + request.sessionId + " cnxn " +
- finished = true;
- queuedRequests.clear();
+import com.yahoo.zookeeper.server.Request;
+import com.yahoo.zookeeper.server.RequestProcessor;
+ * This RequestProcessor matches the incoming committed requests with the
+ * locally submitted requests. The trick is that locally submitted requests that
+ * change the state of the system will come back as incoming committed requests,
+ * so we need to match them up.
+public class CommitProcessor extends Thread implements RequestProcessor {
+ private static final Logger LOG = Logger.getLogger(CommitProcessor.class);
+ * Requests that we are holding until the commit comes in.
+ LinkedList<Request> queuedRequests = new LinkedList<Request>();
+ * Requests that have been committed.
+ LinkedList<Request> committedRequests = new LinkedList<Request>();
+ * Pending sync requests
+ LinkedList<Request> pendingSyncs = new LinkedList<Request>();
+ public CommitProcessor(RequestProcessor nextProcessor) {
+ boolean finished = false;
+ Request nextPending = null;
+ ArrayList<Request> toProcess = new ArrayList<Request>();
+ while (!finished) {
+ int len = toProcess.size();
+ for (int i = 0; i < len; i++) {
+ nextProcessor.processRequest(toProcess.get(i));
+ toProcess.clear();
+ if ((queuedRequests.size() == 0 || nextPending != null)
+ && committedRequests.size() == 0) {
+ wait();
+ // First check and see if the commit came in for the pending
+ // request
+ && committedRequests.size() > 0) {
+ Request r = committedRequests.remove();
+ * We match with nextPending so that we can move to the
+ * next request when it is committed. We also want to
+ * use nextPending because it has the cnxn member set
+ * properly.
+ if (nextPending != null
+ && nextPending.sessionId == r.sessionId
+ && nextPending.cxid == r.cxid) {
+ // we want to send our version of the request.
+ // the pointer to the connection in the request
+ nextPending.hdr = r.hdr;
+ nextPending.txn = r.txn;
+ nextPending.zxid = r.zxid;
+ toProcess.add(nextPending);
+ nextPending = null;
+ // this request came from someone else so just
+ // send the commit packet
+ toProcess.add(r);
+ // We haven't matched the pending requests, so go back to
+ // waiting
+ if (nextPending != null) {
+ // Process the next requests in the queuedRequests
+ while (nextPending == null && queuedRequests.size() > 0) {
+ Request request = queuedRequests.remove();
+ nextPending = request;
+ pendingSyncs.add(request);
+ toProcess.add(request);
+ "CommitProcessor exited loop!");
+ synchronized public void commit(Request request) {
+ if (!finished) {
+ if (request == null) {
+ LOG.warn("Committed a null!",
+ new Exception("committing a null! "));
+ committedRequests.add(request);
+ synchronized public void processRequest(Request request) {
+ // request.addRQRec(">commit");
+ // LOG.info("Zoo processReq>>> cxid = " + request.cxid + " type =
+ // " + request.type + " id = " + request.sessionId + " cnxn " +
+ finished = true;
+ queuedRequests.clear();
@@ -1,366 +1,366 @@
- * This class has the control logic for the Follower.
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.net.ConnectException;
-import com.yahoo.jute.OutputArchive;
-public class Follower {
- private static final Logger LOG = Logger.getLogger(Follower.class);
- FollowerZooKeeperServer zk;
- Follower(QuorumPeer self,FollowerZooKeeperServer zk) {
- this.zk=zk;
- private InputArchive leaderIs;
- private OutputArchive leaderOs;
- private BufferedOutputStream bufferedOutput;
- public Socket sock;
- * write a packet to the leader
- * @param pp
- * the proposal packet to be sent to the leader
- void writePacket(QuorumPacket pp) throws IOException {
- long traceMask = ZooTrace.SERVER_PACKET_TRACE_MASK;
- if (pp.getType() == Leader.PING) {
- ZooTrace.logQuorumPacket(LOG, traceMask, 'o', pp);
- synchronized (leaderOs) {
- leaderOs.writeRecord(pp, "packet");
- bufferedOutput.flush();
- * read a packet from the leader
- * the packet to be instantiated
- void readPacket(QuorumPacket pp) throws IOException {
- synchronized (leaderIs) {
- leaderIs.readRecord(pp, "packet");
- ZooTrace.logQuorumPacket(LOG, traceMask, 'i', pp);
- * the main method called by the follower to follow the leader
- void followLeader() throws InterruptedException {
- InetSocketAddress addr = null;
- // Find the leader by id
- for (QuorumServer s : self.quorumPeers) {
- if (s.id == self.currentVote.id) {
- addr = s.addr;
- if (addr == null) {
- LOG.error("Couldn't find the leader with id = "
- + self.currentVote.id);
- LOG.warn("Following " + addr);
- sock = new Socket();
- QuorumPacket ack = new QuorumPacket(Leader.ACK, 0, null, null);
- sock.setSoTimeout(self.tickTime * self.initLimit);
- for (int tries = 0; tries < 5; tries++) {
- //sock = new Socket();
- //sock.setSoTimeout(self.tickTime * self.initLimit);
- sock.connect(addr, self.tickTime * self.syncLimit);
- sock.setTcpNoDelay(true);
- } catch (ConnectException e) {
- if (tries == 4) {
- else{
- leaderIs = BinaryInputArchive.getArchive(new BufferedInputStream(
- sock.getInputStream()));
- bufferedOutput = new BufferedOutputStream(sock.getOutputStream());
- leaderOs = BinaryOutputArchive.getArchive(bufferedOutput);
- QuorumPacket qp = new QuorumPacket();
- qp.setType(Leader.LASTZXID);
- long sentLastZxid = self.getLastLoggedZxid();
- qp.setZxid(sentLastZxid);
- writePacket(qp);
- readPacket(qp);
- long newLeaderZxid = qp.getZxid();
- if (qp.getType() != Leader.NEWLEADER) {
- LOG.error("First packet should have been NEWLEADER");
- throw new IOException("First packet should have been NEWLEADER");
- synchronized (zk) {
- if (qp.getType() == Leader.DIFF) {
- LOG.warn("Getting a diff from the leader!");
- zk.loadData();
- else if (qp.getType() == Leader.SNAP) {
- LOG.warn("Getting a snapshot from leader");
- // The leader is going to dump the database
- zk.loadData(leaderIs);
- String signature = leaderIs.readString("signature");
- if (!signature.equals("BenWasHere")) {
- LOG.error("Missing signature. Got " + signature);
- throw new IOException("Missing signature");
- } else if (qp.getType() == Leader.TRUNC) {
- //we need to truncate the log to the lastzxid of the leader
- LOG.warn("Truncating log to get in sync with the leader "
- + Long.toHexString(qp.getZxid()));
- zk.truncateLog(qp.getZxid());
- else {
- LOG.error("Got unexpected packet from leader "
- + qp.getType() + " exiting ... " );
- zk.dataTree.lastProcessedZxid = newLeaderZxid;
- ack.setZxid(newLeaderZxid & ~0xffffffffL);
- writePacket(ack);
- sock.setSoTimeout(self.tickTime * self.syncLimit);
- while (self.running) {
- switch (qp.getType()) {
- case Leader.PING:
- // Send back the ping with our session data
- ByteArrayOutputStream bos = new ByteArrayOutputStream();
- DataOutputStream dos = new DataOutputStream(bos);
- HashMap<Long, Integer> touchTable = ((FollowerZooKeeperServer) zk)
- .getTouchSnapshot();
- for (Entry<Long, Integer> entry : touchTable.entrySet()) {
- dos.writeLong(entry.getKey());
- dos.writeInt(entry.getValue());
- qp.setData(bos.toByteArray());
- case Leader.PROPOSAL:
- BinaryInputArchive ia = BinaryInputArchive
- .getArchive(new ByteArrayInputStream(qp.getData()));
- Record txn = ZooKeeperServer.deserializeTxn(ia, hdr);
- if (hdr.getZxid() != lastQueued + 1) {
- LOG.warn("Got zxid "
- + Long.toHexString(hdr.getZxid())
- + " expected "
- + Long.toHexString(lastQueued + 1));
- lastQueued = hdr.getZxid();
- zk.logRequest(hdr, txn);
- case Leader.COMMIT:
- zk.commit(qp.getZxid());
- case Leader.UPTODATE:
- zk.snapshot();
- self.cnxnFactory.setZooKeeperServer(zk);
- case Leader.REVALIDATE:
- ByteArrayInputStream bis = new ByteArrayInputStream(qp
- .getData());
- DataInputStream dis = new DataInputStream(bis);
- long sessionId = dis.readLong();
- boolean valid = dis.readBoolean();
- synchronized (pendingRevalidations) {
- ServerCnxn cnxn = pendingRevalidations
- .remove(sessionId);
- LOG.warn("Missing "
- + Long.toHexString(sessionId)
- + " for validation");
- cnxn.finishSessionInit(valid);
- "Session " + sessionId
- + " is valid: " + valid);
- case Leader.SYNC:
- zk.sync();
- // clear pending revalitions
- pendingRevalidations.clear();
- pendingRevalidations.notifyAll();
- private long lastQueued;
- ConcurrentHashMap<Long, ServerCnxn> pendingRevalidations = new ConcurrentHashMap<Long, ServerCnxn>();
- * validate a seesion for a client
- * @param clientId
- * the client to be revailidated
- * @param timeout
- * the timeout for which the session is valid
- void validateSession(ServerCnxn cnxn, long clientId, int timeout)
- throws IOException, InterruptedException {
- DataOutputStream dos = new DataOutputStream(baos);
- dos.writeLong(clientId);
- dos.writeInt(timeout);
- dos.close();
- QuorumPacket qp = new QuorumPacket(Leader.REVALIDATE, -1, baos
- .toByteArray(), null);
- pendingRevalidations.put(clientId, cnxn);
- "To validate session "
- + Long.toHexString(clientId));
- * send a request packet to the leader
- * the request from the client
- void request(Request request) throws IOException {
- DataOutputStream oa = new DataOutputStream(baos);
- oa.writeLong(request.sessionId);
- oa.writeInt(request.cxid);
- oa.writeInt(request.type);
- if (request.request != null) {
- int len = request.request.remaining();
- byte b[] = new byte[len];
- request.request.get(b);
- oa.write(b);
- oa.close();
- QuorumPacket qp = new QuorumPacket(Leader.REQUEST, -1, baos
- .toByteArray(), request.authInfo);
-// QuorumPacket qp;
-// if(request.type == OpCode.sync){
-// qp = new QuorumPacket(Leader.SYNC, -1, baos
-// .toByteArray(), request.authInfo);
-// }
-// else{
-// qp = new QuorumPacket(Leader.REQUEST, -1, baos
- return zk.getZxid();
- } catch (NullPointerException e) {
- // set the zookeeper server to null
- self.cnxnFactory.setZooKeeperServer(null);
- // clear all the connections
- self.cnxnFactory.clear();
- // shutdown previous zookeeper
- LOG.error("FIXMSG",new Exception("shutdown Follower"));
+ * This class has the control logic for the Follower.
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.net.ConnectException;
+import com.yahoo.jute.OutputArchive;
+public class Follower {
+ private static final Logger LOG = Logger.getLogger(Follower.class);
+ FollowerZooKeeperServer zk;
+ Follower(QuorumPeer self,FollowerZooKeeperServer zk) {
+ this.zk=zk;
+ private InputArchive leaderIs;
+ private OutputArchive leaderOs;
+ private BufferedOutputStream bufferedOutput;
+ public Socket sock;
+ * write a packet to the leader
+ * @param pp
+ * the proposal packet to be sent to the leader
+ void writePacket(QuorumPacket pp) throws IOException {
+ long traceMask = ZooTrace.SERVER_PACKET_TRACE_MASK;
+ if (pp.getType() == Leader.PING) {
+ ZooTrace.logQuorumPacket(LOG, traceMask, 'o', pp);
+ synchronized (leaderOs) {
+ leaderOs.writeRecord(pp, "packet");
+ bufferedOutput.flush();
+ * read a packet from the leader
+ * the packet to be instantiated
+ void readPacket(QuorumPacket pp) throws IOException {
+ synchronized (leaderIs) {
+ leaderIs.readRecord(pp, "packet");
+ ZooTrace.logQuorumPacket(LOG, traceMask, 'i', pp);
+ * the main method called by the follower to follow the leader
+ void followLeader() throws InterruptedException {
+ InetSocketAddress addr = null;
+ // Find the leader by id
+ for (QuorumServer s : self.quorumPeers) {
+ if (s.id == self.currentVote.id) {
+ addr = s.addr;
+ if (addr == null) {
+ LOG.warn("Couldn't find the leader with id = "
+ + self.currentVote.id);
+ LOG.info("Following " + addr);
+ sock = new Socket();
+ QuorumPacket ack = new QuorumPacket(Leader.ACK, 0, null, null);
+ sock.setSoTimeout(self.tickTime * self.initLimit);
+ for (int tries = 0; tries < 5; tries++) {
+ //sock = new Socket();
+ //sock.setSoTimeout(self.tickTime * self.initLimit);
+ sock.connect(addr, self.tickTime * self.syncLimit);
+ sock.setTcpNoDelay(true);
+ } catch (ConnectException e) {
+ if (tries == 4) {
+ LOG.error("Unexpected exception",e);
+ leaderIs = BinaryInputArchive.getArchive(new BufferedInputStream(
+ sock.getInputStream()));
+ bufferedOutput = new BufferedOutputStream(sock.getOutputStream());
+ leaderOs = BinaryOutputArchive.getArchive(bufferedOutput);
+ QuorumPacket qp = new QuorumPacket();
+ qp.setType(Leader.LASTZXID);
+ long sentLastZxid = self.getLastLoggedZxid();
+ qp.setZxid(sentLastZxid);
+ writePacket(qp);
+ readPacket(qp);
+ long newLeaderZxid = qp.getZxid();
+ if (qp.getType() != Leader.NEWLEADER) {
+ LOG.error("First packet should have been NEWLEADER");
+ throw new IOException("First packet should have been NEWLEADER");
+ synchronized (zk) {
+ if (qp.getType() == Leader.DIFF) {
+ LOG.info("Getting a diff from the leader!");
+ zk.loadData();
+ else if (qp.getType() == Leader.SNAP) {
+ LOG.info("Getting a snapshot from leader");
+ // The leader is going to dump the database
+ zk.loadData(leaderIs);
+ String signature = leaderIs.readString("signature");
+ if (!signature.equals("BenWasHere")) {
+ LOG.error("Missing signature. Got " + signature);
+ throw new IOException("Missing signature");
+ } else if (qp.getType() == Leader.TRUNC) {
+ //we need to truncate the log to the lastzxid of the leader
+ LOG.warn("Truncating log to get in sync with the leader "
+ + Long.toHexString(qp.getZxid()));
+ zk.truncateLog(qp.getZxid());
+ else {
+ LOG.error("Got unexpected packet from leader "
+ + qp.getType() + " exiting ... " );
+ zk.dataTree.lastProcessedZxid = newLeaderZxid;
+ ack.setZxid(newLeaderZxid & ~0xffffffffL);
+ writePacket(ack);
+ sock.setSoTimeout(self.tickTime * self.syncLimit);
+ while (self.running) {
+ switch (qp.getType()) {
+ case Leader.PING:
+ // Send back the ping with our session data
+ ByteArrayOutputStream bos = new ByteArrayOutputStream();
+ DataOutputStream dos = new DataOutputStream(bos);
+ HashMap<Long, Integer> touchTable = ((FollowerZooKeeperServer) zk)
+ .getTouchSnapshot();
+ for (Entry<Long, Integer> entry : touchTable.entrySet()) {
+ dos.writeLong(entry.getKey());
+ dos.writeInt(entry.getValue());
+ qp.setData(bos.toByteArray());
+ case Leader.PROPOSAL:
+ BinaryInputArchive ia = BinaryInputArchive
+ .getArchive(new ByteArrayInputStream(qp.getData()));
+ Record txn = ZooKeeperServer.deserializeTxn(ia, hdr);
+ if (hdr.getZxid() != lastQueued + 1) {
+ LOG.warn("Got zxid "
+ + Long.toHexString(hdr.getZxid())
+ + " expected "
+ + Long.toHexString(lastQueued + 1));
+ lastQueued = hdr.getZxid();
+ zk.logRequest(hdr, txn);
+ case Leader.COMMIT:
+ zk.commit(qp.getZxid());
+ case Leader.UPTODATE:
+ zk.snapshot();
+ self.cnxnFactory.setZooKeeperServer(zk);
+ case Leader.REVALIDATE:
+ ByteArrayInputStream bis = new ByteArrayInputStream(qp
+ .getData());
+ DataInputStream dis = new DataInputStream(bis);
+ long sessionId = dis.readLong();
+ boolean valid = dis.readBoolean();
+ synchronized (pendingRevalidations) {
+ ServerCnxn cnxn = pendingRevalidations
+ .remove(sessionId);
+ LOG.warn("Missing "
+ + Long.toHexString(sessionId)
+ + " for validation");
+ cnxn.finishSessionInit(valid);
+ "Session " + sessionId
+ + " is valid: " + valid);
+ case Leader.SYNC:
+ zk.sync();
+ e.printStackTrace();
+ // clear pending revalitions
+ pendingRevalidations.clear();
+ pendingRevalidations.notifyAll();
+ private long lastQueued;
+ ConcurrentHashMap<Long, ServerCnxn> pendingRevalidations = new ConcurrentHashMap<Long, ServerCnxn>();
+ * validate a seesion for a client
+ * @param clientId
+ * the client to be revailidated
+ * @param timeout
+ * the timeout for which the session is valid
+ void validateSession(ServerCnxn cnxn, long clientId, int timeout)
+ throws IOException, InterruptedException {
+ DataOutputStream dos = new DataOutputStream(baos);
+ dos.writeLong(clientId);
+ dos.writeInt(timeout);
+ dos.close();
+ QuorumPacket qp = new QuorumPacket(Leader.REVALIDATE, -1, baos
+ .toByteArray(), null);
+ pendingRevalidations.put(clientId, cnxn);
+ "To validate session "
+ + Long.toHexString(clientId));
+ * send a request packet to the leader
+ * the request from the client
+ void request(Request request) throws IOException {
+ DataOutputStream oa = new DataOutputStream(baos);
+ oa.writeLong(request.sessionId);
+ oa.writeInt(request.cxid);
+ oa.writeInt(request.type);
+ if (request.request != null) {
+ int len = request.request.remaining();
+ byte b[] = new byte[len];
+ request.request.get(b);
+ oa.write(b);
+ oa.close();
+ QuorumPacket qp = new QuorumPacket(Leader.REQUEST, -1, baos
+ .toByteArray(), request.authInfo);
+// QuorumPacket qp;
+// if(request.type == OpCode.sync){
+// qp = new QuorumPacket(Leader.SYNC, -1, baos
+// .toByteArray(), request.authInfo);
+// }
+// else{
+// qp = new QuorumPacket(Leader.REQUEST, -1, baos
+ return zk.getZxid();
+ } catch (NullPointerException e) {
+ // set the zookeeper server to null
+ self.cnxnFactory.setZooKeeperServer(null);
+ // clear all the connections
+ self.cnxnFactory.clear();
+ // shutdown previous zookeeper
+ LOG.error("FIXMSG",new Exception("shutdown Follower"));
@@ -1,390 +1,389 @@
- * There will be an instance of this class created by the Leader for each
- * follower.All communication for a given Follower will be handled by this
- * class.
-public class FollowerHandler extends Thread {
- private static final Logger LOG = Logger.getLogger(FollowerHandler.class);
- public Socket s;
- Leader leader;
- long tickOfLastAck;
- * The packets to be sent to the follower
- LinkedBlockingQueue<QuorumPacket> queuedPackets = new LinkedBlockingQueue<QuorumPacket>();
- private BinaryInputArchive ia;
- private BinaryOutputArchive oa;
- FollowerHandler(Socket s, Leader leader) throws IOException {
- super("FollowerHandler-" + s.getRemoteSocketAddress());
- this.s = s;
- leader.addFollowerHandler(this);
- * If this packet is queued, the sender thread will exit
- QuorumPacket proposalOfDeath = new QuorumPacket();
- * This method will use the thread to send packets added to the
- * queuedPackets list
- private void sendPackets() throws InterruptedException {
- QuorumPacket p;
- p = queuedPackets.take();
- if (p == proposalOfDeath) {
- // Packet of death!
- if (p.getType() == Leader.PING) {
- ZooTrace.logQuorumPacket(LOG, traceMask, 'o', p);
- oa.writeRecord(p, "packet");
- if (!s.isClosed()) {
- static public String packetToString(QuorumPacket p) {
- if (true)
- String type = null;
- String mess = null;
- switch (p.getType()) {
- case Leader.ACK:
- type = "ACK";
- type = "COMMIT";
- case Leader.LASTZXID:
- type = "LASTZXID";
- case Leader.NEWLEADER:
- type = "NEWLEADER";
- type = "PING";
- type = "PROPOSAL";
- .getArchive(new ByteArrayInputStream(p.getData()));
- txn = ZooKeeperServer.deserializeTxn(ia, hdr);
- // mess = "transaction: " + txn.toString();
- case Leader.REQUEST:
- type = "REQUEST";
- type = "REVALIDATE";
- ByteArrayInputStream bis = new ByteArrayInputStream(p.getData());
- long id = dis.readLong();
- mess = " sessionid = " + id;
- type = "UPTODATE";
- type = "UNKNOWN" + p.getType();
- String entry = null;
- if (type != null) {
- entry = type + " " + Long.toHexString(p.getZxid()) + " " + mess;
- return entry;
- * This thread will receive packets from the follower and process them and
- * also listen to new connections from new followers.
- ia = BinaryInputArchive.getArchive(new BufferedInputStream(s
- .getInputStream()));
- bufferedOutput = new BufferedOutputStream(s.getOutputStream());
- oa = BinaryOutputArchive.getArchive(bufferedOutput);
- ia.readRecord(qp, "packet");
- if (qp.getType() != Leader.LASTZXID) {
- LOG.error("First packet " + qp.toString()
- + " is not LASTZXID!");
- long peerLastZxid = qp.getZxid();
- int packetToSend = Leader.SNAP;
- boolean logTxns = true;
- long zxidToSend = 0;
- // we are sending the diff
- synchronized(leader.zk.committedLog) {
- if (leader.zk.committedLog.size() != 0) {
- if ((leader.zk.maxCommittedLog >= peerLastZxid)
- && (leader.zk.minCommittedLog <= peerLastZxid)) {
- packetToSend = Leader.DIFF;
- zxidToSend = leader.zk.maxCommittedLog;
- for (Proposal propose: leader.zk.committedLog) {
- if (propose.packet.getZxid() > peerLastZxid) {
- queuePacket(propose.packet);
- QuorumPacket qcommit = new QuorumPacket(Leader.COMMIT, propose.packet.getZxid(),
- queuePacket(qcommit);
- logTxns = false;
- } }
- long leaderLastZxid = leader.startForwarding(this, peerLastZxid);
- QuorumPacket newLeaderQP = new QuorumPacket(Leader.NEWLEADER,
- leaderLastZxid, null, null);
- oa.writeRecord(newLeaderQP, "packet");
- // a special case when both the ids are the same
- if (peerLastZxid == leaderLastZxid) {
- zxidToSend = leaderLastZxid;
- //check if we decided to send a diff or we need to send a truncate
- // we avoid using epochs for truncating because epochs make things
- // complicated. Two epochs might have the last 32 bits as same.
- // only if we know that there is a committed zxid in the queue that
- // is less than the one the peer has we send a trunc else to make
- // things simple we just send sanpshot.
- if (logTxns && (peerLastZxid > leader.zk.maxCommittedLog)) {
- // this is the only case that we are sure that
- // we can ask the follower to truncate the log
- packetToSend = Leader.TRUNC;
- oa.writeRecord(new QuorumPacket(packetToSend, zxidToSend, null, null), "packet");
- // only if we are not truncating or fast sycning
- if (packetToSend == Leader.SNAP) {
- LOG.warn("Sending snapshot last zxid of peer is "
- + Long.toHexString(peerLastZxid) + " " + " zxid of leader is "
- + Long.toHexString(leaderLastZxid));
- // Dump data to follower
- leader.zk.snapshot(oa);
- oa.writeString("BenWasHere", "signature");
- //
- // Mutation packets will be queued during the serialize,
- // so we need to mark when the follower can actually start
- // using the data
- queuedPackets
- .add(new QuorumPacket(Leader.UPTODATE, -1, null, null));
- // Start sending packets
- new Thread() {
- Thread.currentThread().setName(
- "Sender-" + s.getRemoteSocketAddress());
- sendPackets();
- }.start();
- qp = new QuorumPacket();
- if (qp.getType() == Leader.PING) {
- ZooTrace.logQuorumPacket(LOG, traceMask, 'i', qp);
- tickOfLastAck = leader.self.tick;
- int cxid;
- leader.processAck(qp.getZxid(), s.getLocalSocketAddress());
- // Process the touches
- while (dis.available() > 0) {
- long sess = dis.readLong();
- int to = dis.readInt();
- leader.zk.touch(sess, to);
- bis = new ByteArrayInputStream(qp.getData());
- dis = new DataInputStream(bis);
- dos.writeLong(id);
- boolean valid = leader.zk.touch(id, to);
- "Session " + Long.toHexString(id)
- + " is valid: "+ valid);
- dos.writeBoolean(valid);
- queuedPackets.add(qp);
- bb = ByteBuffer.wrap(qp.getData());
- sessionId = bb.getLong();
- cxid = bb.getInt();
- type = bb.getInt();
- bb = bb.slice();
- if(type == OpCode.sync){
- leader.setSyncHandler(this, sessionId);
- leader.zk.submitRequest(null, sessionId, type, cxid, bb,
- qp.getAuthinfo());
- if (s != null && !s.isClosed()) {
- LOG.warn("******* GOODBYE " + s.getRemoteSocketAddress()
- + " ********");
- // Send the packet of death
- queuedPackets.put(proposalOfDeath);
- shutdown();
- s.close();
- leader.removeFollowerHandler(this);
- public long tickOfLastAck() {
- return tickOfLastAck;
- * ping calls from the leader to the followers
- public void ping() {
- QuorumPacket ping = new QuorumPacket(Leader.PING, leader.lastProposed,
- queuePacket(ping);
- void queuePacket(QuorumPacket p) {
- queuedPackets.add(p);
- public boolean synced() {
- return isAlive()
- && tickOfLastAck >= leader.self.tick - leader.self.syncLimit;
+ * There will be an instance of this class created by the Leader for each
+ * follower.All communication for a given Follower will be handled by this
+ * class.
+public class FollowerHandler extends Thread {
+ private static final Logger LOG = Logger.getLogger(FollowerHandler.class);
+ public Socket s;
+ Leader leader;
+ long tickOfLastAck;
+ * The packets to be sent to the follower
+ LinkedBlockingQueue<QuorumPacket> queuedPackets = new LinkedBlockingQueue<QuorumPacket>();
+ private BinaryInputArchive ia;
+ private BinaryOutputArchive oa;
+ FollowerHandler(Socket s, Leader leader) throws IOException {
+ super("FollowerHandler-" + s.getRemoteSocketAddress());
+ this.s = s;
+ leader.addFollowerHandler(this);
+ * If this packet is queued, the sender thread will exit
+ QuorumPacket proposalOfDeath = new QuorumPacket();
+ * This method will use the thread to send packets added to the
+ * queuedPackets list
+ private void sendPackets() throws InterruptedException {
+ QuorumPacket p;
+ p = queuedPackets.take();
+ if (p == proposalOfDeath) {
+ // Packet of death!
+ if (p.getType() == Leader.PING) {
+ ZooTrace.logQuorumPacket(LOG, traceMask, 'o', p);
+ oa.writeRecord(p, "packet");
+ if (!s.isClosed()) {
+ static public String packetToString(QuorumPacket p) {
+ if (true)
+ String type = null;
+ String mess = null;
+ switch (p.getType()) {
+ case Leader.ACK:
+ type = "ACK";
+ type = "COMMIT";
+ case Leader.LASTZXID:
+ type = "LASTZXID";
+ case Leader.NEWLEADER:
+ type = "NEWLEADER";
+ type = "PING";
+ type = "PROPOSAL";
+ .getArchive(new ByteArrayInputStream(p.getData()));
+ txn = ZooKeeperServer.deserializeTxn(ia, hdr);
+ // mess = "transaction: " + txn.toString();
+ case Leader.REQUEST:
+ type = "REQUEST";
+ type = "REVALIDATE";
+ ByteArrayInputStream bis = new ByteArrayInputStream(p.getData());
+ long id = dis.readLong();
+ mess = " sessionid = " + id;
+ type = "UPTODATE";
+ type = "UNKNOWN" + p.getType();
+ String entry = null;
+ if (type != null) {
+ entry = type + " " + Long.toHexString(p.getZxid()) + " " + mess;
+ return entry;
+ * This thread will receive packets from the follower and process them and
+ * also listen to new connections from new followers.
+ ia = BinaryInputArchive.getArchive(new BufferedInputStream(s
+ .getInputStream()));
+ bufferedOutput = new BufferedOutputStream(s.getOutputStream());
+ oa = BinaryOutputArchive.getArchive(bufferedOutput);
+ ia.readRecord(qp, "packet");
+ if (qp.getType() != Leader.LASTZXID) {
+ LOG.error("First packet " + qp.toString()
+ + " is not LASTZXID!");
+ long peerLastZxid = qp.getZxid();
+ int packetToSend = Leader.SNAP;
+ boolean logTxns = true;
+ long zxidToSend = 0;
+ // we are sending the diff
+ synchronized(leader.zk.committedLog) {
+ if (leader.zk.committedLog.size() != 0) {
+ if ((leader.zk.maxCommittedLog >= peerLastZxid)
+ && (leader.zk.minCommittedLog <= peerLastZxid)) {
+ packetToSend = Leader.DIFF;
+ zxidToSend = leader.zk.maxCommittedLog;
+ for (Proposal propose: leader.zk.committedLog) {
+ if (propose.packet.getZxid() > peerLastZxid) {
+ queuePacket(propose.packet);
+ QuorumPacket qcommit = new QuorumPacket(Leader.COMMIT, propose.packet.getZxid(),
+ queuePacket(qcommit);
+ logTxns = false;
+ } }
+ long leaderLastZxid = leader.startForwarding(this, peerLastZxid);
+ QuorumPacket newLeaderQP = new QuorumPacket(Leader.NEWLEADER,
+ leaderLastZxid, null, null);
+ oa.writeRecord(newLeaderQP, "packet");
+ // a special case when both the ids are the same
+ if (peerLastZxid == leaderLastZxid) {
+ zxidToSend = leaderLastZxid;
+ //check if we decided to send a diff or we need to send a truncate
+ // we avoid using epochs for truncating because epochs make things
+ // complicated. Two epochs might have the last 32 bits as same.
+ // only if we know that there is a committed zxid in the queue that
+ // is less than the one the peer has we send a trunc else to make
+ // things simple we just send sanpshot.
+ if (logTxns && (peerLastZxid > leader.zk.maxCommittedLog)) {
+ // this is the only case that we are sure that
+ // we can ask the follower to truncate the log
+ packetToSend = Leader.TRUNC;
+ oa.writeRecord(new QuorumPacket(packetToSend, zxidToSend, null, null), "packet");
+ // only if we are not truncating or fast sycning
+ if (packetToSend == Leader.SNAP) {
+ LOG.warn("Sending snapshot last zxid of peer is "
+ + Long.toHexString(peerLastZxid) + " " + " zxid of leader is "
+ + Long.toHexString(leaderLastZxid));
+ // Dump data to follower
+ leader.zk.snapshot(oa);
+ oa.writeString("BenWasHere", "signature");
+ //
+ // Mutation packets will be queued during the serialize,
+ // so we need to mark when the follower can actually start
+ // using the data
+ queuedPackets
+ .add(new QuorumPacket(Leader.UPTODATE, -1, null, null));
+ // Start sending packets
+ new Thread() {
+ Thread.currentThread().setName(
+ "Sender-" + s.getRemoteSocketAddress());
+ sendPackets();
+ }.start();
+ qp = new QuorumPacket();
+ if (qp.getType() == Leader.PING) {
+ ZooTrace.logQuorumPacket(LOG, traceMask, 'i', qp);
+ tickOfLastAck = leader.self.tick;
+ int cxid;
+ leader.processAck(qp.getZxid(), s.getLocalSocketAddress());
+ // Process the touches
+ while (dis.available() > 0) {
+ long sess = dis.readLong();
+ int to = dis.readInt();
+ leader.zk.touch(sess, to);
+ bis = new ByteArrayInputStream(qp.getData());
+ dis = new DataInputStream(bis);
+ dos.writeLong(id);
+ boolean valid = leader.zk.touch(id, to);
+ "Session " + Long.toHexString(id)
+ + " is valid: "+ valid);
+ dos.writeBoolean(valid);
+ queuedPackets.add(qp);
+ bb = ByteBuffer.wrap(qp.getData());
+ sessionId = bb.getLong();
+ cxid = bb.getInt();
+ type = bb.getInt();
+ bb = bb.slice();
+ if(type == OpCode.sync){
+ leader.setSyncHandler(this, sessionId);
+ leader.zk.submitRequest(null, sessionId, type, cxid, bb,
+ qp.getAuthinfo());
+ if (s != null && !s.isClosed()) {
+ LOG.warn("******* GOODBYE " + s.getRemoteSocketAddress()
+ + " ********");
+ // Send the packet of death
+ queuedPackets.put(proposalOfDeath);
+ shutdown();
+ s.close();
+ leader.removeFollowerHandler(this);
+ public long tickOfLastAck() {
+ return tickOfLastAck;
+ * ping calls from the leader to the followers
+ public void ping() {
+ QuorumPacket ping = new QuorumPacket(Leader.PING, leader.lastProposed,
+ queuePacket(ping);
+ void queuePacket(QuorumPacket p) {
+ queuedPackets.add(p);
+ public boolean synced() {
+ return isAlive()
+ && tickOfLastAck >= leader.self.tick - leader.self.syncLimit;
@@ -1,571 +1,571 @@
-import static com.yahoo.zookeeper.server.ServerConfig.getClientPort;
-import static com.yahoo.zookeeper.server.ServerConfig.getDataDir;
-import static com.yahoo.zookeeper.server.ServerConfig.getDataLogDir;
-import static com.yahoo.zookeeper.server.quorum.QuorumPeerConfig.getElectionAlg;
-import static com.yahoo.zookeeper.server.quorum.QuorumPeerConfig.getElectionPort;
-import static com.yahoo.zookeeper.server.quorum.QuorumPeerConfig.getInitLimit;
-import static com.yahoo.zookeeper.server.quorum.QuorumPeerConfig.getServerId;
-import static com.yahoo.zookeeper.server.quorum.QuorumPeerConfig.getServers;
-import static com.yahoo.zookeeper.server.quorum.QuorumPeerConfig.getSyncLimit;
-import static com.yahoo.zookeeper.server.quorum.QuorumPeerConfig.getTickTime;
-import com.yahoo.zookeeper.server.NIOServerCnxn;
- * This class manages the quorum protocol. There are three states this server
- * can be in:
- * <ol>
- * <li>Leader election - each server will elect a leader (proposing itself as a
- * leader initially).</li>
- * <li>Follower - the server will synchronize with the leader and replicate any
- * transactions.</li>
- * <li>Leader - the server will process requests and forward them to followers.
- * A majority of followers must log the request before it can be accepted.
- * </ol>
- * This class will setup a datagram socket that will always respond with its
- * view of the current leader. The response will take the form of:
- * <pre>
- * int xid;
- * long myid;
- * long leader_id;
- * long leader_zxid;
- * </pre>
- * The request for the current leader will consist solely of an xid: int xid;
- * <h2>Configuration file</h2>
- * When the main() method of this class is used to start the program, the file
- * "zoo.cfg" in the current directory will be used to obtain configuration
- * information. zoo.cfg is a Properties file, so keys and values are separated
- * by equals (=) and the key/value pairs are separated by new lines. The
- * following keys are used in the configuration file:
- * <li>dataDir - The directory where the zookeeper data is stored.</li>
- * <li>clientPort - The port used to communicate with clients.</li>
- * <li>tickTime - The duration of a tick in milliseconds. This is the basic
- * unit of time in zookeeper.</li>
- * <li>initLimit - The maximum number of ticks that a follower will wait to
- * initially synchronize with a leader.</li>
- * <li>syncLimit - The maximum number of ticks that a follower will wait for a
- * message (including heartbeats) from the leader.</li>
- * <li>server.<i>id</i> - This is the host:port that the server with the
- * given id will use for the quorum protocol.</li>
- * In addition to the zoo.cfg file. There is a file in the data directory called
- * "myid" that contains the server id as an ASCII decimal value.
-public class QuorumPeer extends Thread implements QuorumStats.Provider {
- private static final Logger LOG = Logger.getLogger(QuorumPeer.class);
- * Create an instance of a quorum peer
- public interface Factory{
- public QuorumPeer create(NIOServerCnxn.Factory cnxnFactory) throws IOException;
- public NIOServerCnxn.Factory createConnectionFactory() throws IOException;
- public static class QuorumServer {
- public QuorumServer(long id, InetSocketAddress addr) {
- this.id = id;
- public InetSocketAddress addr;
- public long id;
- public enum ServerState {
- LOOKING, FOLLOWING, LEADING;
- * The servers that make up the cluster
- ArrayList<QuorumServer> quorumPeers;
- public int getQuorumSize(){
- return quorumPeers.size();
- * My id
- private long myid;
- * get the id of this quorum peer.
- public long getId() {
- return myid;
- * This is who I think the leader currently is.
- volatile Vote currentVote;
- volatile boolean running = true;
- * The number of milliseconds of each tick
- int tickTime;
- * The number of ticks that the initial synchronization phase can take
- int initLimit;
- * The number of ticks that can pass between sending a request and getting
- * an acknowledgement
- int syncLimit;
- * The current tick
- int tick;
- * This class simply responds to requests for the current leader of this
- * The request contains just an xid generated by the requestor.
- * The response has the xid, the id of this server, the id of the leader,
- * and the zxid of the leader.
- * @author breed
- class ResponderThread extends Thread {
- ResponderThread() {
- super("ResponderThread");
- byte b[] = new byte[36];
- ByteBuffer responseBuffer = ByteBuffer.wrap(b);
- DatagramPacket packet = new DatagramPacket(b, b.length);
- udpSocket.receive(packet);
- if (packet.getLength() != 4) {
- LOG.error("Got more than just an xid! Len = "
- + packet.getLength());
- responseBuffer.getInt(); // Skip the xid
- responseBuffer.putLong(myid);
- switch (state) {
- responseBuffer.putLong(currentVote.id);
- responseBuffer.putLong(currentVote.zxid);
- responseBuffer.putLong(leader.lastProposed);
- } catch (NullPointerException npe) {
- // This can happen in state transitions,
- // just ignore the request
- responseBuffer.putLong(follower.getZxid());
- packet.setData(b);
- udpSocket.send(packet);
- packet.setLength(b.length);
- LOG.error("QuorumPeer responder thread exited");
- private ServerState state = ServerState.LOOKING;
- public void setPeerState(ServerState newState){
- state=newState;
- public ServerState getPeerState(){
- DatagramSocket udpSocket;
- private InetSocketAddress myQuorumAddr;
- public InetSocketAddress getQuorumAddress(){
- return myQuorumAddr;
- * the directory where the snapshot is stored.
- private File dataDir;
- * the directory where the logs are stored.
- private File dataLogDir;
- Election electionAlg;
+import static com.yahoo.zookeeper.server.ServerConfig.getClientPort;
+import static com.yahoo.zookeeper.server.ServerConfig.getDataDir;
+import static com.yahoo.zookeeper.server.ServerConfig.getDataLogDir;
+import static com.yahoo.zookeeper.server.quorum.QuorumPeerConfig.getElectionAlg;
+import static com.yahoo.zookeeper.server.quorum.QuorumPeerConfig.getElectionPort;
+import static com.yahoo.zookeeper.server.quorum.QuorumPeerConfig.getInitLimit;
+import static com.yahoo.zookeeper.server.quorum.QuorumPeerConfig.getServerId;
+import static com.yahoo.zookeeper.server.quorum.QuorumPeerConfig.getServers;
+import static com.yahoo.zookeeper.server.quorum.QuorumPeerConfig.getSyncLimit;
+import static com.yahoo.zookeeper.server.quorum.QuorumPeerConfig.getTickTime;
+import com.yahoo.zookeeper.server.NIOServerCnxn;
+ * This class manages the quorum protocol. There are three states this server
+ * can be in:
+ * <ol>
+ * <li>Leader election - each server will elect a leader (proposing itself as a
+ * leader initially).</li>
+ * <li>Follower - the server will synchronize with the leader and replicate any
+ * transactions.</li>
+ * <li>Leader - the server will process requests and forward them to followers.
+ * A majority of followers must log the request before it can be accepted.
+ * </ol>
+ * This class will setup a datagram socket that will always respond with its
+ * view of the current leader. The response will take the form of:
+ * <pre>
+ * int xid;
+ * long myid;
+ * long leader_id;
+ * long leader_zxid;
+ * </pre>
+ * The request for the current leader will consist solely of an xid: int xid;
+ * <h2>Configuration file</h2>
+ * When the main() method of this class is used to start the program, the file
+ * "zoo.cfg" in the current directory will be used to obtain configuration
+ * information. zoo.cfg is a Properties file, so keys and values are separated
+ * by equals (=) and the key/value pairs are separated by new lines. The
+ * following keys are used in the configuration file:
+ * <li>dataDir - The directory where the zookeeper data is stored.</li>
+ * <li>clientPort - The port used to communicate with clients.</li>
+ * <li>tickTime - The duration of a tick in milliseconds. This is the basic
+ * unit of time in zookeeper.</li>
+ * <li>initLimit - The maximum number of ticks that a follower will wait to
+ * initially synchronize with a leader.</li>
+ * <li>syncLimit - The maximum number of ticks that a follower will wait for a
+ * message (including heartbeats) from the leader.</li>
+ * <li>server.<i>id</i> - This is the host:port that the server with the
+ * given id will use for the quorum protocol.</li>
+ * In addition to the zoo.cfg file. There is a file in the data directory called
+ * "myid" that contains the server id as an ASCII decimal value.
+public class QuorumPeer extends Thread implements QuorumStats.Provider {
+ private static final Logger LOG = Logger.getLogger(QuorumPeer.class);
+ * Create an instance of a quorum peer
+ public interface Factory{
+ public QuorumPeer create(NIOServerCnxn.Factory cnxnFactory) throws IOException;
+ public NIOServerCnxn.Factory createConnectionFactory() throws IOException;
+ public static class QuorumServer {
+ public QuorumServer(long id, InetSocketAddress addr) {
+ this.id = id;
+ public InetSocketAddress addr;
+ public long id;
+ public enum ServerState {
+ LOOKING, FOLLOWING, LEADING;
+ * The servers that make up the cluster
+ ArrayList<QuorumServer> quorumPeers;
+ public int getQuorumSize(){
+ return quorumPeers.size();
+ * My id
+ private long myid;
+ * get the id of this quorum peer.
+ public long getId() {
+ return myid;
+ * This is who I think the leader currently is.
+ volatile Vote currentVote;
+ volatile boolean running = true;
+ * The number of milliseconds of each tick
+ int tickTime;
+ * The number of ticks that the initial synchronization phase can take
+ int initLimit;
+ * The number of ticks that can pass between sending a request and getting
+ * an acknowledgement
+ int syncLimit;
+ * The current tick
+ int tick;
+ * This class simply responds to requests for the current leader of this
+ * The request contains just an xid generated by the requestor.
+ * The response has the xid, the id of this server, the id of the leader,
+ * and the zxid of the leader.
+ * @author breed
+ class ResponderThread extends Thread {
+ ResponderThread() {
+ super("ResponderThread");
+ byte b[] = new byte[36];
+ ByteBuffer responseBuffer = ByteBuffer.wrap(b);
+ DatagramPacket packet = new DatagramPacket(b, b.length);
+ udpSocket.receive(packet);
+ if (packet.getLength() != 4) {
+ LOG.warn("Got more than just an xid! Len = "
+ + packet.getLength());
+ responseBuffer.getInt(); // Skip the xid
+ responseBuffer.putLong(myid);
+ switch (state) {
+ responseBuffer.putLong(currentVote.id);
+ responseBuffer.putLong(currentVote.zxid);
+ responseBuffer.putLong(leader.lastProposed);
+ } catch (NullPointerException npe) {
+ // This can happen in state transitions,
+ // just ignore the request
+ responseBuffer.putLong(follower.getZxid());
+ packet.setData(b);
+ udpSocket.send(packet);
+ packet.setLength(b.length);
+ LOG.warn("QuorumPeer responder thread exited");
+ private ServerState state = ServerState.LOOKING;
+ public void setPeerState(ServerState newState){
+ state=newState;
+ public ServerState getPeerState(){
+ DatagramSocket udpSocket;
+ private InetSocketAddress myQuorumAddr;
+ public InetSocketAddress getQuorumAddress(){
+ return myQuorumAddr;
+ * the directory where the snapshot is stored.
+ private File dataDir;
+ * the directory where the logs are stored.
+ private File dataLogDir;
+ Election electionAlg;
int electionPort;
- NIOServerCnxn.Factory cnxnFactory;
- public QuorumPeer(ArrayList<QuorumServer> quorumPeers, File dataDir,
- File dataLogDir, int electionAlg, int electionPort,long myid, int tickTime,
- int initLimit, int syncLimit,NIOServerCnxn.Factory cnxnFactory) throws IOException {
- super("QuorumPeer");
- this.cnxnFactory = cnxnFactory;
- this.quorumPeers = quorumPeers;
+ NIOServerCnxn.Factory cnxnFactory;
+ public QuorumPeer(ArrayList<QuorumServer> quorumPeers, File dataDir,
+ File dataLogDir, int electionAlg, int electionPort,long myid, int tickTime,
+ int initLimit, int syncLimit,NIOServerCnxn.Factory cnxnFactory) throws IOException {
+ super("QuorumPeer");
+ this.cnxnFactory = cnxnFactory;
+ this.quorumPeers = quorumPeers;
this.electionPort = electionPort;
- this.myid = myid;
- this.initLimit = initLimit;
- this.syncLimit = syncLimit;
- currentVote = new Vote(myid, getLastLoggedZxid());
- for (QuorumServer p : quorumPeers) {
- if (p.id == myid) {
- myQuorumAddr = p.addr;
- if (myQuorumAddr == null) {
- throw new SocketException("My id " + myid + " not in the peer list");
- if (electionAlg == 0) {
- udpSocket = new DatagramSocket(myQuorumAddr.getPort());
- new ResponderThread().start();
- this.electionAlg = createElectionAlgorithm(electionAlg);
- QuorumStats.getInstance().setStatsProvider(this);
- * This constructor is only used by the existing unit test code.
- File dataLogDir, int clientPort, int electionAlg, int electionPort,
- long myid, int tickTime, int initLimit, int syncLimit) throws IOException {
- this(quorumPeers,dataDir,dataLogDir,electionAlg,electionPort,myid,tickTime,
- initLimit,syncLimit,new NIOServerCnxn.Factory(clientPort));
- * The constructor uses the quorum peer config to instantiate the class
- public QuorumPeer(NIOServerCnxn.Factory cnxnFactory) throws IOException {
- this(getServers(), new File(getDataDir()), new File(getDataLogDir()),
- getElectionAlg(), getElectionPort(),getServerId(),getTickTime(),
- getInitLimit(), getSyncLimit(),cnxnFactory);
- public Follower follower;
- public Leader leader;
- protected Follower makeFollower(File dataDir,File dataLogDir) throws IOException {
- return new Follower(this, new FollowerZooKeeperServer(dataDir,
- dataLogDir, this,new ZooKeeperServer.BasicDataTreeBuilder()));
- protected Leader makeLeader(File dataDir,File dataLogDir) throws IOException {
- return new Leader(this, new LeaderZooKeeperServer(dataDir, dataLogDir,
- this,new ZooKeeperServer.BasicDataTreeBuilder()));
- private Election createElectionAlgorithm(int electionAlgorithm){
- Election le=null;
- //TODO: use a factory rather than a switch
- switch (electionAlgorithm) {
- // will create a new instance for each run of the protocol
- le = new AuthFastLeaderElection(this, this.electionPort);
- le = new AuthFastLeaderElection(this, this.electionPort, true);
- le = new FastLeaderElection(this,
- new QuorumCnxManager(this.electionPort));
- assert false;
- return le;
- protected Election makeLEStrategy(){
- if(electionAlg==null)
- return new LeaderElection(this);
- return electionAlg;
- synchronized protected void setLeader(Leader newLeader){
- leader=newLeader;
- synchronized protected void setFollower(Follower newFollower){
- follower=newFollower;
- synchronized public ZooKeeperServer getActiveServer(){
- if(leader!=null)
- return leader.zk;
- else if(follower!=null)
- return follower.zk;
- * Main loop
- while (running) {
- LOG.warn("LOOKING");
- currentVote = makeLEStrategy().lookForLeader();
- state = ServerState.LOOKING;
- LOG.warn("FOLLOWING");
- setFollower(makeFollower(dataDir,dataLogDir));
- follower.followLeader();
- follower.shutdown();
- setFollower(null);
- LOG.warn("LEADING");
- setLeader(makeLeader(dataDir,dataLogDir));
- leader.lead();
- setLeader(null);
- if (leader != null) {
- leader.shutdown("Forcing shutdown");
- LOG.error("QuorumPeer main thread exited");
- running = false;
- leader.shutdown("quorum Peer shutdown");
- if (follower != null) {
- cnxnFactory.shutdown();
- udpSocket.close();
- long getLastLoggedZxid() {
- File[] list = dataLogDir.listFiles();
- if (list == null) {
- long maxLog = -1;
- long maxSnapShot = 0;
- for (File f : list) {
- String name = f.getName();
- if (name.startsWith("log.")) {
- long zxid = ZooKeeperServer.getZxidFromName(f.getName(), "log");
- if (zxid > maxLog) {
- maxLog = zxid;
- } else if (name.startsWith("snapshot.")) {
- long zxid = ZooKeeperServer.getZxidFromName(f.getName(),
- "snapshot");
- maxSnapShot = zxid;
- if (maxSnapShot > maxLog) {
- return maxSnapShot;
- long zxid = maxLog;
- FileInputStream logStream = null;
- logStream = new FileInputStream(new File(dataLogDir, "log."
- + Long.toHexString(maxLog)));
- BinaryInputArchive ia = BinaryInputArchive.getArchive(logStream);
- byte[] bytes = ia.readBuffer("txnEntry");
- int B = ia.readByte("EOR");
- if (B != 'B') {
- InputArchive bia = BinaryInputArchive
- hdr.deserialize(bia, "hdr");
- zxid = hdr.getZxid();
- LOG.warn(e.toString());
- if (logStream != null) {
- logStream.close();
- public static void runPeer(QuorumPeer.Factory qpFactory) {
- QuorumStats.registerAsConcrete();
- QuorumPeer self = qpFactory.create(qpFactory.createConnectionFactory());
- self.start();
- self.join();
- System.exit(2);
- public String[] getQuorumPeers() {
- List<String> l = new ArrayList<String>();
- synchronized (leader.followers) {
- for (FollowerHandler fh : leader.followers) {
- if (fh.s == null)
- String s = fh.s.getRemoteSocketAddress().toString();
- if (leader.isFollowerSynced(fh))
- s += "*";
- l.add(s);
- } else if (follower != null) {
- l.add(follower.sock.getRemoteSocketAddress().toString());
- return l.toArray(new String[0]);
- public String getServerState() {
- return QuorumStats.Provider.LOOKING_STATE;
- return QuorumStats.Provider.LEADING_STATE;
- return QuorumStats.Provider.FOLLOWING_STATE;
- return QuorumStats.Provider.UNKNOWN_STATE;
- public static void main(String args[]) {
- if (args.length == 2) {
- ZooKeeperServer.main(args);
- QuorumPeerConfig.parse(args);
- if (!QuorumPeerConfig.isStandalone()) {
- runPeer(new QuorumPeer.Factory() {
- public QuorumPeer create(NIOServerCnxn.Factory cnxnFactory)
- return new QuorumPeer(cnxnFactory);
- return new NIOServerCnxn.Factory(getClientPort());
- }else{
- // there is only server in the quorum -- run as standalone
+ this.myid = myid;
+ this.initLimit = initLimit;
+ this.syncLimit = syncLimit;
+ currentVote = new Vote(myid, getLastLoggedZxid());
+ for (QuorumServer p : quorumPeers) {
+ if (p.id == myid) {
+ myQuorumAddr = p.addr;
+ if (myQuorumAddr == null) {
+ throw new SocketException("My id " + myid + " not in the peer list");
+ if (electionAlg == 0) {
+ udpSocket = new DatagramSocket(myQuorumAddr.getPort());
+ new ResponderThread().start();
+ this.electionAlg = createElectionAlgorithm(electionAlg);
+ QuorumStats.getInstance().setStatsProvider(this);
+ * This constructor is only used by the existing unit test code.
+ File dataLogDir, int clientPort, int electionAlg, int electionPort,
+ long myid, int tickTime, int initLimit, int syncLimit) throws IOException {
+ this(quorumPeers,dataDir,dataLogDir,electionAlg,electionPort,myid,tickTime,
+ initLimit,syncLimit,new NIOServerCnxn.Factory(clientPort));
+ * The constructor uses the quorum peer config to instantiate the class
+ public QuorumPeer(NIOServerCnxn.Factory cnxnFactory) throws IOException {
+ this(getServers(), new File(getDataDir()), new File(getDataLogDir()),
+ getElectionAlg(), getElectionPort(),getServerId(),getTickTime(),
+ getInitLimit(), getSyncLimit(),cnxnFactory);
+ public Follower follower;
+ public Leader leader;
+ protected Follower makeFollower(File dataDir,File dataLogDir) throws IOException {
+ return new Follower(this, new FollowerZooKeeperServer(dataDir,
+ dataLogDir, this,new ZooKeeperServer.BasicDataTreeBuilder()));
+ protected Leader makeLeader(File dataDir,File dataLogDir) throws IOException {
+ return new Leader(this, new LeaderZooKeeperServer(dataDir, dataLogDir,
+ this,new ZooKeeperServer.BasicDataTreeBuilder()));
+ private Election createElectionAlgorithm(int electionAlgorithm){
+ Election le=null;
+ //TODO: use a factory rather than a switch
+ switch (electionAlgorithm) {
+ // will create a new instance for each run of the protocol
+ le = new AuthFastLeaderElection(this, this.electionPort);
+ le = new AuthFastLeaderElection(this, this.electionPort, true);
+ le = new FastLeaderElection(this,
+ new QuorumCnxManager(this.electionPort));
+ assert false;
+ return le;
+ protected Election makeLEStrategy(){
+ if(electionAlg==null)
+ return new LeaderElection(this);
+ return electionAlg;
+ synchronized protected void setLeader(Leader newLeader){
+ leader=newLeader;
+ synchronized protected void setFollower(Follower newFollower){
+ follower=newFollower;
+ synchronized public ZooKeeperServer getActiveServer(){
+ if(leader!=null)
+ return leader.zk;
+ else if(follower!=null)
+ return follower.zk;
+ * Main loop
+ while (running) {
+ LOG.info("LOOKING");
+ currentVote = makeLEStrategy().lookForLeader();
+ state = ServerState.LOOKING;
+ LOG.info("FOLLOWING");
+ setFollower(makeFollower(dataDir,dataLogDir));
+ follower.followLeader();
+ follower.shutdown();
+ setFollower(null);
+ LOG.info("LEADING");
+ setLeader(makeLeader(dataDir,dataLogDir));
+ leader.lead();
+ setLeader(null);
+ if (leader != null) {
+ leader.shutdown("Forcing shutdown");
+ LOG.warn("QuorumPeer main thread exited");
+ running = false;
+ leader.shutdown("quorum Peer shutdown");
+ if (follower != null) {
+ cnxnFactory.shutdown();
+ udpSocket.close();
+ long getLastLoggedZxid() {
+ File[] list = dataLogDir.listFiles();
+ if (list == null) {
+ long maxLog = -1;
+ long maxSnapShot = 0;
+ for (File f : list) {
+ String name = f.getName();
+ if (name.startsWith("log.")) {
+ long zxid = ZooKeeperServer.getZxidFromName(f.getName(), "log");
+ if (zxid > maxLog) {
+ maxLog = zxid;
+ } else if (name.startsWith("snapshot.")) {
+ long zxid = ZooKeeperServer.getZxidFromName(f.getName(),
+ "snapshot");
+ maxSnapShot = zxid;
+ if (maxSnapShot > maxLog) {
+ return maxSnapShot;
+ long zxid = maxLog;
+ FileInputStream logStream = null;
+ logStream = new FileInputStream(new File(dataLogDir, "log."
+ + Long.toHexString(maxLog)));
+ BinaryInputArchive ia = BinaryInputArchive.getArchive(logStream);
+ byte[] bytes = ia.readBuffer("txnEntry");
+ int B = ia.readByte("EOR");
+ if (B != 'B') {
+ InputArchive bia = BinaryInputArchive
+ hdr.deserialize(bia, "hdr");
+ zxid = hdr.getZxid();
+ if (logStream != null) {
+ logStream.close();
+ public static void runPeer(QuorumPeer.Factory qpFactory) {
+ QuorumStats.registerAsConcrete();
+ QuorumPeer self = qpFactory.create(qpFactory.createConnectionFactory());
+ self.start();
+ self.join();
+ System.exit(2);
+ public String[] getQuorumPeers() {
+ List<String> l = new ArrayList<String>();
+ synchronized (leader.followers) {
+ for (FollowerHandler fh : leader.followers) {
+ if (fh.s == null)
+ String s = fh.s.getRemoteSocketAddress().toString();
+ if (leader.isFollowerSynced(fh))
+ s += "*";
+ l.add(s);
+ } else if (follower != null) {
+ l.add(follower.sock.getRemoteSocketAddress().toString());
+ return l.toArray(new String[0]);
+ public String getServerState() {
+ return QuorumStats.Provider.LOOKING_STATE;
+ return QuorumStats.Provider.LEADING_STATE;
+ return QuorumStats.Provider.FOLLOWING_STATE;
+ return QuorumStats.Provider.UNKNOWN_STATE;
+ public static void main(String args[]) {
+ if (args.length == 2) {
+ ZooKeeperServer.main(args);
+ QuorumPeerConfig.parse(args);
+ if (!QuorumPeerConfig.isStandalone()) {
+ runPeer(new QuorumPeer.Factory() {
+ public QuorumPeer create(NIOServerCnxn.Factory cnxnFactory)
+ return new QuorumPeer(cnxnFactory);
+ return new NIOServerCnxn.Factory(getClientPort());
+ }else{
+ // there is only server in the quorum -- run as standalone