|
@@ -29,9 +29,11 @@ import java.nio.ByteBuffer;
|
|
import java.nio.channels.CancelledKeyException;
|
|
import java.nio.channels.CancelledKeyException;
|
|
import java.nio.channels.SelectionKey;
|
|
import java.nio.channels.SelectionKey;
|
|
import java.nio.channels.SocketChannel;
|
|
import java.nio.channels.SocketChannel;
|
|
-import java.util.HashSet;
|
|
|
|
import java.util.List;
|
|
import java.util.List;
|
|
|
|
+import java.util.Queue;
|
|
import java.util.Set;
|
|
import java.util.Set;
|
|
|
|
+import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
import java.util.concurrent.LinkedBlockingQueue;
|
|
import java.util.concurrent.LinkedBlockingQueue;
|
|
|
|
|
|
import org.apache.jute.BinaryInputArchive;
|
|
import org.apache.jute.BinaryInputArchive;
|
|
@@ -46,6 +48,7 @@ import org.apache.zookeeper.data.Id;
|
|
import org.apache.zookeeper.proto.ReplyHeader;
|
|
import org.apache.zookeeper.proto.ReplyHeader;
|
|
import org.apache.zookeeper.proto.RequestHeader;
|
|
import org.apache.zookeeper.proto.RequestHeader;
|
|
import org.apache.zookeeper.proto.WatcherEvent;
|
|
import org.apache.zookeeper.proto.WatcherEvent;
|
|
|
|
+import org.apache.zookeeper.server.NIOServerCnxnFactory.SelectorThread;
|
|
import org.apache.zookeeper.server.quorum.Leader;
|
|
import org.apache.zookeeper.server.quorum.Leader;
|
|
import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer;
|
|
import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer;
|
|
import org.apache.zookeeper.server.quorum.ReadOnlyZooKeeperServer;
|
|
import org.apache.zookeeper.server.quorum.ReadOnlyZooKeeperServer;
|
|
@@ -58,58 +61,63 @@ import org.apache.zookeeper.server.util.OSMXBean;
|
|
public class NIOServerCnxn extends ServerCnxn {
|
|
public class NIOServerCnxn extends ServerCnxn {
|
|
static final Logger LOG = LoggerFactory.getLogger(NIOServerCnxn.class);
|
|
static final Logger LOG = LoggerFactory.getLogger(NIOServerCnxn.class);
|
|
|
|
|
|
- NIOServerCnxnFactory factory;
|
|
|
|
|
|
+ private final NIOServerCnxnFactory factory;
|
|
|
|
|
|
- SocketChannel sock;
|
|
|
|
|
|
+ private SocketChannel sock;
|
|
|
|
+
|
|
|
|
+ private final SelectorThread selectorThread;
|
|
|
|
|
|
private final SelectionKey sk;
|
|
private final SelectionKey sk;
|
|
|
|
|
|
- boolean initialized;
|
|
|
|
|
|
+ private boolean initialized;
|
|
|
|
|
|
- ByteBuffer lenBuffer = ByteBuffer.allocate(4);
|
|
|
|
|
|
+ private final ByteBuffer lenBuffer = ByteBuffer.allocate(4);
|
|
|
|
|
|
- ByteBuffer incomingBuffer = lenBuffer;
|
|
|
|
|
|
+ private ByteBuffer incomingBuffer = lenBuffer;
|
|
|
|
|
|
- LinkedBlockingQueue<ByteBuffer> outgoingBuffers = new LinkedBlockingQueue<ByteBuffer>();
|
|
|
|
|
|
+ private final Queue<ByteBuffer> outgoingBuffers =
|
|
|
|
+ new LinkedBlockingQueue<ByteBuffer>();
|
|
|
|
|
|
- int sessionTimeout;
|
|
|
|
|
|
+ private int sessionTimeout;
|
|
|
|
|
|
private final ZooKeeperServer zkServer;
|
|
private final ZooKeeperServer zkServer;
|
|
|
|
|
|
/**
|
|
/**
|
|
* The number of requests that have been submitted but not yet responded to.
|
|
* The number of requests that have been submitted but not yet responded to.
|
|
*/
|
|
*/
|
|
- int outstandingRequests;
|
|
|
|
|
|
+ private final AtomicInteger outstandingRequests = new AtomicInteger(0);
|
|
|
|
|
|
/**
|
|
/**
|
|
* This is the id that uniquely identifies the session of a client. Once
|
|
* 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.
|
|
* this session is no longer active, the ephemeral nodes will go away.
|
|
*/
|
|
*/
|
|
- long sessionId;
|
|
|
|
|
|
+ private long sessionId;
|
|
|
|
|
|
- static long nextSessionId = 1;
|
|
|
|
- int outstandingLimit = 1;
|
|
|
|
|
|
+ private final int outstandingLimit;
|
|
|
|
|
|
public NIOServerCnxn(ZooKeeperServer zk, SocketChannel sock,
|
|
public NIOServerCnxn(ZooKeeperServer zk, SocketChannel sock,
|
|
- SelectionKey sk, NIOServerCnxnFactory factory) throws IOException {
|
|
|
|
|
|
+ SelectionKey sk, NIOServerCnxnFactory factory,
|
|
|
|
+ SelectorThread selectorThread) throws IOException {
|
|
this.zkServer = zk;
|
|
this.zkServer = zk;
|
|
this.sock = sock;
|
|
this.sock = sock;
|
|
this.sk = sk;
|
|
this.sk = sk;
|
|
this.factory = factory;
|
|
this.factory = factory;
|
|
|
|
+ this.selectorThread = selectorThread;
|
|
if (this.factory.login != null) {
|
|
if (this.factory.login != null) {
|
|
this.zooKeeperSaslServer = new ZooKeeperSaslServer(factory.login);
|
|
this.zooKeeperSaslServer = new ZooKeeperSaslServer(factory.login);
|
|
}
|
|
}
|
|
- if (zk != null) {
|
|
|
|
|
|
+ if (zk != null) {
|
|
outstandingLimit = zk.getGlobalOutstandingLimit();
|
|
outstandingLimit = zk.getGlobalOutstandingLimit();
|
|
|
|
+ } else {
|
|
|
|
+ outstandingLimit = 1;
|
|
}
|
|
}
|
|
sock.socket().setTcpNoDelay(true);
|
|
sock.socket().setTcpNoDelay(true);
|
|
- /* set socket linger to false, so that socket close does not
|
|
|
|
- * block */
|
|
|
|
|
|
+ /* set socket linger to false, so that socket close does not block */
|
|
sock.socket().setSoLinger(false, -1);
|
|
sock.socket().setSoLinger(false, -1);
|
|
InetAddress addr = ((InetSocketAddress) sock.socket()
|
|
InetAddress addr = ((InetSocketAddress) sock.socket()
|
|
.getRemoteSocketAddress()).getAddress();
|
|
.getRemoteSocketAddress()).getAddress();
|
|
authInfo.add(new Id("ip", addr.getHostAddress()));
|
|
authInfo.add(new Id("ip", addr.getHostAddress()));
|
|
- sk.interestOps(SelectionKey.OP_READ);
|
|
|
|
|
|
+ this.sessionTimeout = factory.sessionlessCnxnTimeout;
|
|
}
|
|
}
|
|
|
|
|
|
/* Send close connection packet to the client, doIO will eventually
|
|
/* Send close connection packet to the client, doIO will eventually
|
|
@@ -127,56 +135,32 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
void sendBufferSync(ByteBuffer bb) {
|
|
void sendBufferSync(ByteBuffer bb) {
|
|
try {
|
|
try {
|
|
/* configure socket to be blocking
|
|
/* configure socket to be blocking
|
|
- * so that we dont have to do write in
|
|
|
|
|
|
+ * so that we dont have to do write in
|
|
* a tight while loop
|
|
* a tight while loop
|
|
*/
|
|
*/
|
|
- sock.configureBlocking(true);
|
|
|
|
if (bb != ServerCnxnFactory.closeConn) {
|
|
if (bb != ServerCnxnFactory.closeConn) {
|
|
if (sock != null) {
|
|
if (sock != null) {
|
|
|
|
+ sock.configureBlocking(true);
|
|
sock.write(bb);
|
|
sock.write(bb);
|
|
}
|
|
}
|
|
packetSent();
|
|
packetSent();
|
|
- }
|
|
|
|
|
|
+ }
|
|
} catch (IOException ie) {
|
|
} catch (IOException ie) {
|
|
LOG.error("Error sending data synchronously ", ie);
|
|
LOG.error("Error sending data synchronously ", ie);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
- public void sendBuffer(ByteBuffer bb) {
|
|
|
|
- try {
|
|
|
|
- if (bb != ServerCnxnFactory.closeConn) {
|
|
|
|
- // We check if write interest here because if it is NOT set,
|
|
|
|
- // nothing is queued, so we can try to send the buffer right
|
|
|
|
- // away without waking up the selector
|
|
|
|
- if ((sk.interestOps() & SelectionKey.OP_WRITE) == 0) {
|
|
|
|
- try {
|
|
|
|
- sock.write(bb);
|
|
|
|
- } catch (IOException e) {
|
|
|
|
- // we are just doing best effort right now
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- // if there is nothing left to send, we are done
|
|
|
|
- if (bb.remaining() == 0) {
|
|
|
|
- packetSent();
|
|
|
|
- return;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
|
|
- synchronized(this.factory){
|
|
|
|
- sk.selector().wakeup();
|
|
|
|
- if (LOG.isTraceEnabled()) {
|
|
|
|
- LOG.trace("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(Exception e) {
|
|
|
|
- LOG.error("Unexpected Exception: ", e);
|
|
|
|
|
|
+ /**
|
|
|
|
+ * sendBuffer pushes a byte buffer onto the outgoing buffer queue for
|
|
|
|
+ * asynchronous writes.
|
|
|
|
+ */
|
|
|
|
+ public void sendBuffer(ByteBuffer bb) {
|
|
|
|
+ if (LOG.isTraceEnabled()) {
|
|
|
|
+ LOG.trace("Add a buffer to outgoingBuffers, sk " + sk
|
|
|
|
+ + " is valid: " + sk.isValid());
|
|
}
|
|
}
|
|
|
|
+ outgoingBuffers.add(bb);
|
|
|
|
+ requestInterestOpsUpdate();
|
|
}
|
|
}
|
|
|
|
|
|
/** Read the request payload (everything following the length prefix) */
|
|
/** Read the request payload (everything following the length prefix) */
|
|
@@ -204,6 +188,123 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * This boolean tracks whether the connection is ready for selection or
|
|
|
|
+ * not. A connection is marked as not ready for selection while it is
|
|
|
|
+ * processing an IO request. The flag is used to gatekeep pushing interest
|
|
|
|
+ * op updates onto the selector.
|
|
|
|
+ */
|
|
|
|
+ private final AtomicBoolean selectable = new AtomicBoolean(true);
|
|
|
|
+
|
|
|
|
+ public boolean isSelectable() {
|
|
|
|
+ return sk.isValid() && selectable.get();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void disableSelectable() {
|
|
|
|
+ selectable.set(false);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void enableSelectable() {
|
|
|
|
+ selectable.set(true);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void requestInterestOpsUpdate() {
|
|
|
|
+ if (isSelectable()) {
|
|
|
|
+ selectorThread.addInterestOpsUpdateRequest(sk);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ void handleWrite(SelectionKey k) throws IOException, CloseRequestException {
|
|
|
|
+ if (outgoingBuffers.isEmpty()) {
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /*
|
|
|
|
+ * 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 = NIOServerCnxnFactory.getDirectBuffer();
|
|
|
|
+ if (directBuffer == null) {
|
|
|
|
+ ByteBuffer[] bufferList = new ByteBuffer[outgoingBuffers.size()];
|
|
|
|
+ // Use gathered write call. This updates the positions of the
|
|
|
|
+ // byte buffers to reflect the bytes that were written out.
|
|
|
|
+ sock.write(outgoingBuffers.toArray(bufferList));
|
|
|
|
+
|
|
|
|
+ // Remove the buffers that we have sent
|
|
|
|
+ ByteBuffer bb;
|
|
|
|
+ while ((bb = outgoingBuffers.peek()) != null) {
|
|
|
|
+ if (bb == ServerCnxnFactory.closeConn) {
|
|
|
|
+ throw new CloseRequestException("close requested");
|
|
|
|
+ }
|
|
|
|
+ if (bb.remaining() > 0) {
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ packetSent();
|
|
|
|
+ outgoingBuffers.remove();
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ 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) {
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ /*
|
|
|
|
+ * 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);
|
|
|
|
+
|
|
|
|
+ ByteBuffer bb;
|
|
|
|
+
|
|
|
|
+ // Remove the buffers that we have sent
|
|
|
|
+ while ((bb = outgoingBuffers.peek()) != null) {
|
|
|
|
+ if (bb == ServerCnxnFactory.closeConn) {
|
|
|
|
+ throw new CloseRequestException("close requested");
|
|
|
|
+ }
|
|
|
|
+ if (sent < bb.remaining()) {
|
|
|
|
+ /*
|
|
|
|
+ * We only partially sent this buffer, so we update
|
|
|
|
+ * the position and exit the loop.
|
|
|
|
+ */
|
|
|
|
+ bb.position(bb.position() + sent);
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ packetSent();
|
|
|
|
+ /* We've sent the whole buffer, so drop the buffer */
|
|
|
|
+ sent -= bb.remaining();
|
|
|
|
+ outgoingBuffers.remove();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Handles read/write IO on connection.
|
|
|
|
+ */
|
|
void doIO(SelectionKey k) throws InterruptedException {
|
|
void doIO(SelectionKey k) throws InterruptedException {
|
|
try {
|
|
try {
|
|
if (sock == null) {
|
|
if (sock == null) {
|
|
@@ -241,101 +342,15 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (k.isWritable()) {
|
|
if (k.isWritable()) {
|
|
- // ZooLog.logTraceMessage(LOG,
|
|
|
|
- // ZooLog.CLIENT_DATA_PACKET_TRACE_MASK
|
|
|
|
- // "outgoingBuffers.size() = " +
|
|
|
|
- // outgoingBuffers.size());
|
|
|
|
- if (outgoingBuffers.size() > 0) {
|
|
|
|
- // ZooLog.logTraceMessage(LOG,
|
|
|
|
- // ZooLog.CLIENT_DATA_PACKET_TRACE_MASK,
|
|
|
|
- // "sk " + k + " is valid: " +
|
|
|
|
- // k.isValid());
|
|
|
|
|
|
+ handleWrite(k);
|
|
|
|
|
|
- /*
|
|
|
|
- * 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) {
|
|
|
|
- break;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- /*
|
|
|
|
- * 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);
|
|
|
|
- ByteBuffer bb;
|
|
|
|
-
|
|
|
|
- // Remove the buffers that we have sent
|
|
|
|
- while (outgoingBuffers.size() > 0) {
|
|
|
|
- bb = outgoingBuffers.peek();
|
|
|
|
- if (bb == ServerCnxnFactory.closeConn) {
|
|
|
|
- throw new CloseRequestException("close requested");
|
|
|
|
- }
|
|
|
|
- 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);
|
|
|
|
- break;
|
|
|
|
- }
|
|
|
|
- packetSent();
|
|
|
|
- /* We've sent the whole buffer, so drop the buffer */
|
|
|
|
- sent -= bb.remaining();
|
|
|
|
- outgoingBuffers.remove();
|
|
|
|
- }
|
|
|
|
- // ZooLog.logTraceMessage(LOG,
|
|
|
|
- // ZooLog.CLIENT_DATA_PACKET_TRACE_MASK, "after send,
|
|
|
|
- // outgoingBuffers.size() = " + outgoingBuffers.size());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- synchronized(this.factory){
|
|
|
|
- if (outgoingBuffers.size() == 0) {
|
|
|
|
- if (!initialized
|
|
|
|
- && (sk.interestOps() & SelectionKey.OP_READ) == 0) {
|
|
|
|
- throw new CloseRequestException("responded to info probe");
|
|
|
|
- }
|
|
|
|
- sk.interestOps(sk.interestOps()
|
|
|
|
- & (~SelectionKey.OP_WRITE));
|
|
|
|
- } else {
|
|
|
|
- sk.interestOps(sk.interestOps()
|
|
|
|
- | SelectionKey.OP_WRITE);
|
|
|
|
- }
|
|
|
|
|
|
+ if (!initialized && !getReadInterest() && !getWriteInterest()) {
|
|
|
|
+ throw new CloseRequestException("responded to info probe");
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} catch (CancelledKeyException e) {
|
|
} catch (CancelledKeyException e) {
|
|
- LOG.warn("Exception causing close of session 0x"
|
|
|
|
- + Long.toHexString(sessionId)
|
|
|
|
- + " due to " + e);
|
|
|
|
|
|
+ LOG.warn("CancelledKeyException causing close of session 0x"
|
|
|
|
+ + Long.toHexString(sessionId));
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug("CancelledKeyException stack trace", e);
|
|
LOG.debug("CancelledKeyException stack trace", e);
|
|
}
|
|
}
|
|
@@ -344,14 +359,12 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
// expecting close to log session closure
|
|
// expecting close to log session closure
|
|
close();
|
|
close();
|
|
} catch (EndOfStreamException e) {
|
|
} catch (EndOfStreamException e) {
|
|
- LOG.warn("caught end of stream exception",e); // tell user why
|
|
|
|
-
|
|
|
|
|
|
+ LOG.warn(e.getMessage());
|
|
// expecting close to log session closure
|
|
// expecting close to log session closure
|
|
close();
|
|
close();
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
LOG.warn("Exception causing close of session 0x"
|
|
LOG.warn("Exception causing close of session 0x"
|
|
- + Long.toHexString(sessionId)
|
|
|
|
- + " due to " + e);
|
|
|
|
|
|
+ + Long.toHexString(sessionId) + ": " + e.getMessage());
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug("IOException stack trace", e);
|
|
LOG.debug("IOException stack trace", e);
|
|
}
|
|
}
|
|
@@ -362,42 +375,50 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
private void readRequest() throws IOException {
|
|
private void readRequest() throws IOException {
|
|
zkServer.processPacket(this, incomingBuffer);
|
|
zkServer.processPacket(this, incomingBuffer);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ // Only called as callback from zkServer.processPacket()
|
|
protected void incrOutstandingRequests(RequestHeader h) {
|
|
protected void incrOutstandingRequests(RequestHeader h) {
|
|
if (h.getXid() >= 0) {
|
|
if (h.getXid() >= 0) {
|
|
- synchronized (this) {
|
|
|
|
- outstandingRequests++;
|
|
|
|
- }
|
|
|
|
- synchronized (this.factory) {
|
|
|
|
- // check throttling
|
|
|
|
- if (zkServer.getInProcess() > outstandingLimit) {
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Throttling recv " + zkServer.getInProcess());
|
|
|
|
- }
|
|
|
|
- disableRecv();
|
|
|
|
- // following lines should not be needed since we are
|
|
|
|
- // already reading
|
|
|
|
- // } else {
|
|
|
|
- // enableRecv();
|
|
|
|
|
|
+ outstandingRequests.incrementAndGet();
|
|
|
|
+ // check throttling
|
|
|
|
+ int inProcess = zkServer.getInProcess();
|
|
|
|
+ if (inProcess > outstandingLimit) {
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("Throttling recv " + inProcess);
|
|
}
|
|
}
|
|
|
|
+ disableRecv();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // returns whether we are interested in writing, which is determined
|
|
|
|
+ // by whether we have any pending buffers on the output queue or not
|
|
|
|
+ private boolean getWriteInterest() {
|
|
|
|
+ return !outgoingBuffers.isEmpty();
|
|
|
|
+ }
|
|
|
|
|
|
|
|
+ // returns whether we are interested in taking new requests, which is
|
|
|
|
+ // determined by whether we are currently throttled or not
|
|
|
|
+ private boolean getReadInterest() {
|
|
|
|
+ return !throttled.get();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private final AtomicBoolean throttled = new AtomicBoolean(false);
|
|
|
|
+
|
|
|
|
+ // Throttle acceptance of new requests. If this entailed a state change,
|
|
|
|
+ // register an interest op update request with the selector.
|
|
public void disableRecv() {
|
|
public void disableRecv() {
|
|
- sk.interestOps(sk.interestOps() & (~SelectionKey.OP_READ));
|
|
|
|
|
|
+ if (throttled.compareAndSet(false, true)) {
|
|
|
|
+ requestInterestOpsUpdate();
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ // Disable throttling and resume acceptance of new requests. If this
|
|
|
|
+ // entailed a state change, register an interest op update request with
|
|
|
|
+ // the selector.
|
|
public void enableRecv() {
|
|
public void enableRecv() {
|
|
- synchronized (this.factory) {
|
|
|
|
- sk.selector().wakeup();
|
|
|
|
- if (sk.isValid()) {
|
|
|
|
- int interest = sk.interestOps();
|
|
|
|
- if ((interest & SelectionKey.OP_READ) == 0) {
|
|
|
|
- sk.interestOps(interest | SelectionKey.OP_READ);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
+ if (throttled.compareAndSet(true, false)) {
|
|
|
|
+ requestInterestOpsUpdate();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -412,7 +433,7 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
/**
|
|
/**
|
|
* clean up the socket related to a command and also make sure we flush the
|
|
* clean up the socket related to a command and also make sure we flush the
|
|
* data before we do that
|
|
* data before we do that
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @param pwriter
|
|
* @param pwriter
|
|
* the pwriter for a command socket
|
|
* the pwriter for a command socket
|
|
*/
|
|
*/
|
|
@@ -432,7 +453,7 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* This class wraps the sendBuffer method of NIOServerCnxn. It is
|
|
* This class wraps the sendBuffer method of NIOServerCnxn. It is
|
|
* responsible for chunking up the response to a client. Rather
|
|
* responsible for chunking up the response to a client. Rather
|
|
@@ -441,7 +462,7 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
*/
|
|
*/
|
|
private class SendBufferWriter extends Writer {
|
|
private class SendBufferWriter extends Writer {
|
|
private StringBuffer sb = new StringBuffer();
|
|
private StringBuffer sb = new StringBuffer();
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Check if we are ready to send another chunk.
|
|
* Check if we are ready to send another chunk.
|
|
* @param force force sending, even if not a full chunk
|
|
* @param force force sending, even if not a full chunk
|
|
@@ -475,20 +496,24 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
|
|
|
|
private static final String ZK_NOT_SERVING =
|
|
private static final String ZK_NOT_SERVING =
|
|
"This ZooKeeper instance is not currently serving requests";
|
|
"This ZooKeeper instance is not currently serving requests";
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Set of threads for commmand ports. All the 4
|
|
* Set of threads for commmand ports. All the 4
|
|
* letter commands are run via a thread. Each class
|
|
* letter commands are run via a thread. Each class
|
|
* maps to a corresponding 4 letter command. CommandThread
|
|
* maps to a corresponding 4 letter command. CommandThread
|
|
* is the abstract class from which all the others inherit.
|
|
* is the abstract class from which all the others inherit.
|
|
*/
|
|
*/
|
|
- private abstract class CommandThread extends Thread {
|
|
|
|
|
|
+ private abstract class CommandThread {
|
|
PrintWriter pw;
|
|
PrintWriter pw;
|
|
-
|
|
|
|
|
|
+
|
|
CommandThread(PrintWriter pw) {
|
|
CommandThread(PrintWriter pw) {
|
|
this.pw = pw;
|
|
this.pw = pw;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ public void start() {
|
|
|
|
+ run();
|
|
|
|
+ }
|
|
|
|
+
|
|
public void run() {
|
|
public void run() {
|
|
try {
|
|
try {
|
|
commandRun();
|
|
commandRun();
|
|
@@ -498,52 +523,52 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
cleanupWriterSocket(pw);
|
|
cleanupWriterSocket(pw);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
public abstract void commandRun() throws IOException;
|
|
public abstract void commandRun() throws IOException;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
private class RuokCommand extends CommandThread {
|
|
private class RuokCommand extends CommandThread {
|
|
public RuokCommand(PrintWriter pw) {
|
|
public RuokCommand(PrintWriter pw) {
|
|
super(pw);
|
|
super(pw);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void commandRun() {
|
|
public void commandRun() {
|
|
pw.print("imok");
|
|
pw.print("imok");
|
|
-
|
|
|
|
|
|
+
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
private class TraceMaskCommand extends CommandThread {
|
|
private class TraceMaskCommand extends CommandThread {
|
|
TraceMaskCommand(PrintWriter pw) {
|
|
TraceMaskCommand(PrintWriter pw) {
|
|
super(pw);
|
|
super(pw);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void commandRun() {
|
|
public void commandRun() {
|
|
long traceMask = ZooTrace.getTextTraceLevel();
|
|
long traceMask = ZooTrace.getTextTraceLevel();
|
|
pw.print(traceMask);
|
|
pw.print(traceMask);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
private class SetTraceMaskCommand extends CommandThread {
|
|
private class SetTraceMaskCommand extends CommandThread {
|
|
long trace = 0;
|
|
long trace = 0;
|
|
SetTraceMaskCommand(PrintWriter pw, long trace) {
|
|
SetTraceMaskCommand(PrintWriter pw, long trace) {
|
|
super(pw);
|
|
super(pw);
|
|
this.trace = trace;
|
|
this.trace = trace;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void commandRun() {
|
|
public void commandRun() {
|
|
pw.print(trace);
|
|
pw.print(trace);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
private class EnvCommand extends CommandThread {
|
|
private class EnvCommand extends CommandThread {
|
|
EnvCommand(PrintWriter pw) {
|
|
EnvCommand(PrintWriter pw) {
|
|
super(pw);
|
|
super(pw);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void commandRun() {
|
|
public void commandRun() {
|
|
List<Environment.Entry> env = Environment.list();
|
|
List<Environment.Entry> env = Environment.list();
|
|
@@ -554,15 +579,15 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
pw.print("=");
|
|
pw.print("=");
|
|
pw.println(e.getValue());
|
|
pw.println(e.getValue());
|
|
}
|
|
}
|
|
-
|
|
|
|
- }
|
|
|
|
|
|
+
|
|
|
|
+ }
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
private class ConfCommand extends CommandThread {
|
|
private class ConfCommand extends CommandThread {
|
|
ConfCommand(PrintWriter pw) {
|
|
ConfCommand(PrintWriter pw) {
|
|
super(pw);
|
|
super(pw);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void commandRun() {
|
|
public void commandRun() {
|
|
if (zkServer == null) {
|
|
if (zkServer == null) {
|
|
@@ -572,38 +597,36 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
private class StatResetCommand extends CommandThread {
|
|
private class StatResetCommand extends CommandThread {
|
|
public StatResetCommand(PrintWriter pw) {
|
|
public StatResetCommand(PrintWriter pw) {
|
|
super(pw);
|
|
super(pw);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void commandRun() {
|
|
public void commandRun() {
|
|
if (zkServer == null) {
|
|
if (zkServer == null) {
|
|
pw.println(ZK_NOT_SERVING);
|
|
pw.println(ZK_NOT_SERVING);
|
|
}
|
|
}
|
|
- else {
|
|
|
|
|
|
+ else {
|
|
zkServer.serverStats().reset();
|
|
zkServer.serverStats().reset();
|
|
pw.println("Server stats reset.");
|
|
pw.println("Server stats reset.");
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
private class CnxnStatResetCommand extends CommandThread {
|
|
private class CnxnStatResetCommand extends CommandThread {
|
|
public CnxnStatResetCommand(PrintWriter pw) {
|
|
public CnxnStatResetCommand(PrintWriter pw) {
|
|
super(pw);
|
|
super(pw);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void commandRun() {
|
|
public void commandRun() {
|
|
if (zkServer == null) {
|
|
if (zkServer == null) {
|
|
pw.println(ZK_NOT_SERVING);
|
|
pw.println(ZK_NOT_SERVING);
|
|
} else {
|
|
} else {
|
|
- synchronized(factory.cnxns){
|
|
|
|
- for(ServerCnxn c : factory.cnxns){
|
|
|
|
- c.resetStats();
|
|
|
|
- }
|
|
|
|
|
|
+ for(ServerCnxn c : factory.cnxns){
|
|
|
|
+ c.resetStats();
|
|
}
|
|
}
|
|
pw.println("Connection stats reset.");
|
|
pw.println("Connection stats reset.");
|
|
}
|
|
}
|
|
@@ -614,7 +637,7 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
public DumpCommand(PrintWriter pw) {
|
|
public DumpCommand(PrintWriter pw) {
|
|
super(pw);
|
|
super(pw);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void commandRun() {
|
|
public void commandRun() {
|
|
if (zkServer == null) {
|
|
if (zkServer == null) {
|
|
@@ -625,24 +648,26 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
zkServer.sessionTracker.dumpSessions(pw);
|
|
zkServer.sessionTracker.dumpSessions(pw);
|
|
pw.println("ephemeral nodes dump:");
|
|
pw.println("ephemeral nodes dump:");
|
|
zkServer.dumpEphemerals(pw);
|
|
zkServer.dumpEphemerals(pw);
|
|
|
|
+ pw.println("Connections dump:");
|
|
|
|
+ factory.dumpConnections(pw);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
private class StatCommand extends CommandThread {
|
|
private class StatCommand extends CommandThread {
|
|
int len;
|
|
int len;
|
|
public StatCommand(PrintWriter pw, int len) {
|
|
public StatCommand(PrintWriter pw, int len) {
|
|
super(pw);
|
|
super(pw);
|
|
this.len = len;
|
|
this.len = len;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@SuppressWarnings("unchecked")
|
|
@SuppressWarnings("unchecked")
|
|
@Override
|
|
@Override
|
|
public void commandRun() {
|
|
public void commandRun() {
|
|
if (zkServer == null) {
|
|
if (zkServer == null) {
|
|
pw.println(ZK_NOT_SERVING);
|
|
pw.println(ZK_NOT_SERVING);
|
|
}
|
|
}
|
|
- else {
|
|
|
|
|
|
+ else {
|
|
pw.print("Zookeeper version: ");
|
|
pw.print("Zookeeper version: ");
|
|
pw.println(Version.getFullVersion());
|
|
pw.println(Version.getFullVersion());
|
|
if (zkServer instanceof ReadOnlyZooKeeperServer) {
|
|
if (zkServer instanceof ReadOnlyZooKeeperServer) {
|
|
@@ -652,14 +677,7 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
if (len == statCmd) {
|
|
if (len == statCmd) {
|
|
LOG.info("Stat command output");
|
|
LOG.info("Stat command output");
|
|
pw.println("Clients:");
|
|
pw.println("Clients:");
|
|
- // clone should be faster than iteration
|
|
|
|
- // ie give up the cnxns lock faster
|
|
|
|
- HashSet<NIOServerCnxn> cnxnset;
|
|
|
|
- synchronized(factory.cnxns){
|
|
|
|
- cnxnset = (HashSet<NIOServerCnxn>)factory
|
|
|
|
- .cnxns.clone();
|
|
|
|
- }
|
|
|
|
- for(NIOServerCnxn c : cnxnset){
|
|
|
|
|
|
+ for(ServerCnxn c : factory.cnxns){
|
|
c.dumpConnectionInfo(pw, true);
|
|
c.dumpConnectionInfo(pw, true);
|
|
pw.println();
|
|
pw.println();
|
|
}
|
|
}
|
|
@@ -669,28 +687,22 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
pw.print("Node count: ");
|
|
pw.print("Node count: ");
|
|
pw.println(zkServer.getZKDatabase().getNodeCount());
|
|
pw.println(zkServer.getZKDatabase().getNodeCount());
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
private class ConsCommand extends CommandThread {
|
|
private class ConsCommand extends CommandThread {
|
|
public ConsCommand(PrintWriter pw) {
|
|
public ConsCommand(PrintWriter pw) {
|
|
super(pw);
|
|
super(pw);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@SuppressWarnings("unchecked")
|
|
@SuppressWarnings("unchecked")
|
|
@Override
|
|
@Override
|
|
public void commandRun() {
|
|
public void commandRun() {
|
|
if (zkServer == null) {
|
|
if (zkServer == null) {
|
|
pw.println(ZK_NOT_SERVING);
|
|
pw.println(ZK_NOT_SERVING);
|
|
} else {
|
|
} else {
|
|
- // clone should be faster than iteration
|
|
|
|
- // ie give up the cnxns lock faster
|
|
|
|
- HashSet<NIOServerCnxn> cnxns;
|
|
|
|
- synchronized (factory.cnxns) {
|
|
|
|
- cnxns = (HashSet<NIOServerCnxn>) factory.cnxns.clone();
|
|
|
|
- }
|
|
|
|
- for (NIOServerCnxn c : cnxns) {
|
|
|
|
|
|
+ for (ServerCnxn c : factory.cnxns) {
|
|
c.dumpConnectionInfo(pw, false);
|
|
c.dumpConnectionInfo(pw, false);
|
|
pw.println();
|
|
pw.println();
|
|
}
|
|
}
|
|
@@ -698,7 +710,7 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
private class WatchCommand extends CommandThread {
|
|
private class WatchCommand extends CommandThread {
|
|
int len = 0;
|
|
int len = 0;
|
|
public WatchCommand(PrintWriter pw, int len) {
|
|
public WatchCommand(PrintWriter pw, int len) {
|
|
@@ -821,7 +833,7 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
/** cancel the selection key to remove the socket handling
|
|
/** cancel the selection key to remove the socket handling
|
|
* from selector. This is to prevent netcat problem wherein
|
|
* from selector. This is to prevent netcat problem wherein
|
|
* netcat immediately closes the sending side after sending the
|
|
* netcat immediately closes the sending side after sending the
|
|
- * commands and still keeps the receiving channel open.
|
|
|
|
|
|
+ * commands and still keeps the receiving channel open.
|
|
* The idea is to remove the selectionkey from the selector
|
|
* The idea is to remove the selectionkey from the selector
|
|
* so that the selector does not notice the closed read on the
|
|
* so that the selector does not notice the closed read on the
|
|
* socket channel and keep the socket alive to write the data to
|
|
* socket channel and keep the socket alive to write the data to
|
|
@@ -897,6 +909,9 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
IsroCommand isro = new IsroCommand(pwriter);
|
|
IsroCommand isro = new IsroCommand(pwriter);
|
|
isro.start();
|
|
isro.start();
|
|
return true;
|
|
return true;
|
|
|
|
+ } else if (len == telnetCloseCmd) {
|
|
|
|
+ cleanupWriterSocket(null);
|
|
|
|
+ return true;
|
|
}
|
|
}
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
@@ -925,11 +940,7 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
}
|
|
}
|
|
|
|
|
|
public long getOutstandingRequests() {
|
|
public long getOutstandingRequests() {
|
|
- synchronized (this) {
|
|
|
|
- synchronized (this.factory) {
|
|
|
|
- return outstandingRequests;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
+ return outstandingRequests.get();
|
|
}
|
|
}
|
|
|
|
|
|
/*
|
|
/*
|
|
@@ -941,53 +952,45 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
return sessionTimeout;
|
|
return sessionTimeout;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Used by "dump" 4-letter command to list all connection in
|
|
|
|
+ * cnxnExpiryMap
|
|
|
|
+ */
|
|
@Override
|
|
@Override
|
|
public String toString() {
|
|
public String toString() {
|
|
- return "NIOServerCnxn object with sock = " + sock + " and sk = " + sk;
|
|
|
|
|
|
+ return "ip: " + sock.socket().getRemoteSocketAddress() +
|
|
|
|
+ " sessionId: 0x" + Long.toHexString(sessionId);
|
|
}
|
|
}
|
|
|
|
|
|
- /*
|
|
|
|
|
|
+ /**
|
|
* Close the cnxn and remove it from the factory cnxns list.
|
|
* Close the cnxn and remove it from the factory cnxns list.
|
|
- *
|
|
|
|
- * This function returns immediately if the cnxn is not on the cnxns list.
|
|
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
public void close() {
|
|
public void close() {
|
|
- synchronized(factory.cnxns){
|
|
|
|
- // if this is not in cnxns then it's already closed
|
|
|
|
- if (!factory.cnxns.remove(this)) {
|
|
|
|
- return;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- synchronized (factory.ipMap) {
|
|
|
|
- Set<NIOServerCnxn> s =
|
|
|
|
- factory.ipMap.get(sock.socket().getInetAddress());
|
|
|
|
- s.remove(this);
|
|
|
|
- }
|
|
|
|
|
|
+ if (!factory.removeCnxn(this)) {
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
|
|
- factory.unregisterConnection(this);
|
|
|
|
|
|
+ if (zkServer != null) {
|
|
|
|
+ zkServer.removeCnxn(this);
|
|
|
|
+ }
|
|
|
|
|
|
- if (zkServer != null) {
|
|
|
|
- zkServer.removeCnxn(this);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- closeSock();
|
|
|
|
-
|
|
|
|
- if (sk != null) {
|
|
|
|
- try {
|
|
|
|
- // need to cancel this selection key from the selector
|
|
|
|
- sk.cancel();
|
|
|
|
- } catch (Exception e) {
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("ignoring exception during selectionkey cancel", e);
|
|
|
|
- }
|
|
|
|
|
|
+ if (sk != null) {
|
|
|
|
+ try {
|
|
|
|
+ // need to cancel this selection key from the selector
|
|
|
|
+ sk.cancel();
|
|
|
|
+ } catch (Exception e) {
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("ignoring exception during selectionkey cancel", e);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ closeSock();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Close resources associated with the sock of this cnxn.
|
|
|
|
|
|
+ * Close resources associated with the sock of this cnxn.
|
|
*/
|
|
*/
|
|
private void closeSock() {
|
|
private void closeSock() {
|
|
if (sock == null) {
|
|
if (sock == null) {
|
|
@@ -999,6 +1002,18 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
+ (sessionId != 0 ?
|
|
+ (sessionId != 0 ?
|
|
" which had sessionid 0x" + Long.toHexString(sessionId) :
|
|
" which had sessionid 0x" + Long.toHexString(sessionId) :
|
|
" (no session established for client)"));
|
|
" (no session established for client)"));
|
|
|
|
+ closeSock(sock);
|
|
|
|
+ sock = null;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Close resources associated with a sock.
|
|
|
|
+ */
|
|
|
|
+ public static void closeSock(SocketChannel sock) {
|
|
|
|
+ if (sock == null) {
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
try {
|
|
try {
|
|
/*
|
|
/*
|
|
* The following sequence of code is stupid! You would think that
|
|
* The following sequence of code is stupid! You would think that
|
|
@@ -1030,18 +1045,13 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
}
|
|
}
|
|
try {
|
|
try {
|
|
sock.close();
|
|
sock.close();
|
|
- // 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();
|
|
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug("ignoring exception during socketchannel close", e);
|
|
LOG.debug("ignoring exception during socketchannel close", e);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- sock = null;
|
|
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
private final static byte fourBytes[] = new byte[4];
|
|
private final static byte fourBytes[] = new byte[4];
|
|
|
|
|
|
/*
|
|
/*
|
|
@@ -1051,7 +1061,7 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
* org.apache.jute.Record, java.lang.String)
|
|
* org.apache.jute.Record, java.lang.String)
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
- synchronized public void sendResponse(ReplyHeader h, Record r, String tag) {
|
|
|
|
|
|
+ public void sendResponse(ReplyHeader h, Record r, String tag) {
|
|
try {
|
|
try {
|
|
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
|
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
|
// Make space for length
|
|
// Make space for length
|
|
@@ -1071,16 +1081,10 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
bb.putInt(b.length - 4).rewind();
|
|
bb.putInt(b.length - 4).rewind();
|
|
sendBuffer(bb);
|
|
sendBuffer(bb);
|
|
if (h.getXid() > 0) {
|
|
if (h.getXid() > 0) {
|
|
- synchronized(this){
|
|
|
|
- outstandingRequests--;
|
|
|
|
- }
|
|
|
|
// check throttling
|
|
// check throttling
|
|
- synchronized (this.factory) {
|
|
|
|
- if (zkServer.getInProcess() < outstandingLimit
|
|
|
|
- || outstandingRequests < 1) {
|
|
|
|
- sk.selector().wakeup();
|
|
|
|
- enableRecv();
|
|
|
|
- }
|
|
|
|
|
|
+ if (outstandingRequests.decrementAndGet() < 1 ||
|
|
|
|
+ zkServer.getInProcess() < outstandingLimit) {
|
|
|
|
+ enableRecv();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} catch(Exception e) {
|
|
} catch(Exception e) {
|
|
@@ -1094,7 +1098,7 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
* @see org.apache.zookeeper.server.ServerCnxnIface#process(org.apache.zookeeper.proto.WatcherEvent)
|
|
* @see org.apache.zookeeper.server.ServerCnxnIface#process(org.apache.zookeeper.proto.WatcherEvent)
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
- synchronized public void process(WatchedEvent event) {
|
|
|
|
|
|
+ public void process(WatchedEvent event) {
|
|
ReplyHeader h = new ReplyHeader(-1, -1L, 0);
|
|
ReplyHeader h = new ReplyHeader(-1, -1L, 0);
|
|
if (LOG.isTraceEnabled()) {
|
|
if (LOG.isTraceEnabled()) {
|
|
ZooTrace.logTraceMessage(LOG, ZooTrace.EVENT_DELIVERY_TRACE_MASK,
|
|
ZooTrace.logTraceMessage(LOG, ZooTrace.EVENT_DELIVERY_TRACE_MASK,
|
|
@@ -1122,16 +1126,28 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
@Override
|
|
@Override
|
|
public void setSessionId(long sessionId) {
|
|
public void setSessionId(long sessionId) {
|
|
this.sessionId = sessionId;
|
|
this.sessionId = sessionId;
|
|
|
|
+ factory.addSession(sessionId, this);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void setSessionTimeout(int sessionTimeout) {
|
|
public void setSessionTimeout(int sessionTimeout) {
|
|
this.sessionTimeout = sessionTimeout;
|
|
this.sessionTimeout = sessionTimeout;
|
|
|
|
+ factory.touchCnxn(this);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public int getInterestOps() {
|
|
public int getInterestOps() {
|
|
- return sk.isValid() ? sk.interestOps() : 0;
|
|
|
|
|
|
+ if (!isSelectable()) {
|
|
|
|
+ return 0;
|
|
|
|
+ }
|
|
|
|
+ int interestOps = 0;
|
|
|
|
+ if (getReadInterest()) {
|
|
|
|
+ interestOps |= SelectionKey.OP_READ;
|
|
|
|
+ }
|
|
|
|
+ if (getWriteInterest()) {
|
|
|
|
+ interestOps |= SelectionKey.OP_WRITE;
|
|
|
|
+ }
|
|
|
|
+ return interestOps;
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -1142,6 +1158,13 @@ public class NIOServerCnxn extends ServerCnxn {
|
|
return (InetSocketAddress) sock.socket().getRemoteSocketAddress();
|
|
return (InetSocketAddress) sock.socket().getRemoteSocketAddress();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public InetAddress getSocketAddress() {
|
|
|
|
+ if (sock == null) {
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ return sock.socket().getInetAddress();
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
protected ServerStats serverStats() {
|
|
protected ServerStats serverStats() {
|
|
if (zkServer == null) {
|
|
if (zkServer == null) {
|