Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 3E9E6200D2E for ; Tue, 31 Oct 2017 21:26:53 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 3D2431609EF; Tue, 31 Oct 2017 20:26:53 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 824EB1609E6 for ; Tue, 31 Oct 2017 21:26:52 +0100 (CET) Received: (qmail 58761 invoked by uid 500); 31 Oct 2017 20:26:51 -0000 Mailing-List: contact dev-help@zookeeper.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@zookeeper.apache.org Delivered-To: mailing list dev@zookeeper.apache.org Received: (qmail 58750 invoked by uid 99); 31 Oct 2017 20:26:51 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 31 Oct 2017 20:26:51 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 3A4C3DFAE0; Tue, 31 Oct 2017 20:26:51 +0000 (UTC) From: shralex To: dev@zookeeper.apache.org Reply-To: dev@zookeeper.apache.org References: In-Reply-To: Subject: [GitHub] zookeeper pull request #411: ZOOKEEPER-2684 Fix a crashing bug in the mixed ... Content-Type: text/plain Message-Id: <20171031202651.3A4C3DFAE0@git1-us-west.apache.org> Date: Tue, 31 Oct 2017 20:26:51 +0000 (UTC) archived-at: Tue, 31 Oct 2017 20:26:53 -0000 Github user shralex commented on a diff in the pull request: https://github.com/apache/zookeeper/pull/411#discussion_r148117464 --- Diff: src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java --- @@ -246,33 +246,51 @@ public void run() { } /* - * Check if request is pending, if so, update it with the - * committed info + * Check if request is pending, if so, update it with the committed info */ LinkedList sessionQueue = pendingRequests .get(request.sessionId); if (sessionQueue != null) { // If session queue != null, then it is also not empty. Request topPending = sessionQueue.poll(); if (request.cxid != topPending.cxid) { - LOG.error( - "Got cxid 0x" - + Long.toHexString(request.cxid) - + " expected 0x" + Long.toHexString( - topPending.cxid) - + " for client session id " - + Long.toHexString(request.sessionId)); - throw new IOException("Error: unexpected cxid for" - + "client session"); + // TL;DR - we should not encounter this scenario often under normal load. + // We pass the commit to the next processor and put the pending back with a warning. + + // Generally, we can get commit requests that are not at the queue head after + // a session moved (see ZOOKEEPER-2684). Let's denote the previous server of the session + // with A, and the server that the session moved to with B (keep in mind that it is + // possible that the session already moved from B to a new server C, and maybe C=A). + // 1. If request.cxid < topPending.cxid : this means that the session requested this update + // from A, then moved to B (i.e., which is us), and now B receives the commit + // for the update after the session already performed several operations in B + // (and therefore its cxid is higher than that old request). + // 2. If request.cxid > topPending.cxid : this means that the session requested an updated + // from B with cxid that is bigger than the one we know therefore in this case we + // are A, and we lost the connection to the session. Given that we are waiting for a commit + // for that update, it means that we already sent the request to the leader and it will + // be committed at some point (in this case the order of cxid won't follow zxid, since zxid + // is an increasing order). It is not safe for us to delete the session's queue at this + // point, since it is possible that the session has newer requests in it after it moved + // back to us. We just leave the queue as it is, and once the commit arrives (for the old + // request), the finalRequestProcessor will see a closed cnxn handle, and just won't send a + // response. + // Also note that we don't have a local session, therefore we treat the request + // like any other commit for a remote request, i.e., we perform the update without sending + // a response. + + LOG.warn("Got request " + request + + " but we are expecting request " + topPending); + sessionQueue.addFirst(topPending); + } else { + // We want to send to the next processor our version of the request, + // since it contains the session information that is needed + // for post update processing (e.g., using request.cnxn we send a response to the client). + topPending.setHdr(request.getHdr()); --- End diff -- My understanding is that when a request is in the local queue, there is (or could be) a client attached to this server waiting for a response, and there is other bookeeping of requests that are outstanding and have originated from this server (e.g., for setting the max outstanding requests) - we need to update this info when an outstanding request completes. In the other case, the operation originated from a different server and there is no local bookeeeping or a local client session that needs to be notified. ---