You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@curator.apache.org by ra...@apache.org on 2020/04/02 13:17:03 UTC

[curator] branch CURATOR-525-fix-lost-state-race updated (9ff0190 -> d907766)

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

randgalt pushed a change to branch CURATOR-525-fix-lost-state-race
in repository https://gitbox.apache.org/repos/asf/curator.git.


    omit 9ff0190  CURATOR-525
     add 7f9e9cc  CURATOR-549 (#335)
     new ba83de1  CURATOR-525
     new d907766  CURATOR-525 - instead of resetting the connection, change the state to RECONNECTED. I'm concerned about LOST/reset loops. This is still a bad hack and needs to be addressed in the future.

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (9ff0190)
            \
             N -- N -- N   refs/heads/CURATOR-525-fix-lost-state-race (d907766)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../src/main/java/cache/CuratorCacheExample.java   |  92 +++++
 .../src/site/confluence/index.confluence           |   2 +-
 .../framework/state/ConnectionStateManager.java    |   2 +-
 .../framework/recipes/cache/CuratorCache.java      | 136 ++++++++
 .../recipes/cache/CuratorCacheAccessor.java        |  75 +++++
 .../recipes/cache/CuratorCacheBuilder.java         |  54 +++
 .../recipes/cache/CuratorCacheBuilderImpl.java     |  65 ++++
 .../framework/recipes/cache/CuratorCacheImpl.java  | 301 +++++++++++++++++
 .../recipes/cache/CuratorCacheListener.java        |  78 +++++
 .../recipes/cache/CuratorCacheListenerBuilder.java | 129 +++++++
 .../cache/CuratorCacheListenerBuilderImpl.java     | 161 +++++++++
 .../recipes/cache/CuratorCacheStorage.java         |  88 +++++
 .../curator/framework/recipes/cache/NodeCache.java |   3 +
 ...peration.java => NodeCacheListenerWrapper.java} |  35 +-
 .../{EventOperation.java => OutstandingOps.java}   |  38 ++-
 .../framework/recipes/cache/PathChildrenCache.java |   4 +-
 .../cache/PathChildrenCacheListenerWrapper.java    |  78 +++++
 .../recipes/cache/StandardCuratorCacheStorage.java |  74 ++++
 .../curator/framework/recipes/cache/TreeCache.java |   3 +
 .../recipes/cache/TreeCacheListenerWrapper.java    |  81 +++++
 .../framework/recipes/watch/PersistentWatcher.java | 174 ++++++++++
 .../src/site/confluence/curator-cache.confluence   |  36 ++
 .../src/site/confluence/index.confluence           |  10 +-
 .../site/confluence/persistent-watcher.confluence  |  35 ++
 .../framework/recipes/cache/TestCuratorCache.java  | 176 ++++++++++
 .../recipes/cache/TestCuratorCacheConsistency.java | 373 +++++++++++++++++++++
 .../recipes/cache/TestCuratorCacheEdges.java       | 148 ++++++++
 .../cache/TestCuratorCacheEventOrdering.java       |  52 +++
 .../recipes/cache/TestCuratorCacheWrappers.java    | 172 ++++++++++
 .../recipes/cache/TestWrappedNodeCache.java        | 171 ++++++++++
 .../recipes/watch/TestPersistentWatcher.java       | 106 ++++++
 31 files changed, 2914 insertions(+), 38 deletions(-)
 create mode 100644 curator-examples/src/main/java/cache/CuratorCacheExample.java
 create mode 100644 curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/CuratorCache.java
 create mode 100644 curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/CuratorCacheAccessor.java
 create mode 100644 curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/CuratorCacheBuilder.java
 create mode 100644 curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/CuratorCacheBuilderImpl.java
 create mode 100644 curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/CuratorCacheImpl.java
 create mode 100644 curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/CuratorCacheListener.java
 create mode 100644 curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/CuratorCacheListenerBuilder.java
 create mode 100644 curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/CuratorCacheListenerBuilderImpl.java
 create mode 100644 curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/CuratorCacheStorage.java
 copy curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/{EventOperation.java => NodeCacheListenerWrapper.java} (61%)
 copy curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/{EventOperation.java => OutstandingOps.java} (53%)
 create mode 100644 curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCacheListenerWrapper.java
 create mode 100644 curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/StandardCuratorCacheStorage.java
 create mode 100644 curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCacheListenerWrapper.java
 create mode 100644 curator-recipes/src/main/java/org/apache/curator/framework/recipes/watch/PersistentWatcher.java
 create mode 100644 curator-recipes/src/site/confluence/curator-cache.confluence
 create mode 100644 curator-recipes/src/site/confluence/persistent-watcher.confluence
 create mode 100644 curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestCuratorCache.java
 create mode 100644 curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestCuratorCacheConsistency.java
 create mode 100644 curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestCuratorCacheEdges.java
 create mode 100644 curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestCuratorCacheEventOrdering.java
 create mode 100644 curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestCuratorCacheWrappers.java
 create mode 100644 curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestWrappedNodeCache.java
 create mode 100644 curator-recipes/src/test/java/org/apache/curator/framework/recipes/watch/TestPersistentWatcher.java


[curator] 01/02: CURATOR-525

Posted by ra...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

randgalt pushed a commit to branch CURATOR-525-fix-lost-state-race
in repository https://gitbox.apache.org/repos/asf/curator.git

commit ba83de17ea9c99ff5fb73a8efa8d939ceda0d1f9
Author: randgalt <ra...@apache.org>
AuthorDate: Tue Mar 31 20:22:43 2020 -0500

    CURATOR-525
    
    There is a race whereby the ZooKeeper connection can be healed before Curator is finished processing the new connection state. When this happens
    the Curator instance becomes a Zombie stuck in the LOST state. This fix is a "hack". ConnectionStateManager will notice that the connection state is
    LOST but that the Curator instance reports that it is connected. When this happens, it is logged and the connection is reset.
---
 .../framework/imps/CuratorFrameworkImpl.java       | 27 ++++++++++++++++
 .../framework/state/ConnectionStateManager.java    | 18 +++++++++++
 .../curator/framework/imps/TestFrameworkEdges.java | 36 ++++++++++++++++++++++
 3 files changed, 81 insertions(+)

diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java
index bfe61bf..1abfc28 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java
@@ -816,6 +816,13 @@ public class CuratorFrameworkImpl implements CuratorFramework
         return ensembleTracker;
     }
 
+    @VisibleForTesting
+    volatile CountDownLatch debugCheckBackgroundRetryLatch;
+    @VisibleForTesting
+    volatile CountDownLatch debugCheckBackgroundRetryReadyLatch;
+    @VisibleForTesting
+    volatile KeeperException.Code injectedCode;
+
     @SuppressWarnings({"ThrowableResultOfMethodCallIgnored"})
     private <DATA_TYPE> boolean checkBackgroundRetry(OperationAndData<DATA_TYPE> operationAndData, CuratorEvent event)
     {
@@ -851,6 +858,26 @@ public class CuratorFrameworkImpl implements CuratorFramework
                 e = new Exception("Unknown result codegetResultCode()");
             }
 
+            if ( debugCheckBackgroundRetryLatch != null )       // scaffolding to test CURATOR-525
+            {
+                if ( debugCheckBackgroundRetryReadyLatch != null )
+                {
+                    debugCheckBackgroundRetryReadyLatch.countDown();
+                }
+                try
+                {
+                    debugCheckBackgroundRetryLatch.await();
+                    if (injectedCode != null)
+                    {
+                        code = injectedCode;
+                    }
+                }
+                catch ( InterruptedException ex )
+                {
+                    Thread.currentThread().interrupt();
+                }
+            }
+
             validateConnection(codeToState(code));
             logError("Background operation retry gave up", e);
         }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java
index 7285431..32ddb78 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java
@@ -285,6 +285,24 @@ public class ConnectionStateManager implements Closeable
                         checkSessionExpiration();
                     }
                 }
+
+                synchronized(this)
+                {
+                    if ( (currentConnectionState == ConnectionState.LOST) && client.getZookeeperClient().isConnected() )
+                    {
+                        // CURATOR-525 - there is a race whereby LOST is sometimes set after the connection has been repaired
+                        // this "hack" fixes it by resetting the connection
+                        log.warn("ConnectionState is LOST but isConnected() is true. Resetting connection.");
+                        try
+                        {
+                            client.getZookeeperClient().reset();
+                        }
+                        catch ( Exception e )
+                        {
+                            log.error("Could not reset connection after LOST/isConnected mismatch");
+                        }
+                    }
+                }
             }
             catch ( InterruptedException e )
             {
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java
index 5a7c415..6fcd553 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java
@@ -75,6 +75,42 @@ public class TestFrameworkEdges extends BaseClassForTests
         System.setProperty("zookeeper.extendedTypesEnabled", "true");
     }
 
+    @Test(description = "test case for CURATOR-525")
+    public void testValidateConnectionEventRaces() throws Exception
+    {
+        // test for CURATOR-525 - there is a race whereby Curator can go to LOST
+        // after the connection has been repaired. Prior to the fix, the Curator
+        // instance would become a zombie, never leaving the LOST state
+        try (CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), 2000, 1000, new RetryOneTime(1)))
+        {
+            CuratorFrameworkImpl clientImpl = (CuratorFrameworkImpl)client;
+
+            client.start();
+            client.getChildren().forPath("/");
+            client.create().forPath("/foo");
+
+            BlockingQueue<ConnectionState> stateQueue = new LinkedBlockingQueue<>();
+            client.getConnectionStateListenable().addListener((__, newState) -> stateQueue.add(newState));
+
+            server.stop();
+            Assert.assertEquals(timing.takeFromQueue(stateQueue), ConnectionState.SUSPENDED);
+            Assert.assertEquals(timing.takeFromQueue(stateQueue), ConnectionState.LOST);
+
+            clientImpl.debugCheckBackgroundRetryReadyLatch = new CountDownLatch(1);
+            clientImpl.debugCheckBackgroundRetryLatch = new CountDownLatch(1);
+
+            client.delete().guaranteed().inBackground().forPath("/foo");
+            timing.awaitLatch(clientImpl.debugCheckBackgroundRetryReadyLatch);
+            server.restart();
+            Assert.assertEquals(timing.takeFromQueue(stateQueue), ConnectionState.RECONNECTED);
+            clientImpl.injectedCode = KeeperException.Code.SESSIONEXPIRED;  // simulate an expiration being handled after the connection is repaired
+            clientImpl.debugCheckBackgroundRetryLatch.countDown();
+            Assert.assertEquals(timing.takeFromQueue(stateQueue), ConnectionState.LOST);
+
+            Assert.assertEquals(timing.takeFromQueue(stateQueue), ConnectionState.RECONNECTED);
+        }
+    }
+
     @Test
     public void testInjectSessionExpiration() throws Exception
     {


[curator] 02/02: CURATOR-525 - instead of resetting the connection, change the state to RECONNECTED. I'm concerned about LOST/reset loops. This is still a bad hack and needs to be addressed in the future.

Posted by ra...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

randgalt pushed a commit to branch CURATOR-525-fix-lost-state-race
in repository https://gitbox.apache.org/repos/asf/curator.git

commit d9077664cdf9c99042ff5ff34a3aa5a665d0d829
Author: randgalt <ra...@apache.org>
AuthorDate: Thu Apr 2 08:16:41 2020 -0500

    CURATOR-525 - instead of resetting the connection, change the state to RECONNECTED. I'm concerned about LOST/reset loops. This is still a bad hack and needs to be addressed in the future.
---
 .../java/org/apache/curator/framework/state/ConnectionStateManager.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java
index 32ddb78..ca631b6 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java
@@ -295,7 +295,7 @@ public class ConnectionStateManager implements Closeable
                         log.warn("ConnectionState is LOST but isConnected() is true. Resetting connection.");
                         try
                         {
-                            client.getZookeeperClient().reset();
+                            addStateChange(ConnectionState.RECONNECTED);
                         }
                         catch ( Exception e )
                         {