You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by Dutch Meyer <ho...@gmail.com> on 2013/12/10 18:07:35 UTC

Possible bug - xid mismatch, out of order responses, "unexpected server response: expected 0x..., but recieved 0x..."

I've been tracking an error we see occasionally on our cluster, we're
currently running behind trunk at build
047b07a298d84e9755c6e06c035787ce397f4958.

We've been seeing this error, it's quite rare and so far I've had no luck
reproducing it in a controlled setting.

The symptom is that C clients see errors of the form:


  ZOO_ERROR@handle_socket_error_msg@2726: Socket [10.11.13.2:2181]

  zk retcode=-2, errno=115(Operation now in progress):
  unexpected server response: expected 0x529a8be8, but received 0x529a8be6

(note the expected/received entries are reversed here, we always receive a
larger entry than we were expecting).

Kazoo clients are also failing similarly, with the error:

  zookeeper: xids do not match, expected %r received %r', 1435, 1436

Generally we see these failures in groups, where multiple clients will see
these failures from one server over a 5 or ten second windows.  Sometimes
one client can fail with the error multiple times in that period.

I'd appreciate any insight anyone can give me into why this is happening
and how we might fix it.  Has anyone seen this before?  Any hunches what
code or conditions I might investigate to reliably trigger or fix the
error?  I'd just greatly appreciate any help in identifying the problem.

-- 
-=-Dutch

Re: Possible bug - xid mismatch, out of order responses, "unexpected server response: expected 0x..., but recieved 0x..."

Posted by Dutch Meyer <ho...@gmail.com>.
Just to close the loop here - this is a race in upstream zookeeper.  I'll
open a jira ticket today.

In the commit processor, if we are at the primary request handler on line
167:

                while (!stopped && !isWaitingForCommit() &&
                       !isProcessingCommit() &&
                       (request = queuedRequests.poll()) != null) {
                    if (needCommit(request)) {
                        nextPending.set(request);
                    } else {
                        sendToNextProcessor(request);
                    }
                }

A request can be handled in this block and be quickly processed and
completed on another thread. If queuedRequests is empty, we then exit the
block. Next, before this thread makes any more progress, we can get 2 more
requests, one get_children(say), and a sync placed on queuedRequests for
the processor. Then, if we are very unlucky, the sync request can complete
and this object's commit() routine is called (from
FollowerZookeeperServer), which places the sync request on the previously
empty committedRequests queue. At that point, this thread continues.

We reach line 182, which is a check on sync requests.

                if (!stopped && !isProcessingRequest() &&
                    (request = committedRequests.poll()) != null) {

Here we are not processing any requests, because the original request above
has completed. We haven't dequeued either the read or the sync request in
this processor. Next, the poll above will pull the sync request off the
queue, and in the following block, the sync will get forwarded to the next
processor.

This is a problem because the read request hasn't been forwarded yet, so
requests are now out of order.

--Dutch

On Mon, Jan 13, 2014 at 3:16 PM, Dutch Meyer <ho...@gmail.com> wrote:

> I have another instance of this issue while running at DEBUG trace level.
>
>
> ----------------------------------------------------------------------------------------------------------------------------
> 2014-01-01 12:55:12,278 - WARN  [NIOWorkerThread-4:NIOServerCnxn@362][] -
> Unable to read additional data from client sessionid 0x200000209eb0002,
> likely client has closed socket
> 2014-01-01 12:55:12,278 - INFO  [NIOWorkerThread-4:NIOServerCnxn@1000][]
> - Closed socket connection for client /10.11.19.2:54313 which had
> sessionid 0x200000209eb0002
> 2014-01-01 12:55:12,279 - DEBUG [NIOWorkerThread-4:NIOServerCnxn@1036][]
> - 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:658)
>         at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:378)
>         at
> org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1032)
>         at
> org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1005)
>         at
> org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:989)
>         at
> org.apache.zookeeper.server.NIOServerCnxn.doIO(NIOServerCnxn.java:364)
>         at
> org.apache.zookeeper.server.NIOServerCnxnFactory$IOWorkRequest.doWork(NIOServerCnxnFactory.java:530)
>         at
> org.apache.zookeeper.server.WorkerService$ScheduledWorkRequest.run(WorkerService.java:152)
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>         at java.lang.Thread.run(Thread.java:679)
> 2014-01-01 12:55:12,799 - INFO
>  [NIOServerCxnFactory.AcceptThread:/10.11.19.2:2181
> :NIOServerCnxnFactory$AcceptThread@296][] - Accepted socket connection
> from /10.11.19.2:54326
> 2014-01-01 12:55:12,799 - DEBUG [NIOWorkerThread-1:ZooKeeperServer@761][]
> - Session establishment request from client /10.11.19.2:54326 client's
> lastZxid is 0x2000003b3
> 2014-01-01 12:55:12,800 - INFO  [NIOWorkerThread-1:ZooKeeperServer@816][]
> - Client attempting to renew session 0x200000209eb0002 at /
> 10.11.19.2:54326
> 2014-01-01 12:55:12,800 - INFO  [NIOWorkerThread-1:Learner@114][] -
> Revalidating client: 0x200000209eb0002
> 2014-01-01 12:55:12,802 - INFO
>  [QuorumPeer[myid=3175620633077743617]/10.11.19.2:2181:ZooKeeperServer@567][]
> - Established session 0x200000209eb0002 with negotiated timeout 10000 for
> client /10.11.19.2:54326
>
> ...
>
> 2014-01-01 12:55:14,291 - DEBUG
> [FollowerRequestProcessor:3175620633077743617:CommitProcessor@315][] -
> Processing request:: sessionid:0x200000209eb0002 type:getChildren cxid:0x1
> zxid:0xfffffffffffffffe txntype:unknown reqpath:/j/l/m
>
> 2014-01-01 12:55:14,298 - DEBUG
> [FollowerRequestProcessor:3175620633077743617:CommitProcessor@315][] -
> Processing request:: sessionid:0x200000209eb0002 type:sync: cxid:0x2
> zxid:0xfffffffffffffffe txntype:unknown reqpath:/j/k
>
> 2014-01-01 12:55:14,299 - DEBUG
> [FollowerRequestProcessor:3175620633077743617:CommitProcessor@315][] -
> Processing request:: sessionid:0x200000209eb0002 type:getChildren cxid:0x3
> zxid:0xfffffffffffffffe txntype:unknown reqpath:/a/b/c/d/e/f/g/h.i
>
> 2014-01-01 12:55:14,300 - DEBUG
> [QuorumPeer[myid=3175620633077743617]/10.11.19.2:2181:CommitProcessor@302][]
> - Committing request:: sessionid:0x200000209eb0002 type:sync: cxid:0x2
> zxid:0xfffffffffffffffe txntype:unknown reqpath:/j/k
>
> 2014-01-01 12:55:14,306 - DEBUG
> [CommitProcWorkThread-1:FinalRequestProcessor@88][] - Processing
> request:: sessionid:0x200000209eb0002 type:sync: cxid:0x2
> zxid:0xfffffffffffffffe txntype:unknown reqpath:/j/k
>
> 2014-01-01 12:55:14,307 - DEBUG
> [CommitProcWorkThread-1:FinalRequestProcessor@159][] -
> sessionid:0x200000209eb0002 type:sync: cxid:0x2 zxid:0xfffffffffffffffe
> txntype:unknown reqpath:/j/k
>
> ----------------------------------------------------------------------------------------------------------------------------
>
> Leading to, on the (remote) client:
>
> ----------------------------------------------------------------------------------------------------------------------------
> Jan  1 04:54:49 myhost 2014-01-01 04:54:49,310 MainProcess: ERROR:pid
> 677:tid 140289981269760:kazoo.protocol.connection:('xids do not match,
> expected %r received %r', 1, 2)#012Traceback (most recent call last):#012
>  File "/usr/lib/python2.7/dist-packages/kazoo/protocol/connection.py", line
> 512, in _connect_loop#012    response = self._read_socket(read_timeout)#012
>  File "/usr/lib/python2.7/dist-packages/kazoo/protocol/connection.py", line
> 397, in _read_socket#012    return self._read_response(header, buffer,
> offset)#012  File
> "/usr/lib/python2.7/dist-packages/kazoo/protocol/connection.py", line 322,
> in _read_response#012    'received %r', xid, header.xid)#012RuntimeError:
> ('xids do not match, expected %r received %r', 1, 2)
>
> ----------------------------------------------------------------------------------------------------------------------------
>
> I'm confused as to how the sync (cxid==0x2) can apparently hit the final
> request processor before get_children (cxid==0x1) does.  If anyone has any
> insight (or guesses) into this problem I'd be very curious to hear your
> thoughts.   I'd like to push this through to a patch, if appropriate, but I
> don't see the root cause here.
>
> Thanks,
> --Dutch
>
>
>
>
> On Tue, Dec 10, 2013 at 10:04 AM, Dutch Meyer <ho...@gmail.com> wrote:
>
>> No authentication.
>>
>>
>> On Tue, Dec 10, 2013 at 9:54 AM, Raúl Gutiérrez Segalés <
>> rgs@itevenworks.net> wrote:
>>
>>> Hi,
>>>
>>> On 10 December 2013 09:07, Dutch Meyer <ho...@gmail.com> wrote:
>>>
>>>> I've been tracking an error we see occasionally on our cluster, we're
>>>> currently running behind trunk at build
>>>> 047b07a298d84e9755c6e06c035787ce397f4958.
>>>>
>>>> We've been seeing this error, it's quite rare and so far I've had no
>>>> luck
>>>> reproducing it in a controlled setting.
>>>>
>>>> The symptom is that C clients see errors of the form:
>>>>
>>>>
>>>>   ZOO_ERROR@handle_socket_error_msg@2726: Socket [10.11.13.2:2181]
>>>>
>>>>   zk retcode=-2, errno=115(Operation now in progress):
>>>>   unexpected server response: expected 0x529a8be8, but received
>>>> 0x529a8be6
>>>>
>>>> (note the expected/received entries are reversed here, we always
>>>> receive a
>>>> larger entry than we were expecting).
>>>>
>>>> Kazoo clients are also failing similarly, with the error:
>>>>
>>>>   zookeeper: xids do not match, expected %r received %r', 1435, 1436
>>>>
>>>> Generally we see these failures in groups, where multiple clients will
>>>> see
>>>> these failures from one server over a 5 or ten second windows.
>>>>  Sometimes
>>>> one client can fail with the error multiple times in that period.
>>>>
>>>> I'd appreciate any insight anyone can give me into why this is happening
>>>> and how we might fix it.  Has anyone seen this before?  Any hunches what
>>>> code or conditions I might investigate to reliably trigger or fix the
>>>> error?  I'd just greatly appreciate any help in identifying the problem.
>>>>
>>>
>>> Are you using authentication? I wonder if your read/write ops are racing
>>> with your add_auth calls
>>>  which would cause the out of order xids.
>>>
>>>
>>> -rgs
>>>
>>
>>
>>
>> --
>> -=-Dutch
>>
>
>
>
> --
> -=-Dutch
>



-- 
-=-Dutch

Re: Possible bug - xid mismatch, out of order responses, "unexpected server response: expected 0x..., but recieved 0x..."

Posted by Dutch Meyer <ho...@gmail.com>.
I have another instance of this issue while running at DEBUG trace level.

----------------------------------------------------------------------------------------------------------------------------
2014-01-01 12:55:12,278 - WARN  [NIOWorkerThread-4:NIOServerCnxn@362][] -
Unable to read additional data from client sessionid 0x200000209eb0002,
likely client has closed socket
2014-01-01 12:55:12,278 - INFO  [NIOWorkerThread-4:NIOServerCnxn@1000][] -
Closed socket connection for client /10.11.19.2:54313 which had sessionid
0x200000209eb0002
2014-01-01 12:55:12,279 - DEBUG [NIOWorkerThread-4:NIOServerCnxn@1036][] -
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:658)
        at sun.nio.ch.SocketAdaptor.shutdownInput(SocketAdaptor.java:378)
        at
org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1032)
        at
org.apache.zookeeper.server.NIOServerCnxn.closeSock(NIOServerCnxn.java:1005)
        at
org.apache.zookeeper.server.NIOServerCnxn.close(NIOServerCnxn.java:989)
        at
org.apache.zookeeper.server.NIOServerCnxn.doIO(NIOServerCnxn.java:364)
        at
org.apache.zookeeper.server.NIOServerCnxnFactory$IOWorkRequest.doWork(NIOServerCnxnFactory.java:530)
        at
org.apache.zookeeper.server.WorkerService$ScheduledWorkRequest.run(WorkerService.java:152)
        at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
        at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
        at java.lang.Thread.run(Thread.java:679)
2014-01-01 12:55:12,799 - INFO
 [NIOServerCxnFactory.AcceptThread:/10.11.19.2:2181
:NIOServerCnxnFactory$AcceptThread@296][] - Accepted socket connection from
/10.11.19.2:54326
2014-01-01 12:55:12,799 - DEBUG [NIOWorkerThread-1:ZooKeeperServer@761][] -
Session establishment request from client /10.11.19.2:54326 client's
lastZxid is 0x2000003b3
2014-01-01 12:55:12,800 - INFO  [NIOWorkerThread-1:ZooKeeperServer@816][] -
Client attempting to renew session 0x200000209eb0002 at /10.11.19.2:54326
2014-01-01 12:55:12,800 - INFO  [NIOWorkerThread-1:Learner@114][] -
Revalidating client: 0x200000209eb0002
2014-01-01 12:55:12,802 - INFO
 [QuorumPeer[myid=3175620633077743617]/10.11.19.2:2181:ZooKeeperServer@567][]
- Established session 0x200000209eb0002 with negotiated timeout 10000 for
client /10.11.19.2:54326

...

2014-01-01 12:55:14,291 - DEBUG
[FollowerRequestProcessor:3175620633077743617:CommitProcessor@315][] -
Processing request:: sessionid:0x200000209eb0002 type:getChildren cxid:0x1
zxid:0xfffffffffffffffe txntype:unknown reqpath:/j/l/m

2014-01-01 12:55:14,298 - DEBUG
[FollowerRequestProcessor:3175620633077743617:CommitProcessor@315][] -
Processing request:: sessionid:0x200000209eb0002 type:sync: cxid:0x2
zxid:0xfffffffffffffffe txntype:unknown reqpath:/j/k

2014-01-01 12:55:14,299 - DEBUG
[FollowerRequestProcessor:3175620633077743617:CommitProcessor@315][] -
Processing request:: sessionid:0x200000209eb0002 type:getChildren cxid:0x3
zxid:0xfffffffffffffffe txntype:unknown reqpath:/a/b/c/d/e/f/g/h.i

2014-01-01 12:55:14,300 - DEBUG
[QuorumPeer[myid=3175620633077743617]/10.11.19.2:2181:CommitProcessor@302][]
- Committing request:: sessionid:0x200000209eb0002 type:sync: cxid:0x2
zxid:0xfffffffffffffffe txntype:unknown reqpath:/j/k

2014-01-01 12:55:14,306 - DEBUG
[CommitProcWorkThread-1:FinalRequestProcessor@88][] - Processing request::
sessionid:0x200000209eb0002 type:sync: cxid:0x2 zxid:0xfffffffffffffffe
txntype:unknown reqpath:/j/k

2014-01-01 12:55:14,307 - DEBUG
[CommitProcWorkThread-1:FinalRequestProcessor@159][] -
sessionid:0x200000209eb0002 type:sync: cxid:0x2 zxid:0xfffffffffffffffe
txntype:unknown reqpath:/j/k
----------------------------------------------------------------------------------------------------------------------------

Leading to, on the (remote) client:
----------------------------------------------------------------------------------------------------------------------------
Jan  1 04:54:49 myhost 2014-01-01 04:54:49,310 MainProcess: ERROR:pid
677:tid 140289981269760:kazoo.protocol.connection:('xids do not match,
expected %r received %r', 1, 2)#012Traceback (most recent call last):#012
 File "/usr/lib/python2.7/dist-packages/kazoo/protocol/connection.py", line
512, in _connect_loop#012    response = self._read_socket(read_timeout)#012
 File "/usr/lib/python2.7/dist-packages/kazoo/protocol/connection.py", line
397, in _read_socket#012    return self._read_response(header, buffer,
offset)#012  File
"/usr/lib/python2.7/dist-packages/kazoo/protocol/connection.py", line 322,
in _read_response#012    'received %r', xid, header.xid)#012RuntimeError:
('xids do not match, expected %r received %r', 1, 2)
----------------------------------------------------------------------------------------------------------------------------

I'm confused as to how the sync (cxid==0x2) can apparently hit the final
request processor before get_children (cxid==0x1) does.  If anyone has any
insight (or guesses) into this problem I'd be very curious to hear your
thoughts.   I'd like to push this through to a patch, if appropriate, but I
don't see the root cause here.

Thanks,
--Dutch




On Tue, Dec 10, 2013 at 10:04 AM, Dutch Meyer <ho...@gmail.com> wrote:

> No authentication.
>
>
> On Tue, Dec 10, 2013 at 9:54 AM, Raúl Gutiérrez Segalés <
> rgs@itevenworks.net> wrote:
>
>> Hi,
>>
>> On 10 December 2013 09:07, Dutch Meyer <ho...@gmail.com> wrote:
>>
>>> I've been tracking an error we see occasionally on our cluster, we're
>>> currently running behind trunk at build
>>> 047b07a298d84e9755c6e06c035787ce397f4958.
>>>
>>> We've been seeing this error, it's quite rare and so far I've had no luck
>>> reproducing it in a controlled setting.
>>>
>>> The symptom is that C clients see errors of the form:
>>>
>>>
>>>   ZOO_ERROR@handle_socket_error_msg@2726: Socket [10.11.13.2:2181]
>>>
>>>   zk retcode=-2, errno=115(Operation now in progress):
>>>   unexpected server response: expected 0x529a8be8, but received
>>> 0x529a8be6
>>>
>>> (note the expected/received entries are reversed here, we always receive
>>> a
>>> larger entry than we were expecting).
>>>
>>> Kazoo clients are also failing similarly, with the error:
>>>
>>>   zookeeper: xids do not match, expected %r received %r', 1435, 1436
>>>
>>> Generally we see these failures in groups, where multiple clients will
>>> see
>>> these failures from one server over a 5 or ten second windows.  Sometimes
>>> one client can fail with the error multiple times in that period.
>>>
>>> I'd appreciate any insight anyone can give me into why this is happening
>>> and how we might fix it.  Has anyone seen this before?  Any hunches what
>>> code or conditions I might investigate to reliably trigger or fix the
>>> error?  I'd just greatly appreciate any help in identifying the problem.
>>>
>>
>> Are you using authentication? I wonder if your read/write ops are racing
>> with your add_auth calls
>>  which would cause the out of order xids.
>>
>>
>> -rgs
>>
>
>
>
> --
> -=-Dutch
>



-- 
-=-Dutch

Re: Possible bug - xid mismatch, out of order responses, "unexpected server response: expected 0x..., but recieved 0x..."

Posted by Dutch Meyer <ho...@gmail.com>.
No authentication.


On Tue, Dec 10, 2013 at 9:54 AM, Raúl Gutiérrez Segalés <rgs@itevenworks.net
> wrote:

> Hi,
>
> On 10 December 2013 09:07, Dutch Meyer <ho...@gmail.com> wrote:
>
>> I've been tracking an error we see occasionally on our cluster, we're
>> currently running behind trunk at build
>> 047b07a298d84e9755c6e06c035787ce397f4958.
>>
>> We've been seeing this error, it's quite rare and so far I've had no luck
>> reproducing it in a controlled setting.
>>
>> The symptom is that C clients see errors of the form:
>>
>>
>>   ZOO_ERROR@handle_socket_error_msg@2726: Socket [10.11.13.2:2181]
>>
>>   zk retcode=-2, errno=115(Operation now in progress):
>>   unexpected server response: expected 0x529a8be8, but received 0x529a8be6
>>
>> (note the expected/received entries are reversed here, we always receive a
>> larger entry than we were expecting).
>>
>> Kazoo clients are also failing similarly, with the error:
>>
>>   zookeeper: xids do not match, expected %r received %r', 1435, 1436
>>
>> Generally we see these failures in groups, where multiple clients will see
>> these failures from one server over a 5 or ten second windows.  Sometimes
>> one client can fail with the error multiple times in that period.
>>
>> I'd appreciate any insight anyone can give me into why this is happening
>> and how we might fix it.  Has anyone seen this before?  Any hunches what
>> code or conditions I might investigate to reliably trigger or fix the
>> error?  I'd just greatly appreciate any help in identifying the problem.
>>
>
> Are you using authentication? I wonder if your read/write ops are racing
> with your add_auth calls
> which would cause the out of order xids.
>
>
> -rgs
>



-- 
-=-Dutch

Re: Possible bug - xid mismatch, out of order responses, "unexpected server response: expected 0x..., but recieved 0x..."

Posted by Raúl Gutiérrez Segalés <rg...@itevenworks.net>.
Hi,

On 10 December 2013 09:07, Dutch Meyer <ho...@gmail.com> wrote:

> I've been tracking an error we see occasionally on our cluster, we're
> currently running behind trunk at build
> 047b07a298d84e9755c6e06c035787ce397f4958.
>
> We've been seeing this error, it's quite rare and so far I've had no luck
> reproducing it in a controlled setting.
>
> The symptom is that C clients see errors of the form:
>
>
>   ZOO_ERROR@handle_socket_error_msg@2726: Socket [10.11.13.2:2181]
>
>   zk retcode=-2, errno=115(Operation now in progress):
>   unexpected server response: expected 0x529a8be8, but received 0x529a8be6
>
> (note the expected/received entries are reversed here, we always receive a
> larger entry than we were expecting).
>
> Kazoo clients are also failing similarly, with the error:
>
>   zookeeper: xids do not match, expected %r received %r', 1435, 1436
>
> Generally we see these failures in groups, where multiple clients will see
> these failures from one server over a 5 or ten second windows.  Sometimes
> one client can fail with the error multiple times in that period.
>
> I'd appreciate any insight anyone can give me into why this is happening
> and how we might fix it.  Has anyone seen this before?  Any hunches what
> code or conditions I might investigate to reliably trigger or fix the
> error?  I'd just greatly appreciate any help in identifying the problem.
>

Are you using authentication? I wonder if your read/write ops are racing
with your add_auth calls
which would cause the out of order xids.


-rgs