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/05/11 13:57:40 UTC

curator git commit: 1. Support partially resolved paths 2. Added method to get siblings 3. Auto resolve unresolved paths on set/update using the model being set/updated

Repository: curator
Updated Branches:
  refs/heads/CURATOR-397 7ed263cc3 -> 6ea7221ab


1. Support partially resolved paths
2. Added method to get siblings
3. Auto resolve unresolved paths on set/update using the model being set/updated


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

Branch: refs/heads/CURATOR-397
Commit: 6ea7221ab48dd0597e3b0f2ba4cdc33147ff7d4d
Parents: 7ed263c
Author: randgalt <ra...@apache.org>
Authored: Thu May 11 15:55:31 2017 +0200
Committer: randgalt <ra...@apache.org>
Committed: Thu May 11 15:55:31 2017 +0200

----------------------------------------------------------------------
 .../curator/x/async/modeled/ModelSpec.java      | 17 ++++++
 .../x/async/modeled/ModeledFramework.java       | 25 ++++++++
 .../apache/curator/x/async/modeled/ZPath.java   |  7 +++
 .../details/CachedModeledFrameworkImpl.java     | 13 +++++
 .../x/async/modeled/details/ModelSpecImpl.java  |  6 ++
 .../modeled/details/ModeledFrameworkImpl.java   | 60 ++++++++++++++++----
 .../x/async/modeled/details/ZPathImpl.java      |  7 +--
 .../curator/x/async/modeled/TestZPath.java      | 25 +++++++-
 8 files changed, 140 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/6ea7221a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ModelSpec.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ModelSpec.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ModelSpec.java
index 0567635..652eabd 100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ModelSpec.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ModelSpec.java
@@ -94,6 +94,23 @@ public interface ModelSpec<T> extends Resolvable
     ModelSpec<T> at(Object child);
 
     /**
+     * <p>
+     *     Return a new CuratorModel instance with all the same options but applying to the parent node of this CuratorModel's
+     *     path. E.g. if this CuratorModel instance applies to "/a/b/c", calling <code>modeled.parent()</code> returns an instance that applies to
+     *     "/a/b".
+     * </p>
+     *
+     * <p>
+     *     The replacement is the <code>toString()</code> value of child or,
+     *     if it implements {@link org.apache.curator.x.async.modeled.NodeName},
+     *     the value of <code>nodeName()</code>.
+     * </p>
+     *
+     * @return new Modeled Spec instance
+     */
+    ModelSpec<T> parent();
+
+    /**
      * Return a new CuratorModel instance with all the same options but using the given path.
      *
      * @param path new path

http://git-wip-us.apache.org/repos/asf/curator/blob/6ea7221a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ModeledFramework.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ModeledFramework.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ModeledFramework.java
index cb511d3..42447d8 100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ModeledFramework.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ModeledFramework.java
@@ -132,6 +132,23 @@ public interface ModeledFramework<T>
     ModeledFramework<T> at(Object child);
 
     /**
+     * <p>
+     *     Return a new Modeled Curator instance with all the same options but applying to the parent node of this Modeled Curator's
+     *     path. E.g. if this Modeled Curator instance applies to "/a/b/c", calling <code>modeled.parent()</code> returns an instance that applies to
+     *     "/a/b".
+     * </p>
+     *
+     * <p>
+     *     The replacement is the <code>toString()</code> value of child or,
+     *     if it implements {@link org.apache.curator.x.async.modeled.NodeName},
+     *     the value of <code>nodeName()</code>.
+     * </p>
+     *
+     * @return new Modeled Curator instance
+     */
+    ModeledFramework<T> parent();
+
+    /**
      * Return a Modeled Curator instance with all the same options but using the given path.
      *
      * @param path new path
@@ -263,6 +280,14 @@ public interface ModeledFramework<T>
     AsyncStage<List<ZPath>> children();
 
     /**
+     * Return the child paths of this instance's parent path (in no particular order)
+     *
+     * @return AsyncStage
+     * @see org.apache.curator.x.async.AsyncStage
+     */
+    AsyncStage<List<ZPath>> siblings();
+
+    /**
      * Create operation instance that can be passed among other operations to
      * {@link #inTransaction(java.util.List)} to be executed as a single transaction. Note:
      * due to ZooKeeper transaction limits, this is a _not_ a "set or update" operation but only

http://git-wip-us.apache.org/repos/asf/curator/blob/6ea7221a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ZPath.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ZPath.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ZPath.java
index ba6fc21..0e32023 100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ZPath.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ZPath.java
@@ -241,6 +241,13 @@ public interface ZPath extends Resolvable
     boolean isRoot();
 
     /**
+     * Return true if this path is fully resolved (i.e. has no unresoled parameters)
+     *
+     * @return true/false
+     */
+    boolean isResolved();
+
+    /**
      * Return true if this path starts with the given path. i.e.
      * <code>ZPath.from("/one/two/three").startsWith(ZPath.from("/one/two"))</code> returns true
      *

http://git-wip-us.apache.org/repos/asf/curator/blob/6ea7221a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/CachedModeledFrameworkImpl.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/CachedModeledFrameworkImpl.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/CachedModeledFrameworkImpl.java
index 7df2e98..4cfb70c 100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/CachedModeledFrameworkImpl.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/CachedModeledFrameworkImpl.java
@@ -129,6 +129,12 @@ class CachedModeledFrameworkImpl<T> implements CachedModeledFramework<T>
     }
 
     @Override
+    public ModeledFramework<T> parent()
+    {
+        throw new UnsupportedOperationException("Not supported for CachedModeledFramework. Instead, call parent() on the ModeledFramework before calling cached()");
+    }
+
+    @Override
     public CachedModeledFramework<T> withPath(ZPath path)
     {
         return new CachedModeledFrameworkImpl<>(client.withPath(path), cache, executor, asyncDefaultMode);
@@ -222,6 +228,13 @@ class CachedModeledFrameworkImpl<T> implements CachedModeledFramework<T>
     }
 
     @Override
+    public AsyncStage<List<ZPath>> siblings()
+    {
+        Set<ZPath> paths = cache.currentChildren(client.modelSpec().path().parent()).keySet();
+        return completed(Lists.newArrayList(paths));
+    }
+
+    @Override
     public CuratorOp createOp(T model)
     {
         return client.createOp(model);

http://git-wip-us.apache.org/repos/asf/curator/blob/6ea7221a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModelSpecImpl.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModelSpecImpl.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModelSpecImpl.java
index 847ce61..b75fa12 100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModelSpecImpl.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModelSpecImpl.java
@@ -64,6 +64,12 @@ public class ModelSpecImpl<T> implements ModelSpec<T>, SchemaValidator
     }
 
     @Override
+    public ModelSpec<T> parent()
+    {
+        return withPath(path.parent());
+    }
+
+    @Override
     public ModelSpec<T> resolved(Object... parameters)
     {
         return withPath(path.resolved(parameters));

http://git-wip-us.apache.org/repos/asf/curator/blob/6ea7221a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModeledFrameworkImpl.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModeledFrameworkImpl.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModeledFrameworkImpl.java
index 7be713c..abd6af7 100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModeledFrameworkImpl.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModeledFrameworkImpl.java
@@ -157,7 +157,7 @@ public class ModeledFrameworkImpl<T> implements ModeledFramework<T>
             byte[] bytes = modelSpec.serializer().serialize(item);
             return dslClient.create()
                 .withOptions(modelSpec.createOptions(), modelSpec.createMode(), fixAclList(modelSpec.aclList()), storingStatIn, modelSpec.ttl(), version)
-                .forPath(modelSpec.path().fullPath(), bytes);
+                .forPath(resolveForSet(item), bytes);
         }
         catch ( Exception e )
         {
@@ -165,11 +165,6 @@ public class ModeledFrameworkImpl<T> implements ModeledFramework<T>
         }
     }
 
-    private List<ACL> fixAclList(List<ACL> aclList)
-    {
-        return (aclList.size() > 0) ? aclList : null;   // workaround for old, bad design. empty list not accepted
-    }
-
     @Override
     public AsyncStage<T> read()
     {
@@ -201,7 +196,7 @@ public class ModeledFrameworkImpl<T> implements ModeledFramework<T>
         {
             byte[] bytes = modelSpec.serializer().serialize(item);
             AsyncPathAndBytesable<AsyncStage<Stat>> next = isCompressed() ? dslClient.setData().compressedWithVersion(version) : dslClient.setData();
-            return next.forPath(modelSpec.path().fullPath(), bytes);
+            return next.forPath(resolveForSet(item), bytes);
         }
         catch ( Exception e )
         {
@@ -230,7 +225,18 @@ public class ModeledFrameworkImpl<T> implements ModeledFramework<T>
     @Override
     public AsyncStage<List<ZPath>> children()
     {
-        AsyncStage<List<String>> asyncStage = watchableClient.getChildren().forPath(modelSpec.path().fullPath());
+        return internalGetChildren(modelSpec.path());
+    }
+
+    @Override
+    public AsyncStage<List<ZPath>> siblings()
+    {
+        return internalGetChildren(modelSpec.path().parent());
+    }
+
+    private AsyncStage<List<ZPath>> internalGetChildren(ZPath path)
+    {
+        AsyncStage<List<String>> asyncStage = watchableClient.getChildren().forPath(path.fullPath());
         ModelStage<List<ZPath>> modelStage = ModelStage.make(asyncStage.event());
         asyncStage.whenComplete((children, e) -> {
             if ( e != null )
@@ -239,13 +245,30 @@ public class ModeledFrameworkImpl<T> implements ModeledFramework<T>
             }
             else
             {
-                modelStage.complete(children.stream().map(child -> modelSpec.path().at(child)).collect(Collectors.toList()));
+                modelStage.complete(children.stream().map(path::at).collect(Collectors.toList()));
             }
         });
         return modelStage;
     }
 
     @Override
+    public ModeledFramework<T> parent()
+    {
+        ModelSpec<T> newModelSpec = modelSpec.parent();
+        return new ModeledFrameworkImpl<>(
+            client,
+            dslClient,
+            watchableClient,
+            newModelSpec,
+            watchMode,
+            watcherFilter,
+            unhandledErrorListener,
+            resultFilter,
+            isWatched
+        );
+    }
+
+    @Override
     public ModeledFramework<T> at(Object child)
     {
         ModelSpec<T> newModelSpec = modelSpec.at(child);
@@ -290,7 +313,7 @@ public class ModeledFrameworkImpl<T> implements ModeledFramework<T>
         return client.transactionOp()
             .create()
             .withOptions(modelSpec.createMode(), fixAclList(modelSpec.aclList()), modelSpec.createOptions().contains(CreateOption.compress), modelSpec.ttl())
-            .forPath(modelSpec.path().fullPath(), modelSpec.serializer().serialize(model));
+            .forPath(resolveForSet(model), modelSpec.serializer().serialize(model));
     }
 
     @Override
@@ -305,9 +328,9 @@ public class ModeledFrameworkImpl<T> implements ModeledFramework<T>
         AsyncTransactionSetDataBuilder builder = client.transactionOp().setData();
         if ( isCompressed() )
         {
-            return builder.withVersionCompressed(version).forPath(modelSpec.path().fullPath(), modelSpec.serializer().serialize(model));
+            return builder.withVersionCompressed(version).forPath(resolveForSet(model), modelSpec.serializer().serialize(model));
         }
-        return builder.withVersion(version).forPath(modelSpec.path().fullPath(), modelSpec.serializer().serialize(model));
+        return builder.withVersion(version).forPath(resolveForSet(model), modelSpec.serializer().serialize(model));
     }
 
     @Override
@@ -372,4 +395,17 @@ public class ModeledFrameworkImpl<T> implements ModeledFramework<T>
         return modelStage;
     }
 
+    private String resolveForSet(T model)
+    {
+        if ( modelSpec.path().isResolved() )
+        {
+            return modelSpec.path().fullPath();
+        }
+        return modelSpec.path().resolved(model).fullPath();
+    }
+
+    private List<ACL> fixAclList(List<ACL> aclList)
+    {
+        return (aclList.size() > 0) ? aclList : null;   // workaround for old, bad design. empty list not accepted
+    }
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/6ea7221a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ZPathImpl.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ZPathImpl.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ZPathImpl.java
index 209b8da..35a6bd2 100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ZPathImpl.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ZPathImpl.java
@@ -209,12 +209,8 @@ public class ZPathImpl implements ZPath
         Iterator<Object> iterator = parameters.iterator();
         List<String> nodeNames = nodes.stream()
             .map(name -> {
-                if ( isParameter(name) )
+                if ( isParameter(name) && iterator.hasNext() )
                 {
-                    if ( !iterator.hasNext() )
-                    {
-                        throw new IllegalStateException(String.format("Parameter missing for [%s]", toString()));
-                    }
                     return NodeName.nameFrom(iterator.next());
                 }
                 return name;
@@ -223,6 +219,7 @@ public class ZPathImpl implements ZPath
         return new ZPathImpl(nodeNames, null);
     }
 
+    @Override
     public boolean isResolved()
     {
         return isResolved;

http://git-wip-us.apache.org/repos/asf/curator/blob/6ea7221a/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/TestZPath.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/TestZPath.java b/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/TestZPath.java
index f3d6c87..5136282 100644
--- a/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/TestZPath.java
+++ b/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/TestZPath.java
@@ -54,11 +54,11 @@ public class TestZPath
         Assert.assertFalse(path.startsWith(ZPath.root.at("two")));
 
         ZPath checkIdLike = ZPath.parse("/one/{two}/three");
-        Assert.assertTrue(((ZPathImpl)checkIdLike).isResolved());
+        Assert.assertTrue(checkIdLike.isResolved());
         checkIdLike = ZPath.parse("/one/" + ZPath.parameter() + "/three");
-        Assert.assertTrue(((ZPathImpl)checkIdLike).isResolved());
+        Assert.assertTrue(checkIdLike.isResolved());
         checkIdLike = ZPath.parse("/one/" + ZPath.parameter("others") + "/three");
-        Assert.assertTrue(((ZPathImpl)checkIdLike).isResolved());
+        Assert.assertTrue(checkIdLike.isResolved());
     }
 
     @Test
@@ -104,4 +104,23 @@ public class TestZPath
         Assert.assertEquals(ZPath.from("a", parameter(), "b", parameter()).toString(), "/a/{id}/b/{id}");
         Assert.assertEquals(ZPath.from("a", parameter("foo"), "b", parameter("bar")).toString(), "/a/{foo}/b/{bar}");
     }
+
+    @Test
+    public void testPartialResolution()
+    {
+        ZPath path = ZPath.parseWithIds("/one/{1}/two/{2}");
+        Assert.assertFalse(path.parent().isResolved());
+        Assert.assertFalse(path.parent().parent().isResolved());
+        Assert.assertTrue(path.parent().parent().parent().isResolved());
+        Assert.assertFalse(path.isResolved());
+
+        path = path.resolved("p1");
+        Assert.assertFalse(path.isResolved());
+        Assert.assertTrue(path.parent().isResolved());
+        Assert.assertEquals(path.toString(), "/one/p1/two/{2}");
+
+        path = path.resolved("p2");
+        Assert.assertTrue(path.isResolved());
+        Assert.assertEquals(path.toString(), "/one/p1/two/p2");
+    }
 }