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 22:04:04 UTC

[01/11] curator git commit: CURATOR-278: In transactions compressed option does not work with some fluent api combinations

Repository: curator
Updated Branches:
  refs/heads/CURATOR-3.0 9a03ea939 -> ae8dc466c


CURATOR-278: In transactions compressed option does not work with some fluent api combinations


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

Branch: refs/heads/CURATOR-3.0
Commit: c1e2a308d891cba4251adcb18a91d074077979b9
Parents: 31c0465
Author: Grant Henke <gr...@gmail.com>
Authored: Sat Nov 7 13:39:54 2015 -0600
Committer: Grant Henke <gr...@gmail.com>
Committed: Sat Nov 7 13:39:54 2015 -0600

----------------------------------------------------------------------
 .../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(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/c1e2a308/curator-framework/src/main/java/org/apache/curator/framework/api/VersionPathAndBytesable.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/VersionPathAndBytesable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/VersionPathAndBytesable.java
new file mode 100644
index 0000000..556fc67
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/VersionPathAndBytesable.java
@@ -0,0 +1,25 @@
+/**
+ * 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.api;
+
+public interface VersionPathAndBytesable<T> extends
+        Versionable<PathAndBytesable<T>>,
+        PathAndBytesable<T>
+{
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/c1e2a308/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java
index 6ac3069..f2ac146 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java
@@ -18,6 +18,7 @@
  */
 package org.apache.curator.framework.api.transaction;
 
+import org.apache.curator.framework.api.ACLCreateModePathAndBytesable;
 import org.apache.curator.framework.api.ACLPathAndBytesable;
 import org.apache.curator.framework.api.Compressible;
 import org.apache.curator.framework.api.CreateModable;
@@ -27,6 +28,6 @@ public interface TransactionCreateBuilder extends
     PathAndBytesable<CuratorTransactionBridge>,
     CreateModable<ACLPathAndBytesable<CuratorTransactionBridge>>,
     ACLPathAndBytesable<CuratorTransactionBridge>,
-    Compressible<ACLPathAndBytesable<CuratorTransactionBridge>>
-{
+    ACLCreateModePathAndBytesable<CuratorTransactionBridge>,
+    Compressible<ACLCreateModePathAndBytesable<CuratorTransactionBridge>>  {
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/c1e2a308/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java
index 777537a..cc9e01c 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java
@@ -20,11 +20,13 @@ package org.apache.curator.framework.api.transaction;
 
 import org.apache.curator.framework.api.Compressible;
 import org.apache.curator.framework.api.PathAndBytesable;
+import org.apache.curator.framework.api.VersionPathAndBytesable;
 import org.apache.curator.framework.api.Versionable;
 
 public interface TransactionSetDataBuilder extends
     PathAndBytesable<CuratorTransactionBridge>,
     Versionable<PathAndBytesable<CuratorTransactionBridge>>,
-    Compressible<PathAndBytesable<CuratorTransactionBridge>>
+    VersionPathAndBytesable<CuratorTransactionBridge>,
+    Compressible<VersionPathAndBytesable<CuratorTransactionBridge>>
 {
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/c1e2a308/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
index b72b7b6..39dd871 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
@@ -89,14 +89,14 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
                 CreateBuilderImpl.this.withMode(mode);
                 return this;
             }
-            
+
             @Override
-            public ACLPathAndBytesable<CuratorTransactionBridge> compressed()
+            public ACLCreateModePathAndBytesable<CuratorTransactionBridge> compressed()
             {
                 CreateBuilderImpl.this.compressed();
                 return this;
             }
-            
+
             @Override
             public CuratorTransactionBridge forPath(String path) throws Exception
             {
@@ -105,12 +105,12 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
 
             @Override
             public CuratorTransactionBridge forPath(String path, byte[] data) throws Exception
-            {               
+            {
                 if ( compress )
                 {
                     data = client.getCompressionProvider().compress(path, data);
                 }
-                
+
                 String fixedPath = client.fixForNamespace(path);
                 transaction.add(Op.create(fixedPath, data, acling.getAclList(path), createMode), OperationType.CREATE, path);
                 return curatorTransaction;
@@ -493,7 +493,7 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
                 */
                 protectedId = UUID.randomUUID().toString();
             }
-            
+
             throw e;
         }
     }

http://git-wip-us.apache.org/repos/asf/curator/blob/c1e2a308/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java
index 8e93cbf..4117930 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java
@@ -20,7 +20,6 @@ package org.apache.curator.framework.imps;
 
 import org.apache.curator.RetryLoop;
 import org.apache.curator.TimeTrace;
-import org.apache.curator.framework.api.ACLPathAndBytesable;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.BackgroundPathAndBytesable;
 import org.apache.curator.framework.api.CuratorEvent;
@@ -28,6 +27,7 @@ import org.apache.curator.framework.api.CuratorEventType;
 import org.apache.curator.framework.api.PathAndBytesable;
 import org.apache.curator.framework.api.SetDataBackgroundVersionable;
 import org.apache.curator.framework.api.SetDataBuilder;
+import org.apache.curator.framework.api.VersionPathAndBytesable;
 import org.apache.curator.framework.api.transaction.CuratorTransactionBridge;
 import org.apache.curator.framework.api.transaction.OperationType;
 import org.apache.curator.framework.api.transaction.TransactionSetDataBuilder;
@@ -64,7 +64,7 @@ class SetDataBuilderImpl implements SetDataBuilder, BackgroundOperation<PathAndB
                 {
                     data = client.getCompressionProvider().compress(path, data);
                 }
-                
+
                 String      fixedPath = client.fixForNamespace(path);
                 transaction.add(Op.setData(fixedPath, data, version), OperationType.SET_DATA, path);
                 return curatorTransaction;
@@ -84,9 +84,9 @@ class SetDataBuilderImpl implements SetDataBuilder, BackgroundOperation<PathAndB
             }
 
             @Override
-            public PathAndBytesable<CuratorTransactionBridge> compressed() {
+            public VersionPathAndBytesable<CuratorTransactionBridge> compressed() {
                 compress = true;
-                
+
                 return this;
             }
         };

http://git-wip-us.apache.org/repos/asf/curator/blob/c1e2a308/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactions.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactions.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactions.java
index ae2cf1d..fedc779 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactions.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactions.java
@@ -27,6 +27,7 @@ import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.Stat;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -59,7 +60,7 @@ public class TestTransactions extends BaseClassForTests
             {
                 // correct
             }
-            
+
             Assert.assertNull(client.checkExists().forPath("/bar"));
         }
         finally
@@ -106,6 +107,49 @@ public class TestTransactions extends BaseClassForTests
     }
 
     @Test
+    public void     testWithCompression() throws Exception
+    {
+        CuratorFramework        client = CuratorFrameworkFactory.builder().connectString(server.getConnectString()).retryPolicy(new RetryOneTime(1)).namespace("galt").build();
+        client.start();
+        try
+        {
+            Collection<CuratorTransactionResult>    results =
+                    client.inTransaction()
+                        .create().compressed().forPath("/foo", "one".getBytes())
+                    .and()
+                        .create().compressed().withACL(ZooDefs.Ids.READ_ACL_UNSAFE).forPath("/bar", "two".getBytes())
+                    .and()
+                        .create().compressed().withMode(CreateMode.PERSISTENT_SEQUENTIAL).forPath("/test-", "three".getBytes())
+                    .and()
+                        .create().compressed().withMode(CreateMode.PERSISTENT).withACL(ZooDefs.Ids.READ_ACL_UNSAFE).forPath("/baz", "four".getBytes())
+                    .and()
+                        .setData().compressed().withVersion(0).forPath("/foo", "five".getBytes())
+                    .and()
+                        .commit();
+
+            Assert.assertTrue(client.checkExists().forPath("/foo") != null);
+            Assert.assertEquals(client.getData().decompressed().forPath("/foo"), "five".getBytes());
+
+            Assert.assertTrue(client.checkExists().forPath("/bar") != null);
+            Assert.assertEquals(client.getData().decompressed().forPath("/bar"), "two".getBytes());
+            Assert.assertEquals(client.getACL().forPath("/bar"), ZooDefs.Ids.READ_ACL_UNSAFE);
+
+            CuratorTransactionResult    ephemeralResult = Iterables.find(results, CuratorTransactionResult.ofTypeAndPath(OperationType.CREATE, "/test-"));
+            Assert.assertNotNull(ephemeralResult);
+            Assert.assertNotEquals(ephemeralResult.getResultPath(), "/test-");
+            Assert.assertTrue(ephemeralResult.getResultPath().startsWith("/test-"));
+
+            Assert.assertTrue(client.checkExists().forPath("/baz") != null);
+            Assert.assertEquals(client.getData().decompressed().forPath("/baz"), "four".getBytes());
+            Assert.assertEquals(client.getACL().forPath("/baz"), ZooDefs.Ids.READ_ACL_UNSAFE);
+        }
+        finally
+        {
+            client.close();
+        }
+    }
+
+    @Test
     public void     testBasic() throws Exception
     {
         CuratorFramework        client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));


[03/11] 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/CURATOR-3.0
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 )


[09/11] curator git commit: Merge branch 'master' into CURATOR-3.0

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

Conflicts:
	curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java
	curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java
	curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
	curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java
	curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java


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

Branch: refs/heads/CURATOR-3.0
Commit: ba38d4c5ea59466c4f1dcda6dadcab9bf3262257
Parents: 9a03ea9 5d485b0
Author: randgalt <ra...@apache.org>
Authored: Mon Jan 18 15:57:02 2016 -0500
Committer: randgalt <ra...@apache.org>
Committed: Mon Jan 18 15:57:02 2016 -0500

----------------------------------------------------------------------
 .../framework/api/VersionPathAndBytesable.java  | 25 ++++++++
 .../transaction/TransactionCreateBuilder.java   |  2 +
 .../transaction/TransactionSetDataBuilder.java  |  2 +
 .../framework/imps/CreateBuilderImpl.java       | 10 +--
 .../framework/imps/SetDataBuilderImpl.java      |  2 +-
 .../framework/imps/TestTransactionsOld.java     | 46 +++++++++++++-
 .../recipes/cache/DefaultTreeCacheSelector.java | 37 +++++++++++
 .../framework/recipes/cache/TreeCache.java      | 39 +++++++++---
 .../recipes/cache/TreeCacheSelector.java        | 66 ++++++++++++++++++++
 .../framework/recipes/cache/TestTreeCache.java  | 57 +++++++++++++++--
 10 files changed, 264 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/ba38d4c5/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java
----------------------------------------------------------------------
diff --cc curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java
index cba0cba,f2ac146..d20c834
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java
@@@ -23,10 -24,10 +24,11 @@@ import org.apache.curator.framework.api
  import org.apache.curator.framework.api.CreateModable;
  import org.apache.curator.framework.api.PathAndBytesable;
  
 -public interface TransactionCreateBuilder extends
 -    PathAndBytesable<CuratorTransactionBridge>,
 -    CreateModable<ACLPathAndBytesable<CuratorTransactionBridge>>,
 -    ACLPathAndBytesable<CuratorTransactionBridge>,
 -    ACLCreateModePathAndBytesable<CuratorTransactionBridge>,
 -    Compressible<ACLCreateModePathAndBytesable<CuratorTransactionBridge>>  {
 +public interface TransactionCreateBuilder<T> extends
 +    PathAndBytesable<T>,
 +    CreateModable<ACLPathAndBytesable<T>>,
 +    ACLPathAndBytesable<T>,
++    ACLCreateModePathAndBytesable<T>,
 +    Compressible<ACLPathAndBytesable<T>>
 +{
  }

http://git-wip-us.apache.org/repos/asf/curator/blob/ba38d4c5/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java
----------------------------------------------------------------------
diff --cc curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java
index 2d4d255,cc9e01c..84b25bb
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java
@@@ -20,11 -20,13 +20,13 @@@ package org.apache.curator.framework.ap
  
  import org.apache.curator.framework.api.Compressible;
  import org.apache.curator.framework.api.PathAndBytesable;
+ import org.apache.curator.framework.api.VersionPathAndBytesable;
  import org.apache.curator.framework.api.Versionable;
  
 -public interface TransactionSetDataBuilder extends
 -    PathAndBytesable<CuratorTransactionBridge>,
 -    Versionable<PathAndBytesable<CuratorTransactionBridge>>,
 -    VersionPathAndBytesable<CuratorTransactionBridge>,
 -    Compressible<VersionPathAndBytesable<CuratorTransactionBridge>>
 +public interface TransactionSetDataBuilder<T> extends
 +    PathAndBytesable<T>,
 +    Versionable<PathAndBytesable<T>>,
++    VersionPathAndBytesable<T>,
 +    Compressible<PathAndBytesable<T>>
  {
  }

http://git-wip-us.apache.org/repos/asf/curator/blob/ba38d4c5/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
----------------------------------------------------------------------
diff --cc curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
index ada4bae,e11d74f..4d532ff
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
@@@ -101,31 -89,31 +101,31 @@@ class CreateBuilderImpl implements Crea
                  CreateBuilderImpl.this.withMode(mode);
                  return this;
              }
-             
+ 
              @Override
 -            public ACLCreateModePathAndBytesable<CuratorTransactionBridge> compressed()
 +            public ACLPathAndBytesable<T> compressed()
              {
                  CreateBuilderImpl.this.compressed();
                  return this;
              }
-             
+ 
              @Override
 -            public CuratorTransactionBridge forPath(String path) throws Exception
 +            public T forPath(String path) throws Exception
              {
                  return forPath(path, client.getDefaultData());
              }
  
              @Override
 -            public CuratorTransactionBridge forPath(String path, byte[] data) throws Exception
 +            public T forPath(String path, byte[] data) throws Exception
-             {               
+             {
                  if ( compress )
                  {
                      data = client.getCompressionProvider().compress(path, data);
                  }
-                 
+ 
                  String fixedPath = client.fixForNamespace(path);
                  transaction.add(Op.create(fixedPath, data, acling.getAclList(path), createMode), OperationType.CREATE, path);
 -                return curatorTransaction;
 +                return context;
              }
          };
      }

http://git-wip-us.apache.org/repos/asf/curator/blob/ba38d4c5/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java
----------------------------------------------------------------------
diff --cc curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java
index 3ea704c,4117930..b150783
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java
@@@ -61,10 -64,10 +61,10 @@@ class SetDataBuilderImpl implements Set
                  {
                      data = client.getCompressionProvider().compress(path, data);
                  }
-                 
+ 
                  String      fixedPath = client.fixForNamespace(path);
                  transaction.add(Op.setData(fixedPath, data, version), OperationType.SET_DATA, path);
 -                return curatorTransaction;
 +                return context;
              }
  
              @Override

http://git-wip-us.apache.org/repos/asf/curator/blob/ba38d4c5/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactionsOld.java
----------------------------------------------------------------------
diff --cc curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactionsOld.java
index f0147d5,0000000..2c42d61
mode 100644,000000..100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactionsOld.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactionsOld.java
@@@ -1,141 -1,0 +1,185 @@@
 +/**
 + * 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.imps;
 +
 +import com.google.common.collect.Iterables;
 +import org.apache.curator.framework.CuratorFramework;
 +import org.apache.curator.framework.CuratorFrameworkFactory;
 +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.utils.CloseableUtils;
 +import org.apache.zookeeper.CreateMode;
 +import org.apache.zookeeper.KeeperException;
++import org.apache.zookeeper.ZooDefs;
 +import org.apache.zookeeper.data.Stat;
 +import org.testng.Assert;
 +import org.testng.annotations.Test;
 +import java.util.Collection;
 +
 +@SuppressWarnings("deprecation")
 +public class TestTransactionsOld extends BaseClassForTests
 +{
 +    @Test
 +    public void     testCheckVersion() throws Exception
 +    {
 +        CuratorFramework        client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
 +        try
 +        {
 +            client.start();
 +            client.create().forPath("/foo");
 +            Stat        stat = client.setData().forPath("/foo", "new".getBytes());  // up the version
 +
 +            try
 +            {
 +                client.inTransaction()
 +                    .check().withVersion(stat.getVersion() + 1).forPath("/foo") // force a bad version
 +                .and()
 +                    .create().forPath("/bar")
 +                .and()
 +                    .commit();
 +
 +                Assert.fail();
 +            }
 +            catch ( KeeperException.BadVersionException correct )
 +            {
 +                // correct
 +            }
-             
++
 +            Assert.assertNull(client.checkExists().forPath("/bar"));
 +        }
 +        finally
 +        {
 +            CloseableUtils.closeQuietly(client);
 +        }
 +    }
 +
 +    @Test
 +    public void     testWithNamespace() throws Exception
 +    {
 +        CuratorFramework        client = CuratorFrameworkFactory.builder().connectString(server.getConnectString()).retryPolicy(new RetryOneTime(1)).namespace("galt").build();
 +        try
 +        {
 +            client.start();
 +            Collection<CuratorTransactionResult>    results =
 +                client.inTransaction()
 +                    .create().forPath("/foo", "one".getBytes())
 +                .and()
 +                    .create().withMode(CreateMode.PERSISTENT_SEQUENTIAL).forPath("/test-", "one".getBytes())
 +                .and()
 +                    .setData().forPath("/foo", "two".getBytes())
 +                .and()
 +                    .create().forPath("/foo/bar")
 +                .and()
 +                    .delete().forPath("/foo/bar")
 +                .and()
 +                    .commit();
 +
 +            Assert.assertTrue(client.checkExists().forPath("/foo") != null);
 +            Assert.assertTrue(client.usingNamespace(null).checkExists().forPath("/galt/foo") != null);
 +            Assert.assertEquals(client.getData().forPath("/foo"), "two".getBytes());
 +            Assert.assertTrue(client.checkExists().forPath("/foo/bar") == null);
 +
 +            CuratorTransactionResult    ephemeralResult = Iterables.find(results, CuratorTransactionResult.ofTypeAndPath(OperationType.CREATE, "/test-"));
 +            Assert.assertNotNull(ephemeralResult);
 +            Assert.assertNotEquals(ephemeralResult.getResultPath(), "/test-");
 +            Assert.assertTrue(ephemeralResult.getResultPath().startsWith("/test-"));
 +        }
 +        finally
 +        {
 +            CloseableUtils.closeQuietly(client);
 +        }
 +    }
 +
 +    @Test
++    public void     testWithCompression() throws Exception
++    {
++        CuratorFramework        client = CuratorFrameworkFactory.builder().connectString(server.getConnectString()).retryPolicy(new RetryOneTime(1)).namespace("galt").build();
++        client.start();
++        try
++        {
++            Collection<CuratorTransactionResult>    results =
++                    client.inTransaction()
++                        .create().compressed().forPath("/foo", "one".getBytes())
++                    .and()
++                        .create().compressed().withACL(ZooDefs.Ids.READ_ACL_UNSAFE).forPath("/bar", "two".getBytes())
++                    .and()
++                        .create().compressed().withMode(CreateMode.PERSISTENT_SEQUENTIAL).forPath("/test-", "three".getBytes())
++                    .and()
++                        .create().compressed().withMode(CreateMode.PERSISTENT).withACL(ZooDefs.Ids.READ_ACL_UNSAFE).forPath("/baz", "four".getBytes())
++                    .and()
++                        .setData().compressed().withVersion(0).forPath("/foo", "five".getBytes())
++                    .and()
++                        .commit();
++
++            Assert.assertTrue(client.checkExists().forPath("/foo") != null);
++            Assert.assertEquals(client.getData().decompressed().forPath("/foo"), "five".getBytes());
++
++            Assert.assertTrue(client.checkExists().forPath("/bar") != null);
++            Assert.assertEquals(client.getData().decompressed().forPath("/bar"), "two".getBytes());
++            Assert.assertEquals(client.getACL().forPath("/bar"), ZooDefs.Ids.READ_ACL_UNSAFE);
++
++            CuratorTransactionResult    ephemeralResult = Iterables.find(results, CuratorTransactionResult.ofTypeAndPath(OperationType.CREATE, "/test-"));
++            Assert.assertNotNull(ephemeralResult);
++            Assert.assertNotEquals(ephemeralResult.getResultPath(), "/test-");
++            Assert.assertTrue(ephemeralResult.getResultPath().startsWith("/test-"));
++
++            Assert.assertTrue(client.checkExists().forPath("/baz") != null);
++            Assert.assertEquals(client.getData().decompressed().forPath("/baz"), "four".getBytes());
++            Assert.assertEquals(client.getACL().forPath("/baz"), ZooDefs.Ids.READ_ACL_UNSAFE);
++        }
++        finally
++        {
++            client.close();
++        }
++    }
++
++    @Test
 +    public void     testBasic() throws Exception
 +    {
 +        CuratorFramework        client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
 +        try
 +        {
 +            client.start();
 +            Collection<CuratorTransactionResult>    results =
 +                client.inTransaction()
 +                    .create().forPath("/foo")
 +                .and()
 +                    .create().forPath("/foo/bar", "snafu".getBytes())
 +                .and()
 +                    .commit();
 +
 +            Assert.assertTrue(client.checkExists().forPath("/foo/bar") != null);
 +            Assert.assertEquals(client.getData().forPath("/foo/bar"), "snafu".getBytes());
 +
 +            CuratorTransactionResult    fooResult = Iterables.find(results, CuratorTransactionResult.ofTypeAndPath(OperationType.CREATE, "/foo"));
 +            CuratorTransactionResult    fooBarResult = Iterables.find(results, CuratorTransactionResult.ofTypeAndPath(OperationType.CREATE, "/foo/bar"));
 +            Assert.assertNotNull(fooResult);
 +            Assert.assertNotNull(fooBarResult);
 +            Assert.assertNotSame(fooResult, fooBarResult);
 +            Assert.assertEquals(fooResult.getResultPath(), "/foo");
 +            Assert.assertEquals(fooBarResult.getResultPath(), "/foo/bar");
 +        }
 +        finally
 +        {
 +            CloseableUtils.closeQuietly(client);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/curator/blob/ba38d4c5/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
----------------------------------------------------------------------
diff --cc curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
index 2558984,4d00266..81590f7
--- 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
@@@ -289,8 -296,9 +303,10 @@@ public class TreeCache implements Close
  
          void wasDeleted() throws Exception
          {
-             stat.set(null);
-             data.set(null);
+             Stat oldStat = stat.getAndSet(null);
+             byte[] oldData = data.getAndSet(null);
 -            client.clearWatcherReferences(this);
++            client.watches().remove(this).ofType(WatcherType.Any).inBackground().forPath(path);
++
              ConcurrentMap<String, TreeNode> childMap = children.getAndSet(null);
              if ( childMap != null )
              {
@@@ -532,12 -540,14 +548,15 @@@
       * @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 = Preconditions.checkNotNull(client, "client cannot be null");
++        Preconditions.checkNotNull(client, "client cannot be null");
 +        this.client = client.newWatcherRemoveCuratorFramework();
          this.cacheData = cacheData;
          this.dataIsCompressed = dataIsCompressed;
          this.maxDepth = maxDepth;

http://git-wip-us.apache.org/repos/asf/curator/blob/ba38d4c5/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
----------------------------------------------------------------------
diff --cc curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
index 151ea7e,ad7c417..173fcd7
--- 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
@@@ -377,9 -424,10 +424,9 @@@ public class TestTreeCache extends Base
          assertEvent(TreeCacheEvent.Type.NODE_ADDED, "/test/me");
  
          KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString());
 -        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();
      }


[10/11] curator git commit: work on merging compressed transaction code

Posted by ra...@apache.org.
work on merging compressed transaction code


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

Branch: refs/heads/CURATOR-3.0
Commit: 82a1df4e2635a1d7fc04c667dc09bdad513b991b
Parents: ba38d4c
Author: randgalt <ra...@apache.org>
Authored: Mon Jan 18 16:01:43 2016 -0500
Committer: randgalt <ra...@apache.org>
Committed: Mon Jan 18 16:01:43 2016 -0500

----------------------------------------------------------------------
 .../framework/api/transaction/TransactionCreateBuilder.java        | 2 +-
 .../java/org/apache/curator/framework/imps/CreateBuilderImpl.java  | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/82a1df4e/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java
index d20c834..f7b2850 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java
@@ -29,6 +29,6 @@ public interface TransactionCreateBuilder<T> extends
     CreateModable<ACLPathAndBytesable<T>>,
     ACLPathAndBytesable<T>,
     ACLCreateModePathAndBytesable<T>,
-    Compressible<ACLPathAndBytesable<T>>
+    Compressible<ACLCreateModePathAndBytesable<T>>
 {
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/82a1df4e/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
index 4d532ff..2eca53c 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
@@ -103,7 +103,7 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
             }
 
             @Override
-            public ACLPathAndBytesable<T> compressed()
+            public ACLCreateModePathAndBytesable<T> compressed()
             {
                 CreateBuilderImpl.this.compressed();
                 return this;


[05/11] 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/CURATOR-3.0
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;


[08/11] 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/CURATOR-3.0
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(-)
----------------------------------------------------------------------



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

Posted by ra...@apache.org.
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/CURATOR-3.0
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");


[04/11] 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/CURATOR-3.0
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"));


[07/11] 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/CURATOR-3.0
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
----------------------------------------------------------------------


[11/11] curator git commit: work on merging compressed transaction code

Posted by ra...@apache.org.
work on merging compressed transaction code


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

Branch: refs/heads/CURATOR-3.0
Commit: ae8dc466cfd1ed31490bb65321406fabd0e5e560
Parents: 82a1df4
Author: randgalt <ra...@apache.org>
Authored: Mon Jan 18 16:03:55 2016 -0500
Committer: randgalt <ra...@apache.org>
Committed: Mon Jan 18 16:03:55 2016 -0500

----------------------------------------------------------------------
 .../framework/api/transaction/TransactionSetDataBuilder.java      | 2 +-
 .../org/apache/curator/framework/imps/SetDataBuilderImpl.java     | 3 ++-
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/ae8dc466/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java
index 84b25bb..42a0820 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java
@@ -27,6 +27,6 @@ public interface TransactionSetDataBuilder<T> extends
     PathAndBytesable<T>,
     Versionable<PathAndBytesable<T>>,
     VersionPathAndBytesable<T>,
-    Compressible<PathAndBytesable<T>>
+    Compressible<VersionPathAndBytesable<T>>
 {
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/ae8dc466/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java
index b150783..7057c11 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java
@@ -27,6 +27,7 @@ import org.apache.curator.framework.api.CuratorEventType;
 import org.apache.curator.framework.api.PathAndBytesable;
 import org.apache.curator.framework.api.SetDataBackgroundVersionable;
 import org.apache.curator.framework.api.SetDataBuilder;
+import org.apache.curator.framework.api.VersionPathAndBytesable;
 import org.apache.curator.framework.api.transaction.OperationType;
 import org.apache.curator.framework.api.transaction.TransactionSetDataBuilder;
 import org.apache.zookeeper.AsyncCallback;
@@ -81,7 +82,7 @@ class SetDataBuilderImpl implements SetDataBuilder, BackgroundOperation<PathAndB
             }
 
             @Override
-            public PathAndBytesable<T> compressed()
+            public VersionPathAndBytesable<T> compressed()
             {
                 compress = true;
                 return this;


[06/11] curator git commit: Merge branch 'CURATOR-278' of https://github.com/granthenke/curator into CURATOR-278

Posted by ra...@apache.org.
Merge branch 'CURATOR-278' of https://github.com/granthenke/curator into CURATOR-278


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

Branch: refs/heads/CURATOR-3.0
Commit: bef5bb9bd2ff5ba7862d89120b5d3443587bd39a
Parents: 36a72d9 c1e2a30
Author: Cam McKenzie <ca...@apache.org>
Authored: Mon Jan 18 09:28:49 2016 +1100
Committer: Cam McKenzie <ca...@apache.org>
Committed: Mon Jan 18 09:28:49 2016 +1100

----------------------------------------------------------------------
 .../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(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/bef5bb9b/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
----------------------------------------------------------------------