You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@curator.apache.org by ca...@apache.org on 2015/04/22 01:07:41 UTC

[44/50] curator git commit: If the server successfully creates a Curator-protected node by crashes before returning to the client, Curator will incorrectly set the wrong value for the name field in the background Event causing this problem. 'name' should

If the server successfully creates a Curator-protected node by crashes before returning to the client, Curator will incorrectly set the wrong value for the name field in the background Event causing this problem. 'name' should be the full path instead of just the ZNode name


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

Branch: refs/heads/CURATOR-154
Commit: c805679c8d4307360710a24cddc071cbbcb5ca75
Parents: c9a5c4a
Author: randgalt <ra...@apache.org>
Authored: Mon Apr 20 20:16:40 2015 -0500
Committer: randgalt <ra...@apache.org>
Committed: Mon Apr 20 20:16:40 2015 -0500

----------------------------------------------------------------------
 .../framework/imps/CreateBuilderImpl.java       |  14 +-
 .../curator/framework/imps/TestFramework.java   | 215 +++++++++----------
 .../framework/imps/TestFrameworkEdges.java      |  86 +++++++-
 3 files changed, 192 insertions(+), 123 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/c805679c/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 5244d04..6eef4d4 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
@@ -597,6 +597,9 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
         };
     }
 
+    @VisibleForTesting
+    volatile boolean debugForceFindProtectedNode = false;
+
     private void pathInBackground(final String path, final byte[] data, final String givenPath)
     {
         final AtomicBoolean firstTime = new AtomicBoolean(true);
@@ -620,9 +623,10 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
             void callPerformBackgroundOperation() throws Exception
             {
                 boolean callSuper = true;
-                boolean localFirstTime = firstTime.getAndSet(false);
+                boolean localFirstTime = firstTime.getAndSet(false) && !debugForceFindProtectedNode;
                 if ( !localFirstTime && doProtected )
                 {
+                    debugForceFindProtectedNode = false;
                     String createdPath = null;
                     try
                     {
@@ -637,7 +641,7 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
                     {
                         try
                         {
-                            sendBackgroundResponse(KeeperException.Code.OK.intValue(), createdPath, backgrounding.getContext(), ZKPaths.getNodeFromPath(createdPath), this);
+                            sendBackgroundResponse(KeeperException.Code.OK.intValue(), createdPath, backgrounding.getContext(), createdPath, this);
                         }
                         catch ( Exception e )
                         {
@@ -676,11 +680,12 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
                     @Override
                     public String call() throws Exception
                     {
-                        boolean localFirstTime = firstTime.getAndSet(false);
+                        boolean localFirstTime = firstTime.getAndSet(false) && !debugForceFindProtectedNode;
 
                         String createdPath = null;
                         if ( !localFirstTime && doProtected )
                         {
+                            debugForceFindProtectedNode = false;
                             createdPath = findProtectedNodeInForeground(path);
                         }
 
@@ -751,7 +756,8 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
         return returnPath;
     }
 
-    private String adjustPath(String path) throws Exception
+    @VisibleForTesting
+    String adjustPath(String path) throws Exception
     {
         if ( doProtected )
         {

http://git-wip-us.apache.org/repos/asf/curator/blob/c805679c/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFramework.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFramework.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFramework.java
index 0d98f1d..22276e6 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFramework.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFramework.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
  * 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
@@ -16,10 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.framework.imps;
 
 import com.google.common.collect.Lists;
-
 import org.apache.curator.framework.AuthInfo;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
@@ -43,7 +43,6 @@ import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
 import org.testng.Assert;
 import org.testng.annotations.Test;
-
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.BlockingQueue;
@@ -55,14 +54,14 @@ import java.util.concurrent.TimeUnit;
 public class TestFramework extends BaseClassForTests
 {
     @Test
-    public void     testConnectionState() throws Exception
+    public void testConnectionState() throws Exception
     {
-        Timing                  timing = new Timing();
-        CuratorFramework        client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
+        Timing timing = new Timing();
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
         try
         {
-            final BlockingQueue<ConnectionState>        queue = new LinkedBlockingQueue<ConnectionState>();
-            ConnectionStateListener                     listener = new ConnectionStateListener()
+            final BlockingQueue<ConnectionState> queue = new LinkedBlockingQueue<ConnectionState>();
+            ConnectionStateListener listener = new ConnectionStateListener()
             {
                 @Override
                 public void stateChanged(CuratorFramework client, ConnectionState newState)
@@ -85,15 +84,15 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testNamespaceWithWatcher() throws Exception
+    public void testNamespaceWithWatcher() throws Exception
     {
         CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder();
         CuratorFramework client = builder.connectString(server.getConnectString()).namespace("aisa").retryPolicy(new RetryOneTime(1)).build();
         client.start();
         try
         {
-            final BlockingQueue<String>     queue = new LinkedBlockingQueue<String>();
-            Watcher                         watcher = new Watcher()
+            final BlockingQueue<String> queue = new LinkedBlockingQueue<String>();
+            Watcher watcher = new Watcher()
             {
                 @Override
                 public void process(WatchedEvent event)
@@ -112,7 +111,7 @@ public class TestFramework extends BaseClassForTests
             client.getChildren().usingWatcher(watcher).forPath("/base");
             client.create().forPath("/base/child");
 
-            String      path = queue.take();
+            String path = queue.take();
             Assert.assertEquals(path, "/base");
         }
         finally
@@ -122,15 +121,15 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testNamespaceInBackground() throws Exception
+    public void testNamespaceInBackground() throws Exception
     {
         CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder();
         CuratorFramework client = builder.connectString(server.getConnectString()).namespace("aisa").retryPolicy(new RetryOneTime(1)).build();
         client.start();
         try
         {
-            final BlockingQueue<String>     queue = new LinkedBlockingQueue<String>();
-            CuratorListener                 listener = new CuratorListener()
+            final BlockingQueue<String> queue = new LinkedBlockingQueue<String>();
+            CuratorListener listener = new CuratorListener()
             {
                 @Override
                 public void eventReceived(CuratorFramework client, CuratorEvent event) throws Exception
@@ -146,12 +145,12 @@ public class TestFramework extends BaseClassForTests
             client.create().forPath("/base");
             client.checkExists().inBackground().forPath("/base");
 
-            String      path = queue.poll(10, TimeUnit.SECONDS);
+            String path = queue.poll(10, TimeUnit.SECONDS);
             Assert.assertEquals(path, "/base");
 
             client.getCuratorListenable().removeListener(listener);
 
-            BackgroundCallback      callback = new BackgroundCallback()
+            BackgroundCallback callback = new BackgroundCallback()
             {
                 @Override
                 public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
@@ -170,7 +169,7 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testCreateACLSingleAuth() throws Exception
+    public void testCreateACLSingleAuth() throws Exception
     {
         CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder();
         CuratorFramework client = builder
@@ -224,10 +223,10 @@ public class TestFramework extends BaseClassForTests
         {
             client.close();
         }
-    }    
+    }
 
     @Test
-    public void     testACLDeprecatedApis() throws Exception
+    public void testACLDeprecatedApis() throws Exception
     {
         CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder()
             .connectString(server.getConnectString())
@@ -281,10 +280,10 @@ public class TestFramework extends BaseClassForTests
 
             // Try setting data with me1:pass1
             client = builder
-                    .connectString(server.getConnectString())
-                    .authorization("digest", "me2:pass2".getBytes())
-                    .retryPolicy(new RetryOneTime(1))
-                    .build();
+                .connectString(server.getConnectString())
+                .authorization("digest", "me2:pass2".getBytes())
+                .retryPolicy(new RetryOneTime(1))
+                .build();
             client.start();
             try
             {
@@ -320,7 +319,7 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testCreateACLWithReset() throws Exception
+    public void testCreateACLWithReset() throws Exception
     {
         Timing timing = new Timing();
         CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder();
@@ -381,15 +380,15 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testCreateParents() throws Exception
+    public void testCreateParents() throws Exception
     {
-        CuratorFrameworkFactory.Builder      builder = CuratorFrameworkFactory.builder();
+        CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder();
         CuratorFramework client = builder.connectString(server.getConnectString()).retryPolicy(new RetryOneTime(1)).build();
         client.start();
         try
         {
             client.create().creatingParentsIfNeeded().forPath("/one/two/three", "foo".getBytes());
-            byte[]      data = client.getData().forPath("/one/two/three");
+            byte[] data = client.getData().forPath("/one/two/three");
             Assert.assertEquals(data, "foo".getBytes());
 
             client.create().creatingParentsIfNeeded().forPath("/one/two/another", "bar".getBytes());
@@ -403,11 +402,11 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testEnsurePathWithNamespace() throws Exception
+    public void testEnsurePathWithNamespace() throws Exception
     {
         final String namespace = "jz";
 
-        CuratorFrameworkFactory.Builder      builder = CuratorFrameworkFactory.builder();
+        CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder();
         CuratorFramework client = builder.connectString(server.getConnectString()).retryPolicy(new RetryOneTime(1)).namespace(namespace).build();
         client.start();
         try
@@ -427,16 +426,16 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testNamespace() throws Exception
+    public void testNamespace() throws Exception
     {
         final String namespace = "TestNamespace";
-        
-        CuratorFrameworkFactory.Builder      builder = CuratorFrameworkFactory.builder();
+
+        CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder();
         CuratorFramework client = builder.connectString(server.getConnectString()).retryPolicy(new RetryOneTime(1)).namespace(namespace).build();
         client.start();
         try
         {
-            String      actualPath = client.create().forPath("/test");
+            String actualPath = client.create().forPath("/test");
             Assert.assertEquals(actualPath, "/test");
             Assert.assertNotNull(client.getZookeeperClient().getZooKeeper().exists("/" + namespace + "/test", false));
             Assert.assertNull(client.getZookeeperClient().getZooKeeper().exists("/test", false));
@@ -446,7 +445,7 @@ public class TestFramework extends BaseClassForTests
             Assert.assertNotNull(client.getZookeeperClient().getZooKeeper().exists("/non", false));
 
             client.create().forPath("/test/child", "hey".getBytes());
-            byte[]      bytes = client.getData().forPath("/test/child");
+            byte[] bytes = client.getData().forPath("/test/child");
             Assert.assertEquals(bytes, "hey".getBytes());
 
             bytes = client.usingNamespace(null).getData().forPath("/" + namespace + "/test/child");
@@ -459,14 +458,14 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testCustomCallback() throws Exception
+    public void testCustomCallback() throws Exception
     {
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
         client.start();
         try
         {
-            final CountDownLatch    latch = new CountDownLatch(1);
-            BackgroundCallback      callback = new BackgroundCallback()
+            final CountDownLatch latch = new CountDownLatch(1);
+            BackgroundCallback callback = new BackgroundCallback()
             {
                 @Override
                 public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
@@ -490,7 +489,7 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testSync() throws Exception
+    public void testSync() throws Exception
     {
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
         client.start();
@@ -515,7 +514,7 @@ public class TestFramework extends BaseClassForTests
             client.create().forPath("/head");
             Assert.assertNotNull(client.checkExists().forPath("/head"));
 
-            CountDownLatch      latch = new CountDownLatch(1);
+            CountDownLatch latch = new CountDownLatch(1);
             client.sync("/head", latch);
             Assert.assertTrue(latch.await(10, TimeUnit.SECONDS));
         }
@@ -526,7 +525,7 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testSyncNew() throws Exception
+    public void testSyncNew() throws Exception
     {
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
         client.start();
@@ -535,7 +534,7 @@ public class TestFramework extends BaseClassForTests
             client.create().forPath("/head");
             Assert.assertNotNull(client.checkExists().forPath("/head"));
 
-            final CountDownLatch      latch = new CountDownLatch(1);
+            final CountDownLatch latch = new CountDownLatch(1);
             BackgroundCallback callback = new BackgroundCallback()
             {
                 @Override
@@ -557,32 +556,32 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testBackgroundDelete() throws Exception
+    public void testBackgroundDelete() throws Exception
     {
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
         client.start();
         try
         {
             client.getCuratorListenable().addListener
-            (
-                new CuratorListener()
-                {
-                    @Override
-                    public void eventReceived(CuratorFramework client, CuratorEvent event) throws Exception
+                (
+                    new CuratorListener()
                     {
-                        if ( event.getType() == CuratorEventType.DELETE )
+                        @Override
+                        public void eventReceived(CuratorFramework client, CuratorEvent event) throws Exception
                         {
-                            Assert.assertEquals(event.getPath(), "/head");
-                            ((CountDownLatch)event.getContext()).countDown();
+                            if ( event.getType() == CuratorEventType.DELETE )
+                            {
+                                Assert.assertEquals(event.getPath(), "/head");
+                                ((CountDownLatch)event.getContext()).countDown();
+                            }
                         }
                     }
-                }
-            );
+                );
 
             client.create().forPath("/head");
             Assert.assertNotNull(client.checkExists().forPath("/head"));
 
-            CountDownLatch      latch = new CountDownLatch(1);
+            CountDownLatch latch = new CountDownLatch(1);
             client.delete().inBackground(latch).forPath("/head");
             Assert.assertTrue(latch.await(10, TimeUnit.SECONDS));
             Assert.assertNull(client.checkExists().forPath("/head"));
@@ -594,7 +593,7 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testBackgroundDeleteWithChildren() throws Exception
+    public void testBackgroundDeleteWithChildren() throws Exception
     {
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
         client.start();
@@ -619,7 +618,7 @@ public class TestFramework extends BaseClassForTests
             client.create().creatingParentsIfNeeded().forPath("/one/two/three/four");
             Assert.assertNotNull(client.checkExists().forPath("/one/two/three/four"));
 
-            CountDownLatch      latch = new CountDownLatch(1);
+            CountDownLatch latch = new CountDownLatch(1);
             client.delete().deletingChildrenIfNeeded().inBackground(latch).forPath("/one/two");
             Assert.assertTrue(latch.await(10, TimeUnit.SECONDS));
             Assert.assertNull(client.checkExists().forPath("/one/two"));
@@ -631,7 +630,7 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testDelete() throws Exception
+    public void testDelete() throws Exception
     {
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
         client.start();
@@ -651,7 +650,7 @@ public class TestFramework extends BaseClassForTests
     @Test
     public void testDeleteWithChildren() throws Exception
     {
-        CuratorFrameworkFactory.Builder      builder = CuratorFrameworkFactory.builder();
+        CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder();
         CuratorFramework client = builder.connectString(server.getConnectString()).retryPolicy(new RetryOneTime(1)).build();
         client.start();
         try
@@ -671,7 +670,7 @@ public class TestFramework extends BaseClassForTests
     @Test
     public void testDeleteGuaranteedWithChildren() throws Exception
     {
-        CuratorFrameworkFactory.Builder      builder = CuratorFrameworkFactory.builder();
+        CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder();
         CuratorFramework client = builder.connectString(server.getConnectString()).retryPolicy(new RetryOneTime(1)).build();
         client.start();
         try
@@ -689,7 +688,7 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testGetSequentialChildren() throws Exception
+    public void testGetSequentialChildren() throws Exception
     {
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
         client.start();
@@ -702,7 +701,7 @@ public class TestFramework extends BaseClassForTests
                 client.create().withMode(CreateMode.EPHEMERAL_SEQUENTIAL).forPath("/head/child");
             }
 
-            List<String>        children = client.getChildren().forPath("/head");
+            List<String> children = client.getChildren().forPath("/head");
             Assert.assertEquals(children.size(), 10);
         }
         finally
@@ -712,50 +711,50 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testBackgroundGetDataWithWatch() throws Exception
+    public void testBackgroundGetDataWithWatch() throws Exception
     {
-        final byte[]        data1 = {1, 2, 3};
-        final byte[]        data2 = {4, 5, 6, 7};
+        final byte[] data1 = {1, 2, 3};
+        final byte[] data2 = {4, 5, 6, 7};
 
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
         client.start();
         try
         {
-            final CountDownLatch          watchedLatch = new CountDownLatch(1);
+            final CountDownLatch watchedLatch = new CountDownLatch(1);
             client.getCuratorListenable().addListener
-            (
-                new CuratorListener()
-                {
-                    @Override
-                    public void eventReceived(CuratorFramework client, CuratorEvent event) throws Exception
+                (
+                    new CuratorListener()
                     {
-                        if ( event.getType() == CuratorEventType.GET_DATA )
-                        {
-                            Assert.assertEquals(event.getPath(), "/test");
-                            Assert.assertEquals(event.getData(), data1);
-                            ((CountDownLatch)event.getContext()).countDown();
-                        }
-                        else if ( event.getType() == CuratorEventType.WATCHED )
+                        @Override
+                        public void eventReceived(CuratorFramework client, CuratorEvent event) throws Exception
                         {
-                            if ( event.getWatchedEvent().getType() == Watcher.Event.EventType.NodeDataChanged )
+                            if ( event.getType() == CuratorEventType.GET_DATA )
                             {
                                 Assert.assertEquals(event.getPath(), "/test");
-                                watchedLatch.countDown();
+                                Assert.assertEquals(event.getData(), data1);
+                                ((CountDownLatch)event.getContext()).countDown();
+                            }
+                            else if ( event.getType() == CuratorEventType.WATCHED )
+                            {
+                                if ( event.getWatchedEvent().getType() == Watcher.Event.EventType.NodeDataChanged )
+                                {
+                                    Assert.assertEquals(event.getPath(), "/test");
+                                    watchedLatch.countDown();
+                                }
                             }
                         }
                     }
-                }
-            );
+                );
 
             client.create().forPath("/test", data1);
 
-            CountDownLatch      backgroundLatch = new CountDownLatch(1);
+            CountDownLatch backgroundLatch = new CountDownLatch(1);
             client.getData().watched().inBackground(backgroundLatch).forPath("/test");
             Assert.assertTrue(backgroundLatch.await(10, TimeUnit.SECONDS));
 
             client.setData().forPath("/test", data2);
             Assert.assertTrue(watchedLatch.await(10, TimeUnit.SECONDS));
-            byte[]      checkData = client.getData().forPath("/test");
+            byte[] checkData = client.getData().forPath("/test");
             Assert.assertEquals(checkData, data2);
         }
         finally
@@ -765,29 +764,29 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testBackgroundCreate() throws Exception
+    public void testBackgroundCreate() throws Exception
     {
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
         client.start();
         try
         {
             client.getCuratorListenable().addListener
-            (
-                new CuratorListener()
-                {
-                    @Override
-                    public void eventReceived(CuratorFramework client, CuratorEvent event) throws Exception
+                (
+                    new CuratorListener()
                     {
-                        if ( event.getType() == CuratorEventType.CREATE )
+                        @Override
+                        public void eventReceived(CuratorFramework client, CuratorEvent event) throws Exception
                         {
-                            Assert.assertEquals(event.getPath(), "/test");
-                            ((CountDownLatch)event.getContext()).countDown();
+                            if ( event.getType() == CuratorEventType.CREATE )
+                            {
+                                Assert.assertEquals(event.getPath(), "/test");
+                                ((CountDownLatch)event.getContext()).countDown();
+                            }
                         }
                     }
-                }
-            );
+                );
 
-            CountDownLatch     latch = new CountDownLatch(1);
+            CountDownLatch latch = new CountDownLatch(1);
             client.create().inBackground(latch).forPath("/test", new byte[]{1, 2, 3});
             Assert.assertTrue(latch.await(10, TimeUnit.SECONDS));
         }
@@ -798,20 +797,20 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testCreateModes() throws Exception
+    public void testCreateModes() throws Exception
     {
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
         client.start();
         try
         {
-            byte[]  writtenBytes = {1, 2, 3};
+            byte[] writtenBytes = {1, 2, 3};
             client.create().forPath("/test", writtenBytes); // should be persistent
 
             client.close();
             client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
             client.start();
 
-            byte[]  readBytes = client.getData().forPath("/test");
+            byte[] readBytes = client.getData().forPath("/test");
             Assert.assertEquals(writtenBytes, readBytes);
 
             client.create().withMode(CreateMode.EPHEMERAL).forPath("/ghost", writtenBytes);
@@ -822,10 +821,10 @@ public class TestFramework extends BaseClassForTests
 
             readBytes = client.getData().forPath("/test");
             Assert.assertEquals(writtenBytes, readBytes);
-            Stat    stat = client.checkExists().forPath("/ghost");
+            Stat stat = client.checkExists().forPath("/ghost");
             Assert.assertNull(stat);
 
-            String  realPath = client.create().withMode(CreateMode.PERSISTENT_SEQUENTIAL).forPath("/pseq", writtenBytes);
+            String realPath = client.create().withMode(CreateMode.PERSISTENT_SEQUENTIAL).forPath("/pseq", writtenBytes);
             Assert.assertNotSame(realPath, "/pseq");
 
             client.close();
@@ -852,13 +851,13 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
-    public void     testSimple() throws Exception
+    public void testSimple() throws Exception
     {
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
         client.start();
         try
         {
-            String    path = client.create().withMode(CreateMode.PERSISTENT).forPath("/test", new byte[]{1, 2, 3});
+            String path = client.create().withMode(CreateMode.PERSISTENT).forPath("/test", new byte[]{1, 2, 3});
             Assert.assertEquals(path, "/test");
         }
         finally
@@ -866,9 +865,9 @@ public class TestFramework extends BaseClassForTests
             client.close();
         }
     }
-    
+
     @Test
-    public void     testSequentialWithTrailingSeparator() throws Exception
+    public void testSequentialWithTrailingSeparator() throws Exception
     {
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
         client.start();
@@ -876,12 +875,12 @@ public class TestFramework extends BaseClassForTests
         {
             client.create().forPath("/test");
             //This should create a node in the form of "/test/00000001"
-            String    path = client.create().withMode(CreateMode.PERSISTENT_SEQUENTIAL).forPath("/test/");
+            String path = client.create().withMode(CreateMode.PERSISTENT_SEQUENTIAL).forPath("/test/");
             Assert.assertTrue(path.startsWith("/test/"));
         }
         finally
         {
             client.close();
         }
-    }    
+    }
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/c805679c/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java
index 9c02c7d..af7ae2b 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
  * 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
@@ -44,6 +44,7 @@ import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.data.Stat;
 import org.testng.Assert;
 import org.testng.annotations.Test;
+import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Semaphore;
@@ -56,6 +57,69 @@ public class TestFrameworkEdges extends BaseClassForTests
     private final Timing timing = new Timing();
 
     @Test
+    public void testPathsFromProtectingInBackground() throws Exception
+    {
+        for ( CreateMode mode : CreateMode.values() )
+        {
+            internalTestPathsFromProtectingInBackground(mode);
+        }
+    }
+
+    private void internalTestPathsFromProtectingInBackground(CreateMode mode) throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), 1, new RetryOneTime(1));
+        try
+        {
+            client.start();
+
+            client.create().creatingParentsIfNeeded().forPath("/a/b/c");
+
+            final BlockingQueue<String> paths = new ArrayBlockingQueue<String>(2);
+            BackgroundCallback callback = new BackgroundCallback()
+            {
+                @Override
+                public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+                {
+                    paths.put(event.getName());
+                    paths.put(event.getPath());
+                }
+            };
+            final String TEST_PATH = "/a/b/c/test-";
+            client.create().withMode(mode).inBackground(callback).forPath(TEST_PATH);
+
+            String name1 = paths.take();
+            String path1 = paths.take();
+
+            client.close();
+
+            client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), 1, new RetryOneTime(1));
+            client.start();
+            CreateBuilderImpl createBuilder = (CreateBuilderImpl)client.create().withProtection();
+
+            client.create().forPath(createBuilder.adjustPath(TEST_PATH));
+
+            createBuilder.debugForceFindProtectedNode = true;
+            createBuilder.withMode(mode).inBackground(callback).forPath(TEST_PATH);
+
+            String name2 = paths.take();
+            String path2 = paths.take();
+
+            Assert.assertEquals(ZKPaths.getPathAndNode(name1).getPath(), ZKPaths.getPathAndNode(TEST_PATH).getPath());
+            Assert.assertEquals(ZKPaths.getPathAndNode(name2).getPath(), ZKPaths.getPathAndNode(TEST_PATH).getPath());
+            Assert.assertEquals(ZKPaths.getPathAndNode(path1).getPath(), ZKPaths.getPathAndNode(TEST_PATH).getPath());
+            Assert.assertEquals(ZKPaths.getPathAndNode(path2).getPath(), ZKPaths.getPathAndNode(TEST_PATH).getPath());
+
+            client.delete().deletingChildrenIfNeeded().forPath("/a/b/c");
+            client.delete().forPath("/a/b");
+            client.delete().forPath("/a");
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
     public void connectionLossWithBackgroundTest() throws Exception
     {
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), 1, new RetryOneTime(1));
@@ -66,20 +130,20 @@ public class TestFrameworkEdges extends BaseClassForTests
             client.getZookeeperClient().blockUntilConnectedOrTimedOut();
             server.close();
             client.getChildren().inBackground
-            (
-                new BackgroundCallback()
-                {
-                    public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+                (
+                    new BackgroundCallback()
                     {
-                        latch.countDown();
+                        public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+                        {
+                            latch.countDown();
+                        }
                     }
-                }
-            ).forPath("/");
+                ).forPath("/");
             Assert.assertTrue(timing.awaitLatch(latch));
         }
         finally
         {
-            client.close();
+            CloseableUtils.closeQuietly(client);
         }
     }