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:07 UTC

[08/47] curator git commit: Added alternate form of reconfig and allow for storing stat

Added alternate form of reconfig and allow for storing stat


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

Branch: refs/heads/CURATOR-3.0
Commit: 6222282c9c58cb75b8ea47b69d2008b70e863c59
Parents: 9a09b5d
Author: randgalt <ra...@apache.org>
Authored: Thu Jan 5 15:58:27 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Thu Jan 5 15:58:27 2017 -0500

----------------------------------------------------------------------
 .../org/apache/curator/x/crimps/Crimps.java     | 33 +++++++++++
 .../curator/x/crimps/async/AsyncCrimps.java     | 49 ++++++++--------
 .../crimps/async/CrimpedConfigEnsembleable.java | 33 +++++++++++
 .../async/CrimpedConfigureEnsembleable.java     | 33 -----------
 .../x/crimps/async/CrimpedEnsembleable.java     | 30 ++++++++++
 .../x/crimps/async/CrimpedEnsembleableImpl.java | 60 ++++++++++++++++++++
 .../apache/curator/x/crimps/async/Crimps.java   | 31 ----------
 .../curator/x/crimps/async/TestCrimps.java      |  3 +-
 8 files changed, 183 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/6222282c/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/Crimps.java
----------------------------------------------------------------------
diff --git a/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/Crimps.java b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/Crimps.java
new file mode 100644
index 0000000..bf00f33
--- /dev/null
+++ b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/Crimps.java
@@ -0,0 +1,33 @@
+/**
+ * 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;
+
+import org.apache.curator.x.crimps.async.AsyncCrimps;
+
+public class Crimps
+{
+    public static AsyncCrimps async()
+    {
+        return new AsyncCrimps(null);
+    }
+
+    private Crimps()
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/6222282c/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 69695bc..b9a330c 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
@@ -53,6 +53,11 @@ public class AsyncCrimps
         };
     }
 
+    public AsyncCrimps(UnhandledErrorListener unhandledErrorListener)
+    {
+        this.unhandledErrorListener = unhandledErrorListener;
+    }
+
     public AsyncCrimps withUnhandledErrorListener(UnhandledErrorListener unhandledErrorListener)
     {
         return new AsyncCrimps(unhandledErrorListener);
@@ -98,7 +103,7 @@ public class AsyncCrimps
         return build(builder, statSupplier);
     }
 
-    public CrimpedConfigureEnsembleable joiningLeaving(Backgroundable<ErrorListenerReconfigBuilderMain> builder, List<String> joining, List<String> leaving)
+    public CrimpedEnsembleable ensemble(Backgroundable<ErrorListenerReconfigBuilderMain> builder, List<String> newMembers)
     {
         CrimpedBackgroundCallback<byte[]> callback = new CrimpedBackgroundCallback<>(dataSupplier);
 
@@ -112,7 +117,24 @@ public class AsyncCrimps
             main = builder.inBackground(callback);
         }
 
-        ConfigureEnsembleable configBuilder;
+        return new CrimpedEnsembleableImpl(main.withNewMembers(newMembers), callback);
+    }
+
+    public CrimpedEnsembleable ensemble(Backgroundable<ErrorListenerReconfigBuilderMain> builder, List<String> joining, List<String> leaving)
+    {
+        CrimpedBackgroundCallback<byte[]> callback = new CrimpedBackgroundCallback<>(dataSupplier);
+
+        ReconfigBuilderMain main;
+        if ( unhandledErrorListener != null )
+        {
+            main = builder.inBackground(callback).withUnhandledErrorListener(unhandledErrorListener);
+        }
+        else
+        {
+            main = builder.inBackground(callback);
+        }
+
+        Statable<ConfigureEnsembleable> configBuilder;
         if ( nonEmpty(joining) && nonEmpty(leaving) )
         {
             configBuilder = main.joining(joining).leaving(leaving);
@@ -130,24 +152,7 @@ public class AsyncCrimps
             throw new IllegalArgumentException("leaving and joining cannot both be empty");
         }
 
-        return new CrimpedConfigureEnsembleable()
-        {
-            private Ensembleable<byte[]> localEnsembleable = configBuilder;
-
-            @Override
-            public Ensembleable<CompletionStage<byte[]>> fromConfig(long config) throws Exception
-            {
-                localEnsembleable = configBuilder.fromConfig(config);
-                return this;
-            }
-
-            @Override
-            public CompletionStage<byte[]> forEnsemble() throws Exception
-            {
-                localEnsembleable.forEnsemble();
-                return callback;
-            }
-        };
+        return new CrimpedEnsembleableImpl(configBuilder, callback);
     }
 
     public <T> CrimpedPathAndBytesable<T> build(BackgroundPathAndBytesable<T> builder, BackgroundProc<T> backgroundProc)
@@ -189,8 +194,4 @@ public class AsyncCrimps
         return (list != null) && !list.isEmpty();
     }
 
-    AsyncCrimps(UnhandledErrorListener unhandledErrorListener)
-    {
-        this.unhandledErrorListener = unhandledErrorListener;
-    }
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/6222282c/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedConfigEnsembleable.java
----------------------------------------------------------------------
diff --git a/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedConfigEnsembleable.java b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedConfigEnsembleable.java
new file mode 100644
index 0000000..9021daf
--- /dev/null
+++ b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedConfigEnsembleable.java
@@ -0,0 +1,33 @@
+/**
+ * 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.Ensembleable;
+import java.util.concurrent.CompletionStage;
+
+public interface CrimpedConfigEnsembleable extends
+    Ensembleable<CompletionStage<byte[]>>
+{
+    /**
+     * Sets the configuration version to use.
+     * @param config The version of the configuration.
+     * @throws Exception errors
+     */
+    Ensembleable<CompletionStage<byte[]>> fromConfig(long config) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/6222282c/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedConfigureEnsembleable.java
----------------------------------------------------------------------
diff --git a/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedConfigureEnsembleable.java b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedConfigureEnsembleable.java
deleted file mode 100644
index d1211ce..0000000
--- a/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedConfigureEnsembleable.java
+++ /dev/null
@@ -1,33 +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.Ensembleable;
-import java.util.concurrent.CompletionStage;
-
-public interface CrimpedConfigureEnsembleable extends
-    Ensembleable<CompletionStage<byte[]>>
-{
-    /**
-     * Sets the configuration version to use.
-     * @param config The version of the configuration.
-     * @throws Exception errors
-     */
-    Ensembleable<CompletionStage<byte[]>> fromConfig(long config) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/6222282c/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedEnsembleable.java
----------------------------------------------------------------------
diff --git a/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedEnsembleable.java b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedEnsembleable.java
new file mode 100644
index 0000000..3069e39
--- /dev/null
+++ b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedEnsembleable.java
@@ -0,0 +1,30 @@
+/**
+ * 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.Ensembleable;
+import org.apache.curator.framework.api.Statable;
+import java.util.concurrent.CompletionStage;
+
+public interface CrimpedEnsembleable extends
+    CrimpedConfigEnsembleable,
+    Statable<CrimpedConfigEnsembleable>,
+    Ensembleable<CompletionStage<byte[]>>
+{
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/6222282c/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedEnsembleableImpl.java
----------------------------------------------------------------------
diff --git a/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedEnsembleableImpl.java b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedEnsembleableImpl.java
new file mode 100644
index 0000000..116ce56
--- /dev/null
+++ b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/CrimpedEnsembleableImpl.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.curator.framework.api.ConfigureEnsembleable;
+import org.apache.curator.framework.api.Ensembleable;
+import org.apache.curator.framework.api.Statable;
+import org.apache.zookeeper.data.Stat;
+import java.util.concurrent.CompletionStage;
+
+class CrimpedEnsembleableImpl implements CrimpedEnsembleable
+{
+    private final CrimpedBackgroundCallback<byte[]> callback;
+    private final Statable<ConfigureEnsembleable> configBuilder;
+    private ConfigureEnsembleable configureEnsembleable;
+
+    CrimpedEnsembleableImpl(Statable<ConfigureEnsembleable> configBuilder, CrimpedBackgroundCallback<byte[]> callback)
+    {
+        this.configBuilder = configBuilder;
+        this.callback = callback;
+        configureEnsembleable = configBuilder.storingStatIn(new Stat());
+    }
+
+    @Override
+    public CompletionStage<byte[]> forEnsemble() throws Exception
+    {
+        configureEnsembleable.forEnsemble();
+        return callback;
+    }
+
+    @Override
+    public CrimpedConfigEnsembleable storingStatIn(Stat stat)
+    {
+        configureEnsembleable = configBuilder.storingStatIn(stat);
+        return this;
+    }
+
+    @Override
+    public Ensembleable<CompletionStage<byte[]>> fromConfig(long config) throws Exception
+    {
+        configureEnsembleable.fromConfig(config);
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/6222282c/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/Crimps.java
----------------------------------------------------------------------
diff --git a/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/Crimps.java b/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/Crimps.java
deleted file mode 100644
index 519a7c5..0000000
--- a/curator-x-crimps/src/main/java/org/apache/curator/x/crimps/async/Crimps.java
+++ /dev/null
@@ -1,31 +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;
-
-public class Crimps
-{
-    public static AsyncCrimps async()
-    {
-        return new AsyncCrimps(null);
-    }
-
-    private Crimps()
-    {
-    }
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/6222282c/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 428cb78..7f5647a 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
@@ -22,6 +22,7 @@ import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.x.crimps.Crimps;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
@@ -101,7 +102,7 @@ public class TestCrimps extends BaseClassForTests
         {
             client.start();
 
-            CompletionStage<byte[]> f = async.joiningLeaving(client.reconfig(), Arrays.asList("1", "2"), Arrays.asList("3", "4")).forEnsemble();
+            CompletionStage<byte[]> f = async.ensemble(client.reconfig(), Arrays.asList("1", "2"), Arrays.asList("3", "4")).forEnsemble();
             assertException(f, KeeperException.Code.UNIMPLEMENTED);
         }
     }