You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ha...@apache.org on 2019/07/03 18:54:24 UTC

[zookeeper] branch master updated: ZOOKEEPER-2418: txnlog diff sync can skip sending some transactions t…

This is an automated email from the ASF dual-hosted git repository.

hanm pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/zookeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new 4cadbb1  ZOOKEEPER-2418: txnlog diff sync can skip sending some transactions t…
4cadbb1 is described below

commit 4cadbb1a649b70a2243bc4c1e5f736df4d35c462
Author: Brian Nixon <ni...@fb.com>
AuthorDate: Wed Jul 3 11:54:12 2019 -0700

    ZOOKEEPER-2418: txnlog diff sync can skip sending some transactions t…
    
    …o followers
    
    Author: Brian Nixon <ni...@fb.com>
    
    Reviewers: Michael Han <ha...@apache.org>
    
    Closes #972 from enixon/diff-sync-skip
---
 .../zookeeper/server/quorum/LearnerHandler.java    | 23 +++++--
 .../server/quorum/LearnerHandlerTest.java          | 76 +++++++---------------
 2 files changed, 40 insertions(+), 59 deletions(-)

diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerHandler.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerHandler.java
index a3d5a4d..c33fa04 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerHandler.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerHandler.java
@@ -852,11 +852,24 @@ public class LearnerHandler extends ZooKeeperThread {
                     currentZxid = queueCommittedProposals(txnLogItr, peerLastZxid,
                                                          minCommittedLog, maxCommittedLog);
 
-                    LOG.debug("Queueing committedLog 0x" + Long.toHexString(currentZxid));
-                    Iterator<Proposal> committedLogItr = db.getCommittedLog().iterator();
-                    currentZxid = queueCommittedProposals(committedLogItr, currentZxid,
-                                                         null, maxCommittedLog);
-                    needSnap = false;
+                    if (currentZxid < minCommittedLog) {
+                        LOG.info("Detected gap between end of txnlog: 0x{} and start of committedLog: 0x{}",
+                                Long.toHexString(currentZxid),
+                                Long.toHexString(minCommittedLog));
+                        currentZxid = peerLastZxid;
+                        // Clear out currently queued requests and revert
+                        // to sending a snapshot.
+                        queuedPackets.clear();
+                        needOpPacket = true;
+                    } else {
+                        LOG.debug("Queueing committedLog 0x{}",
+                                Long.toHexString(currentZxid));
+                        Iterator<Proposal> committedLogItr =
+                                db.getCommittedLog().iterator();
+                        currentZxid = queueCommittedProposals(committedLogItr,
+                                currentZxid, null, maxCommittedLog);
+                        needSnap = false;
+                    }
                 }
                 // closing the resources
                 if (txnLogItr instanceof TxnLogProposalIterator) {
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerHandlerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerHandlerTest.java
index 8ec83a0..f1dc7cc 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerHandlerTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerHandlerTest.java
@@ -488,60 +488,6 @@ public class LearnerHandlerTest extends ZKTestCase {
     }
 
     /**
-     * Test cases when learner has new-epcoh zxid
-     * (zxid &amp; 0xffffffffL) == 0;
-     */
-    @Test
-    public void testNewEpochZxidWithTxnlogOnly() throws Exception {
-        long peerZxid;
-        db.txnLog.add(createProposal(getZxid(1, 1)));
-        db.txnLog.add(createProposal(getZxid(2, 1)));
-        db.txnLog.add(createProposal(getZxid(2, 2)));
-        db.txnLog.add(createProposal(getZxid(4, 1)));
-
-        // After leader election, lastProcessedZxid will point to new epoch
-        db.lastProcessedZxid = getZxid(6, 0);
-
-        // Peer has zxid of epoch 3
-        peerZxid = getZxid(3, 0);
-        assertFalse(learnerHandler.syncFollower(peerZxid, leader));
-        // We send DIFF to (6,0) and forward any packet starting at (4,1)
-        assertOpType(Leader.DIFF, getZxid(6, 0), getZxid(4, 1));
-        // DIFF + 1 proposals + 1 commit
-        assertEquals(3, learnerHandler.getQueuedPackets().size());
-        queuedPacketMatches(new long[] { getZxid(4, 1)});
-        reset();
-
-        // Peer has zxid of epoch 4
-        peerZxid = getZxid(4, 0);
-        assertFalse(learnerHandler.syncFollower(peerZxid, leader));
-        // We send DIFF to (6,0) and forward any packet starting at (4,1)
-        assertOpType(Leader.DIFF, getZxid(6, 0), getZxid(4, 1));
-        // DIFF + 1 proposals + 1 commit
-        assertEquals(3, learnerHandler.getQueuedPackets().size());
-        queuedPacketMatches(new long[] { getZxid(4, 1)});
-        reset();
-
-        // Peer has zxid of epoch 5
-        peerZxid = getZxid(5, 0);
-        assertFalse(learnerHandler.syncFollower(peerZxid, leader));
-        // We send DIFF to (6,0) and forward any packet starting at (5,0)
-        assertOpType(Leader.DIFF, getZxid(6, 0), getZxid(5, 0));
-        // DIFF only
-        assertEquals(1, learnerHandler.getQueuedPackets().size());
-        reset();
-
-        // Peer has zxid of epoch 6
-        peerZxid = getZxid(6, 0);
-        assertFalse(learnerHandler.syncFollower(peerZxid, leader));
-        // We send DIFF to (6,0) and forward any packet starting at (6, 0)
-        assertOpType(Leader.DIFF, getZxid(6, 0), getZxid(6, 0));
-        // DIFF only
-        assertEquals(1, learnerHandler.getQueuedPackets().size());
-        reset();
-    }
-
-    /**
      * Test cases when there is a duplicate txn in the committedLog. This
      * should never happen unless there is a bug in initialization code
      * but the learner should never see duplicate packets
@@ -595,4 +541,26 @@ public class LearnerHandlerTest extends ZKTestCase {
         assertEquals(0, learnerHandler.getQueuedPackets().size());
         reset();
     }
+
+    /**
+     * Test cases when the leader's disk is slow. There can be a gap
+     * between the txnLog and the committedLog. Make sure we detect this
+     * and send a snap instead of a diff.
+     */
+    @Test
+    public void testTxnLogGap() throws Exception {
+        long peerZxid;
+        db.txnLog.add(createProposal(2));
+        db.txnLog.add(createProposal(3));
+        db.txnLog.add(createProposal(4));
+
+        db.lastProcessedZxid = 8;
+        db.committedLog.add(createProposal(7));
+        db.committedLog.add(createProposal(8));
+
+        // Peer zxid is in txnlog range
+        peerZxid = 3;
+        assertTrue(learnerHandler.syncFollower(peerZxid, leader));
+        reset();
+    }
 }