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 2016/01/18 03:35:00 UTC

[1/6] curator git commit: Added TreeCacheSelector to allow for controling which nodes a TreeCache processes

Repository: curator
Updated Branches:
  refs/heads/master bef5bb9bd -> 5d485b072


Added TreeCacheSelector to allow for controling which nodes a TreeCache processes


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

Branch: refs/heads/master
Commit: a01f2a00dcbf608f5e297683e51a5244ffd679bf
Parents: 45332f3
Author: randgalt <ra...@apache.org>
Authored: Tue Jan 12 14:13:54 2016 -0500
Committer: randgalt <ra...@apache.org>
Committed: Tue Jan 12 14:13:54 2016 -0500

----------------------------------------------------------------------
 .../recipes/cache/DefaultTreeCacheSelector.java | 37 +++++++++++
 .../framework/recipes/cache/TreeCache.java      | 32 +++++++---
 .../recipes/cache/TreeCacheSelector.java        | 66 ++++++++++++++++++++
 .../framework/recipes/cache/TestTreeCache.java  | 48 ++++++++++++++
 4 files changed, 175 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/a01f2a00/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/DefaultTreeCacheSelector.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/DefaultTreeCacheSelector.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/DefaultTreeCacheSelector.java
new file mode 100644
index 0000000..822f098
--- /dev/null
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/DefaultTreeCacheSelector.java
@@ -0,0 +1,37 @@
+/**
+ * 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.framework.recipes.cache;
+
+/**
+ * Default TreeCache selector - returns true for all methods
+ */
+public class DefaultTreeCacheSelector implements TreeCacheSelector
+{
+    @Override
+    public boolean traverseChildren(String fullPath)
+    {
+        return true;
+    }
+
+    @Override
+    public boolean acceptChild(String fullPath)
+    {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/a01f2a00/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
index 4f3ffb6..eeb54a5 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
@@ -72,6 +72,7 @@ public class TreeCache implements Closeable
 {
     private static final Logger LOG = LoggerFactory.getLogger(TreeCache.class);
     private final boolean createParentNodes;
+    private final TreeCacheSelector selector;
 
     public static final class Builder
     {
@@ -82,6 +83,7 @@ public class TreeCache implements Closeable
         private CloseableExecutorService executorService = null;
         private int maxDepth = Integer.MAX_VALUE;
         private boolean createParentNodes = false;
+        private TreeCacheSelector selector = new DefaultTreeCacheSelector();
 
         private Builder(CuratorFramework client, String path)
         {
@@ -99,7 +101,7 @@ public class TreeCache implements Closeable
             {
                 executor = new CloseableExecutorService(Executors.newSingleThreadExecutor(defaultThreadFactory));
             }
-            return new TreeCache(client, path, cacheData, dataIsCompressed, maxDepth, executor, createParentNodes);
+            return new TreeCache(client, path, cacheData, dataIsCompressed, maxDepth, executor, createParentNodes, selector);
         }
 
         /**
@@ -176,6 +178,18 @@ public class TreeCache implements Closeable
             this.createParentNodes = createParentNodes;
             return this;
         }
+
+        /**
+         * By default, {@link DefaultTreeCacheSelector} is used. Change the selector here.
+         *
+         * @param selector new selector
+         * @return this for chaining
+         */
+        public Builder setSelector(TreeCacheSelector selector)
+        {
+            this.selector = selector;
+            return this;
+        }
     }
 
     /**
@@ -220,7 +234,7 @@ public class TreeCache implements Closeable
 
         private void refresh() throws Exception
         {
-            if (depth < maxDepth)
+            if ((depth < maxDepth) && selector.traverseChildren(path))
             {
                 outstandingOps.addAndGet(2);
                 doRefreshData();
@@ -232,7 +246,7 @@ public class TreeCache implements Closeable
 
         private void refreshChildren() throws Exception
         {
-            if (depth < maxDepth)
+            if ((depth < maxDepth) && selector.traverseChildren(path))
             {
                 outstandingOps.incrementAndGet();
                 doRefreshChildren();
@@ -395,7 +409,7 @@ public class TreeCache implements Closeable
                     List<String> newChildren = new ArrayList<String>();
                     for ( String child : event.getChildren() )
                     {
-                        if ( !childMap.containsKey(child) )
+                        if ( !childMap.containsKey(child) && selector.acceptChild(ZKPaths.makePath(path, child)) )
                         {
                             newChildren.add(child);
                         }
@@ -515,7 +529,7 @@ public class TreeCache implements Closeable
      */
     public TreeCache(CuratorFramework client, String path)
     {
-        this(client, path, true, false, Integer.MAX_VALUE, new CloseableExecutorService(Executors.newSingleThreadExecutor(defaultThreadFactory), true), false);
+        this(client, path, true, false, Integer.MAX_VALUE, new CloseableExecutorService(Executors.newSingleThreadExecutor(defaultThreadFactory), true), false, new DefaultTreeCacheSelector());
     }
 
     /**
@@ -525,16 +539,18 @@ public class TreeCache implements Closeable
      * @param dataIsCompressed if true, data in the path is compressed
      * @param executorService  Closeable ExecutorService to use for the TreeCache's background thread
      * @param createParentNodes true to create parent nodes as containers
+     * @param selector         the selector to use
      */
-    TreeCache(CuratorFramework client, String path, boolean cacheData, boolean dataIsCompressed, int maxDepth, final CloseableExecutorService executorService, boolean createParentNodes)
+    TreeCache(CuratorFramework client, String path, boolean cacheData, boolean dataIsCompressed, int maxDepth, final CloseableExecutorService executorService, boolean createParentNodes, TreeCacheSelector selector)
     {
         this.createParentNodes = createParentNodes;
+        this.selector = Preconditions.checkNotNull(selector, "selector cannot be null");
         this.root = new TreeNode(validatePath(path), null);
-        this.client = client;
+        this.client = Preconditions.checkNotNull(client, "client cannot be null");
         this.cacheData = cacheData;
         this.dataIsCompressed = dataIsCompressed;
         this.maxDepth = maxDepth;
-        this.executorService = executorService;
+        this.executorService = Preconditions.checkNotNull(executorService, "executorService cannot be null");
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/curator/blob/a01f2a00/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCacheSelector.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCacheSelector.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCacheSelector.java
new file mode 100644
index 0000000..5d98ad9
--- /dev/null
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCacheSelector.java
@@ -0,0 +1,66 @@
+/**
+ * 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.framework.recipes.cache;
+
+/**
+ * <p>
+ *     Controls which nodes a TreeCache processes. When iterating
+ *     over the children of a parent node, a given node's children are
+ *     queried only if {@link #traverseChildren(String)} returns true.
+ *     When caching the list of nodes for a parent node, a given node is
+ *     stored only if {@link #acceptChild(String)} returns true.
+ * </p>
+ *
+ * <p>
+ *     E.g. Given:
+ * <pre>
+ * root
+ *     n1-a
+ *     n1-b
+ *         n2-a
+ *         n2-b
+ *             n3-a
+ *     n1-c
+ *     n1-d
+ * </pre>
+ *     You could have a TreeCache only work with the nodes: n1-a, n1-b, n2-a, n2-b, n1-d
+ *     by returning false from traverseChildren() for "/root/n1-b/n2-b" and returning
+ *     false from acceptChild("/root/n1-c").
+ * </p>
+ */
+public interface TreeCacheSelector
+{
+    /**
+     * Return true if children of this path should be cached.
+     * i.e. if false is returned, this node is not queried to
+     * determine if it has children or not
+     *
+     * @param fullPath full path of the ZNode
+     * @return true/false
+     */
+    boolean traverseChildren(String fullPath);
+
+    /**
+     * Return true if this node should be returned from the cache
+     *
+     * @param fullPath full path of the ZNode
+     * @return true/false
+     */
+    boolean acceptChild(String fullPath);
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/a01f2a00/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
index 0bccb54..767b3ae 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
@@ -22,6 +22,7 @@ package org.apache.curator.framework.recipes.cache;
 import com.google.common.collect.ImmutableSet;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.test.KillSession;
+import org.apache.curator.utils.CloseableExecutorService;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.CreateMode;
 import org.testng.Assert;
@@ -31,6 +32,53 @@ import java.util.concurrent.Semaphore;
 public class TestTreeCache extends BaseTestTreeCache
 {
     @Test
+    public void testSelector() throws Exception
+    {
+        client.create().forPath("/root");
+        client.create().forPath("/root/n1-a");
+        client.create().forPath("/root/n1-b");
+        client.create().forPath("/root/n1-b/n2-a");
+        client.create().forPath("/root/n1-b/n2-b");
+        client.create().forPath("/root/n1-b/n2-b/n3-a");
+        client.create().forPath("/root/n1-c");
+        client.create().forPath("/root/n1-d");
+
+        TreeCacheSelector selector = new TreeCacheSelector()
+        {
+            @Override
+            public boolean traverseChildren(String fullPath)
+            {
+                return !fullPath.equals("/root/n1-b/n2-b");
+            }
+
+            @Override
+            public boolean acceptChild(String fullPath)
+            {
+                return !fullPath.equals("/root/n1-c");
+            }
+        };
+        TreeCache treeCache = TreeCache.newBuilder(client, "/root").setSelector(selector).build();
+        try
+        {
+            treeCache.getListenable().addListener(eventListener);
+            treeCache.start();
+
+            assertEvent(TreeCacheEvent.Type.NODE_ADDED, "/root");
+            assertEvent(TreeCacheEvent.Type.NODE_ADDED, "/root/n1-a");
+            assertEvent(TreeCacheEvent.Type.NODE_ADDED, "/root/n1-b");
+            assertEvent(TreeCacheEvent.Type.NODE_ADDED, "/root/n1-d");
+            assertEvent(TreeCacheEvent.Type.NODE_ADDED, "/root/n1-b/n2-a");
+            assertEvent(TreeCacheEvent.Type.NODE_ADDED, "/root/n1-b/n2-b");
+            assertEvent(TreeCacheEvent.Type.INITIALIZED);
+            assertNoMoreEvents();
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(treeCache);
+        }
+    }
+
+    @Test
     public void testStartup() throws Exception
     {
         client.create().forPath("/test");


[4/6] curator git commit: removed unneeded import

Posted by ra...@apache.org.
removed unneeded import


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

Branch: refs/heads/master
Commit: bae4846a0cf45f19d20558837d1ac8457b616c19
Parents: 0a3240f
Author: randgalt <ra...@apache.org>
Authored: Fri Jan 15 12:54:13 2016 -0500
Committer: randgalt <ra...@apache.org>
Committed: Fri Jan 15 12:54:13 2016 -0500

----------------------------------------------------------------------
 .../org/apache/curator/framework/recipes/cache/TestTreeCache.java   | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/bae4846a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
index b5b9c9f..ad7c417 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
@@ -22,7 +22,6 @@ package org.apache.curator.framework.recipes.cache;
 import com.google.common.collect.ImmutableSet;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.test.KillSession;
-import org.apache.curator.utils.CloseableExecutorService;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.CreateMode;
 import org.testng.Assert;


[6/6] curator git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/curator

Posted by ra...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/curator


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

Branch: refs/heads/master
Commit: 5d485b07218a72a7b01fe5cfd69b1b7c7b9910a8
Parents: c1ea06b bef5bb9
Author: randgalt <ra...@apache.org>
Authored: Sun Jan 17 21:34:50 2016 -0500
Committer: randgalt <ra...@apache.org>
Committed: Sun Jan 17 21:34:50 2016 -0500

----------------------------------------------------------------------
 .../framework/api/VersionPathAndBytesable.java  | 25 +++++++++++
 .../transaction/TransactionCreateBuilder.java   |  5 ++-
 .../transaction/TransactionSetDataBuilder.java  |  4 +-
 .../framework/imps/CreateBuilderImpl.java       | 12 ++---
 .../framework/imps/SetDataBuilderImpl.java      |  8 ++--
 .../framework/imps/TestTransactions.java        | 46 +++++++++++++++++++-
 6 files changed, 86 insertions(+), 14 deletions(-)
----------------------------------------------------------------------



[5/6] curator git commit: Merge branch 'master' into CURATOR-288

Posted by ra...@apache.org.
Merge branch 'master' into CURATOR-288


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

Branch: refs/heads/master
Commit: c1ea06b1bb5ada979b17e3d1e1f6a65897d1927e
Parents: bae4846 36a72d9
Author: randgalt <ra...@apache.org>
Authored: Sun Jan 17 21:34:27 2016 -0500
Committer: randgalt <ra...@apache.org>
Committed: Sun Jan 17 21:34:27 2016 -0500

----------------------------------------------------------------------
 .../org/apache/curator/ConnectionState.java     |  4 ++
 .../apache/curator/CuratorZookeeperClient.java  |  2 +
 .../main/java/org/apache/curator/RetryLoop.java |  2 +
 .../apache/curator/SessionFailRetryLoop.java    |  2 +
 .../exhibitor/ExhibitorEnsembleProvider.java    |  2 +
 .../org/apache/curator/utils/ThreadUtils.java   | 23 ++++++++++
 .../src/main/java/locking/LockingExample.java   |  7 ++-
 .../curator/framework/imps/Backgrounding.java   |  2 +
 .../framework/imps/CreateBuilderImpl.java       |  3 ++
 .../framework/imps/CuratorFrameworkImpl.java    | 46 +++++++++++++-------
 .../framework/imps/DeleteBuilderImpl.java       |  2 +
 .../framework/imps/FailedDeleteManager.java     |  5 ++-
 .../FindAndDeleteProtectedNodeInBackground.java |  3 ++
 .../framework/imps/GetDataBuilderImpl.java      |  2 +
 .../curator/framework/imps/NamespaceImpl.java   |  2 +
 .../framework/imps/NamespaceWatcher.java        |  2 +
 .../framework/imps/OperationAndData.java        | 11 ++++-
 .../framework/listen/ListenerContainer.java     |  2 +
 .../framework/state/ConnectionStateManager.java | 14 +++---
 .../recipes/AfterConnectionEstablished.java     |  1 +
 .../framework/recipes/cache/NodeCache.java      |  4 ++
 .../recipes/cache/PathChildrenCache.java        |  4 ++
 .../framework/recipes/cache/TreeCache.java      |  7 +++
 .../framework/recipes/leader/LeaderLatch.java   |  5 +++
 .../recipes/leader/LeaderSelector.java          | 10 ++---
 .../framework/recipes/locks/ChildReaper.java    |  1 +
 .../recipes/locks/InterProcessMultiLock.java    |  4 ++
 .../recipes/locks/InterProcessSemaphore.java    |  4 ++
 .../recipes/locks/InterProcessSemaphoreV2.java  |  2 +
 .../framework/recipes/locks/LockInternals.java  |  2 +
 .../curator/framework/recipes/locks/Reaper.java |  1 +
 .../framework/recipes/nodes/GroupMember.java    |  3 ++
 .../recipes/nodes/PersistentEphemeralNode.java  |  3 ++
 .../recipes/queue/DistributedQueue.java         | 43 ++++++++++--------
 .../framework/recipes/queue/QueueSharder.java   | 16 ++++---
 .../framework/recipes/shared/SharedValue.java   |  2 +
 ...estResetConnectionWithBackgroundFailure.java | 36 +++++++--------
 .../curator/test/TestingZooKeeperMain.java      | 31 ++++++++++++-
 .../entity/JsonServiceInstanceMarshaller.java   |  3 ++
 .../entity/JsonServiceInstancesMarshaller.java  |  2 +
 .../server/rest/DiscoveryResource.java          |  6 +++
 .../discovery/server/rest/InstanceCleanup.java  |  2 +
 .../discovery/details/ServiceDiscoveryImpl.java |  3 ++
 .../x/rpc/idl/discovery/DiscoveryService.java   |  8 ++++
 .../idl/discovery/DiscoveryServiceLowLevel.java |  7 +++
 .../idl/services/CuratorProjectionService.java  | 25 +++++++++++
 46 files changed, 295 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/c1ea06b1/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
----------------------------------------------------------------------


[2/6] curator git commit: Return old data for NODE_REMOVED

Posted by ra...@apache.org.
Return old data for NODE_REMOVED


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

Branch: refs/heads/master
Commit: 6b8113b65dce1958a405186ff8ee3c9b4f14f362
Parents: a01f2a0
Author: randgalt <ra...@apache.org>
Authored: Fri Jan 15 12:23:34 2016 -0500
Committer: randgalt <ra...@apache.org>
Committed: Fri Jan 15 12:23:34 2016 -0500

----------------------------------------------------------------------
 .../org/apache/curator/framework/recipes/cache/TreeCache.java  | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/6b8113b6/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
index eeb54a5..f2ba186 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
@@ -296,8 +296,8 @@ public class TreeCache implements Closeable
 
         void wasDeleted() throws Exception
         {
-            stat.set(null);
-            data.set(null);
+            Stat oldStat = stat.getAndSet(null);
+            byte[] oldData = data.getAndSet(null);
             client.clearWatcherReferences(this);
             ConcurrentMap<String, TreeNode> childMap = children.getAndSet(null);
             if ( childMap != null )
@@ -318,7 +318,7 @@ public class TreeCache implements Closeable
             NodeState oldState = nodeState.getAndSet(NodeState.DEAD);
             if ( oldState == NodeState.LIVE )
             {
-                publishEvent(TreeCacheEvent.Type.NODE_REMOVED, path);
+                publishEvent(TreeCacheEvent.Type.NODE_REMOVED, new ChildData(path, oldStat, oldData));
             }
 
             if ( parent == null )


[3/6] curator git commit: updated tests for new values in NODE_REMOVED

Posted by ra...@apache.org.
updated tests for new values in NODE_REMOVED


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

Branch: refs/heads/master
Commit: 0a3240f05bff69a87dd6a743d3afac61f8954494
Parents: 6b8113b
Author: randgalt <ra...@apache.org>
Authored: Fri Jan 15 12:47:33 2016 -0500
Committer: randgalt <ra...@apache.org>
Committed: Fri Jan 15 12:47:33 2016 -0500

----------------------------------------------------------------------
 .../curator/framework/recipes/cache/TestTreeCache.java    | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/0a3240f0/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
index 767b3ae..b5b9c9f 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
@@ -373,12 +373,12 @@ public class TestTreeCache extends BaseTestTreeCache
         assertEvent(TreeCacheEvent.Type.INITIALIZED);
 
         client.delete().forPath("/test/foo");
-        assertEvent(TreeCacheEvent.Type.NODE_REMOVED, "/test/foo");
+        assertEvent(TreeCacheEvent.Type.NODE_REMOVED, "/test/foo", "one".getBytes());
         client.create().forPath("/test/foo", "two".getBytes());
         assertEvent(TreeCacheEvent.Type.NODE_ADDED, "/test/foo");
 
         client.delete().forPath("/test/foo");
-        assertEvent(TreeCacheEvent.Type.NODE_REMOVED, "/test/foo");
+        assertEvent(TreeCacheEvent.Type.NODE_REMOVED, "/test/foo", "two".getBytes());
         client.create().forPath("/test/foo", "two".getBytes());
         assertEvent(TreeCacheEvent.Type.NODE_ADDED, "/test/foo");
 
@@ -428,7 +428,7 @@ public class TestTreeCache extends BaseTestTreeCache
         assertEvent(TreeCacheEvent.Type.CONNECTION_SUSPENDED);
         assertEvent(TreeCacheEvent.Type.CONNECTION_LOST);
         assertEvent(TreeCacheEvent.Type.CONNECTION_RECONNECTED);
-        assertEvent(TreeCacheEvent.Type.NODE_REMOVED, "/test/me");
+        assertEvent(TreeCacheEvent.Type.NODE_REMOVED, "/test/me", "data".getBytes());
 
         assertNoMoreEvents();
     }
@@ -460,7 +460,7 @@ public class TestTreeCache extends BaseTestTreeCache
         Assert.assertEquals(new String(cache.getCurrentData("/test/one").getData()), "sup!");
 
         client.delete().forPath("/test/one");
-        assertEvent(TreeCacheEvent.Type.NODE_REMOVED, "/test/one");
+        assertEvent(TreeCacheEvent.Type.NODE_REMOVED, "/test/one", "sup!".getBytes());
         Assert.assertEquals(cache.getCurrentChildren("/test").keySet(), ImmutableSet.of());
 
         assertNoMoreEvents();
@@ -508,7 +508,7 @@ public class TestTreeCache extends BaseTestTreeCache
             Assert.assertEquals(new String(cache2.getCurrentData("/test/one").getData()), "sup!");
 
             client.delete().forPath("/test/one");
-            assertEvent(TreeCacheEvent.Type.NODE_REMOVED, "/test/one");
+            assertEvent(TreeCacheEvent.Type.NODE_REMOVED, "/test/one", "sup!".getBytes());
             Assert.assertNull(cache.getCurrentData("/test/one"));
             semaphore.acquire();
             Assert.assertNull(cache2.getCurrentData("/test/one"));