You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@zookeeper.apache.org by "Patania, Nick" <Ni...@morganstanley.com> on 2011/02/11 00:16:30 UTC

tickTime and sessionTimeout

Hello,

I run the following test:


*         Two clients connect to a zookeeper server; the tickTime on the server is 500, and the sessionTimeout on the client is 1000.

*         Kill the host running one of the clients.

*         The second client receives a session timeout.

Is there any reason why this might happen?

Thanks
Nick Patania


--------------------------------------------------------------------------
NOTICE: Morgan Stanley is not acting as a municipal advisor and the opinions or views contained herein are not intended to be, and do not constitute, advice within the meaning of Section 975 of the Dodd-Frank Wall Street Reform and Consumer Protection Act. If you have received this communication in error, please destroy all electronic and paper copies and notify the sender immediately. Mistransmission is not intended to waive confidentiality or privilege. Morgan Stanley reserves the right, to the extent permitted under applicable law, to monitor electronic communications. This message is subject to terms available at the following link: http://www.morganstanley.com/disclaimers. If you cannot access these links, please notify us by reply message and we will send the contents to you. By messaging with Morgan Stanley you consent to the foregoing.

Re: tickTime and sessionTimeout

Posted by Nick Patania <Ni...@morganstanley.com>.
Attached.

On Tue, Feb 15, 2011 at 2:23 PM, Fournier, Camille F. [Tech]
<Ca...@gs.com> wrote:
> I mean, full stack dump for the entire process.
>
> -----Original Message-----
> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
> Sent: Tuesday, February 15, 2011 2:11 PM
> To: user@zookeeper.apache.org
> Subject: Re: tickTime and sessionTimeout
>
> The two "suspicious" traces are full.  The full epoll trace is:
>
> "NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799" daemon prio=10
> tid=0xaec2b000 nid=0x6945 runnable [0xaed65000]
>   java.lang.Thread.State: RUNNABLE
>        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>        - locked <0xdfa04348> (a sun.nio.ch.Util$1)
>        - locked <0xdfa04358> (a java.util.Collections$UnmodifiableSet)
>        - locked <0xdfa04308> (a sun.nio.ch.EPollSelectorImpl)
>        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:232)
>
> On Tue, Feb 15, 2011 at 2:06 PM, Fournier, Camille F. [Tech]
> <Ca...@gs.com> wrote:
>> Can you send the full stack trace?
>>
>> -----Original Message-----
>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>> Sent: Tuesday, February 15, 2011 12:16 PM
>> To: user@zookeeper.apache.org
>> Subject: Re: tickTime and sessionTimeout
>>
>> I did some quick and dirty profiling, and during the period leading to
>> the expiration of SESSION_1, two of the server's threads are
>> suspiciously occupied as follows:
>>
>> "SyncThread:0" prio=10 tid=0xaf1cec00 nid=0x6947 runnable [0xaebad000]
>>   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 <0xdfa07648> (a java.lang.Object)
>>        at java.nio.channels.spi.AbstractSelectableChannel.implCloseChannel(AbstractSelectableChannel.java:201)
>>        at java.nio.channels.spi.AbstractInterruptibleChannel.close(AbstractInterruptibleChannel.java:97)
>>        - locked <0xdfa075e8> (a java.lang.Object)
>>        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 <0xdfa1c0d0> (a java.util.HashSet)
>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>        - locked <0xdfa04240> (a
>> org.apache.zookeeper.server.NIOServerCnxn$Factory)
>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>
>> "NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799" daemon prio=10
>> tid=0xaec2b000 nid=0x6945 waiting for monitor entry [
>> 0xaed65000]
>>   java.lang.Thread.State: BLOCKED (on object monitor)
>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:235)
>>        - waiting to lock <0xdfa04240> (a
>> org.apache.zookeeper.server.NIOServerCnxn$Factory)
>>
>> These two threads are in this state for around 1.66 seconds.  Does
>> this mean something to anyone?
>>
>> Note how the second thread "normally" seems to be in epoll_wait:
>>
>> "NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799" daemon prio=10
>> tid=0xaec2b000 nid=0x6945 runnable [0xaed65000]
>>   java.lang.Thread.State: RUNNABLE
>>        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>
>>> Is the server a virtualized host? I still think this might shed some
>>> light, what are you seeing for this? before/during/after the event:
>>
>> It's not a virtualized host.
>>
>> stat prints:
>>
>> Latency min/avg/max: 0/0/62
>>
>> But it's 62 (exactly) before, during and after (I run it every 50ms),
>> which makes me suspicious of the output.
>>
>>> It is blocked a
>>> bit behind the expiring of SESSION_0 due to the synchronous nature of touching
>>> the session table
>>
>> That can't possibly take hundreds of milliseconds, can it?
>>
>> On Mon, Feb 14, 2011 at 1:32 AM, Patrick Hunt <ph...@apache.org> wrote:
>>> Is the server a virtualized host? I still think this might shed some
>>> light, what are you seeing for this? before/during/after the event:
>>>
>>>>>>> Use the "stat" 4
>>>>>>> letter word to see the server's request processing latency, see if
>>>>>>> that's high (higer than the timeout is bad news). Again, checkout the
>>>>>>> troubleshooting guide.
>>>
>>> On Fri, Feb 11, 2011 at 4:32 PM, Fournier, Camille F. [Tech]
>>> <Ca...@gs.com> wrote:
>>>> I think the gist of the problem is that while the ZK Server is expiring SESSION_0, SESSION_1 is coming in and trying to send a ping. It is blocked a bit behind the expiring of SESSION_0 due to the synchronous nature of touching the session table, and then its request for a ping will be processed behind the session expiration processing for SESSION_0. So either the expirer takes long enough dealing with SESSION_0 that it immediately wants to expire SESSION_1 on next processing, or, the processing of the session expiration takes just long enough for SESSION_1 to not get a response to its heartbeat back from the server, which then causes it to disconnect and reconnect, and in the interim the server determines the session dead due to timeout.
>>>>
>>>> Long story short, those timeouts are too short for the server to reliably execute responses in time to guarantee they won't be incorrectly activated.
>>>>
>>>> C
>>>>
>>>> -----Original Message-----
>>>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>>>> Sent: Friday, February 11, 2011 6:19 PM
>>>> To: user@zookeeper.apache.org
>>>> Subject: Re: tickTime and sessionTimeout
>>>>
>>>> 3.3.2-1031432
>>>>
>>>> On Fri, Feb 11, 2011 at 6:15 PM, Fournier, Camille F. [Tech]
>>>> <Ca...@gs.com> wrote:
>>>>> Which version of ZK?
>>>>>
>>>>> -----Original Message-----
>>>>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>>>>> Sent: Friday, February 11, 2011 6:10 PM
>>>>> To: user@zookeeper.apache.org
>>>>> Subject: Re: tickTime and sessionTimeout
>>>>>
>>>>> A single server.  I've intentionally made it trivial to demonstrate
>>>>> the behavior.
>>>>> If I increase the timeout, the issue goes away.
>>>>>
>>>>> On Fri, Feb 11, 2011 at 6:00 PM, Fournier, Camille F. [Tech]
>>>>> <Ca...@gs.com> wrote:
>>>>>> What is your ZooKeeper setup here? And do you continue to see this issue if you increase your session timeout?
>>>>>>
>>>>>> C
>>>>>>
>>>>>> -----Original Message-----
>>>>>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>>>>>> Sent: Friday, February 11, 2011 5:24 PM
>>>>>> To: user@zookeeper.apache.org
>>>>>> Subject: Re: tickTime and sessionTimeout
>>>>>>
>>>>>> Patrick, thanks for your input.
>>>>>>
>>>>>> I have rerun the test several times now while logging GC on the server
>>>>>> and running ping from CLIENT_1:
>>>>>>
>>>>>>  - No GC happens on the server during the period of interest (a
>>>>>> couple of young generation runs happen before I kill HOST_0, and they
>>>>>> complete in under 3ms).
>>>>>>  - Round trip times for ping from CLIENT_1 are consistently under
>>>>>> 250us throughout.
>>>>>>
>>>>>> Regarding client GC -- I can consistently reproduce this using a C
>>>>>> client.  Regarding the theory of swapping on CLIENT_1 -- if that were
>>>>>> the cause, the problem wouldn't be 100% reproducible.  I also looked
>>>>>> through the client log for SESSION_1 -- I see "Got ping response ...
>>>>>> after 1ms" repeatedly, followed by "Client session timed out, have not
>>>>>> heard from server in 666ms"...
>>>>>>
>>>>>> On Fri, Feb 11, 2011 at 1:46 PM, Patrick Hunt <ph...@apache.org> wrote:
>>>>>>> Those are pretty short timeouts, many sources of delay could be
>>>>>>> causing this. Network jitter/latency, GC/swap (server or client), IO
>>>>>>> write latency, etc... See if any of this might be your issue:
>>>>>>> https://cwiki.apache.org/confluence/display/ZOOKEEPER/Troubleshooting
>>>>>>>
>>>>>>> I can clearly see this sequence in your log for session1:
>>>>>>>
>>>>>>> ---
>>>>>>> 2011-02-11 14:18:52,672 -  sessionid:SESSION_1 type:ping
>>>>>>>
>>>>>>> 2011-02-11 14:18:54,502 - Expiring session SESSION_1, timeout of 1000ms exceeded
>>>>>>>
>>>>>>> 2011-02-11 14:18:55,011 - Processing request:: sessionid:SESSION_1 type:ping
>>>>>>> ---
>>>>>>>
>>>>>>> from the looks of it session 1 doesn't send a ping to the server for
>>>>>>> ~2.5 seconds, as a result it's expired.
>>>>>>>
>>>>>>> You should also look at your session 1 client log and see what it's
>>>>>>> view of the world is like. (is it gc/swapping?). Use the "stat" 4
>>>>>>> letter word to see the server's request processing latency, see if
>>>>>>> that's high (higer than the timeout is bad news). Again, checkout the
>>>>>>> troubleshooting guide.
>>>>>>>
>>>>>>> Patrick
>>>>>>>
>>>>>>> ps please use pastebin or attachment, otw the formatting of wrecked
>>>>>>> and it's harder to read the log
>>>>>>>
>>>>>>> On Fri, Feb 11, 2011 at 6:41 AM, Nick Patania
>>>>>>> <Ni...@morganstanley.com> wrote:
>>>>>>>> This is the portion that seems relevant.  For readability, I replaced
>>>>>>>> the host and session for the host that I kill with HOST_0 and
>>>>>>>> SESSION_0 (I expect these to timeout).  The client that should be
>>>>>>>> healthy is HOST_1 and SESSION_1.
>>>>>>>>
>>>>>>>>
>>>>>>>> 2011-02-11 14:18:51,901 - DEBUG
>>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>>> sessionid:SESSION_0 type:ping cxid:0xfffffffffffffffe
>>>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>>>> 2011-02-11 14:18:51,901 - DEBUG
>>>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_0
>>>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>>>> txntype:unknown reqpath:n/a
>>>>>>>> 2011-02-11 14:18:52,005 - DEBUG
>>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>>>> 2011-02-11 14:18:52,005 - DEBUG
>>>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>>>> txntype:unknown reqpath:n/a
>>>>>>>> 2011-02-11 14:18:52,339 - DEBUG
>>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>>>> 2011-02-11 14:18:52,339 - DEBUG
>>>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>>>> txntype:unknown reqpath:n/a
>>>>>>>> 2011-02-11 14:18:52,672 - DEBUG
>>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>>>> 2011-02-11 14:18:52,672 - DEBUG
>>>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>>>> txntype:unknown reqpath:n/a
>>>>>>>> 2011-02-11 14:18:53,002 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>>>>>>> Expiring session SESSION_0, timeout of 1000ms exceeded
>>>>>>>> 2011-02-11 14:18:53,002 - INFO
>>>>>>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>>>>>>> termination for sessionid: SESSION_0
>>>>>>>> 2011-02-11 14:18:53,010 - DEBUG
>>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>>> sessionid:SESSION_0 type:closeSession cxid:0x0 zxid:0x103 txntype:-11
>>>>>>>> reqpath:n/a
>>>>>>>> 2011-02-11 14:18:53,010 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>>>>>>> Closed socket connection for client /HOST_0:34618 which had sessionid
>>>>>>>> SESSION_0
>>>>>>>> 2011-02-11 14:18:54,502 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>>>>>>> Expiring session SESSION_1, timeout of 1000ms exceeded
>>>>>>>> 2011-02-11 14:18:54,502 - INFO
>>>>>>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>>>>>>> termination for sessionid: SESSION_1
>>>>>>>> 2011-02-11 14:18:55,011 - DEBUG
>>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>>>> 2011-02-11 14:18:55,011 - DEBUG
>>>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>>>> txntype:unknown reqpath:n/a
>>>>>>>> 2011-02-11 14:18:55,011 - INFO
>>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@251]
>>>>>>>> - Accepted socket connection from /HOST_1:40556
>>>>>>>> 2011-02-11 14:18:55,019 - DEBUG
>>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>>> sessionid:SESSION_1 type:closeSession cxid:0x0 zxid:0x104 txntype:-11
>>>>>>>> reqpath:n/a
>>>>>>>> 2011-02-11 14:18:55,019 - DEBUG
>>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:ZooKeeperServer@590] -
>>>>>>>> Dropping request: No session with sessionid SESSION_1 exists, probably
>>>>>>>> expired and removed
>>>>>>>> 2011-02-11 14:18:55,019 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>>>>>>> Closed socket connection for client /HOST_1:40555 which had sessionid
>>>>>>>> SESSION_1
>>>>>>>> 2011-02-11 14:18:55,020 - DEBUG [SyncThread:0:NIOServerCnxn@1451] -
>>>>>>>> ignoring exception during output shutdown
>>>>>>>> java.net.SocketException: Transport endpoint is not connected
>>>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdownOutput(SocketChannelImpl.java:651)
>>>>>>>>        at sun.nio.ch.SocketAdaptor.shutdownOutput(SocketAdaptor.java:368)
>>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1447)
>>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>>>>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>>>>>>> 2011-02-11 14:18:55,021 - DEBUG [SyncThread:0:NIOServerCnxn@1459] -
>>>>>>>> ignoring exception during input shutdown
>>>>>>>> java.net.SocketException: Transport endpoint is not connected
>>>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdownInput(SocketChannelImpl.java:640)
>>>>>>>>        at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:360)
>>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1455)
>>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>>>>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>>>>>>> 2011-02-11 14:18:55,022 - WARN
>>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@272]
>>>>>>>> - Ignoring unexpected runtime exception
>>>>>>>> java.nio.channels.CancelledKeyException
>>>>>>>>        at sun.nio.ch.SelectionKeyImpl.ensureValid(SelectionKeyImpl.java:55)
>>>>>>>>        at sun.nio.ch.SelectionKeyImpl.readyOps(SelectionKeyImpl.java:69)
>>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:241)
>>>>>>>> 2011-02-11 14:18:55,023 - DEBUG
>>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@735] -
>>>>>>>> Session establishment request from client /HOST_1:40556 client's
>>>>>>>> lastZxid is 0x0
>>>>>>>> 2011-02-11 14:18:55,023 - INFO
>>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@770] - Client
>>>>>>>> attempting to renew session SESSION_1 at /HOST_1:40556
>>>>>>>> 2011-02-11 14:18:55,024 - INFO
>>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1573] -
>>>>>>>> Invalid session SESSION_1 for client /HOST_1:40556, probably expired
>>>>>>>> 2011-02-11 14:18:55,025 - WARN
>>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@634] -
>>>>>>>> EndOfStreamException: Unable to read additional data from client
>>>>>>>> sessionid SESSION_1, likely client has closed socket
>>>>>>>> 2011-02-11 14:18:55,025 - INFO
>>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1435] -
>>>>>>>> Closed socket connection for client /HOST_1:40556 which had sessionid
>>>>>>>> SESSION_1
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Feb 10, 2011 at 8:11 PM, Benjamin Reed <br...@yahoo-inc.com> wrote:
>>>>>>>>> do you see anything in the server log?
>>>>>>>>>
>>>>>>>>> ben
>>>>>>>>>
>>>>>>>>> On 02/10/2011 03:16 PM, Patania, Nick wrote:
>>>>>>>>>>
>>>>>>>>>> Hello,
>>>>>>>>>>
>>>>>>>>>> I run the following test:
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> *         Two clients connect to a zookeeper server; the tickTime on the
>>>>>>>>>> server is 500, and the sessionTimeout on the client is 1000.
>>>>>>>>>>
>>>>>>>>>> *         Kill the host running one of the clients.
>>>>>>>>>>
>>>>>>>>>> *         The second client receives a session timeout.
>>>>>>>>>>
>>>>>>>>>> Is there any reason why this might happen?
>>>>>>>>>>
>>>>>>>>>> Thanks
>>>>>>>>>> Nick Patania
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> --------------------------------------------------------------------------
>>>>>>>>>> NOTICE: Morgan Stanley is not acting as a municipal advisor and the
>>>>>>>>>> opinions or views contained herein are not intended to be, and do not
>>>>>>>>>> constitute, advice within the meaning of Section 975 of the Dodd-Frank Wall
>>>>>>>>>> Street Reform and Consumer Protection Act. If you have received this
>>>>>>>>>> communication in error, please destroy all electronic and paper copies and
>>>>>>>>>> notify the sender immediately. Mistransmission is not intended to waive
>>>>>>>>>> confidentiality or privilege. Morgan Stanley reserves the right, to the
>>>>>>>>>> extent permitted under applicable law, to monitor electronic communications.
>>>>>>>>>> This message is subject to terms available at the following link:
>>>>>>>>>> http://www.morganstanley.com/disclaimers. If you cannot access these links,
>>>>>>>>>> please notify us by reply message and we will send the contents to you. By
>>>>>>>>>> messaging with Morgan Stanley you consent to the foregoing.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>

RE: tickTime and sessionTimeout

Posted by "Fournier, Camille F. [Tech]" <Ca...@gs.com>.
I mean, full stack dump for the entire process.

-----Original Message-----
From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
Sent: Tuesday, February 15, 2011 2:11 PM
To: user@zookeeper.apache.org
Subject: Re: tickTime and sessionTimeout

The two "suspicious" traces are full.  The full epoll trace is:

"NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799" daemon prio=10
tid=0xaec2b000 nid=0x6945 runnable [0xaed65000]
   java.lang.Thread.State: RUNNABLE
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
        - locked <0xdfa04348> (a sun.nio.ch.Util$1)
        - locked <0xdfa04358> (a java.util.Collections$UnmodifiableSet)
        - locked <0xdfa04308> (a sun.nio.ch.EPollSelectorImpl)
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:232)

On Tue, Feb 15, 2011 at 2:06 PM, Fournier, Camille F. [Tech]
<Ca...@gs.com> wrote:
> Can you send the full stack trace?
>
> -----Original Message-----
> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
> Sent: Tuesday, February 15, 2011 12:16 PM
> To: user@zookeeper.apache.org
> Subject: Re: tickTime and sessionTimeout
>
> I did some quick and dirty profiling, and during the period leading to
> the expiration of SESSION_1, two of the server's threads are
> suspiciously occupied as follows:
>
> "SyncThread:0" prio=10 tid=0xaf1cec00 nid=0x6947 runnable [0xaebad000]
>   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 <0xdfa07648> (a java.lang.Object)
>        at java.nio.channels.spi.AbstractSelectableChannel.implCloseChannel(AbstractSelectableChannel.java:201)
>        at java.nio.channels.spi.AbstractInterruptibleChannel.close(AbstractInterruptibleChannel.java:97)
>        - locked <0xdfa075e8> (a java.lang.Object)
>        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 <0xdfa1c0d0> (a java.util.HashSet)
>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>        - locked <0xdfa04240> (a
> org.apache.zookeeper.server.NIOServerCnxn$Factory)
>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>
> "NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799" daemon prio=10
> tid=0xaec2b000 nid=0x6945 waiting for monitor entry [
> 0xaed65000]
>   java.lang.Thread.State: BLOCKED (on object monitor)
>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:235)
>        - waiting to lock <0xdfa04240> (a
> org.apache.zookeeper.server.NIOServerCnxn$Factory)
>
> These two threads are in this state for around 1.66 seconds.  Does
> this mean something to anyone?
>
> Note how the second thread "normally" seems to be in epoll_wait:
>
> "NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799" daemon prio=10
> tid=0xaec2b000 nid=0x6945 runnable [0xaed65000]
>   java.lang.Thread.State: RUNNABLE
>        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>
>> Is the server a virtualized host? I still think this might shed some
>> light, what are you seeing for this? before/during/after the event:
>
> It's not a virtualized host.
>
> stat prints:
>
> Latency min/avg/max: 0/0/62
>
> But it's 62 (exactly) before, during and after (I run it every 50ms),
> which makes me suspicious of the output.
>
>> It is blocked a
>> bit behind the expiring of SESSION_0 due to the synchronous nature of touching
>> the session table
>
> That can't possibly take hundreds of milliseconds, can it?
>
> On Mon, Feb 14, 2011 at 1:32 AM, Patrick Hunt <ph...@apache.org> wrote:
>> Is the server a virtualized host? I still think this might shed some
>> light, what are you seeing for this? before/during/after the event:
>>
>>>>>> Use the "stat" 4
>>>>>> letter word to see the server's request processing latency, see if
>>>>>> that's high (higer than the timeout is bad news). Again, checkout the
>>>>>> troubleshooting guide.
>>
>> On Fri, Feb 11, 2011 at 4:32 PM, Fournier, Camille F. [Tech]
>> <Ca...@gs.com> wrote:
>>> I think the gist of the problem is that while the ZK Server is expiring SESSION_0, SESSION_1 is coming in and trying to send a ping. It is blocked a bit behind the expiring of SESSION_0 due to the synchronous nature of touching the session table, and then its request for a ping will be processed behind the session expiration processing for SESSION_0. So either the expirer takes long enough dealing with SESSION_0 that it immediately wants to expire SESSION_1 on next processing, or, the processing of the session expiration takes just long enough for SESSION_1 to not get a response to its heartbeat back from the server, which then causes it to disconnect and reconnect, and in the interim the server determines the session dead due to timeout.
>>>
>>> Long story short, those timeouts are too short for the server to reliably execute responses in time to guarantee they won't be incorrectly activated.
>>>
>>> C
>>>
>>> -----Original Message-----
>>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>>> Sent: Friday, February 11, 2011 6:19 PM
>>> To: user@zookeeper.apache.org
>>> Subject: Re: tickTime and sessionTimeout
>>>
>>> 3.3.2-1031432
>>>
>>> On Fri, Feb 11, 2011 at 6:15 PM, Fournier, Camille F. [Tech]
>>> <Ca...@gs.com> wrote:
>>>> Which version of ZK?
>>>>
>>>> -----Original Message-----
>>>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>>>> Sent: Friday, February 11, 2011 6:10 PM
>>>> To: user@zookeeper.apache.org
>>>> Subject: Re: tickTime and sessionTimeout
>>>>
>>>> A single server.  I've intentionally made it trivial to demonstrate
>>>> the behavior.
>>>> If I increase the timeout, the issue goes away.
>>>>
>>>> On Fri, Feb 11, 2011 at 6:00 PM, Fournier, Camille F. [Tech]
>>>> <Ca...@gs.com> wrote:
>>>>> What is your ZooKeeper setup here? And do you continue to see this issue if you increase your session timeout?
>>>>>
>>>>> C
>>>>>
>>>>> -----Original Message-----
>>>>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>>>>> Sent: Friday, February 11, 2011 5:24 PM
>>>>> To: user@zookeeper.apache.org
>>>>> Subject: Re: tickTime and sessionTimeout
>>>>>
>>>>> Patrick, thanks for your input.
>>>>>
>>>>> I have rerun the test several times now while logging GC on the server
>>>>> and running ping from CLIENT_1:
>>>>>
>>>>>  - No GC happens on the server during the period of interest (a
>>>>> couple of young generation runs happen before I kill HOST_0, and they
>>>>> complete in under 3ms).
>>>>>  - Round trip times for ping from CLIENT_1 are consistently under
>>>>> 250us throughout.
>>>>>
>>>>> Regarding client GC -- I can consistently reproduce this using a C
>>>>> client.  Regarding the theory of swapping on CLIENT_1 -- if that were
>>>>> the cause, the problem wouldn't be 100% reproducible.  I also looked
>>>>> through the client log for SESSION_1 -- I see "Got ping response ...
>>>>> after 1ms" repeatedly, followed by "Client session timed out, have not
>>>>> heard from server in 666ms"...
>>>>>
>>>>> On Fri, Feb 11, 2011 at 1:46 PM, Patrick Hunt <ph...@apache.org> wrote:
>>>>>> Those are pretty short timeouts, many sources of delay could be
>>>>>> causing this. Network jitter/latency, GC/swap (server or client), IO
>>>>>> write latency, etc... See if any of this might be your issue:
>>>>>> https://cwiki.apache.org/confluence/display/ZOOKEEPER/Troubleshooting
>>>>>>
>>>>>> I can clearly see this sequence in your log for session1:
>>>>>>
>>>>>> ---
>>>>>> 2011-02-11 14:18:52,672 -  sessionid:SESSION_1 type:ping
>>>>>>
>>>>>> 2011-02-11 14:18:54,502 - Expiring session SESSION_1, timeout of 1000ms exceeded
>>>>>>
>>>>>> 2011-02-11 14:18:55,011 - Processing request:: sessionid:SESSION_1 type:ping
>>>>>> ---
>>>>>>
>>>>>> from the looks of it session 1 doesn't send a ping to the server for
>>>>>> ~2.5 seconds, as a result it's expired.
>>>>>>
>>>>>> You should also look at your session 1 client log and see what it's
>>>>>> view of the world is like. (is it gc/swapping?). Use the "stat" 4
>>>>>> letter word to see the server's request processing latency, see if
>>>>>> that's high (higer than the timeout is bad news). Again, checkout the
>>>>>> troubleshooting guide.
>>>>>>
>>>>>> Patrick
>>>>>>
>>>>>> ps please use pastebin or attachment, otw the formatting of wrecked
>>>>>> and it's harder to read the log
>>>>>>
>>>>>> On Fri, Feb 11, 2011 at 6:41 AM, Nick Patania
>>>>>> <Ni...@morganstanley.com> wrote:
>>>>>>> This is the portion that seems relevant.  For readability, I replaced
>>>>>>> the host and session for the host that I kill with HOST_0 and
>>>>>>> SESSION_0 (I expect these to timeout).  The client that should be
>>>>>>> healthy is HOST_1 and SESSION_1.
>>>>>>>
>>>>>>>
>>>>>>> 2011-02-11 14:18:51,901 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>> sessionid:SESSION_0 type:ping cxid:0xfffffffffffffffe
>>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:51,901 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_0
>>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>>> txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:52,005 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:52,005 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>>> txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:52,339 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:52,339 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>>> txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:52,672 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:52,672 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>>> txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:53,002 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>>>>>> Expiring session SESSION_0, timeout of 1000ms exceeded
>>>>>>> 2011-02-11 14:18:53,002 - INFO
>>>>>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>>>>>> termination for sessionid: SESSION_0
>>>>>>> 2011-02-11 14:18:53,010 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>> sessionid:SESSION_0 type:closeSession cxid:0x0 zxid:0x103 txntype:-11
>>>>>>> reqpath:n/a
>>>>>>> 2011-02-11 14:18:53,010 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>>>>>> Closed socket connection for client /HOST_0:34618 which had sessionid
>>>>>>> SESSION_0
>>>>>>> 2011-02-11 14:18:54,502 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>>>>>> Expiring session SESSION_1, timeout of 1000ms exceeded
>>>>>>> 2011-02-11 14:18:54,502 - INFO
>>>>>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>>>>>> termination for sessionid: SESSION_1
>>>>>>> 2011-02-11 14:18:55,011 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:55,011 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>>> txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:55,011 - INFO
>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@251]
>>>>>>> - Accepted socket connection from /HOST_1:40556
>>>>>>> 2011-02-11 14:18:55,019 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>> sessionid:SESSION_1 type:closeSession cxid:0x0 zxid:0x104 txntype:-11
>>>>>>> reqpath:n/a
>>>>>>> 2011-02-11 14:18:55,019 - DEBUG
>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:ZooKeeperServer@590] -
>>>>>>> Dropping request: No session with sessionid SESSION_1 exists, probably
>>>>>>> expired and removed
>>>>>>> 2011-02-11 14:18:55,019 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>>>>>> Closed socket connection for client /HOST_1:40555 which had sessionid
>>>>>>> SESSION_1
>>>>>>> 2011-02-11 14:18:55,020 - DEBUG [SyncThread:0:NIOServerCnxn@1451] -
>>>>>>> ignoring exception during output shutdown
>>>>>>> java.net.SocketException: Transport endpoint is not connected
>>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdownOutput(SocketChannelImpl.java:651)
>>>>>>>        at sun.nio.ch.SocketAdaptor.shutdownOutput(SocketAdaptor.java:368)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1447)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>>>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>>>>>> 2011-02-11 14:18:55,021 - DEBUG [SyncThread:0:NIOServerCnxn@1459] -
>>>>>>> ignoring exception during input shutdown
>>>>>>> java.net.SocketException: Transport endpoint is not connected
>>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdownInput(SocketChannelImpl.java:640)
>>>>>>>        at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:360)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1455)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>>>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>>>>>> 2011-02-11 14:18:55,022 - WARN
>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@272]
>>>>>>> - Ignoring unexpected runtime exception
>>>>>>> java.nio.channels.CancelledKeyException
>>>>>>>        at sun.nio.ch.SelectionKeyImpl.ensureValid(SelectionKeyImpl.java:55)
>>>>>>>        at sun.nio.ch.SelectionKeyImpl.readyOps(SelectionKeyImpl.java:69)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:241)
>>>>>>> 2011-02-11 14:18:55,023 - DEBUG
>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@735] -
>>>>>>> Session establishment request from client /HOST_1:40556 client's
>>>>>>> lastZxid is 0x0
>>>>>>> 2011-02-11 14:18:55,023 - INFO
>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@770] - Client
>>>>>>> attempting to renew session SESSION_1 at /HOST_1:40556
>>>>>>> 2011-02-11 14:18:55,024 - INFO
>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1573] -
>>>>>>> Invalid session SESSION_1 for client /HOST_1:40556, probably expired
>>>>>>> 2011-02-11 14:18:55,025 - WARN
>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@634] -
>>>>>>> EndOfStreamException: Unable to read additional data from client
>>>>>>> sessionid SESSION_1, likely client has closed socket
>>>>>>> 2011-02-11 14:18:55,025 - INFO
>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1435] -
>>>>>>> Closed socket connection for client /HOST_1:40556 which had sessionid
>>>>>>> SESSION_1
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Feb 10, 2011 at 8:11 PM, Benjamin Reed <br...@yahoo-inc.com> wrote:
>>>>>>>> do you see anything in the server log?
>>>>>>>>
>>>>>>>> ben
>>>>>>>>
>>>>>>>> On 02/10/2011 03:16 PM, Patania, Nick wrote:
>>>>>>>>>
>>>>>>>>> Hello,
>>>>>>>>>
>>>>>>>>> I run the following test:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> *         Two clients connect to a zookeeper server; the tickTime on the
>>>>>>>>> server is 500, and the sessionTimeout on the client is 1000.
>>>>>>>>>
>>>>>>>>> *         Kill the host running one of the clients.
>>>>>>>>>
>>>>>>>>> *         The second client receives a session timeout.
>>>>>>>>>
>>>>>>>>> Is there any reason why this might happen?
>>>>>>>>>
>>>>>>>>> Thanks
>>>>>>>>> Nick Patania
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> --------------------------------------------------------------------------
>>>>>>>>> NOTICE: Morgan Stanley is not acting as a municipal advisor and the
>>>>>>>>> opinions or views contained herein are not intended to be, and do not
>>>>>>>>> constitute, advice within the meaning of Section 975 of the Dodd-Frank Wall
>>>>>>>>> Street Reform and Consumer Protection Act. If you have received this
>>>>>>>>> communication in error, please destroy all electronic and paper copies and
>>>>>>>>> notify the sender immediately. Mistransmission is not intended to waive
>>>>>>>>> confidentiality or privilege. Morgan Stanley reserves the right, to the
>>>>>>>>> extent permitted under applicable law, to monitor electronic communications.
>>>>>>>>> This message is subject to terms available at the following link:
>>>>>>>>> http://www.morganstanley.com/disclaimers. If you cannot access these links,
>>>>>>>>> please notify us by reply message and we will send the contents to you. By
>>>>>>>>> messaging with Morgan Stanley you consent to the foregoing.
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>

Re: tickTime and sessionTimeout

Posted by Nick Patania <Ni...@morganstanley.com>.
The two "suspicious" traces are full.  The full epoll trace is:

"NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799" daemon prio=10
tid=0xaec2b000 nid=0x6945 runnable [0xaed65000]
   java.lang.Thread.State: RUNNABLE
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
        - locked <0xdfa04348> (a sun.nio.ch.Util$1)
        - locked <0xdfa04358> (a java.util.Collections$UnmodifiableSet)
        - locked <0xdfa04308> (a sun.nio.ch.EPollSelectorImpl)
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:232)

On Tue, Feb 15, 2011 at 2:06 PM, Fournier, Camille F. [Tech]
<Ca...@gs.com> wrote:
> Can you send the full stack trace?
>
> -----Original Message-----
> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
> Sent: Tuesday, February 15, 2011 12:16 PM
> To: user@zookeeper.apache.org
> Subject: Re: tickTime and sessionTimeout
>
> I did some quick and dirty profiling, and during the period leading to
> the expiration of SESSION_1, two of the server's threads are
> suspiciously occupied as follows:
>
> "SyncThread:0" prio=10 tid=0xaf1cec00 nid=0x6947 runnable [0xaebad000]
>   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 <0xdfa07648> (a java.lang.Object)
>        at java.nio.channels.spi.AbstractSelectableChannel.implCloseChannel(AbstractSelectableChannel.java:201)
>        at java.nio.channels.spi.AbstractInterruptibleChannel.close(AbstractInterruptibleChannel.java:97)
>        - locked <0xdfa075e8> (a java.lang.Object)
>        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 <0xdfa1c0d0> (a java.util.HashSet)
>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>        - locked <0xdfa04240> (a
> org.apache.zookeeper.server.NIOServerCnxn$Factory)
>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>
> "NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799" daemon prio=10
> tid=0xaec2b000 nid=0x6945 waiting for monitor entry [
> 0xaed65000]
>   java.lang.Thread.State: BLOCKED (on object monitor)
>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:235)
>        - waiting to lock <0xdfa04240> (a
> org.apache.zookeeper.server.NIOServerCnxn$Factory)
>
> These two threads are in this state for around 1.66 seconds.  Does
> this mean something to anyone?
>
> Note how the second thread "normally" seems to be in epoll_wait:
>
> "NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799" daemon prio=10
> tid=0xaec2b000 nid=0x6945 runnable [0xaed65000]
>   java.lang.Thread.State: RUNNABLE
>        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>
>> Is the server a virtualized host? I still think this might shed some
>> light, what are you seeing for this? before/during/after the event:
>
> It's not a virtualized host.
>
> stat prints:
>
> Latency min/avg/max: 0/0/62
>
> But it's 62 (exactly) before, during and after (I run it every 50ms),
> which makes me suspicious of the output.
>
>> It is blocked a
>> bit behind the expiring of SESSION_0 due to the synchronous nature of touching
>> the session table
>
> That can't possibly take hundreds of milliseconds, can it?
>
> On Mon, Feb 14, 2011 at 1:32 AM, Patrick Hunt <ph...@apache.org> wrote:
>> Is the server a virtualized host? I still think this might shed some
>> light, what are you seeing for this? before/during/after the event:
>>
>>>>>> Use the "stat" 4
>>>>>> letter word to see the server's request processing latency, see if
>>>>>> that's high (higer than the timeout is bad news). Again, checkout the
>>>>>> troubleshooting guide.
>>
>> On Fri, Feb 11, 2011 at 4:32 PM, Fournier, Camille F. [Tech]
>> <Ca...@gs.com> wrote:
>>> I think the gist of the problem is that while the ZK Server is expiring SESSION_0, SESSION_1 is coming in and trying to send a ping. It is blocked a bit behind the expiring of SESSION_0 due to the synchronous nature of touching the session table, and then its request for a ping will be processed behind the session expiration processing for SESSION_0. So either the expirer takes long enough dealing with SESSION_0 that it immediately wants to expire SESSION_1 on next processing, or, the processing of the session expiration takes just long enough for SESSION_1 to not get a response to its heartbeat back from the server, which then causes it to disconnect and reconnect, and in the interim the server determines the session dead due to timeout.
>>>
>>> Long story short, those timeouts are too short for the server to reliably execute responses in time to guarantee they won't be incorrectly activated.
>>>
>>> C
>>>
>>> -----Original Message-----
>>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>>> Sent: Friday, February 11, 2011 6:19 PM
>>> To: user@zookeeper.apache.org
>>> Subject: Re: tickTime and sessionTimeout
>>>
>>> 3.3.2-1031432
>>>
>>> On Fri, Feb 11, 2011 at 6:15 PM, Fournier, Camille F. [Tech]
>>> <Ca...@gs.com> wrote:
>>>> Which version of ZK?
>>>>
>>>> -----Original Message-----
>>>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>>>> Sent: Friday, February 11, 2011 6:10 PM
>>>> To: user@zookeeper.apache.org
>>>> Subject: Re: tickTime and sessionTimeout
>>>>
>>>> A single server.  I've intentionally made it trivial to demonstrate
>>>> the behavior.
>>>> If I increase the timeout, the issue goes away.
>>>>
>>>> On Fri, Feb 11, 2011 at 6:00 PM, Fournier, Camille F. [Tech]
>>>> <Ca...@gs.com> wrote:
>>>>> What is your ZooKeeper setup here? And do you continue to see this issue if you increase your session timeout?
>>>>>
>>>>> C
>>>>>
>>>>> -----Original Message-----
>>>>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>>>>> Sent: Friday, February 11, 2011 5:24 PM
>>>>> To: user@zookeeper.apache.org
>>>>> Subject: Re: tickTime and sessionTimeout
>>>>>
>>>>> Patrick, thanks for your input.
>>>>>
>>>>> I have rerun the test several times now while logging GC on the server
>>>>> and running ping from CLIENT_1:
>>>>>
>>>>>  - No GC happens on the server during the period of interest (a
>>>>> couple of young generation runs happen before I kill HOST_0, and they
>>>>> complete in under 3ms).
>>>>>  - Round trip times for ping from CLIENT_1 are consistently under
>>>>> 250us throughout.
>>>>>
>>>>> Regarding client GC -- I can consistently reproduce this using a C
>>>>> client.  Regarding the theory of swapping on CLIENT_1 -- if that were
>>>>> the cause, the problem wouldn't be 100% reproducible.  I also looked
>>>>> through the client log for SESSION_1 -- I see "Got ping response ...
>>>>> after 1ms" repeatedly, followed by "Client session timed out, have not
>>>>> heard from server in 666ms"...
>>>>>
>>>>> On Fri, Feb 11, 2011 at 1:46 PM, Patrick Hunt <ph...@apache.org> wrote:
>>>>>> Those are pretty short timeouts, many sources of delay could be
>>>>>> causing this. Network jitter/latency, GC/swap (server or client), IO
>>>>>> write latency, etc... See if any of this might be your issue:
>>>>>> https://cwiki.apache.org/confluence/display/ZOOKEEPER/Troubleshooting
>>>>>>
>>>>>> I can clearly see this sequence in your log for session1:
>>>>>>
>>>>>> ---
>>>>>> 2011-02-11 14:18:52,672 -  sessionid:SESSION_1 type:ping
>>>>>>
>>>>>> 2011-02-11 14:18:54,502 - Expiring session SESSION_1, timeout of 1000ms exceeded
>>>>>>
>>>>>> 2011-02-11 14:18:55,011 - Processing request:: sessionid:SESSION_1 type:ping
>>>>>> ---
>>>>>>
>>>>>> from the looks of it session 1 doesn't send a ping to the server for
>>>>>> ~2.5 seconds, as a result it's expired.
>>>>>>
>>>>>> You should also look at your session 1 client log and see what it's
>>>>>> view of the world is like. (is it gc/swapping?). Use the "stat" 4
>>>>>> letter word to see the server's request processing latency, see if
>>>>>> that's high (higer than the timeout is bad news). Again, checkout the
>>>>>> troubleshooting guide.
>>>>>>
>>>>>> Patrick
>>>>>>
>>>>>> ps please use pastebin or attachment, otw the formatting of wrecked
>>>>>> and it's harder to read the log
>>>>>>
>>>>>> On Fri, Feb 11, 2011 at 6:41 AM, Nick Patania
>>>>>> <Ni...@morganstanley.com> wrote:
>>>>>>> This is the portion that seems relevant.  For readability, I replaced
>>>>>>> the host and session for the host that I kill with HOST_0 and
>>>>>>> SESSION_0 (I expect these to timeout).  The client that should be
>>>>>>> healthy is HOST_1 and SESSION_1.
>>>>>>>
>>>>>>>
>>>>>>> 2011-02-11 14:18:51,901 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>> sessionid:SESSION_0 type:ping cxid:0xfffffffffffffffe
>>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:51,901 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_0
>>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>>> txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:52,005 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:52,005 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>>> txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:52,339 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:52,339 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>>> txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:52,672 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:52,672 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>>> txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:53,002 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>>>>>> Expiring session SESSION_0, timeout of 1000ms exceeded
>>>>>>> 2011-02-11 14:18:53,002 - INFO
>>>>>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>>>>>> termination for sessionid: SESSION_0
>>>>>>> 2011-02-11 14:18:53,010 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>> sessionid:SESSION_0 type:closeSession cxid:0x0 zxid:0x103 txntype:-11
>>>>>>> reqpath:n/a
>>>>>>> 2011-02-11 14:18:53,010 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>>>>>> Closed socket connection for client /HOST_0:34618 which had sessionid
>>>>>>> SESSION_0
>>>>>>> 2011-02-11 14:18:54,502 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>>>>>> Expiring session SESSION_1, timeout of 1000ms exceeded
>>>>>>> 2011-02-11 14:18:54,502 - INFO
>>>>>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>>>>>> termination for sessionid: SESSION_1
>>>>>>> 2011-02-11 14:18:55,011 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:55,011 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>>> txntype:unknown reqpath:n/a
>>>>>>> 2011-02-11 14:18:55,011 - INFO
>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@251]
>>>>>>> - Accepted socket connection from /HOST_1:40556
>>>>>>> 2011-02-11 14:18:55,019 - DEBUG
>>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>>> sessionid:SESSION_1 type:closeSession cxid:0x0 zxid:0x104 txntype:-11
>>>>>>> reqpath:n/a
>>>>>>> 2011-02-11 14:18:55,019 - DEBUG
>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:ZooKeeperServer@590] -
>>>>>>> Dropping request: No session with sessionid SESSION_1 exists, probably
>>>>>>> expired and removed
>>>>>>> 2011-02-11 14:18:55,019 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>>>>>> Closed socket connection for client /HOST_1:40555 which had sessionid
>>>>>>> SESSION_1
>>>>>>> 2011-02-11 14:18:55,020 - DEBUG [SyncThread:0:NIOServerCnxn@1451] -
>>>>>>> ignoring exception during output shutdown
>>>>>>> java.net.SocketException: Transport endpoint is not connected
>>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdownOutput(SocketChannelImpl.java:651)
>>>>>>>        at sun.nio.ch.SocketAdaptor.shutdownOutput(SocketAdaptor.java:368)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1447)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>>>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>>>>>> 2011-02-11 14:18:55,021 - DEBUG [SyncThread:0:NIOServerCnxn@1459] -
>>>>>>> ignoring exception during input shutdown
>>>>>>> java.net.SocketException: Transport endpoint is not connected
>>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdownInput(SocketChannelImpl.java:640)
>>>>>>>        at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:360)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1455)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>>>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>>>>>> 2011-02-11 14:18:55,022 - WARN
>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@272]
>>>>>>> - Ignoring unexpected runtime exception
>>>>>>> java.nio.channels.CancelledKeyException
>>>>>>>        at sun.nio.ch.SelectionKeyImpl.ensureValid(SelectionKeyImpl.java:55)
>>>>>>>        at sun.nio.ch.SelectionKeyImpl.readyOps(SelectionKeyImpl.java:69)
>>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:241)
>>>>>>> 2011-02-11 14:18:55,023 - DEBUG
>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@735] -
>>>>>>> Session establishment request from client /HOST_1:40556 client's
>>>>>>> lastZxid is 0x0
>>>>>>> 2011-02-11 14:18:55,023 - INFO
>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@770] - Client
>>>>>>> attempting to renew session SESSION_1 at /HOST_1:40556
>>>>>>> 2011-02-11 14:18:55,024 - INFO
>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1573] -
>>>>>>> Invalid session SESSION_1 for client /HOST_1:40556, probably expired
>>>>>>> 2011-02-11 14:18:55,025 - WARN
>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@634] -
>>>>>>> EndOfStreamException: Unable to read additional data from client
>>>>>>> sessionid SESSION_1, likely client has closed socket
>>>>>>> 2011-02-11 14:18:55,025 - INFO
>>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1435] -
>>>>>>> Closed socket connection for client /HOST_1:40556 which had sessionid
>>>>>>> SESSION_1
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Feb 10, 2011 at 8:11 PM, Benjamin Reed <br...@yahoo-inc.com> wrote:
>>>>>>>> do you see anything in the server log?
>>>>>>>>
>>>>>>>> ben
>>>>>>>>
>>>>>>>> On 02/10/2011 03:16 PM, Patania, Nick wrote:
>>>>>>>>>
>>>>>>>>> Hello,
>>>>>>>>>
>>>>>>>>> I run the following test:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> *         Two clients connect to a zookeeper server; the tickTime on the
>>>>>>>>> server is 500, and the sessionTimeout on the client is 1000.
>>>>>>>>>
>>>>>>>>> *         Kill the host running one of the clients.
>>>>>>>>>
>>>>>>>>> *         The second client receives a session timeout.
>>>>>>>>>
>>>>>>>>> Is there any reason why this might happen?
>>>>>>>>>
>>>>>>>>> Thanks
>>>>>>>>> Nick Patania
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> --------------------------------------------------------------------------
>>>>>>>>> NOTICE: Morgan Stanley is not acting as a municipal advisor and the
>>>>>>>>> opinions or views contained herein are not intended to be, and do not
>>>>>>>>> constitute, advice within the meaning of Section 975 of the Dodd-Frank Wall
>>>>>>>>> Street Reform and Consumer Protection Act. If you have received this
>>>>>>>>> communication in error, please destroy all electronic and paper copies and
>>>>>>>>> notify the sender immediately. Mistransmission is not intended to waive
>>>>>>>>> confidentiality or privilege. Morgan Stanley reserves the right, to the
>>>>>>>>> extent permitted under applicable law, to monitor electronic communications.
>>>>>>>>> This message is subject to terms available at the following link:
>>>>>>>>> http://www.morganstanley.com/disclaimers. If you cannot access these links,
>>>>>>>>> please notify us by reply message and we will send the contents to you. By
>>>>>>>>> messaging with Morgan Stanley you consent to the foregoing.
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>

RE: tickTime and sessionTimeout

Posted by "Fournier, Camille F. [Tech]" <Ca...@gs.com>.
Can you send the full stack trace?

-----Original Message-----
From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
Sent: Tuesday, February 15, 2011 12:16 PM
To: user@zookeeper.apache.org
Subject: Re: tickTime and sessionTimeout

I did some quick and dirty profiling, and during the period leading to
the expiration of SESSION_1, two of the server's threads are
suspiciously occupied as follows:

"SyncThread:0" prio=10 tid=0xaf1cec00 nid=0x6947 runnable [0xaebad000]
   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 <0xdfa07648> (a java.lang.Object)
        at java.nio.channels.spi.AbstractSelectableChannel.implCloseChannel(AbstractSelectableChannel.java:201)
        at java.nio.channels.spi.AbstractInterruptibleChannel.close(AbstractInterruptibleChannel.java:97)
        - locked <0xdfa075e8> (a java.lang.Object)
        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 <0xdfa1c0d0> (a java.util.HashSet)
        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
        - locked <0xdfa04240> (a
org.apache.zookeeper.server.NIOServerCnxn$Factory)
        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)

"NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799" daemon prio=10
tid=0xaec2b000 nid=0x6945 waiting for monitor entry [
0xaed65000]
   java.lang.Thread.State: BLOCKED (on object monitor)
        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:235)
        - waiting to lock <0xdfa04240> (a
org.apache.zookeeper.server.NIOServerCnxn$Factory)

These two threads are in this state for around 1.66 seconds.  Does
this mean something to anyone?

Note how the second thread "normally" seems to be in epoll_wait:

"NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799" daemon prio=10
tid=0xaec2b000 nid=0x6945 runnable [0xaed65000]
   java.lang.Thread.State: RUNNABLE
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)

> Is the server a virtualized host? I still think this might shed some
> light, what are you seeing for this? before/during/after the event:

It's not a virtualized host.

stat prints:

Latency min/avg/max: 0/0/62

But it's 62 (exactly) before, during and after (I run it every 50ms),
which makes me suspicious of the output.

> It is blocked a
> bit behind the expiring of SESSION_0 due to the synchronous nature of touching
> the session table

That can't possibly take hundreds of milliseconds, can it?

On Mon, Feb 14, 2011 at 1:32 AM, Patrick Hunt <ph...@apache.org> wrote:
> Is the server a virtualized host? I still think this might shed some
> light, what are you seeing for this? before/during/after the event:
>
>>>>> Use the "stat" 4
>>>>> letter word to see the server's request processing latency, see if
>>>>> that's high (higer than the timeout is bad news). Again, checkout the
>>>>> troubleshooting guide.
>
> On Fri, Feb 11, 2011 at 4:32 PM, Fournier, Camille F. [Tech]
> <Ca...@gs.com> wrote:
>> I think the gist of the problem is that while the ZK Server is expiring SESSION_0, SESSION_1 is coming in and trying to send a ping. It is blocked a bit behind the expiring of SESSION_0 due to the synchronous nature of touching the session table, and then its request for a ping will be processed behind the session expiration processing for SESSION_0. So either the expirer takes long enough dealing with SESSION_0 that it immediately wants to expire SESSION_1 on next processing, or, the processing of the session expiration takes just long enough for SESSION_1 to not get a response to its heartbeat back from the server, which then causes it to disconnect and reconnect, and in the interim the server determines the session dead due to timeout.
>>
>> Long story short, those timeouts are too short for the server to reliably execute responses in time to guarantee they won't be incorrectly activated.
>>
>> C
>>
>> -----Original Message-----
>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>> Sent: Friday, February 11, 2011 6:19 PM
>> To: user@zookeeper.apache.org
>> Subject: Re: tickTime and sessionTimeout
>>
>> 3.3.2-1031432
>>
>> On Fri, Feb 11, 2011 at 6:15 PM, Fournier, Camille F. [Tech]
>> <Ca...@gs.com> wrote:
>>> Which version of ZK?
>>>
>>> -----Original Message-----
>>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>>> Sent: Friday, February 11, 2011 6:10 PM
>>> To: user@zookeeper.apache.org
>>> Subject: Re: tickTime and sessionTimeout
>>>
>>> A single server.  I've intentionally made it trivial to demonstrate
>>> the behavior.
>>> If I increase the timeout, the issue goes away.
>>>
>>> On Fri, Feb 11, 2011 at 6:00 PM, Fournier, Camille F. [Tech]
>>> <Ca...@gs.com> wrote:
>>>> What is your ZooKeeper setup here? And do you continue to see this issue if you increase your session timeout?
>>>>
>>>> C
>>>>
>>>> -----Original Message-----
>>>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>>>> Sent: Friday, February 11, 2011 5:24 PM
>>>> To: user@zookeeper.apache.org
>>>> Subject: Re: tickTime and sessionTimeout
>>>>
>>>> Patrick, thanks for your input.
>>>>
>>>> I have rerun the test several times now while logging GC on the server
>>>> and running ping from CLIENT_1:
>>>>
>>>>  - No GC happens on the server during the period of interest (a
>>>> couple of young generation runs happen before I kill HOST_0, and they
>>>> complete in under 3ms).
>>>>  - Round trip times for ping from CLIENT_1 are consistently under
>>>> 250us throughout.
>>>>
>>>> Regarding client GC -- I can consistently reproduce this using a C
>>>> client.  Regarding the theory of swapping on CLIENT_1 -- if that were
>>>> the cause, the problem wouldn't be 100% reproducible.  I also looked
>>>> through the client log for SESSION_1 -- I see "Got ping response ...
>>>> after 1ms" repeatedly, followed by "Client session timed out, have not
>>>> heard from server in 666ms"...
>>>>
>>>> On Fri, Feb 11, 2011 at 1:46 PM, Patrick Hunt <ph...@apache.org> wrote:
>>>>> Those are pretty short timeouts, many sources of delay could be
>>>>> causing this. Network jitter/latency, GC/swap (server or client), IO
>>>>> write latency, etc... See if any of this might be your issue:
>>>>> https://cwiki.apache.org/confluence/display/ZOOKEEPER/Troubleshooting
>>>>>
>>>>> I can clearly see this sequence in your log for session1:
>>>>>
>>>>> ---
>>>>> 2011-02-11 14:18:52,672 -  sessionid:SESSION_1 type:ping
>>>>>
>>>>> 2011-02-11 14:18:54,502 - Expiring session SESSION_1, timeout of 1000ms exceeded
>>>>>
>>>>> 2011-02-11 14:18:55,011 - Processing request:: sessionid:SESSION_1 type:ping
>>>>> ---
>>>>>
>>>>> from the looks of it session 1 doesn't send a ping to the server for
>>>>> ~2.5 seconds, as a result it's expired.
>>>>>
>>>>> You should also look at your session 1 client log and see what it's
>>>>> view of the world is like. (is it gc/swapping?). Use the "stat" 4
>>>>> letter word to see the server's request processing latency, see if
>>>>> that's high (higer than the timeout is bad news). Again, checkout the
>>>>> troubleshooting guide.
>>>>>
>>>>> Patrick
>>>>>
>>>>> ps please use pastebin or attachment, otw the formatting of wrecked
>>>>> and it's harder to read the log
>>>>>
>>>>> On Fri, Feb 11, 2011 at 6:41 AM, Nick Patania
>>>>> <Ni...@morganstanley.com> wrote:
>>>>>> This is the portion that seems relevant.  For readability, I replaced
>>>>>> the host and session for the host that I kill with HOST_0 and
>>>>>> SESSION_0 (I expect these to timeout).  The client that should be
>>>>>> healthy is HOST_1 and SESSION_1.
>>>>>>
>>>>>>
>>>>>> 2011-02-11 14:18:51,901 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>> sessionid:SESSION_0 type:ping cxid:0xfffffffffffffffe
>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:51,901 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_0
>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>> txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:52,005 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:52,005 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>> txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:52,339 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:52,339 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>> txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:52,672 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:52,672 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>> txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:53,002 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>>>>> Expiring session SESSION_0, timeout of 1000ms exceeded
>>>>>> 2011-02-11 14:18:53,002 - INFO
>>>>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>>>>> termination for sessionid: SESSION_0
>>>>>> 2011-02-11 14:18:53,010 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>> sessionid:SESSION_0 type:closeSession cxid:0x0 zxid:0x103 txntype:-11
>>>>>> reqpath:n/a
>>>>>> 2011-02-11 14:18:53,010 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>>>>> Closed socket connection for client /HOST_0:34618 which had sessionid
>>>>>> SESSION_0
>>>>>> 2011-02-11 14:18:54,502 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>>>>> Expiring session SESSION_1, timeout of 1000ms exceeded
>>>>>> 2011-02-11 14:18:54,502 - INFO
>>>>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>>>>> termination for sessionid: SESSION_1
>>>>>> 2011-02-11 14:18:55,011 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:55,011 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>> txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:55,011 - INFO
>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@251]
>>>>>> - Accepted socket connection from /HOST_1:40556
>>>>>> 2011-02-11 14:18:55,019 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>> sessionid:SESSION_1 type:closeSession cxid:0x0 zxid:0x104 txntype:-11
>>>>>> reqpath:n/a
>>>>>> 2011-02-11 14:18:55,019 - DEBUG
>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:ZooKeeperServer@590] -
>>>>>> Dropping request: No session with sessionid SESSION_1 exists, probably
>>>>>> expired and removed
>>>>>> 2011-02-11 14:18:55,019 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>>>>> Closed socket connection for client /HOST_1:40555 which had sessionid
>>>>>> SESSION_1
>>>>>> 2011-02-11 14:18:55,020 - DEBUG [SyncThread:0:NIOServerCnxn@1451] -
>>>>>> ignoring exception during output shutdown
>>>>>> java.net.SocketException: Transport endpoint is not connected
>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdownOutput(SocketChannelImpl.java:651)
>>>>>>        at sun.nio.ch.SocketAdaptor.shutdownOutput(SocketAdaptor.java:368)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1447)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>>>>> 2011-02-11 14:18:55,021 - DEBUG [SyncThread:0:NIOServerCnxn@1459] -
>>>>>> ignoring exception during input shutdown
>>>>>> java.net.SocketException: Transport endpoint is not connected
>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdownInput(SocketChannelImpl.java:640)
>>>>>>        at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:360)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1455)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>>>>> 2011-02-11 14:18:55,022 - WARN
>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@272]
>>>>>> - Ignoring unexpected runtime exception
>>>>>> java.nio.channels.CancelledKeyException
>>>>>>        at sun.nio.ch.SelectionKeyImpl.ensureValid(SelectionKeyImpl.java:55)
>>>>>>        at sun.nio.ch.SelectionKeyImpl.readyOps(SelectionKeyImpl.java:69)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:241)
>>>>>> 2011-02-11 14:18:55,023 - DEBUG
>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@735] -
>>>>>> Session establishment request from client /HOST_1:40556 client's
>>>>>> lastZxid is 0x0
>>>>>> 2011-02-11 14:18:55,023 - INFO
>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@770] - Client
>>>>>> attempting to renew session SESSION_1 at /HOST_1:40556
>>>>>> 2011-02-11 14:18:55,024 - INFO
>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1573] -
>>>>>> Invalid session SESSION_1 for client /HOST_1:40556, probably expired
>>>>>> 2011-02-11 14:18:55,025 - WARN
>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@634] -
>>>>>> EndOfStreamException: Unable to read additional data from client
>>>>>> sessionid SESSION_1, likely client has closed socket
>>>>>> 2011-02-11 14:18:55,025 - INFO
>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1435] -
>>>>>> Closed socket connection for client /HOST_1:40556 which had sessionid
>>>>>> SESSION_1
>>>>>>
>>>>>>
>>>>>> On Thu, Feb 10, 2011 at 8:11 PM, Benjamin Reed <br...@yahoo-inc.com> wrote:
>>>>>>> do you see anything in the server log?
>>>>>>>
>>>>>>> ben
>>>>>>>
>>>>>>> On 02/10/2011 03:16 PM, Patania, Nick wrote:
>>>>>>>>
>>>>>>>> Hello,
>>>>>>>>
>>>>>>>> I run the following test:
>>>>>>>>
>>>>>>>>
>>>>>>>> *         Two clients connect to a zookeeper server; the tickTime on the
>>>>>>>> server is 500, and the sessionTimeout on the client is 1000.
>>>>>>>>
>>>>>>>> *         Kill the host running one of the clients.
>>>>>>>>
>>>>>>>> *         The second client receives a session timeout.
>>>>>>>>
>>>>>>>> Is there any reason why this might happen?
>>>>>>>>
>>>>>>>> Thanks
>>>>>>>> Nick Patania
>>>>>>>>
>>>>>>>>
>>>>>>>> --------------------------------------------------------------------------
>>>>>>>> NOTICE: Morgan Stanley is not acting as a municipal advisor and the
>>>>>>>> opinions or views contained herein are not intended to be, and do not
>>>>>>>> constitute, advice within the meaning of Section 975 of the Dodd-Frank Wall
>>>>>>>> Street Reform and Consumer Protection Act. If you have received this
>>>>>>>> communication in error, please destroy all electronic and paper copies and
>>>>>>>> notify the sender immediately. Mistransmission is not intended to waive
>>>>>>>> confidentiality or privilege. Morgan Stanley reserves the right, to the
>>>>>>>> extent permitted under applicable law, to monitor electronic communications.
>>>>>>>> This message is subject to terms available at the following link:
>>>>>>>> http://www.morganstanley.com/disclaimers. If you cannot access these links,
>>>>>>>> please notify us by reply message and we will send the contents to you. By
>>>>>>>> messaging with Morgan Stanley you consent to the foregoing.
>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>

Re: tickTime and sessionTimeout

Posted by Nick Patania <Ni...@morganstanley.com>.
I did some quick and dirty profiling, and during the period leading to
the expiration of SESSION_1, two of the server's threads are
suspiciously occupied as follows:

"SyncThread:0" prio=10 tid=0xaf1cec00 nid=0x6947 runnable [0xaebad000]
   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 <0xdfa07648> (a java.lang.Object)
        at java.nio.channels.spi.AbstractSelectableChannel.implCloseChannel(AbstractSelectableChannel.java:201)
        at java.nio.channels.spi.AbstractInterruptibleChannel.close(AbstractInterruptibleChannel.java:97)
        - locked <0xdfa075e8> (a java.lang.Object)
        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 <0xdfa1c0d0> (a java.util.HashSet)
        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
        - locked <0xdfa04240> (a
org.apache.zookeeper.server.NIOServerCnxn$Factory)
        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)

"NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799" daemon prio=10
tid=0xaec2b000 nid=0x6945 waiting for monitor entry [
0xaed65000]
   java.lang.Thread.State: BLOCKED (on object monitor)
        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:235)
        - waiting to lock <0xdfa04240> (a
org.apache.zookeeper.server.NIOServerCnxn$Factory)

These two threads are in this state for around 1.66 seconds.  Does
this mean something to anyone?

Note how the second thread "normally" seems to be in epoll_wait:

"NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799" daemon prio=10
tid=0xaec2b000 nid=0x6945 runnable [0xaed65000]
   java.lang.Thread.State: RUNNABLE
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)

> Is the server a virtualized host? I still think this might shed some
> light, what are you seeing for this? before/during/after the event:

It's not a virtualized host.

stat prints:

Latency min/avg/max: 0/0/62

But it's 62 (exactly) before, during and after (I run it every 50ms),
which makes me suspicious of the output.

> It is blocked a
> bit behind the expiring of SESSION_0 due to the synchronous nature of touching
> the session table

That can't possibly take hundreds of milliseconds, can it?

On Mon, Feb 14, 2011 at 1:32 AM, Patrick Hunt <ph...@apache.org> wrote:
> Is the server a virtualized host? I still think this might shed some
> light, what are you seeing for this? before/during/after the event:
>
>>>>> Use the "stat" 4
>>>>> letter word to see the server's request processing latency, see if
>>>>> that's high (higer than the timeout is bad news). Again, checkout the
>>>>> troubleshooting guide.
>
> On Fri, Feb 11, 2011 at 4:32 PM, Fournier, Camille F. [Tech]
> <Ca...@gs.com> wrote:
>> I think the gist of the problem is that while the ZK Server is expiring SESSION_0, SESSION_1 is coming in and trying to send a ping. It is blocked a bit behind the expiring of SESSION_0 due to the synchronous nature of touching the session table, and then its request for a ping will be processed behind the session expiration processing for SESSION_0. So either the expirer takes long enough dealing with SESSION_0 that it immediately wants to expire SESSION_1 on next processing, or, the processing of the session expiration takes just long enough for SESSION_1 to not get a response to its heartbeat back from the server, which then causes it to disconnect and reconnect, and in the interim the server determines the session dead due to timeout.
>>
>> Long story short, those timeouts are too short for the server to reliably execute responses in time to guarantee they won't be incorrectly activated.
>>
>> C
>>
>> -----Original Message-----
>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>> Sent: Friday, February 11, 2011 6:19 PM
>> To: user@zookeeper.apache.org
>> Subject: Re: tickTime and sessionTimeout
>>
>> 3.3.2-1031432
>>
>> On Fri, Feb 11, 2011 at 6:15 PM, Fournier, Camille F. [Tech]
>> <Ca...@gs.com> wrote:
>>> Which version of ZK?
>>>
>>> -----Original Message-----
>>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>>> Sent: Friday, February 11, 2011 6:10 PM
>>> To: user@zookeeper.apache.org
>>> Subject: Re: tickTime and sessionTimeout
>>>
>>> A single server.  I've intentionally made it trivial to demonstrate
>>> the behavior.
>>> If I increase the timeout, the issue goes away.
>>>
>>> On Fri, Feb 11, 2011 at 6:00 PM, Fournier, Camille F. [Tech]
>>> <Ca...@gs.com> wrote:
>>>> What is your ZooKeeper setup here? And do you continue to see this issue if you increase your session timeout?
>>>>
>>>> C
>>>>
>>>> -----Original Message-----
>>>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>>>> Sent: Friday, February 11, 2011 5:24 PM
>>>> To: user@zookeeper.apache.org
>>>> Subject: Re: tickTime and sessionTimeout
>>>>
>>>> Patrick, thanks for your input.
>>>>
>>>> I have rerun the test several times now while logging GC on the server
>>>> and running ping from CLIENT_1:
>>>>
>>>>  - No GC happens on the server during the period of interest (a
>>>> couple of young generation runs happen before I kill HOST_0, and they
>>>> complete in under 3ms).
>>>>  - Round trip times for ping from CLIENT_1 are consistently under
>>>> 250us throughout.
>>>>
>>>> Regarding client GC -- I can consistently reproduce this using a C
>>>> client.  Regarding the theory of swapping on CLIENT_1 -- if that were
>>>> the cause, the problem wouldn't be 100% reproducible.  I also looked
>>>> through the client log for SESSION_1 -- I see "Got ping response ...
>>>> after 1ms" repeatedly, followed by "Client session timed out, have not
>>>> heard from server in 666ms"...
>>>>
>>>> On Fri, Feb 11, 2011 at 1:46 PM, Patrick Hunt <ph...@apache.org> wrote:
>>>>> Those are pretty short timeouts, many sources of delay could be
>>>>> causing this. Network jitter/latency, GC/swap (server or client), IO
>>>>> write latency, etc... See if any of this might be your issue:
>>>>> https://cwiki.apache.org/confluence/display/ZOOKEEPER/Troubleshooting
>>>>>
>>>>> I can clearly see this sequence in your log for session1:
>>>>>
>>>>> ---
>>>>> 2011-02-11 14:18:52,672 -  sessionid:SESSION_1 type:ping
>>>>>
>>>>> 2011-02-11 14:18:54,502 - Expiring session SESSION_1, timeout of 1000ms exceeded
>>>>>
>>>>> 2011-02-11 14:18:55,011 - Processing request:: sessionid:SESSION_1 type:ping
>>>>> ---
>>>>>
>>>>> from the looks of it session 1 doesn't send a ping to the server for
>>>>> ~2.5 seconds, as a result it's expired.
>>>>>
>>>>> You should also look at your session 1 client log and see what it's
>>>>> view of the world is like. (is it gc/swapping?). Use the "stat" 4
>>>>> letter word to see the server's request processing latency, see if
>>>>> that's high (higer than the timeout is bad news). Again, checkout the
>>>>> troubleshooting guide.
>>>>>
>>>>> Patrick
>>>>>
>>>>> ps please use pastebin or attachment, otw the formatting of wrecked
>>>>> and it's harder to read the log
>>>>>
>>>>> On Fri, Feb 11, 2011 at 6:41 AM, Nick Patania
>>>>> <Ni...@morganstanley.com> wrote:
>>>>>> This is the portion that seems relevant.  For readability, I replaced
>>>>>> the host and session for the host that I kill with HOST_0 and
>>>>>> SESSION_0 (I expect these to timeout).  The client that should be
>>>>>> healthy is HOST_1 and SESSION_1.
>>>>>>
>>>>>>
>>>>>> 2011-02-11 14:18:51,901 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>> sessionid:SESSION_0 type:ping cxid:0xfffffffffffffffe
>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:51,901 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_0
>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>> txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:52,005 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:52,005 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>> txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:52,339 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:52,339 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>> txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:52,672 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:52,672 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>> txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:53,002 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>>>>> Expiring session SESSION_0, timeout of 1000ms exceeded
>>>>>> 2011-02-11 14:18:53,002 - INFO
>>>>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>>>>> termination for sessionid: SESSION_0
>>>>>> 2011-02-11 14:18:53,010 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>> sessionid:SESSION_0 type:closeSession cxid:0x0 zxid:0x103 txntype:-11
>>>>>> reqpath:n/a
>>>>>> 2011-02-11 14:18:53,010 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>>>>> Closed socket connection for client /HOST_0:34618 which had sessionid
>>>>>> SESSION_0
>>>>>> 2011-02-11 14:18:54,502 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>>>>> Expiring session SESSION_1, timeout of 1000ms exceeded
>>>>>> 2011-02-11 14:18:54,502 - INFO
>>>>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>>>>> termination for sessionid: SESSION_1
>>>>>> 2011-02-11 14:18:55,011 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:55,011 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>>> txntype:unknown reqpath:n/a
>>>>>> 2011-02-11 14:18:55,011 - INFO
>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@251]
>>>>>> - Accepted socket connection from /HOST_1:40556
>>>>>> 2011-02-11 14:18:55,019 - DEBUG
>>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>>> sessionid:SESSION_1 type:closeSession cxid:0x0 zxid:0x104 txntype:-11
>>>>>> reqpath:n/a
>>>>>> 2011-02-11 14:18:55,019 - DEBUG
>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:ZooKeeperServer@590] -
>>>>>> Dropping request: No session with sessionid SESSION_1 exists, probably
>>>>>> expired and removed
>>>>>> 2011-02-11 14:18:55,019 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>>>>> Closed socket connection for client /HOST_1:40555 which had sessionid
>>>>>> SESSION_1
>>>>>> 2011-02-11 14:18:55,020 - DEBUG [SyncThread:0:NIOServerCnxn@1451] -
>>>>>> ignoring exception during output shutdown
>>>>>> java.net.SocketException: Transport endpoint is not connected
>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdownOutput(SocketChannelImpl.java:651)
>>>>>>        at sun.nio.ch.SocketAdaptor.shutdownOutput(SocketAdaptor.java:368)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1447)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>>>>> 2011-02-11 14:18:55,021 - DEBUG [SyncThread:0:NIOServerCnxn@1459] -
>>>>>> ignoring exception during input shutdown
>>>>>> java.net.SocketException: Transport endpoint is not connected
>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>>>>        at sun.nio.ch.SocketChannelImpl.shutdownInput(SocketChannelImpl.java:640)
>>>>>>        at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:360)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1455)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>>>>> 2011-02-11 14:18:55,022 - WARN
>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@272]
>>>>>> - Ignoring unexpected runtime exception
>>>>>> java.nio.channels.CancelledKeyException
>>>>>>        at sun.nio.ch.SelectionKeyImpl.ensureValid(SelectionKeyImpl.java:55)
>>>>>>        at sun.nio.ch.SelectionKeyImpl.readyOps(SelectionKeyImpl.java:69)
>>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:241)
>>>>>> 2011-02-11 14:18:55,023 - DEBUG
>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@735] -
>>>>>> Session establishment request from client /HOST_1:40556 client's
>>>>>> lastZxid is 0x0
>>>>>> 2011-02-11 14:18:55,023 - INFO
>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@770] - Client
>>>>>> attempting to renew session SESSION_1 at /HOST_1:40556
>>>>>> 2011-02-11 14:18:55,024 - INFO
>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1573] -
>>>>>> Invalid session SESSION_1 for client /HOST_1:40556, probably expired
>>>>>> 2011-02-11 14:18:55,025 - WARN
>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@634] -
>>>>>> EndOfStreamException: Unable to read additional data from client
>>>>>> sessionid SESSION_1, likely client has closed socket
>>>>>> 2011-02-11 14:18:55,025 - INFO
>>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1435] -
>>>>>> Closed socket connection for client /HOST_1:40556 which had sessionid
>>>>>> SESSION_1
>>>>>>
>>>>>>
>>>>>> On Thu, Feb 10, 2011 at 8:11 PM, Benjamin Reed <br...@yahoo-inc.com> wrote:
>>>>>>> do you see anything in the server log?
>>>>>>>
>>>>>>> ben
>>>>>>>
>>>>>>> On 02/10/2011 03:16 PM, Patania, Nick wrote:
>>>>>>>>
>>>>>>>> Hello,
>>>>>>>>
>>>>>>>> I run the following test:
>>>>>>>>
>>>>>>>>
>>>>>>>> *         Two clients connect to a zookeeper server; the tickTime on the
>>>>>>>> server is 500, and the sessionTimeout on the client is 1000.
>>>>>>>>
>>>>>>>> *         Kill the host running one of the clients.
>>>>>>>>
>>>>>>>> *         The second client receives a session timeout.
>>>>>>>>
>>>>>>>> Is there any reason why this might happen?
>>>>>>>>
>>>>>>>> Thanks
>>>>>>>> Nick Patania
>>>>>>>>
>>>>>>>>
>>>>>>>> --------------------------------------------------------------------------
>>>>>>>> NOTICE: Morgan Stanley is not acting as a municipal advisor and the
>>>>>>>> opinions or views contained herein are not intended to be, and do not
>>>>>>>> constitute, advice within the meaning of Section 975 of the Dodd-Frank Wall
>>>>>>>> Street Reform and Consumer Protection Act. If you have received this
>>>>>>>> communication in error, please destroy all electronic and paper copies and
>>>>>>>> notify the sender immediately. Mistransmission is not intended to waive
>>>>>>>> confidentiality or privilege. Morgan Stanley reserves the right, to the
>>>>>>>> extent permitted under applicable law, to monitor electronic communications.
>>>>>>>> This message is subject to terms available at the following link:
>>>>>>>> http://www.morganstanley.com/disclaimers. If you cannot access these links,
>>>>>>>> please notify us by reply message and we will send the contents to you. By
>>>>>>>> messaging with Morgan Stanley you consent to the foregoing.
>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>

Re: tickTime and sessionTimeout

Posted by Patrick Hunt <ph...@apache.org>.
Is the server a virtualized host? I still think this might shed some
light, what are you seeing for this? before/during/after the event:

>>>> Use the "stat" 4
>>>> letter word to see the server's request processing latency, see if
>>>> that's high (higer than the timeout is bad news). Again, checkout the
>>>> troubleshooting guide.

On Fri, Feb 11, 2011 at 4:32 PM, Fournier, Camille F. [Tech]
<Ca...@gs.com> wrote:
> I think the gist of the problem is that while the ZK Server is expiring SESSION_0, SESSION_1 is coming in and trying to send a ping. It is blocked a bit behind the expiring of SESSION_0 due to the synchronous nature of touching the session table, and then its request for a ping will be processed behind the session expiration processing for SESSION_0. So either the expirer takes long enough dealing with SESSION_0 that it immediately wants to expire SESSION_1 on next processing, or, the processing of the session expiration takes just long enough for SESSION_1 to not get a response to its heartbeat back from the server, which then causes it to disconnect and reconnect, and in the interim the server determines the session dead due to timeout.
>
> Long story short, those timeouts are too short for the server to reliably execute responses in time to guarantee they won't be incorrectly activated.
>
> C
>
> -----Original Message-----
> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
> Sent: Friday, February 11, 2011 6:19 PM
> To: user@zookeeper.apache.org
> Subject: Re: tickTime and sessionTimeout
>
> 3.3.2-1031432
>
> On Fri, Feb 11, 2011 at 6:15 PM, Fournier, Camille F. [Tech]
> <Ca...@gs.com> wrote:
>> Which version of ZK?
>>
>> -----Original Message-----
>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>> Sent: Friday, February 11, 2011 6:10 PM
>> To: user@zookeeper.apache.org
>> Subject: Re: tickTime and sessionTimeout
>>
>> A single server.  I've intentionally made it trivial to demonstrate
>> the behavior.
>> If I increase the timeout, the issue goes away.
>>
>> On Fri, Feb 11, 2011 at 6:00 PM, Fournier, Camille F. [Tech]
>> <Ca...@gs.com> wrote:
>>> What is your ZooKeeper setup here? And do you continue to see this issue if you increase your session timeout?
>>>
>>> C
>>>
>>> -----Original Message-----
>>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>>> Sent: Friday, February 11, 2011 5:24 PM
>>> To: user@zookeeper.apache.org
>>> Subject: Re: tickTime and sessionTimeout
>>>
>>> Patrick, thanks for your input.
>>>
>>> I have rerun the test several times now while logging GC on the server
>>> and running ping from CLIENT_1:
>>>
>>>  - No GC happens on the server during the period of interest (a
>>> couple of young generation runs happen before I kill HOST_0, and they
>>> complete in under 3ms).
>>>  - Round trip times for ping from CLIENT_1 are consistently under
>>> 250us throughout.
>>>
>>> Regarding client GC -- I can consistently reproduce this using a C
>>> client.  Regarding the theory of swapping on CLIENT_1 -- if that were
>>> the cause, the problem wouldn't be 100% reproducible.  I also looked
>>> through the client log for SESSION_1 -- I see "Got ping response ...
>>> after 1ms" repeatedly, followed by "Client session timed out, have not
>>> heard from server in 666ms"...
>>>
>>> On Fri, Feb 11, 2011 at 1:46 PM, Patrick Hunt <ph...@apache.org> wrote:
>>>> Those are pretty short timeouts, many sources of delay could be
>>>> causing this. Network jitter/latency, GC/swap (server or client), IO
>>>> write latency, etc... See if any of this might be your issue:
>>>> https://cwiki.apache.org/confluence/display/ZOOKEEPER/Troubleshooting
>>>>
>>>> I can clearly see this sequence in your log for session1:
>>>>
>>>> ---
>>>> 2011-02-11 14:18:52,672 -  sessionid:SESSION_1 type:ping
>>>>
>>>> 2011-02-11 14:18:54,502 - Expiring session SESSION_1, timeout of 1000ms exceeded
>>>>
>>>> 2011-02-11 14:18:55,011 - Processing request:: sessionid:SESSION_1 type:ping
>>>> ---
>>>>
>>>> from the looks of it session 1 doesn't send a ping to the server for
>>>> ~2.5 seconds, as a result it's expired.
>>>>
>>>> You should also look at your session 1 client log and see what it's
>>>> view of the world is like. (is it gc/swapping?). Use the "stat" 4
>>>> letter word to see the server's request processing latency, see if
>>>> that's high (higer than the timeout is bad news). Again, checkout the
>>>> troubleshooting guide.
>>>>
>>>> Patrick
>>>>
>>>> ps please use pastebin or attachment, otw the formatting of wrecked
>>>> and it's harder to read the log
>>>>
>>>> On Fri, Feb 11, 2011 at 6:41 AM, Nick Patania
>>>> <Ni...@morganstanley.com> wrote:
>>>>> This is the portion that seems relevant.  For readability, I replaced
>>>>> the host and session for the host that I kill with HOST_0 and
>>>>> SESSION_0 (I expect these to timeout).  The client that should be
>>>>> healthy is HOST_1 and SESSION_1.
>>>>>
>>>>>
>>>>> 2011-02-11 14:18:51,901 - DEBUG
>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>> sessionid:SESSION_0 type:ping cxid:0xfffffffffffffffe
>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>> 2011-02-11 14:18:51,901 - DEBUG
>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_0
>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>> txntype:unknown reqpath:n/a
>>>>> 2011-02-11 14:18:52,005 - DEBUG
>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>> 2011-02-11 14:18:52,005 - DEBUG
>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>> txntype:unknown reqpath:n/a
>>>>> 2011-02-11 14:18:52,339 - DEBUG
>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>> 2011-02-11 14:18:52,339 - DEBUG
>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>> txntype:unknown reqpath:n/a
>>>>> 2011-02-11 14:18:52,672 - DEBUG
>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>> 2011-02-11 14:18:52,672 - DEBUG
>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>> txntype:unknown reqpath:n/a
>>>>> 2011-02-11 14:18:53,002 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>>>> Expiring session SESSION_0, timeout of 1000ms exceeded
>>>>> 2011-02-11 14:18:53,002 - INFO
>>>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>>>> termination for sessionid: SESSION_0
>>>>> 2011-02-11 14:18:53,010 - DEBUG
>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>> sessionid:SESSION_0 type:closeSession cxid:0x0 zxid:0x103 txntype:-11
>>>>> reqpath:n/a
>>>>> 2011-02-11 14:18:53,010 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>>>> Closed socket connection for client /HOST_0:34618 which had sessionid
>>>>> SESSION_0
>>>>> 2011-02-11 14:18:54,502 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>>>> Expiring session SESSION_1, timeout of 1000ms exceeded
>>>>> 2011-02-11 14:18:54,502 - INFO
>>>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>>>> termination for sessionid: SESSION_1
>>>>> 2011-02-11 14:18:55,011 - DEBUG
>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>>> 2011-02-11 14:18:55,011 - DEBUG
>>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>>> txntype:unknown reqpath:n/a
>>>>> 2011-02-11 14:18:55,011 - INFO
>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@251]
>>>>> - Accepted socket connection from /HOST_1:40556
>>>>> 2011-02-11 14:18:55,019 - DEBUG
>>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>>> sessionid:SESSION_1 type:closeSession cxid:0x0 zxid:0x104 txntype:-11
>>>>> reqpath:n/a
>>>>> 2011-02-11 14:18:55,019 - DEBUG
>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:ZooKeeperServer@590] -
>>>>> Dropping request: No session with sessionid SESSION_1 exists, probably
>>>>> expired and removed
>>>>> 2011-02-11 14:18:55,019 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>>>> Closed socket connection for client /HOST_1:40555 which had sessionid
>>>>> SESSION_1
>>>>> 2011-02-11 14:18:55,020 - DEBUG [SyncThread:0:NIOServerCnxn@1451] -
>>>>> ignoring exception during output shutdown
>>>>> java.net.SocketException: Transport endpoint is not connected
>>>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>>>        at sun.nio.ch.SocketChannelImpl.shutdownOutput(SocketChannelImpl.java:651)
>>>>>        at sun.nio.ch.SocketAdaptor.shutdownOutput(SocketAdaptor.java:368)
>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1447)
>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>>>> 2011-02-11 14:18:55,021 - DEBUG [SyncThread:0:NIOServerCnxn@1459] -
>>>>> ignoring exception during input shutdown
>>>>> java.net.SocketException: Transport endpoint is not connected
>>>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>>>        at sun.nio.ch.SocketChannelImpl.shutdownInput(SocketChannelImpl.java:640)
>>>>>        at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:360)
>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1455)
>>>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>>>> 2011-02-11 14:18:55,022 - WARN
>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@272]
>>>>> - Ignoring unexpected runtime exception
>>>>> java.nio.channels.CancelledKeyException
>>>>>        at sun.nio.ch.SelectionKeyImpl.ensureValid(SelectionKeyImpl.java:55)
>>>>>        at sun.nio.ch.SelectionKeyImpl.readyOps(SelectionKeyImpl.java:69)
>>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:241)
>>>>> 2011-02-11 14:18:55,023 - DEBUG
>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@735] -
>>>>> Session establishment request from client /HOST_1:40556 client's
>>>>> lastZxid is 0x0
>>>>> 2011-02-11 14:18:55,023 - INFO
>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@770] - Client
>>>>> attempting to renew session SESSION_1 at /HOST_1:40556
>>>>> 2011-02-11 14:18:55,024 - INFO
>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1573] -
>>>>> Invalid session SESSION_1 for client /HOST_1:40556, probably expired
>>>>> 2011-02-11 14:18:55,025 - WARN
>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@634] -
>>>>> EndOfStreamException: Unable to read additional data from client
>>>>> sessionid SESSION_1, likely client has closed socket
>>>>> 2011-02-11 14:18:55,025 - INFO
>>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1435] -
>>>>> Closed socket connection for client /HOST_1:40556 which had sessionid
>>>>> SESSION_1
>>>>>
>>>>>
>>>>> On Thu, Feb 10, 2011 at 8:11 PM, Benjamin Reed <br...@yahoo-inc.com> wrote:
>>>>>> do you see anything in the server log?
>>>>>>
>>>>>> ben
>>>>>>
>>>>>> On 02/10/2011 03:16 PM, Patania, Nick wrote:
>>>>>>>
>>>>>>> Hello,
>>>>>>>
>>>>>>> I run the following test:
>>>>>>>
>>>>>>>
>>>>>>> *         Two clients connect to a zookeeper server; the tickTime on the
>>>>>>> server is 500, and the sessionTimeout on the client is 1000.
>>>>>>>
>>>>>>> *         Kill the host running one of the clients.
>>>>>>>
>>>>>>> *         The second client receives a session timeout.
>>>>>>>
>>>>>>> Is there any reason why this might happen?
>>>>>>>
>>>>>>> Thanks
>>>>>>> Nick Patania
>>>>>>>
>>>>>>>
>>>>>>> --------------------------------------------------------------------------
>>>>>>> NOTICE: Morgan Stanley is not acting as a municipal advisor and the
>>>>>>> opinions or views contained herein are not intended to be, and do not
>>>>>>> constitute, advice within the meaning of Section 975 of the Dodd-Frank Wall
>>>>>>> Street Reform and Consumer Protection Act. If you have received this
>>>>>>> communication in error, please destroy all electronic and paper copies and
>>>>>>> notify the sender immediately. Mistransmission is not intended to waive
>>>>>>> confidentiality or privilege. Morgan Stanley reserves the right, to the
>>>>>>> extent permitted under applicable law, to monitor electronic communications.
>>>>>>> This message is subject to terms available at the following link:
>>>>>>> http://www.morganstanley.com/disclaimers. If you cannot access these links,
>>>>>>> please notify us by reply message and we will send the contents to you. By
>>>>>>> messaging with Morgan Stanley you consent to the foregoing.
>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>

RE: tickTime and sessionTimeout

Posted by "Fournier, Camille F. [Tech]" <Ca...@gs.com>.
I think the gist of the problem is that while the ZK Server is expiring SESSION_0, SESSION_1 is coming in and trying to send a ping. It is blocked a bit behind the expiring of SESSION_0 due to the synchronous nature of touching the session table, and then its request for a ping will be processed behind the session expiration processing for SESSION_0. So either the expirer takes long enough dealing with SESSION_0 that it immediately wants to expire SESSION_1 on next processing, or, the processing of the session expiration takes just long enough for SESSION_1 to not get a response to its heartbeat back from the server, which then causes it to disconnect and reconnect, and in the interim the server determines the session dead due to timeout.

Long story short, those timeouts are too short for the server to reliably execute responses in time to guarantee they won't be incorrectly activated.

C

-----Original Message-----
From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com] 
Sent: Friday, February 11, 2011 6:19 PM
To: user@zookeeper.apache.org
Subject: Re: tickTime and sessionTimeout

3.3.2-1031432

On Fri, Feb 11, 2011 at 6:15 PM, Fournier, Camille F. [Tech]
<Ca...@gs.com> wrote:
> Which version of ZK?
>
> -----Original Message-----
> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
> Sent: Friday, February 11, 2011 6:10 PM
> To: user@zookeeper.apache.org
> Subject: Re: tickTime and sessionTimeout
>
> A single server.  I've intentionally made it trivial to demonstrate
> the behavior.
> If I increase the timeout, the issue goes away.
>
> On Fri, Feb 11, 2011 at 6:00 PM, Fournier, Camille F. [Tech]
> <Ca...@gs.com> wrote:
>> What is your ZooKeeper setup here? And do you continue to see this issue if you increase your session timeout?
>>
>> C
>>
>> -----Original Message-----
>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>> Sent: Friday, February 11, 2011 5:24 PM
>> To: user@zookeeper.apache.org
>> Subject: Re: tickTime and sessionTimeout
>>
>> Patrick, thanks for your input.
>>
>> I have rerun the test several times now while logging GC on the server
>> and running ping from CLIENT_1:
>>
>>  - No GC happens on the server during the period of interest (a
>> couple of young generation runs happen before I kill HOST_0, and they
>> complete in under 3ms).
>>  - Round trip times for ping from CLIENT_1 are consistently under
>> 250us throughout.
>>
>> Regarding client GC -- I can consistently reproduce this using a C
>> client.  Regarding the theory of swapping on CLIENT_1 -- if that were
>> the cause, the problem wouldn't be 100% reproducible.  I also looked
>> through the client log for SESSION_1 -- I see "Got ping response ...
>> after 1ms" repeatedly, followed by "Client session timed out, have not
>> heard from server in 666ms"...
>>
>> On Fri, Feb 11, 2011 at 1:46 PM, Patrick Hunt <ph...@apache.org> wrote:
>>> Those are pretty short timeouts, many sources of delay could be
>>> causing this. Network jitter/latency, GC/swap (server or client), IO
>>> write latency, etc... See if any of this might be your issue:
>>> https://cwiki.apache.org/confluence/display/ZOOKEEPER/Troubleshooting
>>>
>>> I can clearly see this sequence in your log for session1:
>>>
>>> ---
>>> 2011-02-11 14:18:52,672 -  sessionid:SESSION_1 type:ping
>>>
>>> 2011-02-11 14:18:54,502 - Expiring session SESSION_1, timeout of 1000ms exceeded
>>>
>>> 2011-02-11 14:18:55,011 - Processing request:: sessionid:SESSION_1 type:ping
>>> ---
>>>
>>> from the looks of it session 1 doesn't send a ping to the server for
>>> ~2.5 seconds, as a result it's expired.
>>>
>>> You should also look at your session 1 client log and see what it's
>>> view of the world is like. (is it gc/swapping?). Use the "stat" 4
>>> letter word to see the server's request processing latency, see if
>>> that's high (higer than the timeout is bad news). Again, checkout the
>>> troubleshooting guide.
>>>
>>> Patrick
>>>
>>> ps please use pastebin or attachment, otw the formatting of wrecked
>>> and it's harder to read the log
>>>
>>> On Fri, Feb 11, 2011 at 6:41 AM, Nick Patania
>>> <Ni...@morganstanley.com> wrote:
>>>> This is the portion that seems relevant.  For readability, I replaced
>>>> the host and session for the host that I kill with HOST_0 and
>>>> SESSION_0 (I expect these to timeout).  The client that should be
>>>> healthy is HOST_1 and SESSION_1.
>>>>
>>>>
>>>> 2011-02-11 14:18:51,901 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>> sessionid:SESSION_0 type:ping cxid:0xfffffffffffffffe
>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:51,901 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_0
>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>> txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:52,005 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:52,005 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>> txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:52,339 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:52,339 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>> txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:52,672 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:52,672 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>> txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:53,002 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>>> Expiring session SESSION_0, timeout of 1000ms exceeded
>>>> 2011-02-11 14:18:53,002 - INFO
>>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>>> termination for sessionid: SESSION_0
>>>> 2011-02-11 14:18:53,010 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>> sessionid:SESSION_0 type:closeSession cxid:0x0 zxid:0x103 txntype:-11
>>>> reqpath:n/a
>>>> 2011-02-11 14:18:53,010 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>>> Closed socket connection for client /HOST_0:34618 which had sessionid
>>>> SESSION_0
>>>> 2011-02-11 14:18:54,502 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>>> Expiring session SESSION_1, timeout of 1000ms exceeded
>>>> 2011-02-11 14:18:54,502 - INFO
>>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>>> termination for sessionid: SESSION_1
>>>> 2011-02-11 14:18:55,011 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:55,011 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>> txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:55,011 - INFO
>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@251]
>>>> - Accepted socket connection from /HOST_1:40556
>>>> 2011-02-11 14:18:55,019 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>> sessionid:SESSION_1 type:closeSession cxid:0x0 zxid:0x104 txntype:-11
>>>> reqpath:n/a
>>>> 2011-02-11 14:18:55,019 - DEBUG
>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:ZooKeeperServer@590] -
>>>> Dropping request: No session with sessionid SESSION_1 exists, probably
>>>> expired and removed
>>>> 2011-02-11 14:18:55,019 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>>> Closed socket connection for client /HOST_1:40555 which had sessionid
>>>> SESSION_1
>>>> 2011-02-11 14:18:55,020 - DEBUG [SyncThread:0:NIOServerCnxn@1451] -
>>>> ignoring exception during output shutdown
>>>> java.net.SocketException: Transport endpoint is not connected
>>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>>        at sun.nio.ch.SocketChannelImpl.shutdownOutput(SocketChannelImpl.java:651)
>>>>        at sun.nio.ch.SocketAdaptor.shutdownOutput(SocketAdaptor.java:368)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1447)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>>> 2011-02-11 14:18:55,021 - DEBUG [SyncThread:0:NIOServerCnxn@1459] -
>>>> ignoring exception during input shutdown
>>>> java.net.SocketException: Transport endpoint is not connected
>>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>>        at sun.nio.ch.SocketChannelImpl.shutdownInput(SocketChannelImpl.java:640)
>>>>        at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:360)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1455)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>>> 2011-02-11 14:18:55,022 - WARN
>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@272]
>>>> - Ignoring unexpected runtime exception
>>>> java.nio.channels.CancelledKeyException
>>>>        at sun.nio.ch.SelectionKeyImpl.ensureValid(SelectionKeyImpl.java:55)
>>>>        at sun.nio.ch.SelectionKeyImpl.readyOps(SelectionKeyImpl.java:69)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:241)
>>>> 2011-02-11 14:18:55,023 - DEBUG
>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@735] -
>>>> Session establishment request from client /HOST_1:40556 client's
>>>> lastZxid is 0x0
>>>> 2011-02-11 14:18:55,023 - INFO
>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@770] - Client
>>>> attempting to renew session SESSION_1 at /HOST_1:40556
>>>> 2011-02-11 14:18:55,024 - INFO
>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1573] -
>>>> Invalid session SESSION_1 for client /HOST_1:40556, probably expired
>>>> 2011-02-11 14:18:55,025 - WARN
>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@634] -
>>>> EndOfStreamException: Unable to read additional data from client
>>>> sessionid SESSION_1, likely client has closed socket
>>>> 2011-02-11 14:18:55,025 - INFO
>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1435] -
>>>> Closed socket connection for client /HOST_1:40556 which had sessionid
>>>> SESSION_1
>>>>
>>>>
>>>> On Thu, Feb 10, 2011 at 8:11 PM, Benjamin Reed <br...@yahoo-inc.com> wrote:
>>>>> do you see anything in the server log?
>>>>>
>>>>> ben
>>>>>
>>>>> On 02/10/2011 03:16 PM, Patania, Nick wrote:
>>>>>>
>>>>>> Hello,
>>>>>>
>>>>>> I run the following test:
>>>>>>
>>>>>>
>>>>>> *         Two clients connect to a zookeeper server; the tickTime on the
>>>>>> server is 500, and the sessionTimeout on the client is 1000.
>>>>>>
>>>>>> *         Kill the host running one of the clients.
>>>>>>
>>>>>> *         The second client receives a session timeout.
>>>>>>
>>>>>> Is there any reason why this might happen?
>>>>>>
>>>>>> Thanks
>>>>>> Nick Patania
>>>>>>
>>>>>>
>>>>>> --------------------------------------------------------------------------
>>>>>> NOTICE: Morgan Stanley is not acting as a municipal advisor and the
>>>>>> opinions or views contained herein are not intended to be, and do not
>>>>>> constitute, advice within the meaning of Section 975 of the Dodd-Frank Wall
>>>>>> Street Reform and Consumer Protection Act. If you have received this
>>>>>> communication in error, please destroy all electronic and paper copies and
>>>>>> notify the sender immediately. Mistransmission is not intended to waive
>>>>>> confidentiality or privilege. Morgan Stanley reserves the right, to the
>>>>>> extent permitted under applicable law, to monitor electronic communications.
>>>>>> This message is subject to terms available at the following link:
>>>>>> http://www.morganstanley.com/disclaimers. If you cannot access these links,
>>>>>> please notify us by reply message and we will send the contents to you. By
>>>>>> messaging with Morgan Stanley you consent to the foregoing.
>>>>>
>>>>>
>>>>
>>>
>>
>

Re: tickTime and sessionTimeout

Posted by Nick Patania <Ni...@morganstanley.com>.
3.3.2-1031432

On Fri, Feb 11, 2011 at 6:15 PM, Fournier, Camille F. [Tech]
<Ca...@gs.com> wrote:
> Which version of ZK?
>
> -----Original Message-----
> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
> Sent: Friday, February 11, 2011 6:10 PM
> To: user@zookeeper.apache.org
> Subject: Re: tickTime and sessionTimeout
>
> A single server.  I've intentionally made it trivial to demonstrate
> the behavior.
> If I increase the timeout, the issue goes away.
>
> On Fri, Feb 11, 2011 at 6:00 PM, Fournier, Camille F. [Tech]
> <Ca...@gs.com> wrote:
>> What is your ZooKeeper setup here? And do you continue to see this issue if you increase your session timeout?
>>
>> C
>>
>> -----Original Message-----
>> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
>> Sent: Friday, February 11, 2011 5:24 PM
>> To: user@zookeeper.apache.org
>> Subject: Re: tickTime and sessionTimeout
>>
>> Patrick, thanks for your input.
>>
>> I have rerun the test several times now while logging GC on the server
>> and running ping from CLIENT_1:
>>
>>  - No GC happens on the server during the period of interest (a
>> couple of young generation runs happen before I kill HOST_0, and they
>> complete in under 3ms).
>>  - Round trip times for ping from CLIENT_1 are consistently under
>> 250us throughout.
>>
>> Regarding client GC -- I can consistently reproduce this using a C
>> client.  Regarding the theory of swapping on CLIENT_1 -- if that were
>> the cause, the problem wouldn't be 100% reproducible.  I also looked
>> through the client log for SESSION_1 -- I see "Got ping response ...
>> after 1ms" repeatedly, followed by "Client session timed out, have not
>> heard from server in 666ms"...
>>
>> On Fri, Feb 11, 2011 at 1:46 PM, Patrick Hunt <ph...@apache.org> wrote:
>>> Those are pretty short timeouts, many sources of delay could be
>>> causing this. Network jitter/latency, GC/swap (server or client), IO
>>> write latency, etc... See if any of this might be your issue:
>>> https://cwiki.apache.org/confluence/display/ZOOKEEPER/Troubleshooting
>>>
>>> I can clearly see this sequence in your log for session1:
>>>
>>> ---
>>> 2011-02-11 14:18:52,672 -  sessionid:SESSION_1 type:ping
>>>
>>> 2011-02-11 14:18:54,502 - Expiring session SESSION_1, timeout of 1000ms exceeded
>>>
>>> 2011-02-11 14:18:55,011 - Processing request:: sessionid:SESSION_1 type:ping
>>> ---
>>>
>>> from the looks of it session 1 doesn't send a ping to the server for
>>> ~2.5 seconds, as a result it's expired.
>>>
>>> You should also look at your session 1 client log and see what it's
>>> view of the world is like. (is it gc/swapping?). Use the "stat" 4
>>> letter word to see the server's request processing latency, see if
>>> that's high (higer than the timeout is bad news). Again, checkout the
>>> troubleshooting guide.
>>>
>>> Patrick
>>>
>>> ps please use pastebin or attachment, otw the formatting of wrecked
>>> and it's harder to read the log
>>>
>>> On Fri, Feb 11, 2011 at 6:41 AM, Nick Patania
>>> <Ni...@morganstanley.com> wrote:
>>>> This is the portion that seems relevant.  For readability, I replaced
>>>> the host and session for the host that I kill with HOST_0 and
>>>> SESSION_0 (I expect these to timeout).  The client that should be
>>>> healthy is HOST_1 and SESSION_1.
>>>>
>>>>
>>>> 2011-02-11 14:18:51,901 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>> sessionid:SESSION_0 type:ping cxid:0xfffffffffffffffe
>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:51,901 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_0
>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>> txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:52,005 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:52,005 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>> txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:52,339 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:52,339 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>> txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:52,672 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:52,672 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>> txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:53,002 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>>> Expiring session SESSION_0, timeout of 1000ms exceeded
>>>> 2011-02-11 14:18:53,002 - INFO
>>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>>> termination for sessionid: SESSION_0
>>>> 2011-02-11 14:18:53,010 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>> sessionid:SESSION_0 type:closeSession cxid:0x0 zxid:0x103 txntype:-11
>>>> reqpath:n/a
>>>> 2011-02-11 14:18:53,010 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>>> Closed socket connection for client /HOST_0:34618 which had sessionid
>>>> SESSION_0
>>>> 2011-02-11 14:18:54,502 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>>> Expiring session SESSION_1, timeout of 1000ms exceeded
>>>> 2011-02-11 14:18:54,502 - INFO
>>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>>> termination for sessionid: SESSION_1
>>>> 2011-02-11 14:18:55,011 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:55,011 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>>> txntype:unknown reqpath:n/a
>>>> 2011-02-11 14:18:55,011 - INFO
>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@251]
>>>> - Accepted socket connection from /HOST_1:40556
>>>> 2011-02-11 14:18:55,019 - DEBUG
>>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>>> sessionid:SESSION_1 type:closeSession cxid:0x0 zxid:0x104 txntype:-11
>>>> reqpath:n/a
>>>> 2011-02-11 14:18:55,019 - DEBUG
>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:ZooKeeperServer@590] -
>>>> Dropping request: No session with sessionid SESSION_1 exists, probably
>>>> expired and removed
>>>> 2011-02-11 14:18:55,019 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>>> Closed socket connection for client /HOST_1:40555 which had sessionid
>>>> SESSION_1
>>>> 2011-02-11 14:18:55,020 - DEBUG [SyncThread:0:NIOServerCnxn@1451] -
>>>> ignoring exception during output shutdown
>>>> java.net.SocketException: Transport endpoint is not connected
>>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>>        at sun.nio.ch.SocketChannelImpl.shutdownOutput(SocketChannelImpl.java:651)
>>>>        at sun.nio.ch.SocketAdaptor.shutdownOutput(SocketAdaptor.java:368)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1447)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>>> 2011-02-11 14:18:55,021 - DEBUG [SyncThread:0:NIOServerCnxn@1459] -
>>>> ignoring exception during input shutdown
>>>> java.net.SocketException: Transport endpoint is not connected
>>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>>        at sun.nio.ch.SocketChannelImpl.shutdownInput(SocketChannelImpl.java:640)
>>>>        at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:360)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1455)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>>> 2011-02-11 14:18:55,022 - WARN
>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@272]
>>>> - Ignoring unexpected runtime exception
>>>> java.nio.channels.CancelledKeyException
>>>>        at sun.nio.ch.SelectionKeyImpl.ensureValid(SelectionKeyImpl.java:55)
>>>>        at sun.nio.ch.SelectionKeyImpl.readyOps(SelectionKeyImpl.java:69)
>>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:241)
>>>> 2011-02-11 14:18:55,023 - DEBUG
>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@735] -
>>>> Session establishment request from client /HOST_1:40556 client's
>>>> lastZxid is 0x0
>>>> 2011-02-11 14:18:55,023 - INFO
>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@770] - Client
>>>> attempting to renew session SESSION_1 at /HOST_1:40556
>>>> 2011-02-11 14:18:55,024 - INFO
>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1573] -
>>>> Invalid session SESSION_1 for client /HOST_1:40556, probably expired
>>>> 2011-02-11 14:18:55,025 - WARN
>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@634] -
>>>> EndOfStreamException: Unable to read additional data from client
>>>> sessionid SESSION_1, likely client has closed socket
>>>> 2011-02-11 14:18:55,025 - INFO
>>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1435] -
>>>> Closed socket connection for client /HOST_1:40556 which had sessionid
>>>> SESSION_1
>>>>
>>>>
>>>> On Thu, Feb 10, 2011 at 8:11 PM, Benjamin Reed <br...@yahoo-inc.com> wrote:
>>>>> do you see anything in the server log?
>>>>>
>>>>> ben
>>>>>
>>>>> On 02/10/2011 03:16 PM, Patania, Nick wrote:
>>>>>>
>>>>>> Hello,
>>>>>>
>>>>>> I run the following test:
>>>>>>
>>>>>>
>>>>>> *         Two clients connect to a zookeeper server; the tickTime on the
>>>>>> server is 500, and the sessionTimeout on the client is 1000.
>>>>>>
>>>>>> *         Kill the host running one of the clients.
>>>>>>
>>>>>> *         The second client receives a session timeout.
>>>>>>
>>>>>> Is there any reason why this might happen?
>>>>>>
>>>>>> Thanks
>>>>>> Nick Patania
>>>>>>
>>>>>>
>>>>>> --------------------------------------------------------------------------
>>>>>> NOTICE: Morgan Stanley is not acting as a municipal advisor and the
>>>>>> opinions or views contained herein are not intended to be, and do not
>>>>>> constitute, advice within the meaning of Section 975 of the Dodd-Frank Wall
>>>>>> Street Reform and Consumer Protection Act. If you have received this
>>>>>> communication in error, please destroy all electronic and paper copies and
>>>>>> notify the sender immediately. Mistransmission is not intended to waive
>>>>>> confidentiality or privilege. Morgan Stanley reserves the right, to the
>>>>>> extent permitted under applicable law, to monitor electronic communications.
>>>>>> This message is subject to terms available at the following link:
>>>>>> http://www.morganstanley.com/disclaimers. If you cannot access these links,
>>>>>> please notify us by reply message and we will send the contents to you. By
>>>>>> messaging with Morgan Stanley you consent to the foregoing.
>>>>>
>>>>>
>>>>
>>>
>>
>

RE: tickTime and sessionTimeout

Posted by "Fournier, Camille F. [Tech]" <Ca...@gs.com>.
Which version of ZK?

-----Original Message-----
From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com] 
Sent: Friday, February 11, 2011 6:10 PM
To: user@zookeeper.apache.org
Subject: Re: tickTime and sessionTimeout

A single server.  I've intentionally made it trivial to demonstrate
the behavior.
If I increase the timeout, the issue goes away.

On Fri, Feb 11, 2011 at 6:00 PM, Fournier, Camille F. [Tech]
<Ca...@gs.com> wrote:
> What is your ZooKeeper setup here? And do you continue to see this issue if you increase your session timeout?
>
> C
>
> -----Original Message-----
> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
> Sent: Friday, February 11, 2011 5:24 PM
> To: user@zookeeper.apache.org
> Subject: Re: tickTime and sessionTimeout
>
> Patrick, thanks for your input.
>
> I have rerun the test several times now while logging GC on the server
> and running ping from CLIENT_1:
>
>  - No GC happens on the server during the period of interest (a
> couple of young generation runs happen before I kill HOST_0, and they
> complete in under 3ms).
>  - Round trip times for ping from CLIENT_1 are consistently under
> 250us throughout.
>
> Regarding client GC -- I can consistently reproduce this using a C
> client.  Regarding the theory of swapping on CLIENT_1 -- if that were
> the cause, the problem wouldn't be 100% reproducible.  I also looked
> through the client log for SESSION_1 -- I see "Got ping response ...
> after 1ms" repeatedly, followed by "Client session timed out, have not
> heard from server in 666ms"...
>
> On Fri, Feb 11, 2011 at 1:46 PM, Patrick Hunt <ph...@apache.org> wrote:
>> Those are pretty short timeouts, many sources of delay could be
>> causing this. Network jitter/latency, GC/swap (server or client), IO
>> write latency, etc... See if any of this might be your issue:
>> https://cwiki.apache.org/confluence/display/ZOOKEEPER/Troubleshooting
>>
>> I can clearly see this sequence in your log for session1:
>>
>> ---
>> 2011-02-11 14:18:52,672 -  sessionid:SESSION_1 type:ping
>>
>> 2011-02-11 14:18:54,502 - Expiring session SESSION_1, timeout of 1000ms exceeded
>>
>> 2011-02-11 14:18:55,011 - Processing request:: sessionid:SESSION_1 type:ping
>> ---
>>
>> from the looks of it session 1 doesn't send a ping to the server for
>> ~2.5 seconds, as a result it's expired.
>>
>> You should also look at your session 1 client log and see what it's
>> view of the world is like. (is it gc/swapping?). Use the "stat" 4
>> letter word to see the server's request processing latency, see if
>> that's high (higer than the timeout is bad news). Again, checkout the
>> troubleshooting guide.
>>
>> Patrick
>>
>> ps please use pastebin or attachment, otw the formatting of wrecked
>> and it's harder to read the log
>>
>> On Fri, Feb 11, 2011 at 6:41 AM, Nick Patania
>> <Ni...@morganstanley.com> wrote:
>>> This is the portion that seems relevant.  For readability, I replaced
>>> the host and session for the host that I kill with HOST_0 and
>>> SESSION_0 (I expect these to timeout).  The client that should be
>>> healthy is HOST_1 and SESSION_1.
>>>
>>>
>>> 2011-02-11 14:18:51,901 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>> sessionid:SESSION_0 type:ping cxid:0xfffffffffffffffe
>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:51,901 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_0
>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>> txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:52,005 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:52,005 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>> txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:52,339 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:52,339 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>> txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:52,672 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:52,672 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>> txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:53,002 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>> Expiring session SESSION_0, timeout of 1000ms exceeded
>>> 2011-02-11 14:18:53,002 - INFO
>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>> termination for sessionid: SESSION_0
>>> 2011-02-11 14:18:53,010 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>> sessionid:SESSION_0 type:closeSession cxid:0x0 zxid:0x103 txntype:-11
>>> reqpath:n/a
>>> 2011-02-11 14:18:53,010 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>> Closed socket connection for client /HOST_0:34618 which had sessionid
>>> SESSION_0
>>> 2011-02-11 14:18:54,502 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>> Expiring session SESSION_1, timeout of 1000ms exceeded
>>> 2011-02-11 14:18:54,502 - INFO
>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>> termination for sessionid: SESSION_1
>>> 2011-02-11 14:18:55,011 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:55,011 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>> txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:55,011 - INFO
>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@251]
>>> - Accepted socket connection from /HOST_1:40556
>>> 2011-02-11 14:18:55,019 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>> sessionid:SESSION_1 type:closeSession cxid:0x0 zxid:0x104 txntype:-11
>>> reqpath:n/a
>>> 2011-02-11 14:18:55,019 - DEBUG
>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:ZooKeeperServer@590] -
>>> Dropping request: No session with sessionid SESSION_1 exists, probably
>>> expired and removed
>>> 2011-02-11 14:18:55,019 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>> Closed socket connection for client /HOST_1:40555 which had sessionid
>>> SESSION_1
>>> 2011-02-11 14:18:55,020 - DEBUG [SyncThread:0:NIOServerCnxn@1451] -
>>> ignoring exception during output shutdown
>>> java.net.SocketException: Transport endpoint is not connected
>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>        at sun.nio.ch.SocketChannelImpl.shutdownOutput(SocketChannelImpl.java:651)
>>>        at sun.nio.ch.SocketAdaptor.shutdownOutput(SocketAdaptor.java:368)
>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1447)
>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>> 2011-02-11 14:18:55,021 - DEBUG [SyncThread:0:NIOServerCnxn@1459] -
>>> ignoring exception during input shutdown
>>> java.net.SocketException: Transport endpoint is not connected
>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>        at sun.nio.ch.SocketChannelImpl.shutdownInput(SocketChannelImpl.java:640)
>>>        at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:360)
>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1455)
>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>> 2011-02-11 14:18:55,022 - WARN
>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@272]
>>> - Ignoring unexpected runtime exception
>>> java.nio.channels.CancelledKeyException
>>>        at sun.nio.ch.SelectionKeyImpl.ensureValid(SelectionKeyImpl.java:55)
>>>        at sun.nio.ch.SelectionKeyImpl.readyOps(SelectionKeyImpl.java:69)
>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:241)
>>> 2011-02-11 14:18:55,023 - DEBUG
>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@735] -
>>> Session establishment request from client /HOST_1:40556 client's
>>> lastZxid is 0x0
>>> 2011-02-11 14:18:55,023 - INFO
>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@770] - Client
>>> attempting to renew session SESSION_1 at /HOST_1:40556
>>> 2011-02-11 14:18:55,024 - INFO
>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1573] -
>>> Invalid session SESSION_1 for client /HOST_1:40556, probably expired
>>> 2011-02-11 14:18:55,025 - WARN
>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@634] -
>>> EndOfStreamException: Unable to read additional data from client
>>> sessionid SESSION_1, likely client has closed socket
>>> 2011-02-11 14:18:55,025 - INFO
>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1435] -
>>> Closed socket connection for client /HOST_1:40556 which had sessionid
>>> SESSION_1
>>>
>>>
>>> On Thu, Feb 10, 2011 at 8:11 PM, Benjamin Reed <br...@yahoo-inc.com> wrote:
>>>> do you see anything in the server log?
>>>>
>>>> ben
>>>>
>>>> On 02/10/2011 03:16 PM, Patania, Nick wrote:
>>>>>
>>>>> Hello,
>>>>>
>>>>> I run the following test:
>>>>>
>>>>>
>>>>> *         Two clients connect to a zookeeper server; the tickTime on the
>>>>> server is 500, and the sessionTimeout on the client is 1000.
>>>>>
>>>>> *         Kill the host running one of the clients.
>>>>>
>>>>> *         The second client receives a session timeout.
>>>>>
>>>>> Is there any reason why this might happen?
>>>>>
>>>>> Thanks
>>>>> Nick Patania
>>>>>
>>>>>
>>>>> --------------------------------------------------------------------------
>>>>> NOTICE: Morgan Stanley is not acting as a municipal advisor and the
>>>>> opinions or views contained herein are not intended to be, and do not
>>>>> constitute, advice within the meaning of Section 975 of the Dodd-Frank Wall
>>>>> Street Reform and Consumer Protection Act. If you have received this
>>>>> communication in error, please destroy all electronic and paper copies and
>>>>> notify the sender immediately. Mistransmission is not intended to waive
>>>>> confidentiality or privilege. Morgan Stanley reserves the right, to the
>>>>> extent permitted under applicable law, to monitor electronic communications.
>>>>> This message is subject to terms available at the following link:
>>>>> http://www.morganstanley.com/disclaimers. If you cannot access these links,
>>>>> please notify us by reply message and we will send the contents to you. By
>>>>> messaging with Morgan Stanley you consent to the foregoing.
>>>>
>>>>
>>>
>>
>

Re: tickTime and sessionTimeout

Posted by Nick Patania <Ni...@morganstanley.com>.
A single server.  I've intentionally made it trivial to demonstrate
the behavior.
If I increase the timeout, the issue goes away.

On Fri, Feb 11, 2011 at 6:00 PM, Fournier, Camille F. [Tech]
<Ca...@gs.com> wrote:
> What is your ZooKeeper setup here? And do you continue to see this issue if you increase your session timeout?
>
> C
>
> -----Original Message-----
> From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com]
> Sent: Friday, February 11, 2011 5:24 PM
> To: user@zookeeper.apache.org
> Subject: Re: tickTime and sessionTimeout
>
> Patrick, thanks for your input.
>
> I have rerun the test several times now while logging GC on the server
> and running ping from CLIENT_1:
>
>  - No GC happens on the server during the period of interest (a
> couple of young generation runs happen before I kill HOST_0, and they
> complete in under 3ms).
>  - Round trip times for ping from CLIENT_1 are consistently under
> 250us throughout.
>
> Regarding client GC -- I can consistently reproduce this using a C
> client.  Regarding the theory of swapping on CLIENT_1 -- if that were
> the cause, the problem wouldn't be 100% reproducible.  I also looked
> through the client log for SESSION_1 -- I see "Got ping response ...
> after 1ms" repeatedly, followed by "Client session timed out, have not
> heard from server in 666ms"...
>
> On Fri, Feb 11, 2011 at 1:46 PM, Patrick Hunt <ph...@apache.org> wrote:
>> Those are pretty short timeouts, many sources of delay could be
>> causing this. Network jitter/latency, GC/swap (server or client), IO
>> write latency, etc... See if any of this might be your issue:
>> https://cwiki.apache.org/confluence/display/ZOOKEEPER/Troubleshooting
>>
>> I can clearly see this sequence in your log for session1:
>>
>> ---
>> 2011-02-11 14:18:52,672 -  sessionid:SESSION_1 type:ping
>>
>> 2011-02-11 14:18:54,502 - Expiring session SESSION_1, timeout of 1000ms exceeded
>>
>> 2011-02-11 14:18:55,011 - Processing request:: sessionid:SESSION_1 type:ping
>> ---
>>
>> from the looks of it session 1 doesn't send a ping to the server for
>> ~2.5 seconds, as a result it's expired.
>>
>> You should also look at your session 1 client log and see what it's
>> view of the world is like. (is it gc/swapping?). Use the "stat" 4
>> letter word to see the server's request processing latency, see if
>> that's high (higer than the timeout is bad news). Again, checkout the
>> troubleshooting guide.
>>
>> Patrick
>>
>> ps please use pastebin or attachment, otw the formatting of wrecked
>> and it's harder to read the log
>>
>> On Fri, Feb 11, 2011 at 6:41 AM, Nick Patania
>> <Ni...@morganstanley.com> wrote:
>>> This is the portion that seems relevant.  For readability, I replaced
>>> the host and session for the host that I kill with HOST_0 and
>>> SESSION_0 (I expect these to timeout).  The client that should be
>>> healthy is HOST_1 and SESSION_1.
>>>
>>>
>>> 2011-02-11 14:18:51,901 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>> sessionid:SESSION_0 type:ping cxid:0xfffffffffffffffe
>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:51,901 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_0
>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>> txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:52,005 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:52,005 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>> txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:52,339 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:52,339 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>> txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:52,672 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:52,672 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>> txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:53,002 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>> Expiring session SESSION_0, timeout of 1000ms exceeded
>>> 2011-02-11 14:18:53,002 - INFO
>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>> termination for sessionid: SESSION_0
>>> 2011-02-11 14:18:53,010 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>> sessionid:SESSION_0 type:closeSession cxid:0x0 zxid:0x103 txntype:-11
>>> reqpath:n/a
>>> 2011-02-11 14:18:53,010 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>> Closed socket connection for client /HOST_0:34618 which had sessionid
>>> SESSION_0
>>> 2011-02-11 14:18:54,502 - INFO  [SessionTracker:ZooKeeperServer@314] -
>>> Expiring session SESSION_1, timeout of 1000ms exceeded
>>> 2011-02-11 14:18:54,502 - INFO
>>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>>> termination for sessionid: SESSION_1
>>> 2011-02-11 14:18:55,011 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:55,011 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>>> txntype:unknown reqpath:n/a
>>> 2011-02-11 14:18:55,011 - INFO
>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@251]
>>> - Accepted socket connection from /HOST_1:40556
>>> 2011-02-11 14:18:55,019 - DEBUG
>>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>>> sessionid:SESSION_1 type:closeSession cxid:0x0 zxid:0x104 txntype:-11
>>> reqpath:n/a
>>> 2011-02-11 14:18:55,019 - DEBUG
>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:ZooKeeperServer@590] -
>>> Dropping request: No session with sessionid SESSION_1 exists, probably
>>> expired and removed
>>> 2011-02-11 14:18:55,019 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>>> Closed socket connection for client /HOST_1:40555 which had sessionid
>>> SESSION_1
>>> 2011-02-11 14:18:55,020 - DEBUG [SyncThread:0:NIOServerCnxn@1451] -
>>> ignoring exception during output shutdown
>>> java.net.SocketException: Transport endpoint is not connected
>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>        at sun.nio.ch.SocketChannelImpl.shutdownOutput(SocketChannelImpl.java:651)
>>>        at sun.nio.ch.SocketAdaptor.shutdownOutput(SocketAdaptor.java:368)
>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1447)
>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>> 2011-02-11 14:18:55,021 - DEBUG [SyncThread:0:NIOServerCnxn@1459] -
>>> ignoring exception during input shutdown
>>> java.net.SocketException: Transport endpoint is not connected
>>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>>        at sun.nio.ch.SocketChannelImpl.shutdownInput(SocketChannelImpl.java:640)
>>>        at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:360)
>>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1455)
>>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>>> 2011-02-11 14:18:55,022 - WARN
>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@272]
>>> - Ignoring unexpected runtime exception
>>> java.nio.channels.CancelledKeyException
>>>        at sun.nio.ch.SelectionKeyImpl.ensureValid(SelectionKeyImpl.java:55)
>>>        at sun.nio.ch.SelectionKeyImpl.readyOps(SelectionKeyImpl.java:69)
>>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:241)
>>> 2011-02-11 14:18:55,023 - DEBUG
>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@735] -
>>> Session establishment request from client /HOST_1:40556 client's
>>> lastZxid is 0x0
>>> 2011-02-11 14:18:55,023 - INFO
>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@770] - Client
>>> attempting to renew session SESSION_1 at /HOST_1:40556
>>> 2011-02-11 14:18:55,024 - INFO
>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1573] -
>>> Invalid session SESSION_1 for client /HOST_1:40556, probably expired
>>> 2011-02-11 14:18:55,025 - WARN
>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@634] -
>>> EndOfStreamException: Unable to read additional data from client
>>> sessionid SESSION_1, likely client has closed socket
>>> 2011-02-11 14:18:55,025 - INFO
>>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1435] -
>>> Closed socket connection for client /HOST_1:40556 which had sessionid
>>> SESSION_1
>>>
>>>
>>> On Thu, Feb 10, 2011 at 8:11 PM, Benjamin Reed <br...@yahoo-inc.com> wrote:
>>>> do you see anything in the server log?
>>>>
>>>> ben
>>>>
>>>> On 02/10/2011 03:16 PM, Patania, Nick wrote:
>>>>>
>>>>> Hello,
>>>>>
>>>>> I run the following test:
>>>>>
>>>>>
>>>>> *         Two clients connect to a zookeeper server; the tickTime on the
>>>>> server is 500, and the sessionTimeout on the client is 1000.
>>>>>
>>>>> *         Kill the host running one of the clients.
>>>>>
>>>>> *         The second client receives a session timeout.
>>>>>
>>>>> Is there any reason why this might happen?
>>>>>
>>>>> Thanks
>>>>> Nick Patania
>>>>>
>>>>>
>>>>> --------------------------------------------------------------------------
>>>>> NOTICE: Morgan Stanley is not acting as a municipal advisor and the
>>>>> opinions or views contained herein are not intended to be, and do not
>>>>> constitute, advice within the meaning of Section 975 of the Dodd-Frank Wall
>>>>> Street Reform and Consumer Protection Act. If you have received this
>>>>> communication in error, please destroy all electronic and paper copies and
>>>>> notify the sender immediately. Mistransmission is not intended to waive
>>>>> confidentiality or privilege. Morgan Stanley reserves the right, to the
>>>>> extent permitted under applicable law, to monitor electronic communications.
>>>>> This message is subject to terms available at the following link:
>>>>> http://www.morganstanley.com/disclaimers. If you cannot access these links,
>>>>> please notify us by reply message and we will send the contents to you. By
>>>>> messaging with Morgan Stanley you consent to the foregoing.
>>>>
>>>>
>>>
>>
>

RE: tickTime and sessionTimeout

Posted by "Fournier, Camille F. [Tech]" <Ca...@gs.com>.
What is your ZooKeeper setup here? And do you continue to see this issue if you increase your session timeout?

C

-----Original Message-----
From: Nick Patania [mailto:Nicholas.Patania@morganstanley.com] 
Sent: Friday, February 11, 2011 5:24 PM
To: user@zookeeper.apache.org
Subject: Re: tickTime and sessionTimeout

Patrick, thanks for your input.

I have rerun the test several times now while logging GC on the server
and running ping from CLIENT_1:

  - No GC happens on the server during the period of interest (a
couple of young generation runs happen before I kill HOST_0, and they
complete in under 3ms).
  - Round trip times for ping from CLIENT_1 are consistently under
250us throughout.

Regarding client GC -- I can consistently reproduce this using a C
client.  Regarding the theory of swapping on CLIENT_1 -- if that were
the cause, the problem wouldn't be 100% reproducible.  I also looked
through the client log for SESSION_1 -- I see "Got ping response ...
after 1ms" repeatedly, followed by "Client session timed out, have not
heard from server in 666ms"...

On Fri, Feb 11, 2011 at 1:46 PM, Patrick Hunt <ph...@apache.org> wrote:
> Those are pretty short timeouts, many sources of delay could be
> causing this. Network jitter/latency, GC/swap (server or client), IO
> write latency, etc... See if any of this might be your issue:
> https://cwiki.apache.org/confluence/display/ZOOKEEPER/Troubleshooting
>
> I can clearly see this sequence in your log for session1:
>
> ---
> 2011-02-11 14:18:52,672 -  sessionid:SESSION_1 type:ping
>
> 2011-02-11 14:18:54,502 - Expiring session SESSION_1, timeout of 1000ms exceeded
>
> 2011-02-11 14:18:55,011 - Processing request:: sessionid:SESSION_1 type:ping
> ---
>
> from the looks of it session 1 doesn't send a ping to the server for
> ~2.5 seconds, as a result it's expired.
>
> You should also look at your session 1 client log and see what it's
> view of the world is like. (is it gc/swapping?). Use the "stat" 4
> letter word to see the server's request processing latency, see if
> that's high (higer than the timeout is bad news). Again, checkout the
> troubleshooting guide.
>
> Patrick
>
> ps please use pastebin or attachment, otw the formatting of wrecked
> and it's harder to read the log
>
> On Fri, Feb 11, 2011 at 6:41 AM, Nick Patania
> <Ni...@morganstanley.com> wrote:
>> This is the portion that seems relevant.  For readability, I replaced
>> the host and session for the host that I kill with HOST_0 and
>> SESSION_0 (I expect these to timeout).  The client that should be
>> healthy is HOST_1 and SESSION_1.
>>
>>
>> 2011-02-11 14:18:51,901 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>> sessionid:SESSION_0 type:ping cxid:0xfffffffffffffffe
>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:51,901 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_0
>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>> txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:52,005 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:52,005 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>> txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:52,339 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:52,339 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>> txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:52,672 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:52,672 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>> txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:53,002 - INFO  [SessionTracker:ZooKeeperServer@314] -
>> Expiring session SESSION_0, timeout of 1000ms exceeded
>> 2011-02-11 14:18:53,002 - INFO
>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>> termination for sessionid: SESSION_0
>> 2011-02-11 14:18:53,010 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>> sessionid:SESSION_0 type:closeSession cxid:0x0 zxid:0x103 txntype:-11
>> reqpath:n/a
>> 2011-02-11 14:18:53,010 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>> Closed socket connection for client /HOST_0:34618 which had sessionid
>> SESSION_0
>> 2011-02-11 14:18:54,502 - INFO  [SessionTracker:ZooKeeperServer@314] -
>> Expiring session SESSION_1, timeout of 1000ms exceeded
>> 2011-02-11 14:18:54,502 - INFO
>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>> termination for sessionid: SESSION_1
>> 2011-02-11 14:18:55,011 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:55,011 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>> txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:55,011 - INFO
>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@251]
>> - Accepted socket connection from /HOST_1:40556
>> 2011-02-11 14:18:55,019 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>> sessionid:SESSION_1 type:closeSession cxid:0x0 zxid:0x104 txntype:-11
>> reqpath:n/a
>> 2011-02-11 14:18:55,019 - DEBUG
>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:ZooKeeperServer@590] -
>> Dropping request: No session with sessionid SESSION_1 exists, probably
>> expired and removed
>> 2011-02-11 14:18:55,019 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>> Closed socket connection for client /HOST_1:40555 which had sessionid
>> SESSION_1
>> 2011-02-11 14:18:55,020 - DEBUG [SyncThread:0:NIOServerCnxn@1451] -
>> ignoring exception during output shutdown
>> java.net.SocketException: Transport endpoint is not connected
>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>        at sun.nio.ch.SocketChannelImpl.shutdownOutput(SocketChannelImpl.java:651)
>>        at sun.nio.ch.SocketAdaptor.shutdownOutput(SocketAdaptor.java:368)
>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1447)
>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>> 2011-02-11 14:18:55,021 - DEBUG [SyncThread:0:NIOServerCnxn@1459] -
>> ignoring exception during input shutdown
>> java.net.SocketException: Transport endpoint is not connected
>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>        at sun.nio.ch.SocketChannelImpl.shutdownInput(SocketChannelImpl.java:640)
>>        at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:360)
>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1455)
>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>> 2011-02-11 14:18:55,022 - WARN
>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@272]
>> - Ignoring unexpected runtime exception
>> java.nio.channels.CancelledKeyException
>>        at sun.nio.ch.SelectionKeyImpl.ensureValid(SelectionKeyImpl.java:55)
>>        at sun.nio.ch.SelectionKeyImpl.readyOps(SelectionKeyImpl.java:69)
>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:241)
>> 2011-02-11 14:18:55,023 - DEBUG
>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@735] -
>> Session establishment request from client /HOST_1:40556 client's
>> lastZxid is 0x0
>> 2011-02-11 14:18:55,023 - INFO
>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@770] - Client
>> attempting to renew session SESSION_1 at /HOST_1:40556
>> 2011-02-11 14:18:55,024 - INFO
>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1573] -
>> Invalid session SESSION_1 for client /HOST_1:40556, probably expired
>> 2011-02-11 14:18:55,025 - WARN
>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@634] -
>> EndOfStreamException: Unable to read additional data from client
>> sessionid SESSION_1, likely client has closed socket
>> 2011-02-11 14:18:55,025 - INFO
>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1435] -
>> Closed socket connection for client /HOST_1:40556 which had sessionid
>> SESSION_1
>>
>>
>> On Thu, Feb 10, 2011 at 8:11 PM, Benjamin Reed <br...@yahoo-inc.com> wrote:
>>> do you see anything in the server log?
>>>
>>> ben
>>>
>>> On 02/10/2011 03:16 PM, Patania, Nick wrote:
>>>>
>>>> Hello,
>>>>
>>>> I run the following test:
>>>>
>>>>
>>>> *         Two clients connect to a zookeeper server; the tickTime on the
>>>> server is 500, and the sessionTimeout on the client is 1000.
>>>>
>>>> *         Kill the host running one of the clients.
>>>>
>>>> *         The second client receives a session timeout.
>>>>
>>>> Is there any reason why this might happen?
>>>>
>>>> Thanks
>>>> Nick Patania
>>>>
>>>>
>>>> --------------------------------------------------------------------------
>>>> NOTICE: Morgan Stanley is not acting as a municipal advisor and the
>>>> opinions or views contained herein are not intended to be, and do not
>>>> constitute, advice within the meaning of Section 975 of the Dodd-Frank Wall
>>>> Street Reform and Consumer Protection Act. If you have received this
>>>> communication in error, please destroy all electronic and paper copies and
>>>> notify the sender immediately. Mistransmission is not intended to waive
>>>> confidentiality or privilege. Morgan Stanley reserves the right, to the
>>>> extent permitted under applicable law, to monitor electronic communications.
>>>> This message is subject to terms available at the following link:
>>>> http://www.morganstanley.com/disclaimers. If you cannot access these links,
>>>> please notify us by reply message and we will send the contents to you. By
>>>> messaging with Morgan Stanley you consent to the foregoing.
>>>
>>>
>>
>

Re: tickTime and sessionTimeout

Posted by Nick Patania <Ni...@morganstanley.com>.
Patrick, thanks for your input.

I have rerun the test several times now while logging GC on the server
and running ping from CLIENT_1:

  - No GC happens on the server during the period of interest (a
couple of young generation runs happen before I kill HOST_0, and they
complete in under 3ms).
  - Round trip times for ping from CLIENT_1 are consistently under
250us throughout.

Regarding client GC -- I can consistently reproduce this using a C
client.  Regarding the theory of swapping on CLIENT_1 -- if that were
the cause, the problem wouldn't be 100% reproducible.  I also looked
through the client log for SESSION_1 -- I see "Got ping response ...
after 1ms" repeatedly, followed by "Client session timed out, have not
heard from server in 666ms"...

On Fri, Feb 11, 2011 at 1:46 PM, Patrick Hunt <ph...@apache.org> wrote:
> Those are pretty short timeouts, many sources of delay could be
> causing this. Network jitter/latency, GC/swap (server or client), IO
> write latency, etc... See if any of this might be your issue:
> https://cwiki.apache.org/confluence/display/ZOOKEEPER/Troubleshooting
>
> I can clearly see this sequence in your log for session1:
>
> ---
> 2011-02-11 14:18:52,672 -  sessionid:SESSION_1 type:ping
>
> 2011-02-11 14:18:54,502 - Expiring session SESSION_1, timeout of 1000ms exceeded
>
> 2011-02-11 14:18:55,011 - Processing request:: sessionid:SESSION_1 type:ping
> ---
>
> from the looks of it session 1 doesn't send a ping to the server for
> ~2.5 seconds, as a result it's expired.
>
> You should also look at your session 1 client log and see what it's
> view of the world is like. (is it gc/swapping?). Use the "stat" 4
> letter word to see the server's request processing latency, see if
> that's high (higer than the timeout is bad news). Again, checkout the
> troubleshooting guide.
>
> Patrick
>
> ps please use pastebin or attachment, otw the formatting of wrecked
> and it's harder to read the log
>
> On Fri, Feb 11, 2011 at 6:41 AM, Nick Patania
> <Ni...@morganstanley.com> wrote:
>> This is the portion that seems relevant.  For readability, I replaced
>> the host and session for the host that I kill with HOST_0 and
>> SESSION_0 (I expect these to timeout).  The client that should be
>> healthy is HOST_1 and SESSION_1.
>>
>>
>> 2011-02-11 14:18:51,901 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>> sessionid:SESSION_0 type:ping cxid:0xfffffffffffffffe
>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:51,901 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_0
>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>> txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:52,005 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:52,005 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>> txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:52,339 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:52,339 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>> txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:52,672 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:52,672 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>> txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:53,002 - INFO  [SessionTracker:ZooKeeperServer@314] -
>> Expiring session SESSION_0, timeout of 1000ms exceeded
>> 2011-02-11 14:18:53,002 - INFO
>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>> termination for sessionid: SESSION_0
>> 2011-02-11 14:18:53,010 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>> sessionid:SESSION_0 type:closeSession cxid:0x0 zxid:0x103 txntype:-11
>> reqpath:n/a
>> 2011-02-11 14:18:53,010 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>> Closed socket connection for client /HOST_0:34618 which had sessionid
>> SESSION_0
>> 2011-02-11 14:18:54,502 - INFO  [SessionTracker:ZooKeeperServer@314] -
>> Expiring session SESSION_1, timeout of 1000ms exceeded
>> 2011-02-11 14:18:54,502 - INFO
>> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
>> termination for sessionid: SESSION_1
>> 2011-02-11 14:18:55,011 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
>> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:55,011 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
>> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
>> txntype:unknown reqpath:n/a
>> 2011-02-11 14:18:55,011 - INFO
>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@251]
>> - Accepted socket connection from /HOST_1:40556
>> 2011-02-11 14:18:55,019 - DEBUG
>> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
>> sessionid:SESSION_1 type:closeSession cxid:0x0 zxid:0x104 txntype:-11
>> reqpath:n/a
>> 2011-02-11 14:18:55,019 - DEBUG
>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:ZooKeeperServer@590] -
>> Dropping request: No session with sessionid SESSION_1 exists, probably
>> expired and removed
>> 2011-02-11 14:18:55,019 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
>> Closed socket connection for client /HOST_1:40555 which had sessionid
>> SESSION_1
>> 2011-02-11 14:18:55,020 - DEBUG [SyncThread:0:NIOServerCnxn@1451] -
>> ignoring exception during output shutdown
>> java.net.SocketException: Transport endpoint is not connected
>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>        at sun.nio.ch.SocketChannelImpl.shutdownOutput(SocketChannelImpl.java:651)
>>        at sun.nio.ch.SocketAdaptor.shutdownOutput(SocketAdaptor.java:368)
>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1447)
>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>> 2011-02-11 14:18:55,021 - DEBUG [SyncThread:0:NIOServerCnxn@1459] -
>> ignoring exception during input shutdown
>> java.net.SocketException: Transport endpoint is not connected
>>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>>        at sun.nio.ch.SocketChannelImpl.shutdownInput(SocketChannelImpl.java:640)
>>        at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:360)
>>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1455)
>>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
>> 2011-02-11 14:18:55,022 - WARN
>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@272]
>> - Ignoring unexpected runtime exception
>> java.nio.channels.CancelledKeyException
>>        at sun.nio.ch.SelectionKeyImpl.ensureValid(SelectionKeyImpl.java:55)
>>        at sun.nio.ch.SelectionKeyImpl.readyOps(SelectionKeyImpl.java:69)
>>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:241)
>> 2011-02-11 14:18:55,023 - DEBUG
>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@735] -
>> Session establishment request from client /HOST_1:40556 client's
>> lastZxid is 0x0
>> 2011-02-11 14:18:55,023 - INFO
>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@770] - Client
>> attempting to renew session SESSION_1 at /HOST_1:40556
>> 2011-02-11 14:18:55,024 - INFO
>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1573] -
>> Invalid session SESSION_1 for client /HOST_1:40556, probably expired
>> 2011-02-11 14:18:55,025 - WARN
>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@634] -
>> EndOfStreamException: Unable to read additional data from client
>> sessionid SESSION_1, likely client has closed socket
>> 2011-02-11 14:18:55,025 - INFO
>> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1435] -
>> Closed socket connection for client /HOST_1:40556 which had sessionid
>> SESSION_1
>>
>>
>> On Thu, Feb 10, 2011 at 8:11 PM, Benjamin Reed <br...@yahoo-inc.com> wrote:
>>> do you see anything in the server log?
>>>
>>> ben
>>>
>>> On 02/10/2011 03:16 PM, Patania, Nick wrote:
>>>>
>>>> Hello,
>>>>
>>>> I run the following test:
>>>>
>>>>
>>>> *         Two clients connect to a zookeeper server; the tickTime on the
>>>> server is 500, and the sessionTimeout on the client is 1000.
>>>>
>>>> *         Kill the host running one of the clients.
>>>>
>>>> *         The second client receives a session timeout.
>>>>
>>>> Is there any reason why this might happen?
>>>>
>>>> Thanks
>>>> Nick Patania
>>>>
>>>>
>>>> --------------------------------------------------------------------------
>>>> NOTICE: Morgan Stanley is not acting as a municipal advisor and the
>>>> opinions or views contained herein are not intended to be, and do not
>>>> constitute, advice within the meaning of Section 975 of the Dodd-Frank Wall
>>>> Street Reform and Consumer Protection Act. If you have received this
>>>> communication in error, please destroy all electronic and paper copies and
>>>> notify the sender immediately. Mistransmission is not intended to waive
>>>> confidentiality or privilege. Morgan Stanley reserves the right, to the
>>>> extent permitted under applicable law, to monitor electronic communications.
>>>> This message is subject to terms available at the following link:
>>>> http://www.morganstanley.com/disclaimers. If you cannot access these links,
>>>> please notify us by reply message and we will send the contents to you. By
>>>> messaging with Morgan Stanley you consent to the foregoing.
>>>
>>>
>>
>

Re: tickTime and sessionTimeout

Posted by Patrick Hunt <ph...@apache.org>.
Those are pretty short timeouts, many sources of delay could be
causing this. Network jitter/latency, GC/swap (server or client), IO
write latency, etc... See if any of this might be your issue:
https://cwiki.apache.org/confluence/display/ZOOKEEPER/Troubleshooting

I can clearly see this sequence in your log for session1:

---
2011-02-11 14:18:52,672 -  sessionid:SESSION_1 type:ping

2011-02-11 14:18:54,502 - Expiring session SESSION_1, timeout of 1000ms exceeded

2011-02-11 14:18:55,011 - Processing request:: sessionid:SESSION_1 type:ping
---

from the looks of it session 1 doesn't send a ping to the server for
~2.5 seconds, as a result it's expired.

You should also look at your session 1 client log and see what it's
view of the world is like. (is it gc/swapping?). Use the "stat" 4
letter word to see the server's request processing latency, see if
that's high (higer than the timeout is bad news). Again, checkout the
troubleshooting guide.

Patrick

ps please use pastebin or attachment, otw the formatting of wrecked
and it's harder to read the log

On Fri, Feb 11, 2011 at 6:41 AM, Nick Patania
<Ni...@morganstanley.com> wrote:
> This is the portion that seems relevant.  For readability, I replaced
> the host and session for the host that I kill with HOST_0 and
> SESSION_0 (I expect these to timeout).  The client that should be
> healthy is HOST_1 and SESSION_1.
>
>
> 2011-02-11 14:18:51,901 - DEBUG
> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
> sessionid:SESSION_0 type:ping cxid:0xfffffffffffffffe
> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
> 2011-02-11 14:18:51,901 - DEBUG
> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_0
> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
> txntype:unknown reqpath:n/a
> 2011-02-11 14:18:52,005 - DEBUG
> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
> 2011-02-11 14:18:52,005 - DEBUG
> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
> txntype:unknown reqpath:n/a
> 2011-02-11 14:18:52,339 - DEBUG
> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
> 2011-02-11 14:18:52,339 - DEBUG
> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
> txntype:unknown reqpath:n/a
> 2011-02-11 14:18:52,672 - DEBUG
> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
> 2011-02-11 14:18:52,672 - DEBUG
> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
> txntype:unknown reqpath:n/a
> 2011-02-11 14:18:53,002 - INFO  [SessionTracker:ZooKeeperServer@314] -
> Expiring session SESSION_0, timeout of 1000ms exceeded
> 2011-02-11 14:18:53,002 - INFO
> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
> termination for sessionid: SESSION_0
> 2011-02-11 14:18:53,010 - DEBUG
> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
> sessionid:SESSION_0 type:closeSession cxid:0x0 zxid:0x103 txntype:-11
> reqpath:n/a
> 2011-02-11 14:18:53,010 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
> Closed socket connection for client /HOST_0:34618 which had sessionid
> SESSION_0
> 2011-02-11 14:18:54,502 - INFO  [SessionTracker:ZooKeeperServer@314] -
> Expiring session SESSION_1, timeout of 1000ms exceeded
> 2011-02-11 14:18:54,502 - INFO
> [ProcessThread:-1:PrepRequestProcessor@387] - Processed session
> termination for sessionid: SESSION_1
> 2011-02-11 14:18:55,011 - DEBUG
> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
> sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
> zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
> 2011-02-11 14:18:55,011 - DEBUG
> [SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
> type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
> txntype:unknown reqpath:n/a
> 2011-02-11 14:18:55,011 - INFO
> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@251]
> - Accepted socket connection from /HOST_1:40556
> 2011-02-11 14:18:55,019 - DEBUG
> [SyncThread:0:FinalRequestProcessor@78] - Processing request::
> sessionid:SESSION_1 type:closeSession cxid:0x0 zxid:0x104 txntype:-11
> reqpath:n/a
> 2011-02-11 14:18:55,019 - DEBUG
> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:ZooKeeperServer@590] -
> Dropping request: No session with sessionid SESSION_1 exists, probably
> expired and removed
> 2011-02-11 14:18:55,019 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
> Closed socket connection for client /HOST_1:40555 which had sessionid
> SESSION_1
> 2011-02-11 14:18:55,020 - DEBUG [SyncThread:0:NIOServerCnxn@1451] -
> ignoring exception during output shutdown
> java.net.SocketException: Transport endpoint is not connected
>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>        at sun.nio.ch.SocketChannelImpl.shutdownOutput(SocketChannelImpl.java:651)
>        at sun.nio.ch.SocketAdaptor.shutdownOutput(SocketAdaptor.java:368)
>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1447)
>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
> 2011-02-11 14:18:55,021 - DEBUG [SyncThread:0:NIOServerCnxn@1459] -
> ignoring exception during input shutdown
> java.net.SocketException: Transport endpoint is not connected
>        at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
>        at sun.nio.ch.SocketChannelImpl.shutdownInput(SocketChannelImpl.java:640)
>        at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:360)
>        at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1455)
>        at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
>        at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
>        at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
>        at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
> 2011-02-11 14:18:55,022 - WARN
> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@272]
> - Ignoring unexpected runtime exception
> java.nio.channels.CancelledKeyException
>        at sun.nio.ch.SelectionKeyImpl.ensureValid(SelectionKeyImpl.java:55)
>        at sun.nio.ch.SelectionKeyImpl.readyOps(SelectionKeyImpl.java:69)
>        at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:241)
> 2011-02-11 14:18:55,023 - DEBUG
> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@735] -
> Session establishment request from client /HOST_1:40556 client's
> lastZxid is 0x0
> 2011-02-11 14:18:55,023 - INFO
> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@770] - Client
> attempting to renew session SESSION_1 at /HOST_1:40556
> 2011-02-11 14:18:55,024 - INFO
> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1573] -
> Invalid session SESSION_1 for client /HOST_1:40556, probably expired
> 2011-02-11 14:18:55,025 - WARN
> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@634] -
> EndOfStreamException: Unable to read additional data from client
> sessionid SESSION_1, likely client has closed socket
> 2011-02-11 14:18:55,025 - INFO
> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1435] -
> Closed socket connection for client /HOST_1:40556 which had sessionid
> SESSION_1
>
>
> On Thu, Feb 10, 2011 at 8:11 PM, Benjamin Reed <br...@yahoo-inc.com> wrote:
>> do you see anything in the server log?
>>
>> ben
>>
>> On 02/10/2011 03:16 PM, Patania, Nick wrote:
>>>
>>> Hello,
>>>
>>> I run the following test:
>>>
>>>
>>> *         Two clients connect to a zookeeper server; the tickTime on the
>>> server is 500, and the sessionTimeout on the client is 1000.
>>>
>>> *         Kill the host running one of the clients.
>>>
>>> *         The second client receives a session timeout.
>>>
>>> Is there any reason why this might happen?
>>>
>>> Thanks
>>> Nick Patania
>>>
>>>
>>> --------------------------------------------------------------------------
>>> NOTICE: Morgan Stanley is not acting as a municipal advisor and the
>>> opinions or views contained herein are not intended to be, and do not
>>> constitute, advice within the meaning of Section 975 of the Dodd-Frank Wall
>>> Street Reform and Consumer Protection Act. If you have received this
>>> communication in error, please destroy all electronic and paper copies and
>>> notify the sender immediately. Mistransmission is not intended to waive
>>> confidentiality or privilege. Morgan Stanley reserves the right, to the
>>> extent permitted under applicable law, to monitor electronic communications.
>>> This message is subject to terms available at the following link:
>>> http://www.morganstanley.com/disclaimers. If you cannot access these links,
>>> please notify us by reply message and we will send the contents to you. By
>>> messaging with Morgan Stanley you consent to the foregoing.
>>
>>
>

Re: tickTime and sessionTimeout

Posted by Nick Patania <Ni...@morganstanley.com>.
This is the portion that seems relevant.  For readability, I replaced
the host and session for the host that I kill with HOST_0 and
SESSION_0 (I expect these to timeout).  The client that should be
healthy is HOST_1 and SESSION_1.


2011-02-11 14:18:51,901 - DEBUG
[SyncThread:0:FinalRequestProcessor@78] - Processing request::
sessionid:SESSION_0 type:ping cxid:0xfffffffffffffffe
zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
2011-02-11 14:18:51,901 - DEBUG
[SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_0
type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
txntype:unknown reqpath:n/a
2011-02-11 14:18:52,005 - DEBUG
[SyncThread:0:FinalRequestProcessor@78] - Processing request::
sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
2011-02-11 14:18:52,005 - DEBUG
[SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
txntype:unknown reqpath:n/a
2011-02-11 14:18:52,339 - DEBUG
[SyncThread:0:FinalRequestProcessor@78] - Processing request::
sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
2011-02-11 14:18:52,339 - DEBUG
[SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
txntype:unknown reqpath:n/a
2011-02-11 14:18:52,672 - DEBUG
[SyncThread:0:FinalRequestProcessor@78] - Processing request::
sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
2011-02-11 14:18:52,672 - DEBUG
[SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
txntype:unknown reqpath:n/a
2011-02-11 14:18:53,002 - INFO  [SessionTracker:ZooKeeperServer@314] -
Expiring session SESSION_0, timeout of 1000ms exceeded
2011-02-11 14:18:53,002 - INFO
[ProcessThread:-1:PrepRequestProcessor@387] - Processed session
termination for sessionid: SESSION_0
2011-02-11 14:18:53,010 - DEBUG
[SyncThread:0:FinalRequestProcessor@78] - Processing request::
sessionid:SESSION_0 type:closeSession cxid:0x0 zxid:0x103 txntype:-11
reqpath:n/a
2011-02-11 14:18:53,010 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
Closed socket connection for client /HOST_0:34618 which had sessionid
SESSION_0
2011-02-11 14:18:54,502 - INFO  [SessionTracker:ZooKeeperServer@314] -
Expiring session SESSION_1, timeout of 1000ms exceeded
2011-02-11 14:18:54,502 - INFO
[ProcessThread:-1:PrepRequestProcessor@387] - Processed session
termination for sessionid: SESSION_1
2011-02-11 14:18:55,011 - DEBUG
[SyncThread:0:FinalRequestProcessor@78] - Processing request::
sessionid:SESSION_1 type:ping cxid:0xfffffffffffffffe
zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a
2011-02-11 14:18:55,011 - DEBUG
[SyncThread:0:FinalRequestProcessor@160] - sessionid:SESSION_1
type:ping cxid:0xfffffffffffffffe zxid:0xfffffffffffffffe
txntype:unknown reqpath:n/a
2011-02-11 14:18:55,011 - INFO
[NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@251]
- Accepted socket connection from /HOST_1:40556
2011-02-11 14:18:55,019 - DEBUG
[SyncThread:0:FinalRequestProcessor@78] - Processing request::
sessionid:SESSION_1 type:closeSession cxid:0x0 zxid:0x104 txntype:-11
reqpath:n/a
2011-02-11 14:18:55,019 - DEBUG
[NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:ZooKeeperServer@590] -
Dropping request: No session with sessionid SESSION_1 exists, probably
expired and removed
2011-02-11 14:18:55,019 - INFO  [SyncThread:0:NIOServerCnxn@1435] -
Closed socket connection for client /HOST_1:40555 which had sessionid
SESSION_1
2011-02-11 14:18:55,020 - DEBUG [SyncThread:0:NIOServerCnxn@1451] -
ignoring exception during output shutdown
java.net.SocketException: Transport endpoint is not connected
	at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
	at sun.nio.ch.SocketChannelImpl.shutdownOutput(SocketChannelImpl.java:651)
	at sun.nio.ch.SocketAdaptor.shutdownOutput(SocketAdaptor.java:368)
	at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1447)
	at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
	at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
	at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
	at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
	at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
	at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
2011-02-11 14:18:55,021 - DEBUG [SyncThread:0:NIOServerCnxn@1459] -
ignoring exception during input shutdown
java.net.SocketException: Transport endpoint is not connected
	at sun.nio.ch.SocketChannelImpl.shutdown(Native Method)
	at sun.nio.ch.SocketChannelImpl.shutdownInput(SocketChannelImpl.java:640)
	at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:360)
	at org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1455)
	at org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:1412)
	at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSessionWithoutWakeup(NIOServerCnxn.java:343)
	at org.apache.zookeeper.server.NIOServerCnxn$Factory.closeSession(NIOServerCnxn.java:330)
	at org.apache.zookeeper.server.FinalRequestProcessor.processRequest(FinalRequestProcessor.java:133)
	at org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:161)
	at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:98)
2011-02-11 14:18:55,022 - WARN
[NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn$Factory@272]
- Ignoring unexpected runtime exception
java.nio.channels.CancelledKeyException
	at sun.nio.ch.SelectionKeyImpl.ensureValid(SelectionKeyImpl.java:55)
	at sun.nio.ch.SelectionKeyImpl.readyOps(SelectionKeyImpl.java:69)
	at org.apache.zookeeper.server.NIOServerCnxn$Factory.run(NIOServerCnxn.java:241)
2011-02-11 14:18:55,023 - DEBUG
[NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@735] -
Session establishment request from client /HOST_1:40556 client's
lastZxid is 0x0
2011-02-11 14:18:55,023 - INFO
[NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@770] - Client
attempting to renew session SESSION_1 at /HOST_1:40556
2011-02-11 14:18:55,024 - INFO
[NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1573] -
Invalid session SESSION_1 for client /HOST_1:40556, probably expired
2011-02-11 14:18:55,025 - WARN
[NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@634] -
EndOfStreamException: Unable to read additional data from client
sessionid SESSION_1, likely client has closed socket
2011-02-11 14:18:55,025 - INFO
[NIOServerCxn.Factory:0.0.0.0/0.0.0.0:4799:NIOServerCnxn@1435] -
Closed socket connection for client /HOST_1:40556 which had sessionid
SESSION_1


On Thu, Feb 10, 2011 at 8:11 PM, Benjamin Reed <br...@yahoo-inc.com> wrote:
> do you see anything in the server log?
>
> ben
>
> On 02/10/2011 03:16 PM, Patania, Nick wrote:
>>
>> Hello,
>>
>> I run the following test:
>>
>>
>> *         Two clients connect to a zookeeper server; the tickTime on the
>> server is 500, and the sessionTimeout on the client is 1000.
>>
>> *         Kill the host running one of the clients.
>>
>> *         The second client receives a session timeout.
>>
>> Is there any reason why this might happen?
>>
>> Thanks
>> Nick Patania
>>
>>
>> --------------------------------------------------------------------------
>> NOTICE: Morgan Stanley is not acting as a municipal advisor and the
>> opinions or views contained herein are not intended to be, and do not
>> constitute, advice within the meaning of Section 975 of the Dodd-Frank Wall
>> Street Reform and Consumer Protection Act. If you have received this
>> communication in error, please destroy all electronic and paper copies and
>> notify the sender immediately. Mistransmission is not intended to waive
>> confidentiality or privilege. Morgan Stanley reserves the right, to the
>> extent permitted under applicable law, to monitor electronic communications.
>> This message is subject to terms available at the following link:
>> http://www.morganstanley.com/disclaimers. If you cannot access these links,
>> please notify us by reply message and we will send the contents to you. By
>> messaging with Morgan Stanley you consent to the foregoing.
>
>

Re: tickTime and sessionTimeout

Posted by Benjamin Reed <br...@yahoo-inc.com>.
do you see anything in the server log?

ben

On 02/10/2011 03:16 PM, Patania, Nick wrote:
> Hello,
>
> I run the following test:
>
>
> *         Two clients connect to a zookeeper server; the tickTime on the server is 500, and the sessionTimeout on the client is 1000.
>
> *         Kill the host running one of the clients.
>
> *         The second client receives a session timeout.
>
> Is there any reason why this might happen?
>
> Thanks
> Nick Patania
>
>
> --------------------------------------------------------------------------
> NOTICE: Morgan Stanley is not acting as a municipal advisor and the opinions or views contained herein are not intended to be, and do not constitute, advice within the meaning of Section 975 of the Dodd-Frank Wall Street Reform and Consumer Protection Act. If you have received this communication in error, please destroy all electronic and paper copies and notify the sender immediately. Mistransmission is not intended to waive confidentiality or privilege. Morgan Stanley reserves the right, to the extent permitted under applicable law, to monitor electronic communications. This message is subject to terms available at the following link: http://www.morganstanley.com/disclaimers. If you cannot access these links, please notify us by reply message and we will send the contents to you. By messaging with Morgan Stanley you consent to the foregoing.