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 2017/02/09 18:36:10 UTC

[11/47] curator git commit: start of watcher support

start of watcher support


Project: http://git-wip-us.apache.org/repos/asf/curator/repo
Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/74f14106
Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/74f14106
Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/74f14106

Branch: refs/heads/CURATOR-3.0
Commit: 74f14106bf389b8cdf92f2838a41372fabdbfa18
Parents: 7689f33
Author: randgalt <ra...@apache.org>
Authored: Thu Jan 5 17:45:27 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Thu Jan 5 17:45:27 2017 -0500

----------------------------------------------------------------------
 .../curator/x/crimps/async/AsyncCrimps.java     | 85 ++++++++++++++++----
 .../apache/curator/x/crimps/async/Crimped.java  | 29 +++++++
 .../x/crimps/async/CrimpedPathAndBytesable.java | 26 ------
 .../curator/x/crimps/async/CrimpedPathable.java | 26 ------
 .../curator/x/crimps/async/CrimpedWatcher.java  | 60 ++++++++++++++
 .../curator/x/crimps/async/TestCrimps.java      | 54 +++++++++++++
 6 files changed, 213 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/74f14106/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/AsyncCrimps.java
----------------------------------------------------------------------
diff --git a/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/AsyncCrimps.java b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/AsyncCrimps.java
index d9324a3..afe97da 100644
--- a/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/AsyncCrimps.java
+++ b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/AsyncCrimps.java
@@ -22,6 +22,7 @@ import org.apache.curator.framework.api.*;
 import org.apache.curator.framework.api.transaction.CuratorMultiTransactionMain;
 import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
 import java.util.List;
@@ -35,6 +36,17 @@ public class AsyncCrimps
     public static final BackgroundProc<Void> ignoredProc = makeProc(e -> null);
     public static final BackgroundProc<byte[]> dataProc = makeProc(CuratorEvent::getData);
     public static final BackgroundProc<Stat> statProc = makeProc(CuratorEvent::getStat);
+    public static final BackgroundProc<Stat> safeStatProc = (event, future) -> {
+        if ( (event.getResultCode() == 0) || (event.getResultCode() == KeeperException.Code.NONODE.intValue()) )
+        {
+            future.complete(event.getStat());
+        }
+        else
+        {
+            future.completeExceptionally(KeeperException.create(KeeperException.Code.get(event.getResultCode()), event.getPath()));
+        }
+        return null;
+    };
     public static final BackgroundProc<List<String>> childrenProc = makeProc(CuratorEvent::getChildren);
     public static final BackgroundProc<List<ACL>> aclProc = makeProc(CuratorEvent::getACLList);
     public static final BackgroundProc<List<CuratorTransactionResult>> opResultsProc = makeProc(CuratorEvent::getOpResults);
@@ -50,7 +62,7 @@ public class AsyncCrimps
             }
             else
             {
-                future.completeExceptionally(KeeperException.create(KeeperException.Code.get(event.getResultCode())));
+                future.completeExceptionally(KeeperException.create(KeeperException.Code.get(event.getResultCode()), event.getPath()));
             }
             return null;
         };
@@ -66,42 +78,57 @@ public class AsyncCrimps
         return new AsyncCrimps(unhandledErrorListener);
     }
 
-    public CrimpedPathAndBytesable<String> name(BackgroundPathAndBytesable<String> builder)
+    public PathAndBytesable<CompletionStage<String>> name(BackgroundPathAndBytesable<String> builder)
     {
         return build(builder, nameProc);
     }
 
-    public CrimpedPathAndBytesable<String> path(BackgroundPathAndBytesable<String> builder)
+    public PathAndBytesable<CompletionStage<String>> path(BackgroundPathAndBytesable<String> builder)
     {
         return build(builder, pathProc);
     }
 
-    public CrimpedPathable<Void> ignored(BackgroundPathable<Void> builder)
+    public Pathable<CompletionStage<Void>> ignored(BackgroundPathable<Void> builder)
     {
         return build(builder, ignoredProc);
     }
 
-    public CrimpedPathable<byte[]> data(BackgroundPathable<byte[]> builder)
+    public Pathable<CompletionStage<byte[]>> data(BackgroundPathable<byte[]> builder)
     {
         return build(builder, dataProc);
     }
 
-    public CrimpedPathable<List<String>> children(BackgroundPathable<List<String>> builder)
+    public Pathable<CompletionStage<List<String>>> children(BackgroundPathable<List<String>> builder)
     {
         return build(builder, childrenProc);
     }
 
-    public CrimpedPathable<Stat> stat(BackgroundPathable<Stat> builder)
+    public Pathable<CompletionStage<Stat>> stat(BackgroundPathable<Stat> builder)
+    {
+        return build(builder, statProc);
+    }
+
+    public Pathable<CompletionStage<Stat>> safeStat(BackgroundPathable<Stat> builder)
+    {
+        return build(builder, safeStatProc);
+    }
+
+    public Pathable<Crimped<Stat>> statWatched(Watchable<BackgroundPathable<Stat>> builder)
     {
         return build(builder, statProc);
     }
 
-    public CrimpedPathable<List<ACL>> acls(BackgroundPathable<List<ACL>> builder)
+    public Pathable<Crimped<Stat>> safeStatWatched(Watchable<BackgroundPathable<Stat>> builder)
+    {
+        return build(builder, safeStatProc);
+    }
+
+    public Pathable<CompletionStage<List<ACL>>> acls(BackgroundPathable<List<ACL>> builder)
     {
         return build(builder, aclProc);
     }
 
-    public CrimpedPathAndBytesable<Stat> statBytes(BackgroundPathAndBytesable<Stat> builder)
+    public PathAndBytesable<CompletionStage<Stat>> statBytes(BackgroundPathAndBytesable<Stat> builder)
     {
         return build(builder, statProc);
     }
@@ -186,12 +213,12 @@ public class AsyncCrimps
         };
     }
 
-    public <T> CrimpedPathAndBytesable<T> build(BackgroundPathAndBytesable<T> builder, BackgroundProc<T> backgroundProc)
+    public <T> PathAndBytesable<CompletionStage<T>> build(BackgroundPathAndBytesable<T> builder, BackgroundProc<T> backgroundProc)
     {
         CrimpedBackgroundCallback<T> callback = new CrimpedBackgroundCallback<T>(backgroundProc);
         ErrorListenerPathAndBytesable<T> localBuilder = builder.inBackground(callback);
         PathAndBytesable<T> finalLocalBuilder = (unhandledErrorListener != null) ? localBuilder.withUnhandledErrorListener(unhandledErrorListener) : localBuilder;
-        return new CrimpedPathAndBytesable<T>()
+        return new PathAndBytesable<CompletionStage<T>>()
         {
             @Override
             public CompletionStage<T> forPath(String path) throws Exception
@@ -209,20 +236,48 @@ public class AsyncCrimps
         };
     }
 
-    public <T> CrimpedPathable<T> build(BackgroundPathable<T> builder, BackgroundProc<T> backgroundProc)
+    public <T> Pathable<Crimped<T>> build(Watchable<BackgroundPathable<T>> builder, BackgroundProc<T> backgroundProc)
+    {
+        CrimpedWatcher crimpedWatcher = new CrimpedWatcher();
+        CrimpedBackgroundCallback<T> callback = new CrimpedBackgroundCallback<T>(backgroundProc);
+        Pathable<T> finalLocalBuilder = toFinalBuilder(callback, builder.usingWatcher(crimpedWatcher));
+        return path -> {
+            finalLocalBuilder.forPath(path);
+            return new Crimped<T>()
+            {
+                @Override
+                public CompletionStage<WatchedEvent> event()
+                {
+                    return crimpedWatcher;
+                }
+
+                @Override
+                public CompletionStage<T> value()
+                {
+                    return callback;
+                }
+            };
+        };
+    }
+
+    public <T> Pathable<CompletionStage<T>> build(BackgroundPathable<T> builder, BackgroundProc<T> backgroundProc)
     {
         CrimpedBackgroundCallback<T> callback = new CrimpedBackgroundCallback<T>(backgroundProc);
-        ErrorListenerPathable<T> localBuilder = builder.inBackground(callback);
-        Pathable<T> finalLocalBuilder = (unhandledErrorListener != null) ? localBuilder.withUnhandledErrorListener(unhandledErrorListener) : localBuilder;
+        Pathable<T> finalLocalBuilder = toFinalBuilder(callback, builder);
         return path -> {
             finalLocalBuilder.forPath(path);
             return callback;
         };
     }
 
+    private <T> Pathable<T> toFinalBuilder(CrimpedBackgroundCallback<T> callback, BackgroundPathable<T> backgroundPathable)
+    {
+        ErrorListenerPathable<T> localBuilder = backgroundPathable.inBackground(callback);
+        return (unhandledErrorListener != null) ? localBuilder.withUnhandledErrorListener(unhandledErrorListener) : localBuilder;
+    }
+
     private static boolean nonEmpty(List<String> list)
     {
         return (list != null) && !list.isEmpty();
     }
-
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/74f14106/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/Crimped.java
----------------------------------------------------------------------
diff --git a/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/Crimped.java b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/Crimped.java
new file mode 100644
index 0000000..26e300a
--- /dev/null
+++ b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/Crimped.java
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.curator.x.crimps.async;
+
+import org.apache.zookeeper.WatchedEvent;
+import java.util.concurrent.CompletionStage;
+
+public interface Crimped<T>
+{
+    CompletionStage<WatchedEvent> event();
+
+    CompletionStage<T> value();
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/74f14106/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedPathAndBytesable.java
----------------------------------------------------------------------
diff --git a/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedPathAndBytesable.java b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedPathAndBytesable.java
deleted file mode 100644
index b62ceec..0000000
--- a/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedPathAndBytesable.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.curator.x.crimps.async;
-
-import org.apache.curator.framework.api.PathAndBytesable;
-import java.util.concurrent.CompletionStage;
-
-public interface CrimpedPathAndBytesable<T> extends PathAndBytesable<CompletionStage<T>>
-{
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/74f14106/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedPathable.java
----------------------------------------------------------------------
diff --git a/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedPathable.java b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedPathable.java
deleted file mode 100644
index aa28797..0000000
--- a/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedPathable.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.curator.x.crimps.async;
-
-import org.apache.curator.framework.api.Pathable;
-import java.util.concurrent.CompletionStage;
-
-public interface CrimpedPathable<T> extends Pathable<CompletionStage<T>>
-{
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/74f14106/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedWatcher.java
----------------------------------------------------------------------
diff --git a/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedWatcher.java b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedWatcher.java
new file mode 100644
index 0000000..964a00c
--- /dev/null
+++ b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedWatcher.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.curator.x.crimps.async;
+
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import java.util.concurrent.CompletableFuture;
+
+class CrimpedWatcher extends CompletableFuture<WatchedEvent> implements Watcher
+{
+    @Override
+    public void process(WatchedEvent event)
+    {
+        switch ( event.getState() )
+        {
+            case ConnectedReadOnly:
+            case SyncConnected:
+            case SaslAuthenticated:
+            {
+                complete(event);
+                break;
+            }
+
+            case Disconnected:
+            {
+                completeExceptionally(KeeperException.create(KeeperException.Code.CONNECTIONLOSS));
+                break;
+            }
+
+            case AuthFailed:
+            {
+                completeExceptionally(KeeperException.create(KeeperException.Code.AUTHFAILED));
+                break;
+            }
+
+            case Expired:
+            {
+                completeExceptionally(KeeperException.create(KeeperException.Code.SESSIONEXPIRED));
+                break;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/74f14106/curator-x-crimps/src/test/java/org/apache/curator/x/crimps/async/TestCrimps.java
----------------------------------------------------------------------
diff --git a/curator-x-crimps/src/test/java/org/apache/curator/x/crimps/async/TestCrimps.java b/curator-x-crimps/src/test/java/org/apache/curator/x/crimps/async/TestCrimps.java
index 926649a..d5b7a6d 100644
--- a/curator-x-crimps/src/test/java/org/apache/curator/x/crimps/async/TestCrimps.java
+++ b/curator-x-crimps/src/test/java/org/apache/curator/x/crimps/async/TestCrimps.java
@@ -25,6 +25,7 @@ import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
 import org.apache.curator.framework.api.transaction.OperationType;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.Timing;
 import org.apache.curator.x.crimps.Crimps;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -34,6 +35,7 @@ import org.testng.annotations.Test;
 import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.CompletionStage;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 
 public class TestCrimps extends BaseClassForTests
@@ -100,6 +102,58 @@ public class TestCrimps extends BaseClassForTests
     }
 
     @Test
+    public void testExists() throws Exception
+    {
+        try ( CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1)) )
+        {
+            client.start();
+
+            CompletionStage<Stat> f = async.safeStat(client.checkExists()).forPath("/test");
+            complete(f.handle((stat, e) -> {
+                Assert.assertNull(e);
+                Assert.assertNull(stat);
+                return null;
+            }));
+
+            f = async.stat(client.checkExists()).forPath("/test");
+            complete(f.handle((stat, e) -> {
+                Assert.assertNotNull(e);
+                Assert.assertNull(stat);
+                return null;
+            }));
+
+            async.path(client.create()).forPath("/test").toCompletableFuture().get();
+            f = async.stat(client.checkExists()).forPath("/test");
+            complete(f.handle((stat, e) -> {
+                Assert.assertNull(e);
+                Assert.assertNotNull(stat);
+                return null;
+            }));
+        }
+    }
+
+    @Test
+    public void testExistsWatched() throws Exception
+    {
+        try ( CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1)) )
+        {
+            client.start();
+
+            Crimped<Stat> crimped = async.statWatched(client.checkExists().creatingParentContainersIfNeeded()).forPath("/one/two");
+            CountDownLatch latch = new CountDownLatch(1);
+            crimped.event().handle((event, e) -> {
+                Assert.assertNotNull(event);
+                latch.countDown();
+                return null;
+            });
+
+            async.path(client.create()).forPath("/one/two");
+
+            Assert.assertTrue(new Timing().awaitLatch(latch));
+        }
+    }
+
+    @Test
     public void testReconfig() throws Exception
     {
         try ( CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1)) )