Quellcode durchsuchen

ZOOKEEPER-2913: testEphemeralNodeDeletion is flaky

- The test code is assuming the follower is up to date, which without a `sync` call.when regression,the follower could not catch up with the leader,so the `/e1` patch still exists
- this issue is not very easy to reproduce,but we can change the codes in the `FollowerZooKeeperServer.commit() ` ,just like this:

```
       Request request = pendingTxns.remove();
        if (request.getHdr().getType() == -11) {
        	try {
    			Thread.sleep(200);
    		} catch (InterruptedException e) {

    		}
        }
        commitProcessor.commit(request);
```
to slow down the commit to simulate the flaky test,then we can check the correctness of this patch
- ----------------------------------------**appendix: zk.sync() doc:**------------------------------------------

> ZooKeeper does not guarantee that at every instance in time, two different clients will have identical views of ZooKeeper data. Due to factors like network delays, one client may perform an update before another client gets notified of the change. Consider the scenario of two clients, A and B. If client A sets the value of a znode /a from 0 to 1, then tells client B to read /a, client B may read the old value of 0, depending on which server it is connected to. If it is important that Client A and Client B read the same value, Client B should should call the sync() method from the ZooKeeper API method before it performs its read.

- more detais in [ZOOKEEPER-2913](https://issues.apache.org/jira/browse/ZOOKEEPER-2913)

Author: maoling <maoling199210191@sina.com>

Reviewers: andor@apache.org

Closes #608 from maoling/ZOOKEEPER-2913
maoling vor 6 Jahren
Ursprung
Commit
e7ac12c952

+ 17 - 1
src/java/test/org/apache/zookeeper/server/quorum/EphemeralNodeDeletionTest.java

@@ -24,7 +24,10 @@ import static org.junit.Assert.assertNull;
 
 import java.io.IOException;
 import java.net.SocketTimeoutException;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 
+import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.ZooDefs.Ids;
@@ -147,7 +150,11 @@ public class EphemeralNodeDeletionTest extends QuorumPeerTestBase {
 
         // close the session and newly created ephemeral node should be deleted
         zk.close();
-
+        
+        SyncCallback cb = new SyncCallback();
+        followerZK.sync(nodePath, cb, null);
+        cb.sync.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS);
+        
         nodeAtFollower = followerZK.exists(nodePath, false);
 
         // Problem 2: Before fix, after session close the ephemeral node
@@ -225,4 +232,13 @@ public class EphemeralNodeDeletionTest extends QuorumPeerTestBase {
             return new CustomQuorumPeer();
         }
     }
+    
+    private static class SyncCallback implements AsyncCallback.VoidCallback {
+        private final CountDownLatch sync = new CountDownLatch(1);
+        
+        @Override
+        public void processResult(int rc, String path, Object ctx) {
+        	sync.countDown();
+        }
+    }
 }