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/12/19 20:03:18 UTC

[11/12] curator git commit: if an async create() with creatingParentContainersIfNeeded() was called with a connection that wasn't properly authorized to create the parent nodes. backgroundCreateParentsThenNode() was treating all KeeperExceptions as ignor

if an async create() with creatingParentContainersIfNeeded() was called with a connection that wasn't properly authorized to create the parent nodes. backgroundCreateParentsThenNode() was treating all KeeperExceptions as ignorable (i.e. retry-able) which would cause an infinite loop.


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

Branch: refs/heads/CURATOR-3.0
Commit: 125dc969942a89f498b06e190cb38f9819c7bb37
Parents: 0a3e849
Author: randgalt <ra...@apache.org>
Authored: Mon Dec 19 10:41:11 2016 -0500
Committer: randgalt <ra...@apache.org>
Committed: Mon Dec 19 10:41:11 2016 -0500

----------------------------------------------------------------------
 .../framework/imps/CreateBuilderImpl.java       |   6 +-
 .../recipes/leader/TestLeaderAcls.java          | 133 +++++++++++++++++++
 2 files changed, 138 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/125dc969/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 d59ba69..98c2a05 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
@@ -562,7 +562,11 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
                 }
                 catch ( KeeperException e )
                 {
-                    // ignore
+                    if ( !RetryLoop.isRetryException(e) )
+                    {
+                        throw e;
+                    }
+                    // otherwise safe to ignore as it will get retried
                 }
                 client.queueOperation(mainOperationAndData);
             }

http://git-wip-us.apache.org/repos/asf/curator/blob/125dc969/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderAcls.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderAcls.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderAcls.java
new file mode 100644
index 0000000..1589c7a
--- /dev/null
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderAcls.java
@@ -0,0 +1,133 @@
+/**
+ * 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.leader;
+
+import org.apache.curator.RetryPolicy;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.ACLProvider;
+import org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.Timing;
+import org.apache.curator.utils.CloseableUtils;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+import java.security.NoSuchAlgorithmException;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+public class TestLeaderAcls extends BaseClassForTests
+{
+    private final Timing timing = new Timing();
+
+    @Test(description = "Validation test for CURATOR-365")
+    public void testAclErrorWithLeader() throws Exception
+    {
+        ACLProvider provider = new ACLProvider()
+        {
+            @Override
+            public List<ACL> getDefaultAcl()
+            {
+                return ZooDefs.Ids.OPEN_ACL_UNSAFE;
+            }
+
+            @Override
+            public List<ACL> getAclForPath(String path)
+            {
+                if ( path.equals("/base") )
+                {
+                    try
+                    {
+                        String testDigest = DigestAuthenticationProvider.generateDigest("test:test");
+                        return Collections.singletonList(new ACL(ZooDefs.Perms.ALL, new Id("digest", testDigest)));
+                    }
+                    catch ( NoSuchAlgorithmException e )
+                    {
+                        e.printStackTrace();
+                    }
+                }
+                return getDefaultAcl();
+            }
+        };
+
+        RetryPolicy retryPolicy = new ExponentialBackoffRetry(timing.milliseconds(), 3);
+        CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder()
+            .connectString(server.getConnectString())
+            .retryPolicy(retryPolicy)
+            .aclProvider(provider)
+            .authorization("digest", "test:test".getBytes())
+            ;
+        CuratorFramework client = builder.build();
+        LeaderLatch latch = null;
+        try
+        {
+            client.start();
+
+            latch = new LeaderLatch(client, "/base");
+            latch.start();
+            Assert.assertTrue(latch.await(timing.forWaiting().seconds(), TimeUnit.SECONDS));
+            latch.close();
+            latch = null;
+
+            CuratorFramework noAuthClient = CuratorFrameworkFactory.newClient(server.getConnectString(), retryPolicy);
+            try
+            {
+                noAuthClient.start();
+
+                final CountDownLatch noAuthLatch = new CountDownLatch(1);
+                UnhandledErrorListener listener = new UnhandledErrorListener()
+                {
+                    @Override
+                    public void unhandledError(String message, Throwable e)
+                    {
+                        if ( e instanceof KeeperException.NoAuthException )
+                        {
+                            noAuthLatch.countDown();
+                        }
+                    }
+                };
+                noAuthClient.getUnhandledErrorListenable().addListener(listener);
+
+                // use a path below "base" as noAuthClient is not authorized to create nodes in "/base"
+                // but also making sure that the code goes through the backgroundCreateParentsThenNode() codepath
+                latch = new LeaderLatch(noAuthClient, "/base/second");
+                latch.start();
+                Assert.assertTrue(timing.awaitLatch(noAuthLatch));
+            }
+            finally
+            {
+                CloseableUtils.closeQuietly(noAuthClient);
+            }
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(latch);
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+}