[
https://issues.apache.org/jira/browse/ZOOKEEPER-1049?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13021448#comment-13021448
]
Chang Song commented on ZOOKEEPER-1049:
---------------------------------------
When this happens
Below thread becomes BLOCKED
"NIOServerCxn.Factory:0.0.0.0/0.0.0.0:17288" - Thread t@11
java.lang.Thread.State: BLOCKED on
org.apache.zookeeper.server.NIOServerCnxn$Factory@399831c3 owned by:
CommitProcessor:3
at
org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:234)
Locked ownable synchronizers:
- None
If you look at CommitProcessor:3
"CommitProcessor:3" - Thread t@27
java.lang.Thread.State: RUNNABLE
at sun.nio.ch.FileDispatcher.preClose0(Native Method)
at sun.nio.ch.SocketDispatcher.preClose(SocketDispatcher.java:41)
at
sun.nio.ch.SocketChannelImpl.implCloseSelectableChannel(SocketChannelImpl.java:684)
- locked java.lang.Object@@3186c91
at
java.nio.channels.spi.AbstractSelectableChannel.implCloseChannel(AbstractSelectableChannel.java:201)
at
java.nio.channels.spi.AbstractInterruptibleChannel.close(AbstractInterruptibleChannel.java:97)
- locked java.lang.Object@@7fc600
at sun.nio.ch.SocketAdaptor.close(SocketAdaptor.java:352)
at
org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1463)
at
org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
- locked java.util.HashSet@@4b94a39
at
org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
at
org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
- locked org.apache.zookeeper.server.NIOServerCnxn$Factory@@399831c
at
org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
at
org.apache.zookeeper.server.quorum.Leader$ToBeAppliedRequestProcessor.processRequest(Leader.java:540)
at
org.apache.zookeeper.server.quorum.CommitProcessor.run(CommitProcessor.java:73)
Locked ownable synchronizers:
- None
scxn.closeSession(request.sessionId)
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 && request.cnxn == null) {
// calling this if we have the cnxn results in the client's
// close session response being lost - we've already closed
// the session/socket here before we can send the closeSession
// in the switch block below
scxn.closeSession(request.sessionId);
return;
}
}
Below is a synchronized method
NIOServerCnxn.Factory.closeSession()
synchronized void closeSession(long sessionId) {
selector.wakeup();
closeSessionWithoutWakeup(sessionId);
}
@SuppressWarnings("unchecked")
private void closeSessionWithoutWakeup(long sessionId) {
HashSet<NIOServerCnxn> cnxns;
synchronized (this.cnxns) {
cnxns = (HashSet<NIOServerCnxn>)this.cnxns.clone();
}
for (NIOServerCnxn cnxn : cnxns) {
if (cnxn.sessionId == sessionId) {
try {
cnxn.close();
} catch (Exception e) {
LOG.warn("exception during session close", e);
}
break;
}
}
}
We measured the time spent in scxn.closeSession(request.sessionId), and the
result is in the previous post.
if (request.hdr != null && request.hdr.getType() == OpCode.closeSession) {
Factory scxn = zks.getServerCnxnFactory();
if (scxn != null && request.cnxn == null) {
scxn.closeSession(request.sessionId);
return;
}
}
> Session expire/close flooding renders heartbeats to delay significantly
> -----------------------------------------------------------------------
>
> Key: ZOOKEEPER-1049
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1049
> Project: ZooKeeper
> Issue Type: Bug
> Components: server
> Affects Versions: 3.3.2
> Environment: CentOS 5.3, three node ZK ensemble
> Reporter: Chang Song
> Priority: Critical
> Attachments: ZookeeperPingTest.zip, zk_ping_latency.pdf
>
>
> Let's say we have 100 clients (group A) already connected to three-node ZK
> ensemble with session timeout of 15 second. And we have 1000 clients (group
> B) already connected to the same ZK ensemble, all watching several nodes
> (with 15 second session timeout)
> Consider a case in which All clients in group B suddenly hung or deadlocked
> (JVM OOME) all at the same time. 15 seconds later, all sessions in group B
> gets expired, creating session closing stampede. Depending on the number of
> this clients in group B, all request/response ZK ensemble should process get
> delayed up to 8 seconds (1000 clients we have tested).
> This delay causes some clients in group A their sessions expired due to delay
> in getting heartbeat response. This causes normal servers to drop out of
> clusters. This is a serious problem in our installation, since some of our
> services running batch servers or CI servers creating the same scenario as
> above almost everyday.
> I am attaching a graph showing ping response time delay.
> I think ordering of creating/closing sessions and ping exchange isn't
> important (quorum state machine). at least ping request / response should be
> handle independently (different queue and different thread) to keep
> realtime-ness of ping.
> As a workaround, we are raising session timeout to 50 seconds.
> But this causes max. failover of cluster to significantly increased, thus
> initial QoS we promised cannot be met.
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira