You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@curator.apache.org by ra...@apache.org on 2017/07/21 19:27:16 UTC

[01/18] curator git commit: initial work on ZooKeeper 3.4.x compatibility mode

Repository: curator
Updated Branches:
  refs/heads/CURATOR-426 46ce6b7ae -> 31d7f9a20


initial work on ZooKeeper 3.4.x compatibility mode


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

Branch: refs/heads/CURATOR-426
Commit: 0641243f75d25a92a9ceb811e8e2611a1e95869e
Parents: 7e611bd
Author: randgalt <ra...@apache.org>
Authored: Wed Jul 19 12:36:24 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Wed Jul 19 12:36:24 2017 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/curator/RetryLoop.java |   2 +-
 .../curator/framework/CuratorFramework.java     |   7 +
 .../framework/CuratorFrameworkFactory.java      |  50 +++++++-
 .../apache/curator/framework/SafeIsTtlMode.java |  43 +++++++
 .../imps/CompatibleCreateCallback.java          |  26 ++++
 .../framework/imps/CreateBuilderImpl.java       | 125 ++++++++++++------
 .../curator/framework/imps/CreateZK35.java      |  47 +++++++
 .../framework/imps/CuratorFrameworkImpl.java    |  21 ++-
 .../framework/imps/WatcherRemovalManager.java   |   2 +-
 .../framework/imps/TestFrameworkEdges.java      |   5 +-
 curator-recipes/pom.xml                         |  16 +++
 .../framework/recipes/nodes/PersistentNode.java |   3 +-
 curator-test-zk34/pom.xml                       | 127 +++++++++++++++++++
 .../curator/framework/imps/TestCleanState.java  |  25 ++++
 .../src/test/resources/log4j.properties         |  27 ++++
 pom.xml                                         |   8 ++
 16 files changed, 483 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/0641243f/curator-client/src/main/java/org/apache/curator/RetryLoop.java
----------------------------------------------------------------------
diff --git a/curator-client/src/main/java/org/apache/curator/RetryLoop.java b/curator-client/src/main/java/org/apache/curator/RetryLoop.java
index 2ea6f97..51df662 100644
--- a/curator-client/src/main/java/org/apache/curator/RetryLoop.java
+++ b/curator-client/src/main/java/org/apache/curator/RetryLoop.java
@@ -136,7 +136,7 @@ public class RetryLoop
             (rc == KeeperException.Code.OPERATIONTIMEOUT.intValue()) ||
             (rc == KeeperException.Code.SESSIONMOVED.intValue()) ||
             (rc == KeeperException.Code.SESSIONEXPIRED.intValue()) ||
-            (rc == KeeperException.Code.NEWCONFIGNOQUORUM.intValue());
+            (rc == -13); // KeeperException.Code.NEWCONFIGNOQUORUM.intValue()) - using hard coded value for ZK 3.4.x compatibility
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/curator/blob/0641243f/curator-framework/src/main/java/org/apache/curator/framework/CuratorFramework.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFramework.java b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFramework.java
index 4135e82..bf6167c 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFramework.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFramework.java
@@ -324,4 +324,11 @@ public interface CuratorFramework extends Closeable
      * @return schema set
      */
     SchemaSet getSchemaSet();
+
+    /**
+     * Return true if this instance is running in ZK 3.4.x compatibility mode
+     *
+     * @return true/false
+     */
+    boolean isZk34CompatibilityMode();
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/0641243f/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
index 915aee1..5ce7762 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
@@ -35,15 +35,16 @@ import org.apache.curator.framework.imps.CuratorTempFrameworkImpl;
 import org.apache.curator.framework.imps.DefaultACLProvider;
 import org.apache.curator.framework.imps.GzipCompressionProvider;
 import org.apache.curator.framework.schema.SchemaSet;
+import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateErrorPolicy;
 import org.apache.curator.framework.state.StandardConnectionStateErrorPolicy;
-import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.utils.DefaultZookeeperFactory;
 import org.apache.curator.utils.ZookeeperFactory;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
+import org.slf4j.LoggerFactory;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Arrays;
@@ -67,6 +68,33 @@ public class CuratorFrameworkFactory
     private static final long DEFAULT_INACTIVE_THRESHOLD_MS = (int)TimeUnit.MINUTES.toMillis(3);
     private static final int DEFAULT_CLOSE_WAIT_MS = (int)TimeUnit.SECONDS.toMillis(1);
 
+    private static final boolean hasZooKeeperAdmin;
+    static
+    {
+        boolean hasIt;
+        try
+        {
+            Class.forName("org.apache.zookeeper.admin.ZooKeeperAdmin");
+            hasIt = true;
+        }
+        catch ( ClassNotFoundException e )
+        {
+            hasIt = false;
+            LoggerFactory.getLogger(CuratorFrameworkFactory.class).info("Running in ZooKeeper 3.4.x compatibility mode");
+        }
+        hasZooKeeperAdmin = hasIt;
+    }
+
+    /**
+     * Return true if the classpath ZooKeeper library is 3.4.x
+     *
+     * @return true/false
+     */
+    public static boolean isZK34()
+    {
+        return !hasZooKeeperAdmin;
+    }
+
     /**
      * Return a new builder that builds a CuratorFramework
      *
@@ -145,6 +173,7 @@ public class CuratorFrameworkFactory
         private ConnectionStateErrorPolicy connectionStateErrorPolicy = new StandardConnectionStateErrorPolicy();
         private ConnectionHandlingPolicy connectionHandlingPolicy = Boolean.getBoolean("curator-use-classic-connection-handling") ? new ClassicConnectionHandlingPolicy() : new StandardConnectionHandlingPolicy();
         private SchemaSet schemaSet = SchemaSet.getDefaultSchemaSet();
+        private boolean zk34CompatibilityMode = isZK34();
 
         /**
          * Apply the current values and build a new CuratorFramework
@@ -386,6 +415,20 @@ public class CuratorFrameworkFactory
         }
 
         /**
+         * If mode is true, create a ZooKeeper 3.4.x compatible client. IMPORTANT: If the client
+         * library used is ZooKeeper 3.4.x <code>zk34CompatibilityMode</code> is enabled by default.
+         *
+         * @since 3.5.0
+         * @param mode true/false
+         * @return this
+         */
+        public Builder zk34CompatibilityMode(boolean mode)
+        {
+            this.zk34CompatibilityMode = mode;
+            return this;
+        }
+
+        /**
          * <p>
          *     Change the connection handling policy. The default policy is {@link StandardConnectionHandlingPolicy}.
          * </p>
@@ -515,6 +558,11 @@ public class CuratorFrameworkFactory
             return schemaSet;
         }
 
+        public boolean isZk34CompatibilityMode()
+        {
+            return zk34CompatibilityMode;
+        }
+
         @Deprecated
         public String getAuthScheme()
         {

http://git-wip-us.apache.org/repos/asf/curator/blob/0641243f/curator-framework/src/main/java/org/apache/curator/framework/SafeIsTtlMode.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/SafeIsTtlMode.java b/curator-framework/src/main/java/org/apache/curator/framework/SafeIsTtlMode.java
new file mode 100644
index 0000000..3c4b9e6
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/SafeIsTtlMode.java
@@ -0,0 +1,43 @@
+/**
+ * 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;
+
+import org.apache.zookeeper.CreateMode;
+
+public class SafeIsTtlMode
+{
+    private static class Internal
+    {
+        private static final Internal instance = new Internal();
+
+        public boolean isTtl(CreateMode mode)
+        {
+            return mode.isTTL();
+        }
+    }
+
+    public static boolean isTtl(CreateMode mode)
+    {
+        return !CuratorFrameworkFactory.isZK34() && Internal.instance.isTtl(mode);
+    }
+
+    private SafeIsTtlMode()
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/0641243f/curator-framework/src/main/java/org/apache/curator/framework/imps/CompatibleCreateCallback.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/CompatibleCreateCallback.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/CompatibleCreateCallback.java
new file mode 100644
index 0000000..30ca391
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/CompatibleCreateCallback.java
@@ -0,0 +1,26 @@
+/**
+ * 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 org.apache.zookeeper.data.Stat;
+
+interface CompatibleCreateCallback
+{
+    void processResult(int rc, String path, Object ctx, String name, Stat stat);
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/0641243f/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 416485a..b58084f 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
@@ -623,49 +623,76 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
             final OperationTrace trace = client.getZookeeperClient().startAdvancedTracer("CreateBuilderImpl-Background");
             final byte[] data = operationAndData.getData().getData();
 
-            client.getZooKeeper().create
-            (
-                operationAndData.getData().getPath(),
-                data,
-                acling.getAclList(operationAndData.getData().getPath()),
-                createMode,
-                new AsyncCallback.Create2Callback() {
-                    @Override
-                    public void processResult(int rc, String path, Object ctx, String name, Stat stat) {
-                        trace.setReturnCode(rc).setRequestBytesLength(data).setPath(path).commit();
+            final CompatibleCreateCallback mainCallback = new CompatibleCreateCallback()
+            {
+                @Override
+                public void processResult(int rc, String path, Object ctx, String name, Stat stat) {
+                    trace.setReturnCode(rc).setRequestBytesLength(data).setPath(path).commit();
 
-                        if ( (stat != null) && (storingStat != null) )
-                        {
-                            storingStat.setAversion(stat.getAversion());
-                            storingStat.setCtime(stat.getCtime());
-                            storingStat.setCversion(stat.getCversion());
-                            storingStat.setCzxid(stat.getCzxid());
-                            storingStat.setDataLength(stat.getDataLength());
-                            storingStat.setEphemeralOwner(stat.getEphemeralOwner());
-                            storingStat.setMtime(stat.getMtime());
-                            storingStat.setMzxid(stat.getMzxid());
-                            storingStat.setNumChildren(stat.getNumChildren());
-                            storingStat.setPzxid(stat.getPzxid());
-                            storingStat.setVersion(stat.getVersion());
-                        }
+                    if ( (stat != null) && (storingStat != null) )
+                    {
+                        storingStat.setAversion(stat.getAversion());
+                        storingStat.setCtime(stat.getCtime());
+                        storingStat.setCversion(stat.getCversion());
+                        storingStat.setCzxid(stat.getCzxid());
+                        storingStat.setDataLength(stat.getDataLength());
+                        storingStat.setEphemeralOwner(stat.getEphemeralOwner());
+                        storingStat.setMtime(stat.getMtime());
+                        storingStat.setMzxid(stat.getMzxid());
+                        storingStat.setNumChildren(stat.getNumChildren());
+                        storingStat.setPzxid(stat.getPzxid());
+                        storingStat.setVersion(stat.getVersion());
+                    }
 
-                        if ( (rc == KeeperException.Code.NONODE.intValue()) && createParentsIfNeeded )
-                        {
-                            backgroundCreateParentsThenNode(client, operationAndData, operationAndData.getData().getPath(), backgrounding, acling.getACLProviderForParents(), createParentsAsContainers);
-                        }
-                        else if ( (rc == KeeperException.Code.NODEEXISTS.intValue()) && setDataIfExists )
-                        {
-                            backgroundSetData(client, operationAndData, operationAndData.getData().getPath(), backgrounding);
-                        }
-                        else
-                        {
-                            sendBackgroundResponse(rc, path, ctx, name, stat, operationAndData);
-                        }
+                    if ( (rc == KeeperException.Code.NONODE.intValue()) && createParentsIfNeeded )
+                    {
+                        backgroundCreateParentsThenNode(client, operationAndData, operationAndData.getData().getPath(), backgrounding, acling.getACLProviderForParents(), createParentsAsContainers);
                     }
-                },
-                backgrounding.getContext(),
-                ttl
-            );
+                    else if ( (rc == KeeperException.Code.NODEEXISTS.intValue()) && setDataIfExists )
+                    {
+                        backgroundSetData(client, operationAndData, operationAndData.getData().getPath(), backgrounding);
+                    }
+                    else
+                    {
+                        sendBackgroundResponse(rc, path, ctx, name, stat, operationAndData);
+                    }
+                }
+            };
+
+            if ( client.isZk34CompatibilityMode() )
+            {
+                AsyncCallback.StringCallback stringCallback = new AsyncCallback.StringCallback()
+                {
+                    @Override
+                    public void processResult(int rc, String path, Object ctx, String name)
+                    {
+                        mainCallback.processResult(rc, path, ctx, name, null);
+                    }
+                };
+                client.getZooKeeper().create
+                    (
+                        operationAndData.getData().getPath(),
+                        data,
+                        acling.getAclList(operationAndData.getData().getPath()),
+                        createMode,
+                        stringCallback,
+                        backgrounding.getContext()
+                    );
+            }
+            else
+            {
+                CreateZK35.create
+                (
+                    client.getZooKeeper(),
+                    operationAndData.getData().getPath(),
+                    data,
+                    acling.getAclList(operationAndData.getData().getPath()),
+                    createMode,
+                    mainCallback,
+                    backgrounding.getContext(),
+                    ttl
+                );
+            }
         }
         catch ( Throwable e )
         {
@@ -1144,14 +1171,28 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
                         {
                             try
                             {
-                                createdPath = client.getZooKeeper().create(path, data, aclList, createMode, storingStat, ttl);
+                                if ( client.isZk34CompatibilityMode() )
+                                {
+                                    createdPath = client.getZooKeeper().create(path, data, aclList, createMode);
+                                }
+                                else
+                                {
+                                    createdPath = client.getZooKeeper().create(path, data, aclList, createMode, storingStat, ttl);
+                                }
                             }
                             catch ( KeeperException.NoNodeException e )
                             {
                                 if ( createParentsIfNeeded )
                                 {
                                     ZKPaths.mkdirs(client.getZooKeeper(), path, false, acling.getACLProviderForParents(), createParentsAsContainers);
-                                    createdPath = client.getZooKeeper().create(path, data, acling.getAclList(path), createMode, storingStat, ttl);
+                                    if ( client.isZk34CompatibilityMode() )
+                                    {
+                                        createdPath = client.getZooKeeper().create(path, data, acling.getAclList(path), createMode);
+                                    }
+                                    else
+                                    {
+                                        createdPath = client.getZooKeeper().create(path, data, acling.getAclList(path), createMode, storingStat, ttl);
+                                    }
                                 }
                                 else
                                 {

http://git-wip-us.apache.org/repos/asf/curator/blob/0641243f/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateZK35.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateZK35.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateZK35.java
new file mode 100644
index 0000000..a32e614
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateZK35.java
@@ -0,0 +1,47 @@
+/**
+ * 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 org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import java.util.List;
+
+// keep reference to AsyncCallback.Create2Callback in separate class for ZK 3.4 compatibility
+class CreateZK35
+{
+    static void create(ZooKeeper zooKeeper, String path, byte data[], List<ACL> acl, CreateMode createMode, final CompatibleCreateCallback compatibleCallback, Object ctx, long ttl)
+    {
+        AsyncCallback.Create2Callback callback = new AsyncCallback.Create2Callback()
+        {
+            @Override
+            public void processResult(int rc, String path, Object ctx, String name, Stat stat)
+            {
+                compatibleCallback.processResult(rc, path, ctx, name, stat);
+            }
+        };
+        zooKeeper.create(path, data, acl, createMode, callback, ctx, ttl);
+    }
+
+    private CreateZK35()
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/0641243f/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java
index 7b620d3..00adb3e 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java
@@ -91,6 +91,7 @@ public class CuratorFrameworkImpl implements CuratorFramework
     private final InternalConnectionHandler internalConnectionHandler;
     private final EnsembleTracker ensembleTracker;
     private final SchemaSet schemaSet;
+    private final boolean zk34CompatibilityMode;
 
     private volatile ExecutorService executorService;
     private final AtomicBoolean logAsErrorConnectionErrors = new AtomicBoolean(false);
@@ -146,6 +147,7 @@ public class CuratorFrameworkImpl implements CuratorFramework
         useContainerParentsIfAvailable = builder.useContainerParentsIfAvailable();
         connectionStateErrorPolicy = Preconditions.checkNotNull(builder.getConnectionStateErrorPolicy(), "errorPolicy cannot be null");
         schemaSet = Preconditions.checkNotNull(builder.getSchemaSet(), "schemaSet cannot be null");
+        zk34CompatibilityMode = builder.isZk34CompatibilityMode();
 
         byte[] builderDefaultData = builder.getDefaultData();
         defaultData = (builderDefaultData != null) ? Arrays.copyOf(builderDefaultData, builderDefaultData.length) : new byte[0];
@@ -155,7 +157,7 @@ public class CuratorFrameworkImpl implements CuratorFramework
         failedRemoveWatcherManager = new FailedRemoveWatchManager(this);
         namespaceFacadeCache = new NamespaceFacadeCache(this);
 
-        ensembleTracker = new EnsembleTracker(this, builder.getEnsembleProvider());
+        ensembleTracker = zk34CompatibilityMode ? null : new EnsembleTracker(this, builder.getEnsembleProvider());
     }
 
     private List<AuthInfo> buildAuths(CuratorFrameworkFactory.Builder builder)
@@ -230,6 +232,7 @@ public class CuratorFrameworkImpl implements CuratorFramework
         connectionStateErrorPolicy = parent.connectionStateErrorPolicy;
         internalConnectionHandler = parent.internalConnectionHandler;
         schemaSet = parent.schemaSet;
+        zk34CompatibilityMode = parent.zk34CompatibilityMode;
         ensembleTracker = null;
     }
 
@@ -316,7 +319,10 @@ public class CuratorFrameworkImpl implements CuratorFramework
                 }
             });
 
-            ensembleTracker.start();
+            if ( ensembleTracker != null )
+            {
+                ensembleTracker.start();
+            }
 
             log.info(schemaSet.toDocumentation());
         }
@@ -366,7 +372,10 @@ public class CuratorFrameworkImpl implements CuratorFramework
                 }
             }
 
-            ensembleTracker.close();
+            if ( ensembleTracker != null )
+            {
+                ensembleTracker.close();
+            }
             listeners.clear();
             unhandledErrorListeners.clear();
             connectionStateManager.close();
@@ -782,6 +791,12 @@ public class CuratorFrameworkImpl implements CuratorFramework
         connectionStateManager.addStateChange(newConnectionState);
     }
 
+    @Override
+    public boolean isZk34CompatibilityMode()
+    {
+        return zk34CompatibilityMode;
+    }
+
     EnsembleTracker getEnsembleTracker()
     {
         return ensembleTracker;

http://git-wip-us.apache.org/repos/asf/curator/blob/0641243f/curator-framework/src/main/java/org/apache/curator/framework/imps/WatcherRemovalManager.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/WatcherRemovalManager.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/WatcherRemovalManager.java
index b9c9044..b85675b 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/WatcherRemovalManager.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/WatcherRemovalManager.java
@@ -57,7 +57,7 @@ public class WatcherRemovalManager
         while ( localEntries.size() > 0 )
         {
             NamespaceWatcher watcher = localEntries.remove(0);
-            if ( entries.remove(watcher) )
+            if ( entries.remove(watcher) && !client.isZk34CompatibilityMode() )
             {
                 try
                 {

http://git-wip-us.apache.org/repos/asf/curator/blob/0641243f/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 887f236..8fee2d3 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
@@ -23,6 +23,7 @@ import org.apache.curator.RetryPolicy;
 import org.apache.curator.RetrySleeper;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.SafeIsTtlMode;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.api.CuratorEventType;
@@ -180,7 +181,7 @@ public class TestFrameworkEdges extends BaseClassForTests
             final String TEST_PATH = "/a/b/c/test-";
             long ttl = timing.forWaiting().milliseconds()*1000;
             CreateBuilder firstCreateBuilder = client.create();
-            if(mode.isTTL()) {
+            if( SafeIsTtlMode.isTtl(mode) ) {
                 firstCreateBuilder.withTtl(ttl);
             }
             firstCreateBuilder.withMode(mode).inBackground(callback).forPath(TEST_PATH);
@@ -195,7 +196,7 @@ public class TestFrameworkEdges extends BaseClassForTests
             
             CreateBuilderImpl createBuilder = (CreateBuilderImpl)client.create();
             createBuilder.withProtection();
-            if(mode.isTTL()) {
+            if(SafeIsTtlMode.isTtl(mode)) {
                 createBuilder.withTtl(ttl);
             }
 

http://git-wip-us.apache.org/repos/asf/curator/blob/0641243f/curator-recipes/pom.xml
----------------------------------------------------------------------
diff --git a/curator-recipes/pom.xml b/curator-recipes/pom.xml
index 77f2a51..1ed413c 100644
--- a/curator-recipes/pom.xml
+++ b/curator-recipes/pom.xml
@@ -87,4 +87,20 @@
             <scope>test</scope>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
 </project>

http://git-wip-us.apache.org/repos/asf/curator/blob/0641243f/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
index af74695..bdf607c 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
@@ -23,6 +23,7 @@ import com.google.common.base.Function;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.SafeIsTtlMode;
 import org.apache.curator.framework.WatcherRemoveCuratorFramework;
 import org.apache.curator.framework.api.ACLBackgroundPathAndBytesable;
 import org.apache.curator.framework.api.BackgroundCallback;
@@ -449,7 +450,7 @@ public class PersistentNode implements Closeable
             CreateModable<ACLBackgroundPathAndBytesable<String>> localCreateMethod = createMethod.get();
             if ( localCreateMethod == null )
             {
-                CreateBuilderMain createBuilder = mode.isTTL() ? client.create().withTtl(ttl) : client.create();
+                CreateBuilderMain createBuilder = SafeIsTtlMode.isTtl(mode) ? client.create().withTtl(ttl) : client.create();
                 CreateModable<ACLBackgroundPathAndBytesable<String>> tempCreateMethod = useProtection ? createBuilder.creatingParentContainersIfNeeded().withProtection() : createBuilder.creatingParentContainersIfNeeded();
                 createMethod.compareAndSet(null, tempCreateMethod);
                 localCreateMethod = createMethod.get();

http://git-wip-us.apache.org/repos/asf/curator/blob/0641243f/curator-test-zk34/pom.xml
----------------------------------------------------------------------
diff --git a/curator-test-zk34/pom.xml b/curator-test-zk34/pom.xml
new file mode 100644
index 0000000..fe5e216
--- /dev/null
+++ b/curator-test-zk34/pom.xml
@@ -0,0 +1,127 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>apache-curator</artifactId>
+        <groupId>org.apache.curator</groupId>
+        <version>3.3.1-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>curator-test-zk34</artifactId>
+
+    <properties>
+        <zookeeper-34-version>3.4.8</zookeeper-34-version>
+        <curator-2-version>2.12.0</curator-2-version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.zookeeper</groupId>
+            <artifactId>zookeeper</artifactId>
+            <version>${zookeeper-34-version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.sun.jmx</groupId>
+                    <artifactId>jmxri</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.sun.jdmk</groupId>
+                    <artifactId>jmxtools</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.jms</groupId>
+                    <artifactId>jms</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-test</artifactId>
+            <version>${curator-2-version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+            </exclusions>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-recipes</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+            </exclusions>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-recipes</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+            </exclusions>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.testng</groupId>
+            <artifactId>testng</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-deploy-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <phase>deploy</phase>
+                        <configuration>
+                            <skip>true</skip>
+                        </configuration>
+                        <goals>
+                            <goal>deploy</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <configuration>
+                    <dependenciesToScan>
+                        <dependency>org.apache.curator:curator-framework</dependency>
+                        <dependency>org.apache.curator:curator-recipes</dependency>
+                    </dependenciesToScan>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/curator/blob/0641243f/curator-test-zk34/src/test/java/org/apache/curator/framework/imps/TestCleanState.java
----------------------------------------------------------------------
diff --git a/curator-test-zk34/src/test/java/org/apache/curator/framework/imps/TestCleanState.java b/curator-test-zk34/src/test/java/org/apache/curator/framework/imps/TestCleanState.java
new file mode 100644
index 0000000..9efeb90
--- /dev/null
+++ b/curator-test-zk34/src/test/java/org/apache/curator/framework/imps/TestCleanState.java
@@ -0,0 +1,25 @@
+package org.apache.curator.framework.imps;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.utils.CloseableUtils;
+import java.util.concurrent.Callable;
+
+public class TestCleanState
+{
+    public static void closeAndTestClean(CuratorFramework client)
+    {
+        CloseableUtils.closeQuietly(client);
+    }
+
+    public static void test(CuratorFramework client, Callable<Void> proc) throws Exception
+    {
+        try
+        {
+            proc.call();
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/0641243f/curator-test-zk34/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/curator-test-zk34/src/test/resources/log4j.properties b/curator-test-zk34/src/test/resources/log4j.properties
new file mode 100644
index 0000000..2a85e0d
--- /dev/null
+++ b/curator-test-zk34/src/test/resources/log4j.properties
@@ -0,0 +1,27 @@
+#
+# 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.
+#
+
+log4j.rootLogger=ERROR, console
+
+log4j.logger.org.apache.curator=DEBUG, console
+log4j.additivity.org.apache.curator=false
+
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%-5p %c %x %m [%t]%n

http://git-wip-us.apache.org/repos/asf/curator/blob/0641243f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index aa8fd86..b816d36 100644
--- a/pom.xml
+++ b/pom.xml
@@ -298,6 +298,7 @@
         <module>curator-x-discovery-server</module>
         <module>curator-x-rpc</module>
         <module>curator-x-async</module>
+        <module>curator-test-zk34</module>
     </modules>
 
     <dependencyManagement>
@@ -347,6 +348,13 @@
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
+                <artifactId>curator-recipes</artifactId>
+                <version>${project.version}</version>
+                <type>test-jar</type>
+            </dependency>
+
+            <dependency>
+                <groupId>org.apache.curator</groupId>
                 <artifactId>curator-test</artifactId>
                 <version>${project.version}</version>
             </dependency>


[17/18] curator git commit: more doc tweaks

Posted by ra...@apache.org.
more doc tweaks


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

Branch: refs/heads/CURATOR-426
Commit: 0906eb539d6b5aa3ad65bf7413f5c955288758b8
Parents: 46ce6b7
Author: randgalt <ra...@apache.org>
Authored: Fri Jul 21 14:24:56 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Fri Jul 21 14:24:56 2017 -0500

----------------------------------------------------------------------
 src/site/confluence/compatibility.confluence | 53 ++++++++++++++++++++---
 src/site/site.xml                            |  2 +-
 2 files changed, 47 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/0906eb53/src/site/confluence/compatibility.confluence
----------------------------------------------------------------------
diff --git a/src/site/confluence/compatibility.confluence b/src/site/confluence/compatibility.confluence
index ef3324f..4103bf2 100644
--- a/src/site/confluence/compatibility.confluence
+++ b/src/site/confluence/compatibility.confluence
@@ -1,9 +1,48 @@
-h1. API Compatibility
+h1. ZooKeeper Version Compatibility
 
-A [[Clirr|http://clirr.sourceforge.net/]] report is generated for each Curator module:
+While ZooKeeper 3.5.x is still considered "beta" by the ZooKeeper development team, the reality is that it is
+used in production by many users. However, ZooKeeper 3.4.x is also used in production. Prior to Apache Curator
+4.0, both versions of ZooKeeper were supported via two versions of Apache Curator. Starting with Curator 4.0
+both versions of ZooKeeper are supported via the same Curator libraries.
+
+h2. ZooKeeper 3.5.x
+
+* Curator 4.0 has a hard dependency on ZooKeeper 3.5.x
+* If you are using ZooKeeper 3.5.x there's nothing additional to do \- just use Curator 4.0
+
+h2. ZooKeeper 3.4.x
+
+Curator 4.0 supports ZooKeeper 3.4.x ensembles in a soft\-compatibility mode. To use this mode
+you must exclude ZooKeeper when adding Curator to your dependency management tool.
+
+_Maven_
+
+{code}
+<dependency>
+    <groupId>org.apache.curator</groupId>
+    <artifactId>curator-recipes</artifactId>
+    <version>${curator-version}</version>
+    <exclusions>
+        <exclusion>
+            <groupId>org.apache.zookeeper</groupId>
+            <artifactId>zookeeper</artifactId>
+        </exclusion>
+    </exclusions>
+</dependency>
+{code}
+
+_Gradle_
+
+{code}
+compile('org.apache.curator:curator-recipes:$curatorVersion') {
+  exclude('org.apache.zookeeper:zookeeper')
+}
+{code}
+
+You must add a dependency on ZooKeeper 3.4.x also.
+
+Curator will detect which ZooKeeper library is in use and automatically set ZooKeeper 3.4 compatibility
+mode as needed. In this mode, all features not supported by 3.4 are disabled. It is up to your
+application code to "do the right thing" and not use these features. Use the {{isZk34CompatibilityMode()}}
+method to determine which mode Curator is using at runtime.
 
-* [[Curator Client Report|curator-client/clirr-report.html]]
-* [[Curator Framework Report|curator-framework/clirr-report.html]]
-* [[Curator Recipes Report|curator-recipes/clirr-report.html]]
-* [[Curator Discovery Report|curator-x-discovery/clirr-report.html]]
-* [[Curator Discovery Server Report|curator-x-discovery-server/clirr-report.html]]

http://git-wip-us.apache.org/repos/asf/curator/blob/0906eb53/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
index d78b5b7..8bff322 100644
--- a/src/site/site.xml
+++ b/src/site/site.xml
@@ -39,7 +39,7 @@
             <twitter>
                 <user>ApacheCurator</user>
                 <showUser>true</showUser>
-                <showFollowers>true</showFollowers>
+                <showFollowers>false</showFollowers>
             </twitter>
             <googleSearch>
                 <sitesearch>${project.url}</sitesearch>


[13/18] curator git commit: added some missing tags to the pom

Posted by ra...@apache.org.
added some missing tags to the pom


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

Branch: refs/heads/CURATOR-426
Commit: 1eb8aa8490fe4a007173e4dd70a78425cd74e02a
Parents: aa976bb
Author: randgalt <ra...@apache.org>
Authored: Thu Jul 20 17:44:25 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Thu Jul 20 17:44:25 2017 -0500

----------------------------------------------------------------------
 curator-test-zk34/pom.xml | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/1eb8aa84/curator-test-zk34/pom.xml
----------------------------------------------------------------------
diff --git a/curator-test-zk34/pom.xml b/curator-test-zk34/pom.xml
index ba38571..d417875 100644
--- a/curator-test-zk34/pom.xml
+++ b/curator-test-zk34/pom.xml
@@ -9,6 +9,10 @@
 
     <artifactId>curator-test-zk34</artifactId>
 
+    <name>Curator ZooKeeper 3.4 Testing</name>
+    <description>Tests for ZoKeeper 3.4 compatibility</description>
+    <inceptionYear>2017</inceptionYear>
+
     <properties>
         <zookeeper-34-version>3.4.8</zookeeper-34-version>
         <curator-2-version>2.12.0</curator-2-version>


[06/18] curator git commit: Added some missing deps, fixed some of the tests for zk 3.4 compat and new KillSession, added a SDummyTest to make Maven happy

Posted by ra...@apache.org.
Added some missing deps, fixed some of the tests for zk 3.4 compat and new KillSession, added a SDummyTest to make Maven happy


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

Branch: refs/heads/CURATOR-426
Commit: 20a3db363273429cec80f511a32a737886c6f3c8
Parents: 097083c
Author: randgalt <ra...@apache.org>
Authored: Thu Jul 20 11:04:51 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Thu Jul 20 11:04:51 2017 -0500

----------------------------------------------------------------------
 .../recipes/locks/TestInterProcessMutexBase.java        | 10 +++++++++-
 .../framework/recipes/nodes/TestPersistentTtlNode.java  |  6 ++++--
 curator-test-zk34/pom.xml                               | 12 ++++++++++++
 .../test/java/org/apache/curator/test/DummyTest.java    | 12 ++++++++++++
 4 files changed, 37 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/20a3db36/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java
index 40c9144..cf44daf 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java
@@ -33,6 +33,7 @@ import org.apache.curator.test.Timing;
 import org.apache.curator.test.compatibility.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.utils.ZKPaths;
+import org.apache.zookeeper.KeeperException;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 import java.util.List;
@@ -128,7 +129,14 @@ public abstract class TestInterProcessMutexBase extends BaseClassForTests
                             }
                             finally
                             {
-                                lock.release();
+                                try
+                                {
+                                    lock.release();
+                                }
+                                catch ( KeeperException.SessionExpiredException dummy )
+                                {
+                                    // happens sometimes with a few tests - ignore
+                                }
                             }
                             return result;
                         }

http://git-wip-us.apache.org/repos/asf/curator/blob/20a3db36/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
index 573a867..7bca01f 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
@@ -24,8 +24,9 @@ import org.apache.curator.framework.recipes.cache.PathChildrenCache;
 import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
 import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
 import org.apache.curator.retry.RetryOneTime;
-import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.Timing;
+import org.apache.curator.test.compatibility.CuratorTestBase;
+import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
 import org.apache.curator.utils.ZKPaths;
 import org.testng.Assert;
 import org.testng.annotations.AfterMethod;
@@ -36,7 +37,8 @@ import java.util.concurrent.TimeUnit;
 
 import static org.apache.curator.framework.recipes.cache.PathChildrenCache.StartMode.BUILD_INITIAL_CACHE;
 
-public class TestPersistentTtlNode extends BaseClassForTests
+@Test(groups = Zk35MethodInterceptor.zk35Group)
+public class TestPersistentTtlNode extends CuratorTestBase
 {
     private final Timing timing = new Timing();
     private final long ttlMs = timing.multiple(.10).milliseconds(); // a small number

http://git-wip-us.apache.org/repos/asf/curator/blob/20a3db36/curator-test-zk34/pom.xml
----------------------------------------------------------------------
diff --git a/curator-test-zk34/pom.xml b/curator-test-zk34/pom.xml
index d4101b0..ba38571 100644
--- a/curator-test-zk34/pom.xml
+++ b/curator-test-zk34/pom.xml
@@ -117,6 +117,18 @@
             <artifactId>slf4j-log4j12</artifactId>
             <scope>test</scope>
         </dependency>
+
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-math</artifactId>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
     <build>

http://git-wip-us.apache.org/repos/asf/curator/blob/20a3db36/curator-test-zk34/src/test/java/org/apache/curator/test/DummyTest.java
----------------------------------------------------------------------
diff --git a/curator-test-zk34/src/test/java/org/apache/curator/test/DummyTest.java b/curator-test-zk34/src/test/java/org/apache/curator/test/DummyTest.java
new file mode 100644
index 0000000..2f6afa8
--- /dev/null
+++ b/curator-test-zk34/src/test/java/org/apache/curator/test/DummyTest.java
@@ -0,0 +1,12 @@
+package org.apache.curator.test;
+
+import org.testng.annotations.Test;
+
+public class DummyTest
+{
+    @Test
+    public void dummyTest()
+    {
+        // NOP
+    }
+}


[04/18] curator git commit: Use the resource plugin to copy the compatibility files instead of adding the source directory. This seems more compatible

Posted by ra...@apache.org.
Use the resource plugin to copy the compatibility files instead of adding the source directory. This seems more compatible


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

Branch: refs/heads/CURATOR-426
Commit: 242b7011a7b8344f772c1a4497aadf6de16d73f5
Parents: 58bc969
Author: randgalt <ra...@apache.org>
Authored: Thu Jul 20 02:14:10 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Thu Jul 20 02:14:10 2017 -0500

----------------------------------------------------------------------
 .../java/org/apache/curator/BasicTests.java     |   2 +-
 .../curator/TestSessionFailRetryLoop.java       |   2 +-
 .../framework/imps/TestCreateReturningStat.java |   4 +-
 .../imps/TestEnabledSessionExpiredState.java    |   4 +-
 .../curator/framework/imps/TestFramework.java   |   2 +-
 .../framework/imps/TestFrameworkEdges.java      |   4 +-
 .../framework/imps/TestReconfiguration.java     |   7 +-
 .../framework/imps/TestRemoveWatches.java       |   4 +-
 .../curator/framework/imps/TestTtlNodes.java    |   5 +-
 .../imps/TestWatcherRemovalManager.java         |   5 +-
 .../recipes/cache/TestEventOrdering.java        |   3 +-
 .../framework/recipes/cache/TestNodeCache.java  |   2 +-
 .../recipes/cache/TestPathChildrenCache.java    |   2 +-
 .../framework/recipes/cache/TestTreeCache.java  |   2 +-
 .../recipes/leader/TestLeaderLatch.java         |   2 +-
 .../recipes/leader/TestLeaderSelector.java      |   4 +-
 .../recipes/locks/TestInterProcessMutex.java    |   2 +-
 .../locks/TestInterProcessMutexBase.java        |   4 +-
 .../nodes/TestPersistentEphemeralNode.java      |   4 +-
 .../recipes/nodes/TestPersistentNode.java       |   2 +-
 curator-test-zk34/pom.xml                       |  20 +-
 .../org/apache/curator/test/Compatibility.java  |   6 +-
 curator-test/pom.xml                            |  19 --
 .../apache/curator/test/CuratorTestBase.java    |  27 --
 .../org/apache/curator/test/KillSession2.java   |  39 ---
 .../java/org/apache/curator/test/Timing2.java   | 299 -------------------
 .../curator/test/Zk35MethodInterceptor.java     |  55 ----
 .../org/apache/curator/test/Compatibility.java  |   6 +-
 .../test/compatibility/CuratorTestBase.java     |  28 ++
 .../test/compatibility/KillSession2.java        |  40 +++
 .../curator/test/compatibility/Timing2.java     | 299 +++++++++++++++++++
 .../compatibility/Zk35MethodInterceptor.java    |  56 ++++
 .../x/async/CompletableBaseClassForTests.java   |   3 +-
 .../discovery/details/TestServiceDiscovery.java |   2 +-
 pom.xml                                         |   7 -
 35 files changed, 474 insertions(+), 498 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-client/src/test/java/org/apache/curator/BasicTests.java
----------------------------------------------------------------------
diff --git a/curator-client/src/test/java/org/apache/curator/BasicTests.java b/curator-client/src/test/java/org/apache/curator/BasicTests.java
index 94d418b..2875f49 100644
--- a/curator-client/src/test/java/org/apache/curator/BasicTests.java
+++ b/curator-client/src/test/java/org/apache/curator/BasicTests.java
@@ -21,7 +21,7 @@ package org.apache.curator;
 import org.apache.curator.ensemble.fixed.FixedEnsembleProvider;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.KillSession2;
+import org.apache.curator.test.compatibility.KillSession2;
 import org.apache.curator.test.Timing;
 import org.apache.curator.utils.ZookeeperFactory;
 import org.apache.zookeeper.CreateMode;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java
----------------------------------------------------------------------
diff --git a/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java b/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java
index e56f4d4..39b0e45 100644
--- a/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java
+++ b/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java
@@ -20,7 +20,7 @@ package org.apache.curator;
 
 import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.KillSession2;
+import org.apache.curator.test.compatibility.KillSession2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.test.Timing;
 import org.testng.Assert;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java
index bef143f..67a960f 100755
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java
@@ -24,9 +24,9 @@ import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.api.CuratorEventType;
 import org.apache.curator.retry.RetryOneTime;
-import org.apache.curator.test.CuratorTestBase;
+import org.apache.curator.test.compatibility.CuratorTestBase;
 import org.apache.curator.test.Timing;
-import org.apache.curator.test.Zk35MethodInterceptor;
+import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.data.Stat;
 import org.testng.Assert;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java
index 63fd36e..f1bbc3b 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java
@@ -25,8 +25,8 @@ import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.KillSession2;
-import org.apache.curator.test.Timing2;
+import org.apache.curator.test.compatibility.KillSession2;
+import org.apache.curator.test.compatibility.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/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 70ae2ea..a1d6f51 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
@@ -31,7 +31,7 @@ import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.Timing;
-import org.apache.curator.test.Timing2;
+import org.apache.curator.test.compatibility.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.utils.EnsurePath;
 import org.apache.curator.utils.ZKPaths;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/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 42e9afa..db5210d 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
@@ -33,9 +33,9 @@ import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.RetryNTimes;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.KillSession2;
+import org.apache.curator.test.compatibility.KillSession2;
 import org.apache.curator.test.TestingServer;
-import org.apache.curator.test.Timing2;
+import org.apache.curator.test.compatibility.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.utils.ZKPaths;
 import org.apache.zookeeper.CreateMode;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java
index 567d71d..acf9df3 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java
@@ -27,13 +27,12 @@ import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.api.CuratorEventType;
 import org.apache.curator.retry.ExponentialBackoffRetry;
-import org.apache.curator.test.CuratorTestBase;
+import org.apache.curator.test.compatibility.CuratorTestBase;
 import org.apache.curator.test.InstanceSpec;
 import org.apache.curator.test.TestingCluster;
 import org.apache.curator.test.TestingZooKeeperServer;
-import org.apache.curator.test.Timing;
-import org.apache.curator.test.Timing2;
-import org.apache.curator.test.Zk35MethodInterceptor;
+import org.apache.curator.test.compatibility.Timing2;
+import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-framework/src/test/java/org/apache/curator/framework/imps/TestRemoveWatches.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestRemoveWatches.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestRemoveWatches.java
index 66f5703..f3b4f8d 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestRemoveWatches.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestRemoveWatches.java
@@ -30,9 +30,9 @@ import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.curator.retry.RetryOneTime;
-import org.apache.curator.test.CuratorTestBase;
+import org.apache.curator.test.compatibility.CuratorTestBase;
 import org.apache.curator.test.Timing;
-import org.apache.curator.test.Zk35MethodInterceptor;
+import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTtlNodes.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTtlNodes.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTtlNodes.java
index 297399d..f253d96 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTtlNodes.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTtlNodes.java
@@ -23,10 +23,9 @@ import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.retry.RetryOneTime;
-import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.CuratorTestBase;
+import org.apache.curator.test.compatibility.CuratorTestBase;
 import org.apache.curator.test.Timing;
-import org.apache.curator.test.Zk35MethodInterceptor;
+import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
 import org.apache.zookeeper.CreateMode;
 import org.testng.Assert;
 import org.testng.annotations.AfterMethod;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWatcherRemovalManager.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWatcherRemovalManager.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWatcherRemovalManager.java
index d3ad66f..74aac1d 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWatcherRemovalManager.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWatcherRemovalManager.java
@@ -24,11 +24,10 @@ import org.apache.curator.framework.WatcherRemoveCuratorFramework;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.retry.RetryOneTime;
-import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.CuratorTestBase;
+import org.apache.curator.test.compatibility.CuratorTestBase;
 import org.apache.curator.test.Timing;
 import org.apache.curator.test.WatchersDebug;
-import org.apache.curator.test.Zk35MethodInterceptor;
+import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestEventOrdering.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestEventOrdering.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestEventOrdering.java
index c50474d..3d369e1 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestEventOrdering.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestEventOrdering.java
@@ -24,8 +24,7 @@ import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.Timing;
-import org.apache.curator.test.Timing2;
+import org.apache.curator.test.compatibility.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.KeeperException;
 import org.testng.Assert;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestNodeCache.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestNodeCache.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestNodeCache.java
index 52c76ad..2e81034 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestNodeCache.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestNodeCache.java
@@ -20,7 +20,7 @@ package org.apache.curator.framework.recipes.cache;
 
 import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.KillSession2;
+import org.apache.curator.test.compatibility.KillSession2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
index f24b846..14074f0 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
@@ -29,7 +29,7 @@ import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.ExecuteCalledWatchingExecutorService;
-import org.apache.curator.test.KillSession2;
+import org.apache.curator.test.compatibility.KillSession2;
 import org.apache.curator.test.TestingServer;
 import org.apache.curator.test.Timing;
 import org.apache.curator.utils.CloseableUtils;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/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 1e203b7..409cc48 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
@@ -23,7 +23,7 @@ import com.google.common.collect.ImmutableSet;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.api.UnhandledErrorListener;
 import org.apache.curator.framework.recipes.cache.TreeCacheEvent.Type;
-import org.apache.curator.test.KillSession2;
+import org.apache.curator.test.compatibility.KillSession2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.CreateMode;
 import org.testng.Assert;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
index 93c955b..ef25ba6 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
@@ -35,7 +35,7 @@ import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.TestingServer;
 import org.apache.curator.test.Timing;
-import org.apache.curator.test.Timing2;
+import org.apache.curator.test.compatibility.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java
index 79fcdb3..808ff8f 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java
@@ -30,10 +30,10 @@ import org.apache.curator.framework.state.SessionConnectionStateErrorPolicy;
 import org.apache.curator.framework.state.StandardConnectionStateErrorPolicy;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.KillSession2;
+import org.apache.curator.test.compatibility.KillSession2;
 import org.apache.curator.test.TestingServer;
 import org.apache.curator.test.Timing;
-import org.apache.curator.test.Timing2;
+import org.apache.curator.test.compatibility.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutex.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutex.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutex.java
index cf82c57..8ab2dc5 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutex.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutex.java
@@ -26,7 +26,7 @@ import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.framework.schema.Schema;
 import org.apache.curator.framework.schema.SchemaSet;
 import org.apache.curator.retry.RetryOneTime;
-import org.apache.curator.test.KillSession2;
+import org.apache.curator.test.compatibility.KillSession2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.CreateMode;
 import org.testng.Assert;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java
index 43ded2e..40c9144 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java
@@ -27,10 +27,10 @@ import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.KillSession2;
+import org.apache.curator.test.compatibility.KillSession2;
 import org.apache.curator.test.TestingServer;
 import org.apache.curator.test.Timing;
-import org.apache.curator.test.Timing2;
+import org.apache.curator.test.compatibility.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.utils.ZKPaths;
 import org.testng.Assert;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java
index 0ce61d5..03665c5 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java
@@ -31,8 +31,8 @@ import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.KillSession2;
-import org.apache.curator.test.Timing2;
+import org.apache.curator.test.compatibility.KillSession2;
+import org.apache.curator.test.compatibility.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.utils.ZKPaths;
 import org.apache.zookeeper.CreateMode;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentNode.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentNode.java
index b848fe4..0fdd7c8 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentNode.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentNode.java
@@ -23,7 +23,7 @@ import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.Timing;
-import org.apache.curator.test.Timing2;
+import org.apache.curator.test.compatibility.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.CreateMode;
 import org.testng.Assert;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-test-zk34/pom.xml
----------------------------------------------------------------------
diff --git a/curator-test-zk34/pom.xml b/curator-test-zk34/pom.xml
index 8d2f073..d4101b0 100644
--- a/curator-test-zk34/pom.xml
+++ b/curator-test-zk34/pom.xml
@@ -149,19 +149,23 @@
             </plugin>
 
             <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>build-helper-maven-plugin</artifactId>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-resources-plugin</artifactId>
                 <executions>
                     <execution>
-                        <id>add-compatibility</id>
+                        <id>copy-curator-test-classes</id>
+                        <phase>validate</phase>
                         <goals>
-                            <goal>add-test-source</goal>
+                            <goal>copy-resources</goal>
                         </goals>
-                        <phase>generate-test-sources</phase>
                         <configuration>
-                            <sources>
-                                <source>../curator-test/src/compatibility/java</source>
-                            </sources>
+                            <outputDirectory>${basedir}/target/generated-test-sources/test-annotations/org/apache/curator/test/compatibility</outputDirectory>
+                            <resources>
+                                <resource>
+                                    <directory>../curator-test/src/main/java/org/apache/curator/test/compatibility</directory>
+                                    <filtering>false</filtering>
+                                </resource>
+                            </resources>
                         </configuration>
                     </execution>
                 </executions>

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-test-zk34/src/test/java/org/apache/curator/test/Compatibility.java
----------------------------------------------------------------------
diff --git a/curator-test-zk34/src/test/java/org/apache/curator/test/Compatibility.java b/curator-test-zk34/src/test/java/org/apache/curator/test/Compatibility.java
index 77ddf2c..bebd7c9 100644
--- a/curator-test-zk34/src/test/java/org/apache/curator/test/Compatibility.java
+++ b/curator-test-zk34/src/test/java/org/apache/curator/test/Compatibility.java
@@ -21,14 +21,14 @@ package org.apache.curator.test;
 import org.apache.curator.utils.InjectSessionExpiration;
 import org.apache.zookeeper.ZooKeeper;
 
-class Compatibility
+public class Compatibility
 {
-    static boolean isZK34()
+    public static boolean isZK34()
     {
         return true;
     }
 
-    static void injectSessionExpiration(ZooKeeper zooKeeper)
+    public static void injectSessionExpiration(ZooKeeper zooKeeper)
     {
         InjectSessionExpiration.injectSessionExpiration(zooKeeper);
     }

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-test/pom.xml
----------------------------------------------------------------------
diff --git a/curator-test/pom.xml b/curator-test/pom.xml
index 0ffe023..fb28d53 100644
--- a/curator-test/pom.xml
+++ b/curator-test/pom.xml
@@ -95,25 +95,6 @@
                     </execution>
                 </executions>
             </plugin>
-
-            <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>build-helper-maven-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <id>add-compatibility</id>
-                        <goals>
-                            <goal>add-source</goal>
-                        </goals>
-                        <phase>generate-sources</phase>
-                        <configuration>
-                            <sources>
-                                <source>src/compatibility/java</source>
-                            </sources>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
         </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-test/src/compatibility/java/org/apache/curator/test/CuratorTestBase.java
----------------------------------------------------------------------
diff --git a/curator-test/src/compatibility/java/org/apache/curator/test/CuratorTestBase.java b/curator-test/src/compatibility/java/org/apache/curator/test/CuratorTestBase.java
deleted file mode 100644
index efa94a3..0000000
--- a/curator-test/src/compatibility/java/org/apache/curator/test/CuratorTestBase.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/**
- * 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.test;
-
-import org.testng.annotations.Listeners;
-
-@Listeners(Zk35MethodInterceptor.class)
-public class CuratorTestBase extends BaseClassForTests
-{
-    protected final Timing2 timing = new Timing2();
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-test/src/compatibility/java/org/apache/curator/test/KillSession2.java
----------------------------------------------------------------------
diff --git a/curator-test/src/compatibility/java/org/apache/curator/test/KillSession2.java b/curator-test/src/compatibility/java/org/apache/curator/test/KillSession2.java
deleted file mode 100644
index 52ab168..0000000
--- a/curator-test/src/compatibility/java/org/apache/curator/test/KillSession2.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * 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.test;
-
-import org.apache.zookeeper.ZooKeeper;
-
-/**
- * <p>
- *     Utility to simulate a ZK session dying.
- * </p>
- */
-public class KillSession2
-{
-    /**
-     * Kill the given ZK session
-     *
-     * @param client the client to kill
-     */
-    public static void     kill(ZooKeeper client)
-    {
-        Compatibility.injectSessionExpiration(client);
-    }
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-test/src/compatibility/java/org/apache/curator/test/Timing2.java
----------------------------------------------------------------------
diff --git a/curator-test/src/compatibility/java/org/apache/curator/test/Timing2.java b/curator-test/src/compatibility/java/org/apache/curator/test/Timing2.java
deleted file mode 100644
index 02b71c4..0000000
--- a/curator-test/src/compatibility/java/org/apache/curator/test/Timing2.java
+++ /dev/null
@@ -1,299 +0,0 @@
-/**
- * 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.test;
-
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-/**
- * Utility to get various testing times.
- *
- * Copied from the old Timing class which is now deprecated. Needed this to support ZK 3.4 compatibility
- */
-public class Timing2
-{
-    private final long value;
-    private final TimeUnit unit;
-    private final int waitingMultiple;
-
-    private static final int DEFAULT_SECONDS = 10;
-    private static final int DEFAULT_WAITING_MULTIPLE = 5;
-    private static final double SESSION_MULTIPLE = 1.5;
-    private static final double SESSION_SLEEP_MULTIPLE = SESSION_MULTIPLE * 1.75;  // has to be at least session + 2/3 of a session to account for missed heartbeat then session expiration
-
-    /**
-     * Use the default base time
-     */
-    public Timing2()
-    {
-        this(Integer.getInteger("timing-multiple", 1), getWaitingMultiple());
-    }
-
-    /**
-     * Use a multiple of the default base time
-     *
-     * @param multiple the multiple
-     */
-    public Timing2(double multiple)
-    {
-        this((long)(DEFAULT_SECONDS * multiple), TimeUnit.SECONDS, getWaitingMultiple());
-    }
-
-    /**
-     * Use a multiple of the default base time
-     *
-     * @param multiple the multiple
-     * @param waitingMultiple multiple of main timing to use when waiting
-     */
-    public Timing2(double multiple, int waitingMultiple)
-    {
-        this((long)(DEFAULT_SECONDS * multiple), TimeUnit.SECONDS, waitingMultiple);
-    }
-
-    /**
-     * @param value base time
-     * @param unit  base time unit
-     */
-    public Timing2(long value, TimeUnit unit)
-    {
-        this(value, unit, getWaitingMultiple());
-    }
-
-    /**
-     * @param value base time
-     * @param unit  base time unit
-     * @param waitingMultiple multiple of main timing to use when waiting
-     */
-    public Timing2(long value, TimeUnit unit, int waitingMultiple)
-    {
-        this.value = value;
-        this.unit = unit;
-        this.waitingMultiple = waitingMultiple;
-    }
-
-    /**
-     * Return the base time in milliseconds
-     *
-     * @return time ms
-     */
-    public int milliseconds()
-    {
-        return (int)TimeUnit.MILLISECONDS.convert(value, unit);
-    }
-
-    /**
-     * Return the base time in seconds
-     *
-     * @return time secs
-     */
-    public int seconds()
-    {
-        return (int)value;
-    }
-
-    /**
-     * Wait on the given latch
-     *
-     * @param latch latch to wait on
-     * @return result of {@link java.util.concurrent.CountDownLatch#await(long, java.util.concurrent.TimeUnit)}
-     */
-    public boolean awaitLatch(CountDownLatch latch)
-    {
-        Timing2 m = forWaiting();
-        try
-        {
-            return latch.await(m.value, m.unit);
-        }
-        catch ( InterruptedException e )
-        {
-            Thread.currentThread().interrupt();
-        }
-        return false;
-    }
-
-    /**
-     * Try to take an item from the given queue
-     *
-     * @param queue queue
-     * @return item
-     * @throws Exception interrupted or timed out
-     */
-    public <T> T takeFromQueue(BlockingQueue<T> queue) throws Exception
-    {
-        Timing2 m = forWaiting();
-        try
-        {
-            T value = queue.poll(m.value, m.unit);
-            if ( value == null )
-            {
-                throw new TimeoutException("Timed out trying to take from queue");
-            }
-            return value;
-        }
-        catch ( InterruptedException e )
-        {
-            Thread.currentThread().interrupt();
-            throw e;
-        }
-    }
-
-    /**
-     * Wait on the given semaphore
-     *
-     * @param semaphore the semaphore
-     * @return result of {@link java.util.concurrent.Semaphore#tryAcquire()}
-     */
-    public boolean acquireSemaphore(Semaphore semaphore)
-    {
-        Timing2 m = forWaiting();
-        try
-        {
-            return semaphore.tryAcquire(m.value, m.unit);
-        }
-        catch ( InterruptedException e )
-        {
-            Thread.currentThread().interrupt();
-        }
-        return false;
-    }
-
-    /**
-     * Wait on the given semaphore
-     *
-     * @param semaphore the semaphore
-     * @param n         number of permits to acquire
-     * @return result of {@link java.util.concurrent.Semaphore#tryAcquire(int, long, java.util.concurrent.TimeUnit)}
-     */
-    public boolean acquireSemaphore(Semaphore semaphore, int n)
-    {
-        Timing2 m = forWaiting();
-        try
-        {
-            return semaphore.tryAcquire(n, m.value, m.unit);
-        }
-        catch ( InterruptedException e )
-        {
-            Thread.currentThread().interrupt();
-        }
-        return false;
-    }
-
-    /**
-     * Return a new timing that is a multiple of the this timing
-     *
-     * @param n the multiple
-     * @return this timing times the multiple
-     */
-    public Timing2 multiple(double n)
-    {
-        return new Timing2((int)(value * n), unit);
-    }
-
-    /**
-     * Return a new timing that is a multiple of the this timing
-     *
-     * @param n the multiple
-     * @param waitingMultiple new waitingMultiple
-     * @return this timing times the multiple
-     */
-    public Timing2 multiple(double n, int waitingMultiple)
-    {
-        return new Timing2((int)(value * n), unit, waitingMultiple);
-    }
-
-    /**
-     * Return a new timing with the standard multiple for waiting on latches, etc.
-     *
-     * @return this timing multiplied
-     */
-    @SuppressWarnings("PointlessArithmeticExpression")
-    public Timing2 forWaiting()
-    {
-        return multiple(waitingMultiple);
-    }
-
-    /**
-     * Return a new timing with a multiple that ensures a ZK session timeout
-     *
-     * @return this timing multiplied
-     */
-    public Timing2 forSessionSleep()
-    {
-        return multiple(SESSION_SLEEP_MULTIPLE, 1);
-    }
-
-    /**
-     * Return a new timing with a multiple for sleeping a smaller amount of time
-     *
-     * @return this timing multiplied
-     */
-    public Timing2 forSleepingABit()
-    {
-        return multiple(.25);
-    }
-
-    /**
-     * Sleep for a small amount of time
-     *
-     * @throws InterruptedException if interrupted
-     */
-    public void sleepABit() throws InterruptedException
-    {
-        forSleepingABit().sleep();
-    }
-
-    /**
-     * Sleep for a the full amount of time
-     *
-     * @throws InterruptedException if interrupted
-     */
-    public void sleep() throws InterruptedException
-    {
-        unit.sleep(value);
-    }
-
-    /**
-     * Return the value to use for ZK session timeout
-     *
-     * @return session timeout
-     */
-    public int session()
-    {
-        return multiple(SESSION_MULTIPLE).milliseconds();
-    }
-
-    /**
-     * Return the value to use for ZK connection timeout
-     *
-     * @return connection timeout
-     */
-    public int connection()
-    {
-        return milliseconds();
-    }
-
-    private static Integer getWaitingMultiple()
-    {
-        return Integer.getInteger("timing-waiting-multiple", DEFAULT_WAITING_MULTIPLE);
-    }
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-test/src/compatibility/java/org/apache/curator/test/Zk35MethodInterceptor.java
----------------------------------------------------------------------
diff --git a/curator-test/src/compatibility/java/org/apache/curator/test/Zk35MethodInterceptor.java b/curator-test/src/compatibility/java/org/apache/curator/test/Zk35MethodInterceptor.java
deleted file mode 100644
index 290910c..0000000
--- a/curator-test/src/compatibility/java/org/apache/curator/test/Zk35MethodInterceptor.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * 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.test;
-
-import org.testng.IMethodInstance;
-import org.testng.IMethodInterceptor;
-import org.testng.ITestContext;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-public class Zk35MethodInterceptor implements IMethodInterceptor
-{
-    public static final String zk35Group = "zk35";
-
-    @Override
-    public List<IMethodInstance> intercept(List<IMethodInstance> methods, ITestContext context)
-    {
-        if ( !Compatibility.isZK34() )
-        {
-            return methods;
-        }
-
-        List<IMethodInstance> filteredMethods = new ArrayList<>();
-        for ( IMethodInstance method : methods )
-        {
-            if ( !isInGroup(method.getMethod().getGroups()) )
-            {
-                filteredMethods.add(method);
-            }
-        }
-        return filteredMethods;
-    }
-
-    private boolean isInGroup(String[] groups)
-    {
-        return (groups != null) && Arrays.asList(groups).contains(zk35Group);
-    }
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-test/src/main/java/org/apache/curator/test/Compatibility.java
----------------------------------------------------------------------
diff --git a/curator-test/src/main/java/org/apache/curator/test/Compatibility.java b/curator-test/src/main/java/org/apache/curator/test/Compatibility.java
index 87eb33e..4fc63df 100644
--- a/curator-test/src/main/java/org/apache/curator/test/Compatibility.java
+++ b/curator-test/src/main/java/org/apache/curator/test/Compatibility.java
@@ -20,14 +20,14 @@ package org.apache.curator.test;
 
 import org.apache.zookeeper.ZooKeeper;
 
-class Compatibility
+public class Compatibility
 {
-    static boolean isZK34()
+    public static boolean isZK34()
     {
         return false;
     }
 
-    static void injectSessionExpiration(ZooKeeper zooKeeper)
+    public static void injectSessionExpiration(ZooKeeper zooKeeper)
     {
         zooKeeper.getTestable().injectSessionExpiration();
     }

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-test/src/main/java/org/apache/curator/test/compatibility/CuratorTestBase.java
----------------------------------------------------------------------
diff --git a/curator-test/src/main/java/org/apache/curator/test/compatibility/CuratorTestBase.java b/curator-test/src/main/java/org/apache/curator/test/compatibility/CuratorTestBase.java
new file mode 100644
index 0000000..a3c2a29
--- /dev/null
+++ b/curator-test/src/main/java/org/apache/curator/test/compatibility/CuratorTestBase.java
@@ -0,0 +1,28 @@
+/**
+ * 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.test.compatibility;
+
+import org.apache.curator.test.BaseClassForTests;
+import org.testng.annotations.Listeners;
+
+@Listeners(Zk35MethodInterceptor.class)
+public class CuratorTestBase extends BaseClassForTests
+{
+    protected final Timing2 timing = new Timing2();
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-test/src/main/java/org/apache/curator/test/compatibility/KillSession2.java
----------------------------------------------------------------------
diff --git a/curator-test/src/main/java/org/apache/curator/test/compatibility/KillSession2.java b/curator-test/src/main/java/org/apache/curator/test/compatibility/KillSession2.java
new file mode 100644
index 0000000..d747d3d
--- /dev/null
+++ b/curator-test/src/main/java/org/apache/curator/test/compatibility/KillSession2.java
@@ -0,0 +1,40 @@
+/**
+ * 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.test.compatibility;
+
+import org.apache.curator.test.Compatibility;
+import org.apache.zookeeper.ZooKeeper;
+
+/**
+ * <p>
+ *     Utility to simulate a ZK session dying.
+ * </p>
+ */
+public class KillSession2
+{
+    /**
+     * Kill the given ZK session
+     *
+     * @param client the client to kill
+     */
+    public static void     kill(ZooKeeper client)
+    {
+        Compatibility.injectSessionExpiration(client);
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-test/src/main/java/org/apache/curator/test/compatibility/Timing2.java
----------------------------------------------------------------------
diff --git a/curator-test/src/main/java/org/apache/curator/test/compatibility/Timing2.java b/curator-test/src/main/java/org/apache/curator/test/compatibility/Timing2.java
new file mode 100644
index 0000000..ad105eb
--- /dev/null
+++ b/curator-test/src/main/java/org/apache/curator/test/compatibility/Timing2.java
@@ -0,0 +1,299 @@
+/**
+ * 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.test.compatibility;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Utility to get various testing times.
+ *
+ * Copied from the old Timing class which is now deprecated. Needed this to support ZK 3.4 compatibility
+ */
+public class Timing2
+{
+    private final long value;
+    private final TimeUnit unit;
+    private final int waitingMultiple;
+
+    private static final int DEFAULT_SECONDS = 10;
+    private static final int DEFAULT_WAITING_MULTIPLE = 5;
+    private static final double SESSION_MULTIPLE = 1.5;
+    private static final double SESSION_SLEEP_MULTIPLE = SESSION_MULTIPLE * 1.75;  // has to be at least session + 2/3 of a session to account for missed heartbeat then session expiration
+
+    /**
+     * Use the default base time
+     */
+    public Timing2()
+    {
+        this(Integer.getInteger("timing-multiple", 1), getWaitingMultiple());
+    }
+
+    /**
+     * Use a multiple of the default base time
+     *
+     * @param multiple the multiple
+     */
+    public Timing2(double multiple)
+    {
+        this((long)(DEFAULT_SECONDS * multiple), TimeUnit.SECONDS, getWaitingMultiple());
+    }
+
+    /**
+     * Use a multiple of the default base time
+     *
+     * @param multiple the multiple
+     * @param waitingMultiple multiple of main timing to use when waiting
+     */
+    public Timing2(double multiple, int waitingMultiple)
+    {
+        this((long)(DEFAULT_SECONDS * multiple), TimeUnit.SECONDS, waitingMultiple);
+    }
+
+    /**
+     * @param value base time
+     * @param unit  base time unit
+     */
+    public Timing2(long value, TimeUnit unit)
+    {
+        this(value, unit, getWaitingMultiple());
+    }
+
+    /**
+     * @param value base time
+     * @param unit  base time unit
+     * @param waitingMultiple multiple of main timing to use when waiting
+     */
+    public Timing2(long value, TimeUnit unit, int waitingMultiple)
+    {
+        this.value = value;
+        this.unit = unit;
+        this.waitingMultiple = waitingMultiple;
+    }
+
+    /**
+     * Return the base time in milliseconds
+     *
+     * @return time ms
+     */
+    public int milliseconds()
+    {
+        return (int)TimeUnit.MILLISECONDS.convert(value, unit);
+    }
+
+    /**
+     * Return the base time in seconds
+     *
+     * @return time secs
+     */
+    public int seconds()
+    {
+        return (int)value;
+    }
+
+    /**
+     * Wait on the given latch
+     *
+     * @param latch latch to wait on
+     * @return result of {@link java.util.concurrent.CountDownLatch#await(long, java.util.concurrent.TimeUnit)}
+     */
+    public boolean awaitLatch(CountDownLatch latch)
+    {
+        Timing2 m = forWaiting();
+        try
+        {
+            return latch.await(m.value, m.unit);
+        }
+        catch ( InterruptedException e )
+        {
+            Thread.currentThread().interrupt();
+        }
+        return false;
+    }
+
+    /**
+     * Try to take an item from the given queue
+     *
+     * @param queue queue
+     * @return item
+     * @throws Exception interrupted or timed out
+     */
+    public <T> T takeFromQueue(BlockingQueue<T> queue) throws Exception
+    {
+        Timing2 m = forWaiting();
+        try
+        {
+            T value = queue.poll(m.value, m.unit);
+            if ( value == null )
+            {
+                throw new TimeoutException("Timed out trying to take from queue");
+            }
+            return value;
+        }
+        catch ( InterruptedException e )
+        {
+            Thread.currentThread().interrupt();
+            throw e;
+        }
+    }
+
+    /**
+     * Wait on the given semaphore
+     *
+     * @param semaphore the semaphore
+     * @return result of {@link java.util.concurrent.Semaphore#tryAcquire()}
+     */
+    public boolean acquireSemaphore(Semaphore semaphore)
+    {
+        Timing2 m = forWaiting();
+        try
+        {
+            return semaphore.tryAcquire(m.value, m.unit);
+        }
+        catch ( InterruptedException e )
+        {
+            Thread.currentThread().interrupt();
+        }
+        return false;
+    }
+
+    /**
+     * Wait on the given semaphore
+     *
+     * @param semaphore the semaphore
+     * @param n         number of permits to acquire
+     * @return result of {@link java.util.concurrent.Semaphore#tryAcquire(int, long, java.util.concurrent.TimeUnit)}
+     */
+    public boolean acquireSemaphore(Semaphore semaphore, int n)
+    {
+        Timing2 m = forWaiting();
+        try
+        {
+            return semaphore.tryAcquire(n, m.value, m.unit);
+        }
+        catch ( InterruptedException e )
+        {
+            Thread.currentThread().interrupt();
+        }
+        return false;
+    }
+
+    /**
+     * Return a new timing that is a multiple of the this timing
+     *
+     * @param n the multiple
+     * @return this timing times the multiple
+     */
+    public Timing2 multiple(double n)
+    {
+        return new Timing2((int)(value * n), unit);
+    }
+
+    /**
+     * Return a new timing that is a multiple of the this timing
+     *
+     * @param n the multiple
+     * @param waitingMultiple new waitingMultiple
+     * @return this timing times the multiple
+     */
+    public Timing2 multiple(double n, int waitingMultiple)
+    {
+        return new Timing2((int)(value * n), unit, waitingMultiple);
+    }
+
+    /**
+     * Return a new timing with the standard multiple for waiting on latches, etc.
+     *
+     * @return this timing multiplied
+     */
+    @SuppressWarnings("PointlessArithmeticExpression")
+    public Timing2 forWaiting()
+    {
+        return multiple(waitingMultiple);
+    }
+
+    /**
+     * Return a new timing with a multiple that ensures a ZK session timeout
+     *
+     * @return this timing multiplied
+     */
+    public Timing2 forSessionSleep()
+    {
+        return multiple(SESSION_SLEEP_MULTIPLE, 1);
+    }
+
+    /**
+     * Return a new timing with a multiple for sleeping a smaller amount of time
+     *
+     * @return this timing multiplied
+     */
+    public Timing2 forSleepingABit()
+    {
+        return multiple(.25);
+    }
+
+    /**
+     * Sleep for a small amount of time
+     *
+     * @throws InterruptedException if interrupted
+     */
+    public void sleepABit() throws InterruptedException
+    {
+        forSleepingABit().sleep();
+    }
+
+    /**
+     * Sleep for a the full amount of time
+     *
+     * @throws InterruptedException if interrupted
+     */
+    public void sleep() throws InterruptedException
+    {
+        unit.sleep(value);
+    }
+
+    /**
+     * Return the value to use for ZK session timeout
+     *
+     * @return session timeout
+     */
+    public int session()
+    {
+        return multiple(SESSION_MULTIPLE).milliseconds();
+    }
+
+    /**
+     * Return the value to use for ZK connection timeout
+     *
+     * @return connection timeout
+     */
+    public int connection()
+    {
+        return milliseconds();
+    }
+
+    private static Integer getWaitingMultiple()
+    {
+        return Integer.getInteger("timing-waiting-multiple", DEFAULT_WAITING_MULTIPLE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-test/src/main/java/org/apache/curator/test/compatibility/Zk35MethodInterceptor.java
----------------------------------------------------------------------
diff --git a/curator-test/src/main/java/org/apache/curator/test/compatibility/Zk35MethodInterceptor.java b/curator-test/src/main/java/org/apache/curator/test/compatibility/Zk35MethodInterceptor.java
new file mode 100644
index 0000000..8072b68
--- /dev/null
+++ b/curator-test/src/main/java/org/apache/curator/test/compatibility/Zk35MethodInterceptor.java
@@ -0,0 +1,56 @@
+/**
+ * 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.test.compatibility;
+
+import org.apache.curator.test.Compatibility;
+import org.testng.IMethodInstance;
+import org.testng.IMethodInterceptor;
+import org.testng.ITestContext;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class Zk35MethodInterceptor implements IMethodInterceptor
+{
+    public static final String zk35Group = "zk35";
+
+    @Override
+    public List<IMethodInstance> intercept(List<IMethodInstance> methods, ITestContext context)
+    {
+        if ( !Compatibility.isZK34() )
+        {
+            return methods;
+        }
+
+        List<IMethodInstance> filteredMethods = new ArrayList<>();
+        for ( IMethodInstance method : methods )
+        {
+            if ( !isInGroup(method.getMethod().getGroups()) )
+            {
+                filteredMethods.add(method);
+            }
+        }
+        return filteredMethods;
+    }
+
+    private boolean isInGroup(String[] groups)
+    {
+        return (groups != null) && Arrays.asList(groups).contains(zk35Group);
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-x-async/src/test/java/org/apache/curator/x/async/CompletableBaseClassForTests.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/test/java/org/apache/curator/x/async/CompletableBaseClassForTests.java b/curator-x-async/src/test/java/org/apache/curator/x/async/CompletableBaseClassForTests.java
index 28c9f11..ae0df3b 100644
--- a/curator-x-async/src/test/java/org/apache/curator/x/async/CompletableBaseClassForTests.java
+++ b/curator-x-async/src/test/java/org/apache/curator/x/async/CompletableBaseClassForTests.java
@@ -20,8 +20,7 @@ package org.apache.curator.x.async;
 
 import com.google.common.base.Throwables;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.Timing;
-import org.apache.curator.test.Timing2;
+import org.apache.curator.test.compatibility.Timing2;
 import org.testng.Assert;
 import java.util.concurrent.CompletionStage;
 import java.util.concurrent.ExecutionException;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java
----------------------------------------------------------------------
diff --git a/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java b/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java
index a2cf157..b67bff9 100644
--- a/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java
+++ b/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java
@@ -25,7 +25,7 @@ import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.KillSession2;
+import org.apache.curator.test.compatibility.KillSession2;
 import org.apache.curator.test.Timing;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.x.discovery.ServiceDiscovery;

http://git-wip-us.apache.org/repos/asf/curator/blob/242b7011/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5a8045e..b816d36 100644
--- a/pom.xml
+++ b/pom.xml
@@ -80,7 +80,6 @@
         <maven-shade-plugin-version>2.4.3</maven-shade-plugin-version>
         <slf4j-version>1.7.6</slf4j-version>
         <clirr-maven-plugin-version>2.8</clirr-maven-plugin-version>
-        <build-helper-maven-plugin-version>3.0.0</build-helper-maven-plugin-version>
 
         <!-- OSGi Properties -->
         <osgi.export.package />
@@ -868,12 +867,6 @@
                     </execution>
                 </executions>
             </plugin>
-
-            <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>build-helper-maven-plugin</artifactId>
-                <version>${build-helper-maven-plugin-version}</version>
-            </plugin>
         </plugins>
     </build>
 </project>


[08/18] curator git commit: license

Posted by ra...@apache.org.
license


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

Branch: refs/heads/CURATOR-426
Commit: 6b6f13f74afa0fe3d8752224c1af82c2709e7f6b
Parents: 96340af
Author: randgalt <ra...@apache.org>
Authored: Thu Jul 20 12:17:50 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Thu Jul 20 12:17:50 2017 -0500

----------------------------------------------------------------------
 .../java/org/apache/curator/test/DummyTest.java   | 18 ++++++++++++++++++
 1 file changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/6b6f13f7/curator-test-zk34/src/test/java/org/apache/curator/test/DummyTest.java
----------------------------------------------------------------------
diff --git a/curator-test-zk34/src/test/java/org/apache/curator/test/DummyTest.java b/curator-test-zk34/src/test/java/org/apache/curator/test/DummyTest.java
index 2f6afa8..f19a85b 100644
--- a/curator-test-zk34/src/test/java/org/apache/curator/test/DummyTest.java
+++ b/curator-test-zk34/src/test/java/org/apache/curator/test/DummyTest.java
@@ -1,3 +1,21 @@
+/**
+ * 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.test;
 
 import org.testng.annotations.Test;


[16/18] curator git commit: merge problem

Posted by ra...@apache.org.
merge problem


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

Branch: refs/heads/CURATOR-426
Commit: afc206c7efad8bf3e596b2e9a523cce3b355f035
Parents: 63f8310
Author: randgalt <ra...@apache.org>
Authored: Fri Jul 21 13:07:10 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Fri Jul 21 13:07:10 2017 -0500

----------------------------------------------------------------------
 src/site/site.xml | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/afc206c7/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
index 7761966..4604664 100644
--- a/src/site/site.xml
+++ b/src/site/site.xml
@@ -76,6 +76,10 @@
             <item name="Schema Support" href="curator-framework/schema.html"/>
         </menu>
 
+        <menu name="Compatibility" inherit="top">
+            <item name="ZooKeeper Versions" href="compatibility.html"/>
+        </menu>
+
         <menu name="Low Level" inherit="top">
             <item name="Framework" href="curator-framework/index.html"/>
             <item name="Utilities" href="utilities.html"/>
@@ -90,7 +94,6 @@
             <item name="Source Code" href="source-repository.html"/>
             <item name="Project Team" href="team-list.html"/>
             <item name="Project Information" href="project-info.html"/>
-            <item name="API Compatibility" href="compatibility.html"/>
             <item name="Javadoc" href="apidocs/index.html"/>
             <item name="Wiki" href="https://cwiki.apache.org/confluence/display/CURATOR"/>
             <item name="Releases" href="https://cwiki.apache.org/confluence/display/CURATOR/Releases"/>


[03/18] curator git commit: Basic concept of zk 3.4.x compatibility proven. The Compatibility class checks for a well-known 3.5 class and sets a static that advertises whether the ZK lib is 3.4.x or 3.5.x. Then, the code "ifs" using this static. The majo

Posted by ra...@apache.org.
Basic concept of zk 3.4.x compatibility proven. The Compatibility class checks for a well-known 3.5 class and sets a static
that advertises whether the ZK lib is 3.4.x or 3.5.x. Then, the code "ifs" using this static. The major work was emulating
the kill session injection (that emulation is done using reflection) and testing. The curator-test-zk module runs
the framework and recipe tests but forces ZooKeeper 3.4.x and uses the Curator 2.x version of curator-test. This
requires a few tricks as the new code uses new methods/classes on the Curator 3.x version of curator-test. I'll write
a readme documenting how this is done.


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

Branch: refs/heads/CURATOR-426
Commit: 58bc969fecc01a1947dd272300d4fb305d221ea2
Parents: 0641243
Author: randgalt <ra...@apache.org>
Authored: Thu Jul 20 01:48:40 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Thu Jul 20 01:48:40 2017 -0500

----------------------------------------------------------------------
 .../org/apache/curator/utils/Compatibility.java |  76 +++++
 .../curator/utils/InjectSessionExpiration.java  | 107 +++++++
 .../java/org/apache/curator/BasicTests.java     |   4 +-
 .../curator/TestSessionFailRetryLoop.java       |  11 +-
 .../framework/CuratorFrameworkFactory.java      |  30 +-
 .../apache/curator/framework/SafeIsTtlMode.java |   3 +-
 .../framework/imps/CreateBuilderImpl.java       |   9 +-
 .../imps/CuratorMultiTransactionImpl.java       |  18 +-
 .../framework/state/ConnectionStateManager.java |   6 +-
 .../curator/framework/imps/TestCleanState.java  |   7 +
 .../framework/imps/TestCreateReturningStat.java |  13 +-
 .../imps/TestEnabledSessionExpiredState.java    |   8 +-
 .../curator/framework/imps/TestFramework.java   |   3 +-
 .../framework/imps/TestFrameworkEdges.java      |   8 +-
 .../framework/imps/TestReconfiguration.java     |   9 +-
 .../framework/imps/TestRemoveWatches.java       |   6 +-
 .../curator/framework/imps/TestTtlNodes.java    |   5 +-
 .../imps/TestWatcherRemovalManager.java         |   5 +-
 .../recipes/cache/TestEventOrdering.java        |   3 +-
 .../framework/recipes/cache/TestNodeCache.java  |   4 +-
 .../recipes/cache/TestPathChildrenCache.java    |   4 +-
 .../framework/recipes/cache/TestTreeCache.java  |   4 +-
 .../recipes/leader/TestLeaderLatch.java         |   3 +-
 .../recipes/leader/TestLeaderSelector.java      |   9 +-
 .../recipes/locks/TestInterProcessMutex.java    |   5 +-
 .../locks/TestInterProcessMutexBase.java        |   7 +-
 .../nodes/TestPersistentEphemeralNode.java      |  14 +-
 .../recipes/nodes/TestPersistentNode.java       |   3 +-
 curator-test-zk34/pom.xml                       |  44 +++
 .../curator/framework/imps/TestCleanState.java  |  25 --
 .../org/apache/curator/test/Compatibility.java  |  35 +++
 curator-test/pom.xml                            |  19 ++
 .../apache/curator/test/CuratorTestBase.java    |  27 ++
 .../org/apache/curator/test/KillSession2.java   |  39 +++
 .../java/org/apache/curator/test/Timing2.java   | 299 +++++++++++++++++++
 .../curator/test/Zk35MethodInterceptor.java     |  55 ++++
 .../org/apache/curator/test/Compatibility.java  |  34 +++
 .../apache/curator/test/KillServerSession.java  | 122 --------
 .../org/apache/curator/test/KillSession.java    |  67 -----
 .../java/org/apache/curator/test/Timing.java    |  75 +----
 .../x/async/CompletableBaseClassForTests.java   |   3 +-
 .../discovery/details/TestServiceDiscovery.java |   8 +-
 pom.xml                                         |   7 +
 43 files changed, 859 insertions(+), 384 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-client/src/main/java/org/apache/curator/utils/Compatibility.java
----------------------------------------------------------------------
diff --git a/curator-client/src/main/java/org/apache/curator/utils/Compatibility.java b/curator-client/src/main/java/org/apache/curator/utils/Compatibility.java
new file mode 100644
index 0000000..58b62a7
--- /dev/null
+++ b/curator-client/src/main/java/org/apache/curator/utils/Compatibility.java
@@ -0,0 +1,76 @@
+/**
+ * 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.utils;
+
+import org.apache.zookeeper.ZooKeeper;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utils to help with ZK 3.4.x compatibility
+ */
+public class Compatibility
+{
+    private static final boolean hasZooKeeperAdmin;
+    static
+    {
+        boolean hasIt;
+        try
+        {
+            Class.forName("org.apache.zookeeper.admin.ZooKeeperAdmin");
+            hasIt = true;
+        }
+        catch ( ClassNotFoundException e )
+        {
+            hasIt = false;
+            LoggerFactory.getLogger(Compatibility.class).info("Running in ZooKeeper 3.4.x compatibility mode");
+        }
+        hasZooKeeperAdmin = hasIt;
+    }
+
+    /**
+     * Return true if the classpath ZooKeeper library is 3.4.x
+     *
+     * @return true/false
+     */
+    public static boolean isZK34()
+    {
+        return !hasZooKeeperAdmin;
+    }
+
+    /**
+     * For ZooKeeper 3.5.x, use the supported <code>zooKeeper.getTestable().injectSessionExpiration()</code>.
+     * For ZooKeeper 3.4.x do the equivalent via reflection
+     *
+     * @param zooKeeper client
+     */
+    public static void injectSessionExpiration(ZooKeeper zooKeeper)
+    {
+        if ( isZK34() )
+        {
+            InjectSessionExpiration.injectSessionExpiration(zooKeeper);
+        }
+        else
+        {
+            // LOL - this method was proposed by me (JZ) in 2013 for totally unrelated reasons
+            // it got added to ZK 3.5 and now does exactly what we need
+            // https://issues.apache.org/jira/browse/ZOOKEEPER-1730
+            zooKeeper.getTestable().injectSessionExpiration();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-client/src/main/java/org/apache/curator/utils/InjectSessionExpiration.java
----------------------------------------------------------------------
diff --git a/curator-client/src/main/java/org/apache/curator/utils/InjectSessionExpiration.java b/curator-client/src/main/java/org/apache/curator/utils/InjectSessionExpiration.java
new file mode 100644
index 0000000..996e9a2
--- /dev/null
+++ b/curator-client/src/main/java/org/apache/curator/utils/InjectSessionExpiration.java
@@ -0,0 +1,107 @@
+/**
+ * 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.utils;
+
+import org.apache.zookeeper.ClientCnxn;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+
+// reflective version of zooKeeper.getTestable().injectSessionExpiration();
+@SuppressWarnings("JavaReflectionMemberAccess")
+public class InjectSessionExpiration
+{
+    private static final Field cnxnField;
+    private static final Field stateField;
+    private static final Field eventThreadField;
+    private static final Field sendThreadField;
+    private static final Method queueEventMethod;
+    private static final Method queueEventOfDeathMethod;
+    private static final Method getClientCnxnSocketMethod;
+    private static final Method wakeupCnxnMethod;
+    static
+    {
+        Field localCnxnField;
+        Field localStateField;
+        Field localEventThreadField;
+        Field localSendThreadField;
+        Method localQueueEventMethod;
+        Method localEventOfDeathMethod;
+        Method localGetClientCnxnSocketMethod;
+        Method localWakeupCnxnMethod;
+        try
+        {
+            Class<?> eventThreadClass = Class.forName("org.apache.zookeeper.ClientCnxn$EventThread");
+            Class<?> sendThreadClass = Class.forName("org.apache.zookeeper.ClientCnxn$SendThread");
+            Class<?> clientCnxnSocketClass = Class.forName("org.apache.zookeeper.ClientCnxnSocket");
+
+            localCnxnField = ZooKeeper.class.getDeclaredField("cnxn");
+            localCnxnField.setAccessible(true);
+            localStateField = ClientCnxn.class.getDeclaredField("state");
+            localStateField.setAccessible(true);
+            localEventThreadField = ClientCnxn.class.getDeclaredField("eventThread");
+            localEventThreadField.setAccessible(true);
+            localSendThreadField = ClientCnxn.class.getDeclaredField("sendThread");
+            localSendThreadField.setAccessible(true);
+            localQueueEventMethod = eventThreadClass.getDeclaredMethod("queueEvent", WatchedEvent.class);
+            localQueueEventMethod.setAccessible(true);
+            localEventOfDeathMethod = eventThreadClass.getDeclaredMethod("queueEventOfDeath");
+            localEventOfDeathMethod.setAccessible(true);
+            localGetClientCnxnSocketMethod = sendThreadClass.getDeclaredMethod("getClientCnxnSocket");
+            localGetClientCnxnSocketMethod.setAccessible(true);
+            localWakeupCnxnMethod = clientCnxnSocketClass.getDeclaredMethod("wakeupCnxn");
+            localWakeupCnxnMethod.setAccessible(true);
+        }
+        catch ( ReflectiveOperationException e )
+        {
+            throw new RuntimeException("Could not access internal ZooKeeper fields", e);
+        }
+        cnxnField = localCnxnField;
+        stateField = localStateField;
+        eventThreadField = localEventThreadField;
+        sendThreadField = localSendThreadField;
+        queueEventMethod = localQueueEventMethod;
+        queueEventOfDeathMethod = localEventOfDeathMethod;
+        getClientCnxnSocketMethod = localGetClientCnxnSocketMethod;
+        wakeupCnxnMethod = localWakeupCnxnMethod;
+    }
+
+    public static void injectSessionExpiration(ZooKeeper zooKeeper)
+    {
+        try
+        {
+            WatchedEvent event = new WatchedEvent(Watcher.Event.EventType.None, Watcher.Event.KeeperState.Expired, null);
+
+            ClientCnxn clientCnxn = (ClientCnxn)cnxnField.get(zooKeeper);
+            Object eventThread = eventThreadField.get(clientCnxn);
+            queueEventMethod.invoke(eventThread, event);
+            queueEventOfDeathMethod.invoke(eventThread);
+            stateField.set(clientCnxn, ZooKeeper.States.CLOSED);
+            Object sendThread = sendThreadField.get(clientCnxn);
+            Object clientCnxnSocket = getClientCnxnSocketMethod.invoke(sendThread);
+            wakeupCnxnMethod.invoke(clientCnxnSocket);
+        }
+        catch ( ReflectiveOperationException e )
+        {
+            throw new RuntimeException("Could not inject session expiration using reflection", e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-client/src/test/java/org/apache/curator/BasicTests.java
----------------------------------------------------------------------
diff --git a/curator-client/src/test/java/org/apache/curator/BasicTests.java b/curator-client/src/test/java/org/apache/curator/BasicTests.java
index eee5047..94d418b 100644
--- a/curator-client/src/test/java/org/apache/curator/BasicTests.java
+++ b/curator-client/src/test/java/org/apache/curator/BasicTests.java
@@ -21,7 +21,7 @@ package org.apache.curator;
 import org.apache.curator.ensemble.fixed.FixedEnsembleProvider;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.KillSession;
+import org.apache.curator.test.KillSession2;
 import org.apache.curator.test.Timing;
 import org.apache.curator.utils.ZookeeperFactory;
 import org.apache.zookeeper.CreateMode;
@@ -100,7 +100,7 @@ public class BasicTests extends BaseClassForTests
                                 // ignore
                             }
 
-                            KillSession.kill(client.getZooKeeper(), server.getConnectString());
+                            KillSession2.kill(client.getZooKeeper());
 
                             Assert.assertTrue(timing.awaitLatch(latch));
                         }

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java
----------------------------------------------------------------------
diff --git a/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java b/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java
index b33939f..e56f4d4 100644
--- a/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java
+++ b/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java
@@ -20,9 +20,8 @@ package org.apache.curator;
 
 import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.KillSession2;
 import org.apache.curator.utils.CloseableUtils;
-import org.apache.curator.retry.RetryOneTime;
-import org.apache.curator.test.KillSession;
 import org.apache.curator.test.Timing;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -58,7 +57,7 @@ public class TestSessionFailRetryLoop extends BaseClassForTests
                                 if ( firstTime.compareAndSet(true, false) )
                                 {
                                     Assert.assertNull(client.getZooKeeper().exists("/foo/bar", false));
-                                    KillSession.kill(client.getZooKeeper(), server.getConnectString());
+                                    KillSession2.kill(client.getZooKeeper());
                                     client.getZooKeeper();
                                     client.blockUntilConnectedOrTimedOut();
                                 }
@@ -132,7 +131,7 @@ public class TestSessionFailRetryLoop extends BaseClassForTests
                                     if ( firstTime.compareAndSet(true, false) )
                                     {
                                         Assert.assertNull(client.getZooKeeper().exists("/foo/bar", false));
-                                        KillSession.kill(client.getZooKeeper(), server.getConnectString());
+                                        KillSession2.kill(client.getZooKeeper());
                                         client.getZooKeeper();
                                         client.blockUntilConnectedOrTimedOut();
                                     }
@@ -197,7 +196,7 @@ public class TestSessionFailRetryLoop extends BaseClassForTests
                                 public Void call() throws Exception
                                 {
                                     Assert.assertNull(client.getZooKeeper().exists("/foo/bar", false));
-                                    KillSession.kill(client.getZooKeeper(), server.getConnectString());
+                                    KillSession2.kill(client.getZooKeeper());
 
                                     timing.sleepABit();
 
@@ -259,7 +258,7 @@ public class TestSessionFailRetryLoop extends BaseClassForTests
                                     public Void call() throws Exception
                                     {
                                         Assert.assertNull(client.getZooKeeper().exists("/foo/bar", false));
-                                        KillSession.kill(client.getZooKeeper(), server.getConnectString());
+                                        KillSession2.kill(client.getZooKeeper());
 
                                         client.getZooKeeper();
                                         client.blockUntilConnectedOrTimedOut();

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
index 5ce7762..18011aa 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
@@ -44,7 +44,6 @@ import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
-import org.slf4j.LoggerFactory;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Arrays;
@@ -52,6 +51,8 @@ import java.util.List;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.curator.utils.Compatibility.isZK34;
+
 /**
  * Factory methods for creating framework-style clients
  */
@@ -68,33 +69,6 @@ public class CuratorFrameworkFactory
     private static final long DEFAULT_INACTIVE_THRESHOLD_MS = (int)TimeUnit.MINUTES.toMillis(3);
     private static final int DEFAULT_CLOSE_WAIT_MS = (int)TimeUnit.SECONDS.toMillis(1);
 
-    private static final boolean hasZooKeeperAdmin;
-    static
-    {
-        boolean hasIt;
-        try
-        {
-            Class.forName("org.apache.zookeeper.admin.ZooKeeperAdmin");
-            hasIt = true;
-        }
-        catch ( ClassNotFoundException e )
-        {
-            hasIt = false;
-            LoggerFactory.getLogger(CuratorFrameworkFactory.class).info("Running in ZooKeeper 3.4.x compatibility mode");
-        }
-        hasZooKeeperAdmin = hasIt;
-    }
-
-    /**
-     * Return true if the classpath ZooKeeper library is 3.4.x
-     *
-     * @return true/false
-     */
-    public static boolean isZK34()
-    {
-        return !hasZooKeeperAdmin;
-    }
-
     /**
      * Return a new builder that builds a CuratorFramework
      *

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-framework/src/main/java/org/apache/curator/framework/SafeIsTtlMode.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/SafeIsTtlMode.java b/curator-framework/src/main/java/org/apache/curator/framework/SafeIsTtlMode.java
index 3c4b9e6..e499a7b 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/SafeIsTtlMode.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/SafeIsTtlMode.java
@@ -18,6 +18,7 @@
  */
 package org.apache.curator.framework;
 
+import org.apache.curator.utils.Compatibility;
 import org.apache.zookeeper.CreateMode;
 
 public class SafeIsTtlMode
@@ -34,7 +35,7 @@ public class SafeIsTtlMode
 
     public static boolean isTtl(CreateMode mode)
     {
-        return !CuratorFrameworkFactory.isZK34() && Internal.instance.isTtl(mode);
+        return !Compatibility.isZK34() && Internal.instance.isTtl(mode);
     }
 
     private SafeIsTtlMode()

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/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 b58084f..1487d6b 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
@@ -176,7 +176,14 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
                 }
 
                 String fixedPath = client.fixForNamespace(path);
-                transaction.add(Op.create(fixedPath, data, acling.getAclList(path), createMode, ttl), OperationType.CREATE, path);
+                if ( client.isZk34CompatibilityMode() )
+                {
+                    transaction.add(Op.create(fixedPath, data, acling.getAclList(path), createMode), OperationType.CREATE, path);
+                }
+                else
+                {
+                    transaction.add(Op.create(fixedPath, data, acling.getAclList(path), createMode, ttl), OperationType.CREATE, path);
+                }
                 return context;
             }
         };

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorMultiTransactionImpl.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorMultiTransactionImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorMultiTransactionImpl.java
index bdab158..9057934 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorMultiTransactionImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorMultiTransactionImpl.java
@@ -35,6 +35,7 @@ import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
 import org.apache.curator.framework.schema.Schema;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.OpResult;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.proto.CreateRequest;
@@ -135,7 +136,22 @@ public class CuratorMultiTransactionImpl implements
             if ( (curatorOp.get().getType() == ZooDefs.OpCode.create) || (curatorOp.get().getType() == ZooDefs.OpCode.createContainer) )
             {
                 CreateRequest createRequest = (CreateRequest)curatorOp.get().toRequestRecord();
-                CreateMode createMode = CreateMode.fromFlag(createRequest.getFlags(), CreateMode.PERSISTENT);
+                CreateMode createMode;
+                if ( client.isZk34CompatibilityMode() )
+                {
+                    try
+                    {
+                        createMode = CreateMode.fromFlag(createRequest.getFlags());
+                    }
+                    catch ( KeeperException.BadArgumentsException dummy )
+                    {
+                        createMode = CreateMode.PERSISTENT;
+                    }
+                }
+                else
+                {
+                    createMode = CreateMode.fromFlag(createRequest.getFlags(), CreateMode.PERSISTENT);
+                }
                 schema.validateCreate(createMode, createRequest.getPath(), createRequest.getData(), createRequest.getAcl());
             }
             else if ( (curatorOp.get().getType() == ZooDefs.OpCode.delete) || (curatorOp.get().getType() == ZooDefs.OpCode.deleteContainer) )

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java
index 56c2250..251baa9 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java
@@ -23,6 +23,7 @@ import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.listen.ListenerContainer;
+import org.apache.curator.utils.Compatibility;
 import org.apache.curator.utils.ThreadUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -306,10 +307,7 @@ public class ConnectionStateManager implements Closeable
                 log.warn(String.format("Session timeout has elapsed while SUSPENDED. Injecting a session expiration. Elapsed ms: %d. Adjusted session timeout ms: %d", elapsedMs, useSessionTimeoutMs));
                 try
                 {
-                    // LOL - this method was proposed by me (JZ) in 2013 for totally unrelated reasons
-                    // it got added to ZK 3.5 and now does exactly what we need
-                    // https://issues.apache.org/jira/browse/ZOOKEEPER-1730
-                    client.getZookeeperClient().getZooKeeper().getTestable().injectSessionExpiration();
+                    Compatibility.injectSessionExpiration(client.getZookeeperClient().getZooKeeper());
                 }
                 catch ( Exception e )
                 {

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCleanState.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCleanState.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCleanState.java
index 9d90616..7a61eea 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCleanState.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCleanState.java
@@ -21,6 +21,7 @@ package org.apache.curator.framework.imps;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.test.WatchersDebug;
 import org.apache.curator.utils.CloseableUtils;
+import org.apache.curator.utils.Compatibility;
 import org.apache.zookeeper.ZooKeeper;
 import java.util.concurrent.Callable;
 
@@ -33,6 +34,12 @@ public class TestCleanState
             return;
         }
 
+        if ( Compatibility.isZK34() )
+        {
+            CloseableUtils.closeQuietly(client);
+            return;
+        }
+
         try
         {
             CuratorFrameworkImpl internalClient = (CuratorFrameworkImpl)client;

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java
index 4e9e78c..bef143f 100755
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java
@@ -18,25 +18,24 @@
  */
 package org.apache.curator.framework.imps;
 
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.api.CuratorEventType;
-import org.apache.curator.framework.api.ExistsBuilder;
-import org.apache.curator.framework.api.PathAndBytesable;
 import org.apache.curator.retry.RetryOneTime;
-import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.CuratorTestBase;
 import org.apache.curator.test.Timing;
+import org.apache.curator.test.Zk35MethodInterceptor;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.data.Stat;
 import org.testng.Assert;
 import org.testng.annotations.Test;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicReference;
 
-public class TestCreateReturningStat extends BaseClassForTests
+@Test(groups = Zk35MethodInterceptor.zk35Group)
+public class TestCreateReturningStat extends CuratorTestBase
 {
     private CuratorFramework createClient()
     {

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java
index f96592a..63fd36e 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java
@@ -25,8 +25,8 @@ import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.KillSession;
-import org.apache.curator.test.Timing;
+import org.apache.curator.test.KillSession2;
+import org.apache.curator.test.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
@@ -41,7 +41,7 @@ import java.util.concurrent.TimeUnit;
 
 public class TestEnabledSessionExpiredState extends BaseClassForTests
 {
-    private final Timing timing = new Timing();
+    private final Timing2 timing = new Timing2();
 
     private CuratorFramework client;
     private BlockingQueue<ConnectionState> states;
@@ -127,7 +127,7 @@ public class TestEnabledSessionExpiredState extends BaseClassForTests
     {
         Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.CONNECTED);
 
-        KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString());
+        KillSession2.kill(client.getZookeeperClient().getZooKeeper());
 
         Assert.assertEquals(states.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.LOST);
         Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.RECONNECTED);

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/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 5d0c5ed..70ae2ea 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
@@ -31,6 +31,7 @@ import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.Timing;
+import org.apache.curator.test.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.utils.EnsurePath;
 import org.apache.curator.utils.ZKPaths;
@@ -261,7 +262,7 @@ public class TestFramework extends BaseClassForTests
             client.getChildren().usingWatcher(watcher).forPath("/base");
             client.create().forPath("/base/child");
 
-            String path = new Timing().takeFromQueue(queue);
+            String path = new Timing2().takeFromQueue(queue);
             Assert.assertEquals(path, "/base");
         }
         finally

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/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 8fee2d3..42e9afa 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
@@ -33,9 +33,9 @@ import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.RetryNTimes;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.KillSession;
+import org.apache.curator.test.KillSession2;
 import org.apache.curator.test.TestingServer;
-import org.apache.curator.test.Timing;
+import org.apache.curator.test.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.utils.ZKPaths;
 import org.apache.zookeeper.CreateMode;
@@ -57,7 +57,7 @@ import org.apache.curator.framework.api.CreateBuilder;
 
 public class TestFrameworkEdges extends BaseClassForTests
 {
-    private final Timing timing = new Timing();
+    private final Timing2 timing = new Timing2();
 
     @Test
     public void testCreateContainersForBadConnect() throws Exception
@@ -391,7 +391,7 @@ public class TestFrameworkEdges extends BaseClassForTests
                 }
             };
             client.checkExists().usingWatcher(watcher).forPath("/sessionTest");
-            KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString());
+            KillSession2.kill(client.getZookeeperClient().getZooKeeper());
             Assert.assertNotNull(client.checkExists().forPath("/sessionTest"));
             Assert.assertTrue(sessionDied.get());
         }

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java
index ef2faed..567d71d 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java
@@ -27,11 +27,13 @@ import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.api.CuratorEventType;
 import org.apache.curator.retry.ExponentialBackoffRetry;
-import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.CuratorTestBase;
 import org.apache.curator.test.InstanceSpec;
 import org.apache.curator.test.TestingCluster;
 import org.apache.curator.test.TestingZooKeeperServer;
 import org.apache.curator.test.Timing;
+import org.apache.curator.test.Timing2;
+import org.apache.curator.test.Zk35MethodInterceptor;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
@@ -56,9 +58,10 @@ import java.util.Properties;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicReference;
 
-public class TestReconfiguration extends BaseClassForTests
+@Test(groups = Zk35MethodInterceptor.zk35Group)
+public class TestReconfiguration extends CuratorTestBase
 {
-    private final Timing timing = new Timing();
+    private final Timing2 timing = new Timing2();
     private TestingCluster cluster;
     private EnsembleProvider ensembleProvider;
 

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-framework/src/test/java/org/apache/curator/framework/imps/TestRemoveWatches.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestRemoveWatches.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestRemoveWatches.java
index 4ac68d3..66f5703 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestRemoveWatches.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestRemoveWatches.java
@@ -30,8 +30,9 @@ import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.curator.retry.RetryOneTime;
-import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.CuratorTestBase;
 import org.apache.curator.test.Timing;
+import org.apache.curator.test.Zk35MethodInterceptor;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
@@ -44,7 +45,8 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
-public class TestRemoveWatches extends BaseClassForTests
+@Test(groups = Zk35MethodInterceptor.zk35Group)
+public class TestRemoveWatches extends CuratorTestBase
 {
     private AtomicReference<ConnectionState> registerConnectionStateListener(CuratorFramework client)
     {

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTtlNodes.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTtlNodes.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTtlNodes.java
index c544474..297399d 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTtlNodes.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTtlNodes.java
@@ -24,7 +24,9 @@ import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.CuratorTestBase;
 import org.apache.curator.test.Timing;
+import org.apache.curator.test.Zk35MethodInterceptor;
 import org.apache.zookeeper.CreateMode;
 import org.testng.Assert;
 import org.testng.annotations.AfterMethod;
@@ -32,7 +34,8 @@ import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 import java.util.concurrent.CountDownLatch;
 
-public class TestTtlNodes extends BaseClassForTests
+@Test(groups = Zk35MethodInterceptor.zk35Group)
+public class TestTtlNodes extends CuratorTestBase
 {
     @BeforeMethod
     @Override

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWatcherRemovalManager.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWatcherRemovalManager.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWatcherRemovalManager.java
index 9c405a2..d3ad66f 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWatcherRemovalManager.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWatcherRemovalManager.java
@@ -25,8 +25,10 @@ import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.CuratorTestBase;
 import org.apache.curator.test.Timing;
 import org.apache.curator.test.WatchersDebug;
+import org.apache.curator.test.Zk35MethodInterceptor;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
@@ -36,7 +38,8 @@ import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 
-public class TestWatcherRemovalManager extends BaseClassForTests
+@Test(groups = Zk35MethodInterceptor.zk35Group)
+public class TestWatcherRemovalManager extends CuratorTestBase
 {
     @Test
     public void testSameWatcherDifferentPaths1Triggered() throws Exception

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestEventOrdering.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestEventOrdering.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestEventOrdering.java
index 7b3a07e..c50474d 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestEventOrdering.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestEventOrdering.java
@@ -25,6 +25,7 @@ import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.Timing;
+import org.apache.curator.test.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.KeeperException;
 import org.testng.Assert;
@@ -41,7 +42,7 @@ import java.util.concurrent.TimeUnit;
 
 public abstract class TestEventOrdering<T extends Closeable> extends BaseClassForTests
 {
-    private final Timing timing = new Timing();
+    private final Timing2 timing = new Timing2();
     private final long start = System.currentTimeMillis();
     private static final int THREAD_QTY = 100;
     private static final int ITERATIONS = 100;

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestNodeCache.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestNodeCache.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestNodeCache.java
index d6d495a..52c76ad 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestNodeCache.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestNodeCache.java
@@ -20,12 +20,12 @@ package org.apache.curator.framework.recipes.cache;
 
 import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.KillSession2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.framework.api.UnhandledErrorListener;
 import org.apache.curator.retry.RetryOneTime;
-import org.apache.curator.test.KillSession;
 import org.apache.curator.test.Timing;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -194,7 +194,7 @@ public class TestNodeCache extends BaseClassForTests
                 }
             );
 
-            KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString());
+            KillSession2.kill(client.getZookeeperClient().getZooKeeper());
             Thread.sleep(timing.multiple(1.5).session());
 
             Assert.assertEquals(cache.getCurrentData().getData(), "start".getBytes());

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
index a09c878..f24b846 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
@@ -29,7 +29,7 @@ import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.ExecuteCalledWatchingExecutorService;
-import org.apache.curator.test.KillSession;
+import org.apache.curator.test.KillSession2;
 import org.apache.curator.test.TestingServer;
 import org.apache.curator.test.Timing;
 import org.apache.curator.utils.CloseableUtils;
@@ -715,7 +715,7 @@ public class TestPathChildrenCache extends BaseClassForTests
             client.create().withMode(CreateMode.EPHEMERAL).forPath("/test/me", "data".getBytes());
             Assert.assertTrue(timing.awaitLatch(childAddedLatch));
 
-            KillSession.kill(client.getZookeeperClient().getZooKeeper());
+            KillSession2.kill(client.getZookeeperClient().getZooKeeper());
             Assert.assertTrue(timing.awaitLatch(lostLatch));
             Assert.assertTrue(timing.awaitLatch(reconnectedLatch));
             Assert.assertTrue(timing.awaitLatch(removedLatch));

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/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 c9c22dd..1e203b7 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
@@ -21,9 +21,9 @@ package org.apache.curator.framework.recipes.cache;
 
 import com.google.common.collect.ImmutableSet;
 import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.test.KillServerSession;
 import org.apache.curator.framework.api.UnhandledErrorListener;
 import org.apache.curator.framework.recipes.cache.TreeCacheEvent.Type;
+import org.apache.curator.test.KillSession2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.CreateMode;
 import org.testng.Assert;
@@ -423,7 +423,7 @@ public class TestTreeCache extends BaseTestTreeCache
         client.create().withMode(CreateMode.EPHEMERAL).forPath("/test/me", "data".getBytes());
         assertEvent(TreeCacheEvent.Type.NODE_ADDED, "/test/me");
 
-        KillServerSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString());
+        KillSession2.kill(client.getZookeeperClient().getZooKeeper());
         assertEvent(TreeCacheEvent.Type.CONNECTION_SUSPENDED);
         assertEvent(TreeCacheEvent.Type.CONNECTION_LOST);
         assertEvent(TreeCacheEvent.Type.CONNECTION_RECONNECTED);

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
index 991e6fc..93c955b 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
@@ -35,6 +35,7 @@ import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.TestingServer;
 import org.apache.curator.test.Timing;
+import org.apache.curator.test.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -136,7 +137,7 @@ public class TestLeaderLatch extends BaseClassForTests
     @Test
     public void testErrorPolicies() throws Exception
     {
-        Timing timing = new Timing();
+        Timing2 timing = new Timing2();
         LeaderLatch latch = null;
         CuratorFramework client = CuratorFrameworkFactory.builder()
             .connectString(server.getConnectString())

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java
index 60619d0..79fcdb3 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java
@@ -30,9 +30,10 @@ import org.apache.curator.framework.state.SessionConnectionStateErrorPolicy;
 import org.apache.curator.framework.state.StandardConnectionStateErrorPolicy;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.KillSession;
+import org.apache.curator.test.KillSession2;
 import org.apache.curator.test.TestingServer;
 import org.apache.curator.test.Timing;
+import org.apache.curator.test.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -56,7 +57,7 @@ public class TestLeaderSelector extends BaseClassForTests
     @Test
     public void testErrorPolicies() throws Exception
     {
-        Timing timing = new Timing();
+        Timing2 timing = new Timing2();
         LeaderSelector selector = null;
         CuratorFramework client = CuratorFrameworkFactory
             .builder()
@@ -147,7 +148,7 @@ public class TestLeaderSelector extends BaseClassForTests
     @Test
     public void testLeaderNodeDeleteOnInterrupt() throws Exception
     {
-        Timing timing = new Timing();
+        Timing2 timing = new Timing2();
         LeaderSelector selector = null;
         CuratorFramework client = null;
         try
@@ -486,7 +487,7 @@ public class TestLeaderSelector extends BaseClassForTests
 
             Assert.assertTrue(timing.acquireSemaphore(semaphore, 1));
 
-            KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString());
+            KillSession2.kill(client.getZookeeperClient().getZooKeeper());
 
             Assert.assertTrue(timing.awaitLatch(interruptedLatch));
             timing.sleepABit();

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutex.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutex.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutex.java
index 68daeb7..cf82c57 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutex.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutex.java
@@ -26,7 +26,7 @@ import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.framework.schema.Schema;
 import org.apache.curator.framework.schema.SchemaSet;
 import org.apache.curator.retry.RetryOneTime;
-import org.apache.curator.test.KillSession;
+import org.apache.curator.test.KillSession2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.CreateMode;
 import org.testng.Assert;
@@ -38,7 +38,6 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
-import java.util.regex.Pattern;
 
 public class TestInterProcessMutex extends TestInterProcessMutexBase
 {
@@ -173,7 +172,7 @@ public class TestInterProcessMutex extends TestInterProcessMutexBase
             Assert.assertTrue(lock.isAcquiredInThisProcess());
 
             // Kill the session, check that lock node still exists
-            KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString());
+            KillSession2.kill(client.getZookeeperClient().getZooKeeper());
             Assert.assertNotNull(client.checkExists().forPath(LOCK_PATH));
 
             // Release the lock and verify that the actual lock node created no longer exists

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java
index febf499..43ded2e 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java
@@ -27,9 +27,10 @@ import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.KillSession;
+import org.apache.curator.test.KillSession2;
 import org.apache.curator.test.TestingServer;
 import org.apache.curator.test.Timing;
+import org.apache.curator.test.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.utils.ZKPaths;
 import org.testng.Assert;
@@ -149,7 +150,7 @@ public abstract class TestInterProcessMutexBase extends BaseClassForTests
     @Test
     public void testKilledSession() throws Exception
     {
-        final Timing timing = new Timing();
+        final Timing2 timing = new Timing2();
 
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new ExponentialBackoffRetry(100, 3));
         client.start();
@@ -191,7 +192,7 @@ public abstract class TestInterProcessMutexBase extends BaseClassForTests
                 );
 
             Assert.assertTrue(timing.acquireSemaphore(semaphore, 1));
-            KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString());
+            KillSession2.kill(client.getZookeeperClient().getZooKeeper());
             Assert.assertTrue(timing.forSessionSleep().acquireSemaphore(semaphore, 1));
         }
         finally

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java
index 7d52b58..0ce61d5 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java
@@ -31,8 +31,8 @@ import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.KillSession;
-import org.apache.curator.test.Timing;
+import org.apache.curator.test.KillSession2;
+import org.apache.curator.test.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.utils.ZKPaths;
 import org.apache.zookeeper.CreateMode;
@@ -69,7 +69,7 @@ public class TestPersistentEphemeralNode extends BaseClassForTests
     private final Collection<CuratorFramework> curatorInstances = Lists.newArrayList();
     private final Collection<PersistentEphemeralNode> createdNodes = Lists.newArrayList();
 
-    private final Timing timing = new Timing();
+    private final Timing2 timing = new Timing2();
 
     @AfterMethod
     @Override
@@ -329,7 +329,7 @@ public class TestPersistentEphemeralNode extends BaseClassForTests
             observer.checkExists().usingWatcher(deletedTrigger).forPath(node.getActualPath());
 
             node.debugCreateNodeLatch = new CountDownLatch(1);
-            KillSession.kill(curator.getZookeeperClient().getZooKeeper());
+            KillSession2.kill(curator.getZookeeperClient().getZooKeeper());
 
             // Make sure the node got deleted
             assertTrue(deletedTrigger.firedWithin(timing.forSessionSleep().seconds(), TimeUnit.SECONDS));
@@ -359,7 +359,7 @@ public class TestPersistentEphemeralNode extends BaseClassForTests
             observer.checkExists().usingWatcher(deletedTrigger).forPath(node.getActualPath());
 
             node.debugCreateNodeLatch = new CountDownLatch(1);
-            KillSession.kill(curator.getZookeeperClient().getZooKeeper());
+            KillSession2.kill(curator.getZookeeperClient().getZooKeeper());
 
             // Make sure the node got deleted...
             assertTrue(deletedTrigger.firedWithin(timing.forSessionSleep().seconds(), TimeUnit.SECONDS));
@@ -400,7 +400,7 @@ public class TestPersistentEphemeralNode extends BaseClassForTests
 
                 node.debugCreateNodeLatch = new CountDownLatch(1);
                 // Kill the session, thus cleaning up the node...
-                KillSession.kill(curator.getZookeeperClient().getZooKeeper());
+                KillSession2.kill(curator.getZookeeperClient().getZooKeeper());
 
                 // Make sure the node ended up getting deleted...
                 assertTrue(deletionTrigger.firedWithin(timing.multiple(1.5).forSessionSleep().seconds(), TimeUnit.SECONDS));
@@ -443,7 +443,7 @@ public class TestPersistentEphemeralNode extends BaseClassForTests
             Trigger deletedTrigger = Trigger.deletedOrSetData();
             observer.checkExists().usingWatcher(deletedTrigger).forPath(node.getActualPath());
 
-            KillSession.kill(nodeCreator.getZookeeperClient().getZooKeeper());
+            KillSession2.kill(nodeCreator.getZookeeperClient().getZooKeeper());
 
             // Make sure the node got deleted...
             assertTrue(deletedTrigger.firedWithin(timing.forWaiting().seconds(), TimeUnit.SECONDS));

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentNode.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentNode.java
index 07d9083..b848fe4 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentNode.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentNode.java
@@ -23,6 +23,7 @@ import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.Timing;
+import org.apache.curator.test.Timing2;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.CreateMode;
 import org.testng.Assert;
@@ -67,7 +68,7 @@ public class TestPersistentNode extends BaseClassForTests
     {
         final byte[] TEST_DATA = "hey".getBytes();
 
-        Timing timing = new Timing();
+        Timing2 timing = new Timing2();
         PersistentNode pen = null;
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
         try

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-test-zk34/pom.xml
----------------------------------------------------------------------
diff --git a/curator-test-zk34/pom.xml b/curator-test-zk34/pom.xml
index fe5e216..8d2f073 100644
--- a/curator-test-zk34/pom.xml
+++ b/curator-test-zk34/pom.xml
@@ -58,6 +58,31 @@
 
         <dependency>
             <groupId>org.apache.curator</groupId>
+            <artifactId>curator-framework</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+            </exclusions>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-framework</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+            </exclusions>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
             <artifactId>curator-recipes</artifactId>
             <exclusions>
                 <exclusion>
@@ -122,6 +147,25 @@
                     </dependenciesToScan>
                 </configuration>
             </plugin>
+
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>add-compatibility</id>
+                        <goals>
+                            <goal>add-test-source</goal>
+                        </goals>
+                        <phase>generate-test-sources</phase>
+                        <configuration>
+                            <sources>
+                                <source>../curator-test/src/compatibility/java</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
     </build>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-test-zk34/src/test/java/org/apache/curator/framework/imps/TestCleanState.java
----------------------------------------------------------------------
diff --git a/curator-test-zk34/src/test/java/org/apache/curator/framework/imps/TestCleanState.java b/curator-test-zk34/src/test/java/org/apache/curator/framework/imps/TestCleanState.java
deleted file mode 100644
index 9efeb90..0000000
--- a/curator-test-zk34/src/test/java/org/apache/curator/framework/imps/TestCleanState.java
+++ /dev/null
@@ -1,25 +0,0 @@
-package org.apache.curator.framework.imps;
-
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.utils.CloseableUtils;
-import java.util.concurrent.Callable;
-
-public class TestCleanState
-{
-    public static void closeAndTestClean(CuratorFramework client)
-    {
-        CloseableUtils.closeQuietly(client);
-    }
-
-    public static void test(CuratorFramework client, Callable<Void> proc) throws Exception
-    {
-        try
-        {
-            proc.call();
-        }
-        finally
-        {
-            CloseableUtils.closeQuietly(client);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-test-zk34/src/test/java/org/apache/curator/test/Compatibility.java
----------------------------------------------------------------------
diff --git a/curator-test-zk34/src/test/java/org/apache/curator/test/Compatibility.java b/curator-test-zk34/src/test/java/org/apache/curator/test/Compatibility.java
new file mode 100644
index 0000000..77ddf2c
--- /dev/null
+++ b/curator-test-zk34/src/test/java/org/apache/curator/test/Compatibility.java
@@ -0,0 +1,35 @@
+/**
+ * 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.test;
+
+import org.apache.curator.utils.InjectSessionExpiration;
+import org.apache.zookeeper.ZooKeeper;
+
+class Compatibility
+{
+    static boolean isZK34()
+    {
+        return true;
+    }
+
+    static void injectSessionExpiration(ZooKeeper zooKeeper)
+    {
+        InjectSessionExpiration.injectSessionExpiration(zooKeeper);
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-test/pom.xml
----------------------------------------------------------------------
diff --git a/curator-test/pom.xml b/curator-test/pom.xml
index fb28d53..0ffe023 100644
--- a/curator-test/pom.xml
+++ b/curator-test/pom.xml
@@ -95,6 +95,25 @@
                     </execution>
                 </executions>
             </plugin>
+
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>add-compatibility</id>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <phase>generate-sources</phase>
+                        <configuration>
+                            <sources>
+                                <source>src/compatibility/java</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-test/src/compatibility/java/org/apache/curator/test/CuratorTestBase.java
----------------------------------------------------------------------
diff --git a/curator-test/src/compatibility/java/org/apache/curator/test/CuratorTestBase.java b/curator-test/src/compatibility/java/org/apache/curator/test/CuratorTestBase.java
new file mode 100644
index 0000000..efa94a3
--- /dev/null
+++ b/curator-test/src/compatibility/java/org/apache/curator/test/CuratorTestBase.java
@@ -0,0 +1,27 @@
+/**
+ * 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.test;
+
+import org.testng.annotations.Listeners;
+
+@Listeners(Zk35MethodInterceptor.class)
+public class CuratorTestBase extends BaseClassForTests
+{
+    protected final Timing2 timing = new Timing2();
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-test/src/compatibility/java/org/apache/curator/test/KillSession2.java
----------------------------------------------------------------------
diff --git a/curator-test/src/compatibility/java/org/apache/curator/test/KillSession2.java b/curator-test/src/compatibility/java/org/apache/curator/test/KillSession2.java
new file mode 100644
index 0000000..52ab168
--- /dev/null
+++ b/curator-test/src/compatibility/java/org/apache/curator/test/KillSession2.java
@@ -0,0 +1,39 @@
+/**
+ * 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.test;
+
+import org.apache.zookeeper.ZooKeeper;
+
+/**
+ * <p>
+ *     Utility to simulate a ZK session dying.
+ * </p>
+ */
+public class KillSession2
+{
+    /**
+     * Kill the given ZK session
+     *
+     * @param client the client to kill
+     */
+    public static void     kill(ZooKeeper client)
+    {
+        Compatibility.injectSessionExpiration(client);
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-test/src/compatibility/java/org/apache/curator/test/Timing2.java
----------------------------------------------------------------------
diff --git a/curator-test/src/compatibility/java/org/apache/curator/test/Timing2.java b/curator-test/src/compatibility/java/org/apache/curator/test/Timing2.java
new file mode 100644
index 0000000..02b71c4
--- /dev/null
+++ b/curator-test/src/compatibility/java/org/apache/curator/test/Timing2.java
@@ -0,0 +1,299 @@
+/**
+ * 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.test;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Utility to get various testing times.
+ *
+ * Copied from the old Timing class which is now deprecated. Needed this to support ZK 3.4 compatibility
+ */
+public class Timing2
+{
+    private final long value;
+    private final TimeUnit unit;
+    private final int waitingMultiple;
+
+    private static final int DEFAULT_SECONDS = 10;
+    private static final int DEFAULT_WAITING_MULTIPLE = 5;
+    private static final double SESSION_MULTIPLE = 1.5;
+    private static final double SESSION_SLEEP_MULTIPLE = SESSION_MULTIPLE * 1.75;  // has to be at least session + 2/3 of a session to account for missed heartbeat then session expiration
+
+    /**
+     * Use the default base time
+     */
+    public Timing2()
+    {
+        this(Integer.getInteger("timing-multiple", 1), getWaitingMultiple());
+    }
+
+    /**
+     * Use a multiple of the default base time
+     *
+     * @param multiple the multiple
+     */
+    public Timing2(double multiple)
+    {
+        this((long)(DEFAULT_SECONDS * multiple), TimeUnit.SECONDS, getWaitingMultiple());
+    }
+
+    /**
+     * Use a multiple of the default base time
+     *
+     * @param multiple the multiple
+     * @param waitingMultiple multiple of main timing to use when waiting
+     */
+    public Timing2(double multiple, int waitingMultiple)
+    {
+        this((long)(DEFAULT_SECONDS * multiple), TimeUnit.SECONDS, waitingMultiple);
+    }
+
+    /**
+     * @param value base time
+     * @param unit  base time unit
+     */
+    public Timing2(long value, TimeUnit unit)
+    {
+        this(value, unit, getWaitingMultiple());
+    }
+
+    /**
+     * @param value base time
+     * @param unit  base time unit
+     * @param waitingMultiple multiple of main timing to use when waiting
+     */
+    public Timing2(long value, TimeUnit unit, int waitingMultiple)
+    {
+        this.value = value;
+        this.unit = unit;
+        this.waitingMultiple = waitingMultiple;
+    }
+
+    /**
+     * Return the base time in milliseconds
+     *
+     * @return time ms
+     */
+    public int milliseconds()
+    {
+        return (int)TimeUnit.MILLISECONDS.convert(value, unit);
+    }
+
+    /**
+     * Return the base time in seconds
+     *
+     * @return time secs
+     */
+    public int seconds()
+    {
+        return (int)value;
+    }
+
+    /**
+     * Wait on the given latch
+     *
+     * @param latch latch to wait on
+     * @return result of {@link java.util.concurrent.CountDownLatch#await(long, java.util.concurrent.TimeUnit)}
+     */
+    public boolean awaitLatch(CountDownLatch latch)
+    {
+        Timing2 m = forWaiting();
+        try
+        {
+            return latch.await(m.value, m.unit);
+        }
+        catch ( InterruptedException e )
+        {
+            Thread.currentThread().interrupt();
+        }
+        return false;
+    }
+
+    /**
+     * Try to take an item from the given queue
+     *
+     * @param queue queue
+     * @return item
+     * @throws Exception interrupted or timed out
+     */
+    public <T> T takeFromQueue(BlockingQueue<T> queue) throws Exception
+    {
+        Timing2 m = forWaiting();
+        try
+        {
+            T value = queue.poll(m.value, m.unit);
+            if ( value == null )
+            {
+                throw new TimeoutException("Timed out trying to take from queue");
+            }
+            return value;
+        }
+        catch ( InterruptedException e )
+        {
+            Thread.currentThread().interrupt();
+            throw e;
+        }
+    }
+
+    /**
+     * Wait on the given semaphore
+     *
+     * @param semaphore the semaphore
+     * @return result of {@link java.util.concurrent.Semaphore#tryAcquire()}
+     */
+    public boolean acquireSemaphore(Semaphore semaphore)
+    {
+        Timing2 m = forWaiting();
+        try
+        {
+            return semaphore.tryAcquire(m.value, m.unit);
+        }
+        catch ( InterruptedException e )
+        {
+            Thread.currentThread().interrupt();
+        }
+        return false;
+    }
+
+    /**
+     * Wait on the given semaphore
+     *
+     * @param semaphore the semaphore
+     * @param n         number of permits to acquire
+     * @return result of {@link java.util.concurrent.Semaphore#tryAcquire(int, long, java.util.concurrent.TimeUnit)}
+     */
+    public boolean acquireSemaphore(Semaphore semaphore, int n)
+    {
+        Timing2 m = forWaiting();
+        try
+        {
+            return semaphore.tryAcquire(n, m.value, m.unit);
+        }
+        catch ( InterruptedException e )
+        {
+            Thread.currentThread().interrupt();
+        }
+        return false;
+    }
+
+    /**
+     * Return a new timing that is a multiple of the this timing
+     *
+     * @param n the multiple
+     * @return this timing times the multiple
+     */
+    public Timing2 multiple(double n)
+    {
+        return new Timing2((int)(value * n), unit);
+    }
+
+    /**
+     * Return a new timing that is a multiple of the this timing
+     *
+     * @param n the multiple
+     * @param waitingMultiple new waitingMultiple
+     * @return this timing times the multiple
+     */
+    public Timing2 multiple(double n, int waitingMultiple)
+    {
+        return new Timing2((int)(value * n), unit, waitingMultiple);
+    }
+
+    /**
+     * Return a new timing with the standard multiple for waiting on latches, etc.
+     *
+     * @return this timing multiplied
+     */
+    @SuppressWarnings("PointlessArithmeticExpression")
+    public Timing2 forWaiting()
+    {
+        return multiple(waitingMultiple);
+    }
+
+    /**
+     * Return a new timing with a multiple that ensures a ZK session timeout
+     *
+     * @return this timing multiplied
+     */
+    public Timing2 forSessionSleep()
+    {
+        return multiple(SESSION_SLEEP_MULTIPLE, 1);
+    }
+
+    /**
+     * Return a new timing with a multiple for sleeping a smaller amount of time
+     *
+     * @return this timing multiplied
+     */
+    public Timing2 forSleepingABit()
+    {
+        return multiple(.25);
+    }
+
+    /**
+     * Sleep for a small amount of time
+     *
+     * @throws InterruptedException if interrupted
+     */
+    public void sleepABit() throws InterruptedException
+    {
+        forSleepingABit().sleep();
+    }
+
+    /**
+     * Sleep for a the full amount of time
+     *
+     * @throws InterruptedException if interrupted
+     */
+    public void sleep() throws InterruptedException
+    {
+        unit.sleep(value);
+    }
+
+    /**
+     * Return the value to use for ZK session timeout
+     *
+     * @return session timeout
+     */
+    public int session()
+    {
+        return multiple(SESSION_MULTIPLE).milliseconds();
+    }
+
+    /**
+     * Return the value to use for ZK connection timeout
+     *
+     * @return connection timeout
+     */
+    public int connection()
+    {
+        return milliseconds();
+    }
+
+    private static Integer getWaitingMultiple()
+    {
+        return Integer.getInteger("timing-waiting-multiple", DEFAULT_WAITING_MULTIPLE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-test/src/compatibility/java/org/apache/curator/test/Zk35MethodInterceptor.java
----------------------------------------------------------------------
diff --git a/curator-test/src/compatibility/java/org/apache/curator/test/Zk35MethodInterceptor.java b/curator-test/src/compatibility/java/org/apache/curator/test/Zk35MethodInterceptor.java
new file mode 100644
index 0000000..290910c
--- /dev/null
+++ b/curator-test/src/compatibility/java/org/apache/curator/test/Zk35MethodInterceptor.java
@@ -0,0 +1,55 @@
+/**
+ * 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.test;
+
+import org.testng.IMethodInstance;
+import org.testng.IMethodInterceptor;
+import org.testng.ITestContext;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class Zk35MethodInterceptor implements IMethodInterceptor
+{
+    public static final String zk35Group = "zk35";
+
+    @Override
+    public List<IMethodInstance> intercept(List<IMethodInstance> methods, ITestContext context)
+    {
+        if ( !Compatibility.isZK34() )
+        {
+            return methods;
+        }
+
+        List<IMethodInstance> filteredMethods = new ArrayList<>();
+        for ( IMethodInstance method : methods )
+        {
+            if ( !isInGroup(method.getMethod().getGroups()) )
+            {
+                filteredMethods.add(method);
+            }
+        }
+        return filteredMethods;
+    }
+
+    private boolean isInGroup(String[] groups)
+    {
+        return (groups != null) && Arrays.asList(groups).contains(zk35Group);
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-test/src/main/java/org/apache/curator/test/Compatibility.java
----------------------------------------------------------------------
diff --git a/curator-test/src/main/java/org/apache/curator/test/Compatibility.java b/curator-test/src/main/java/org/apache/curator/test/Compatibility.java
new file mode 100644
index 0000000..87eb33e
--- /dev/null
+++ b/curator-test/src/main/java/org/apache/curator/test/Compatibility.java
@@ -0,0 +1,34 @@
+/**
+ * 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.test;
+
+import org.apache.zookeeper.ZooKeeper;
+
+class Compatibility
+{
+    static boolean isZK34()
+    {
+        return false;
+    }
+
+    static void injectSessionExpiration(ZooKeeper zooKeeper)
+    {
+        zooKeeper.getTestable().injectSessionExpiration();
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-test/src/main/java/org/apache/curator/test/KillServerSession.java
----------------------------------------------------------------------
diff --git a/curator-test/src/main/java/org/apache/curator/test/KillServerSession.java b/curator-test/src/main/java/org/apache/curator/test/KillServerSession.java
deleted file mode 100644
index ab5fa32..0000000
--- a/curator-test/src/main/java/org/apache/curator/test/KillServerSession.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/**
- * 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.test;
-
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-/**
- * <p>
- *     Utility to simulate a ZK session dying. See: <a href="http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A4">ZooKeeper FAQ</a>
- * </p>
- *
- * <blockquote>
- *     In the case of testing we want to cause a problem, so to explicitly expire a session an
- *     application connects to ZooKeeper, saves the session id and password, creates another
- *     ZooKeeper handle with that id and password, and then closes the new handle. Since both
- *     handles reference the same session, the close on second handle will invalidate the session
- *     causing a SESSION_EXPIRED on the first handle.
- * </blockquote>
- */
-public class KillServerSession
-{
-    /**
-     * Kill the given ZK session
-     *
-     * @param client the client to kill
-     * @param connectString server connection string
-     * @throws Exception errors
-     */
-    public static void     kill(ZooKeeper client, String connectString) throws Exception
-    {
-        kill(client, connectString, new Timing().forWaiting().milliseconds());
-    }
-
-    /**
-     * Kill the given ZK session
-     *
-     * @param client the client to kill
-     * @param connectString server connection string
-     * @param maxMs max time ms to wait for kill
-     * @throws Exception errors
-     */
-    public static void     kill(ZooKeeper client, String connectString, int maxMs) throws Exception
-    {
-        long                    startTicks = System.currentTimeMillis();
-
-        final CountDownLatch    sessionLostLatch = new CountDownLatch(1);
-        Watcher                 sessionLostWatch = new Watcher()
-        {
-            @Override
-            public void process(WatchedEvent event)
-            {
-                sessionLostLatch.countDown();
-            }
-        };
-        client.exists("/___CURATOR_KILL_SESSION___" + System.nanoTime(), sessionLostWatch);
-
-        final CountDownLatch    connectionLatch = new CountDownLatch(1);
-        Watcher                 connectionWatcher = new Watcher()
-        {
-            @Override
-            public void process(WatchedEvent event)
-            {
-                if ( event.getState() == Event.KeeperState.SyncConnected )
-                {
-                    connectionLatch.countDown();
-                }
-            }
-        };
-        ZooKeeper zk = new ZooKeeper(connectString, maxMs, connectionWatcher, client.getSessionId(), client.getSessionPasswd());
-        try
-        {
-            if ( !connectionLatch.await(maxMs, TimeUnit.MILLISECONDS) )
-            {
-                throw new Exception("KillSession could not establish duplicate session");
-            }
-            try
-            {
-                zk.close();
-            }
-            finally
-            {
-                zk = null;
-            }
-
-            while ( client.getState().isConnected() && !sessionLostLatch.await(100, TimeUnit.MILLISECONDS) )
-            {
-                long        elapsed = System.currentTimeMillis() - startTicks;
-                if ( elapsed > maxMs )
-                {
-                    throw new Exception("KillSession timed out waiting for session to expire");
-                }
-            }
-        }
-        finally
-        {
-            if ( zk != null )
-            {
-                zk.close();
-            }
-        }
-    }
-}


[12/18] curator git commit: Adding @Test to the class caused testApiPermutations to run. So, added a disabled Test annotation to it

Posted by ra...@apache.org.
Adding @Test to the class caused testApiPermutations to run. So, added a disabled Test annotation to it


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

Branch: refs/heads/CURATOR-426
Commit: aa976bbe83a8961b33f76aa5b0a7302a3b5f8f39
Parents: a21c31c
Author: randgalt <ra...@apache.org>
Authored: Thu Jul 20 17:36:10 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Thu Jul 20 17:36:10 2017 -0500

----------------------------------------------------------------------
 .../org/apache/curator/framework/imps/TestReconfiguration.java    | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/aa976bbe/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java
index acf9df3..c6ff2bb 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java
@@ -27,10 +27,10 @@ import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.api.CuratorEventType;
 import org.apache.curator.retry.ExponentialBackoffRetry;
-import org.apache.curator.test.compatibility.CuratorTestBase;
 import org.apache.curator.test.InstanceSpec;
 import org.apache.curator.test.TestingCluster;
 import org.apache.curator.test.TestingZooKeeperServer;
+import org.apache.curator.test.compatibility.CuratorTestBase;
 import org.apache.curator.test.compatibility.Timing2;
 import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
 import org.apache.curator.utils.CloseableUtils;
@@ -94,6 +94,7 @@ public class TestReconfiguration extends CuratorTestBase
     }
 
     @SuppressWarnings("ConstantConditions")
+    @Test(enabled = false)
     public void testApiPermutations() throws Exception
     {
         // not an actual test. Specifies all possible API possibilities


[18/18] curator git commit: Merge branch 'master' into CURATOR-426

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


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

Branch: refs/heads/CURATOR-426
Commit: 31d7f9a2040975dee08cb685d61cceb7015f9e32
Parents: 0906eb5 afc206c
Author: randgalt <ra...@apache.org>
Authored: Fri Jul 21 14:27:07 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Fri Jul 21 14:27:07 2017 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/curator/RetryLoop.java |   2 +-
 .../org/apache/curator/utils/Compatibility.java |  76 +++++
 .../curator/utils/InjectSessionExpiration.java  | 107 +++++++
 .../java/org/apache/curator/BasicTests.java     |   4 +-
 .../curator/TestSessionFailRetryLoop.java       |  11 +-
 .../curator/framework/CuratorFramework.java     |   7 +
 .../framework/CuratorFrameworkFactory.java      |  24 +-
 .../apache/curator/framework/SafeIsTtlMode.java |  44 +++
 .../imps/CompatibleCreateCallback.java          |  26 ++
 .../framework/imps/CreateBuilderImpl.java       | 136 ++++++---
 .../curator/framework/imps/CreateZK35.java      |  47 +++
 .../framework/imps/CuratorFrameworkImpl.java    |  24 +-
 .../imps/CuratorMultiTransactionImpl.java       |  18 +-
 .../framework/imps/WatcherRemovalManager.java   |   7 +-
 .../framework/state/ConnectionStateManager.java |   6 +-
 .../curator/framework/imps/TestCleanState.java  |   7 +
 .../framework/imps/TestCreateReturningStat.java |  13 +-
 .../imps/TestEnabledSessionExpiredState.java    |   8 +-
 .../curator/framework/imps/TestFramework.java   |   3 +-
 .../framework/imps/TestFrameworkEdges.java      |  13 +-
 .../framework/imps/TestReconfiguration.java     |  11 +-
 .../framework/imps/TestRemoveWatches.java       |   6 +-
 .../curator/framework/imps/TestTtlNodes.java    |   6 +-
 .../imps/TestWatcherRemovalManager.java         |   6 +-
 curator-recipes/pom.xml                         |  16 +
 .../framework/recipes/nodes/PersistentNode.java |   3 +-
 .../recipes/cache/TestEventOrdering.java        |   4 +-
 .../framework/recipes/cache/TestNodeCache.java  |   4 +-
 .../recipes/cache/TestPathChildrenCache.java    |   4 +-
 .../framework/recipes/cache/TestTreeCache.java  |   7 +-
 .../recipes/leader/TestLeaderLatch.java         |   3 +-
 .../recipes/leader/TestLeaderSelector.java      |   9 +-
 .../recipes/locks/TestInterProcessMutex.java    |   5 +-
 .../locks/TestInterProcessMutexBase.java        |  17 +-
 .../nodes/TestPersistentEphemeralNode.java      |  14 +-
 .../recipes/nodes/TestPersistentNode.java       |   3 +-
 .../recipes/nodes/TestPersistentTtlNode.java    |   6 +-
 .../recipes/shared/TestSharedCount.java         |   6 +-
 curator-test-zk34/README.md                     |  11 +
 curator-test-zk34/pom.xml                       | 191 ++++++++++++
 .../org/apache/curator/test/Compatibility.java  |  35 +++
 .../apache/curator/test/TestCompatibility.java  |  97 ++++++
 .../src/test/resources/log4j.properties         |  27 ++
 .../org/apache/curator/test/Compatibility.java  |  34 +++
 .../apache/curator/test/KillServerSession.java  | 122 --------
 .../org/apache/curator/test/KillSession.java    |  67 -----
 .../java/org/apache/curator/test/Timing.java    |  75 +----
 .../test/compatibility/CuratorTestBase.java     |  28 ++
 .../test/compatibility/KillSession2.java        |  40 +++
 .../curator/test/compatibility/Timing2.java     | 299 +++++++++++++++++++
 .../compatibility/Zk35MethodInterceptor.java    |  56 ++++
 .../x/async/CompletableBaseClassForTests.java   |   4 +-
 .../discovery/details/TestServiceDiscovery.java |   8 +-
 pom.xml                                         |   8 +
 src/site/resources/images/arrow.png             | Bin 0 -> 2159 bytes
 src/site/site.xml                               |   5 +-
 56 files changed, 1427 insertions(+), 393 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/31d7f9a2/curator-recipes/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/curator/blob/31d7f9a2/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/curator/blob/31d7f9a2/src/site/site.xml
----------------------------------------------------------------------


[07/18] curator git commit: need to sleep a bit between setcounts so to avoid missing events

Posted by ra...@apache.org.
need to sleep a bit between setcounts so to avoid missing events


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

Branch: refs/heads/CURATOR-426
Commit: 96340af1d1525b2c5bf50adccc499c6917ff7563
Parents: 20a3db3
Author: randgalt <ra...@apache.org>
Authored: Thu Jul 20 12:17:35 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Thu Jul 20 12:17:35 2017 -0500

----------------------------------------------------------------------
 .../curator/framework/recipes/shared/TestSharedCount.java      | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/96340af1/curator-recipes/src/test/java/org/apache/curator/framework/recipes/shared/TestSharedCount.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/shared/TestSharedCount.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/shared/TestSharedCount.java
index 6a0b7c2..822a01c 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/shared/TestSharedCount.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/shared/TestSharedCount.java
@@ -33,6 +33,7 @@ import org.apache.curator.retry.RetryNTimes;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.Timing;
+import org.apache.curator.test.compatibility.CuratorTestBase;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.WatchedEvent;
 import org.testng.Assert;
@@ -50,7 +51,7 @@ import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
-public class TestSharedCount extends BaseClassForTests
+public class TestSharedCount extends CuratorTestBase
 {
     @Test
     public void testMultiClients() throws Exception
@@ -191,8 +192,11 @@ public class TestSharedCount extends BaseClassForTests
             count.addListener(listener);
 
             Assert.assertTrue(count.trySetCount(1));
+            timing.sleepABit();
             Assert.assertTrue(count.trySetCount(2));
+            timing.sleepABit();
             Assert.assertTrue(count.trySetCount(10));
+            timing.sleepABit();
             Assert.assertEquals(count.getCount(), 10);
 
             Assert.assertTrue(new Timing().awaitLatch(setLatch));


[14/18] curator git commit: Merge branch 'master' into CURATOR-425

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


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

Branch: refs/heads/CURATOR-426
Commit: 334950f6188fe646a4588fd5aafd46e1525686d2
Parents: 1eb8aa8 95b70d2
Author: randgalt <ra...@apache.org>
Authored: Fri Jul 21 12:02:03 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Fri Jul 21 12:02:03 2017 -0500

----------------------------------------------------------------------
 curator-x-rpc/pom.xml                           |   189 -
 .../curator/x/rpc/CuratorProjectionServer.java  |   152 -
 .../AuthorizationConfiguration.java             |    45 -
 ...dedExponentialBackoffRetryConfiguration.java |    69 -
 .../x/rpc/configuration/Configuration.java      |    86 -
 .../rpc/configuration/ConfigurationBuilder.java |    87 -
 .../configuration/ConnectionConfiguration.java  |   129 -
 .../ExponentialBackoffRetryConfiguration.java   |    58 -
 .../configuration/RetryNTimesConfiguration.java |    58 -
 .../configuration/RetryPolicyConfiguration.java |    28 -
 .../curator/x/rpc/connections/Closer.java       |    24 -
 .../x/rpc/connections/ConnectionManager.java    |   141 -
 .../curator/x/rpc/connections/CuratorEntry.java |   170 -
 .../x/rpc/details/RpcBackgroundCallback.java    |    44 -
 .../curator/x/rpc/details/RpcWatcher.java       |    43 -
 .../x/rpc/idl/discovery/DiscoveryInstance.java  |    95 -
 .../idl/discovery/DiscoveryInstanceType.java    |    26 -
 .../rpc/idl/discovery/DiscoveryProjection.java  |    38 -
 .../discovery/DiscoveryProviderProjection.java  |    38 -
 .../x/rpc/idl/discovery/DiscoveryService.java   |   259 -
 .../idl/discovery/DiscoveryServiceLowLevel.java |   159 -
 .../rpc/idl/discovery/ProviderStrategyType.java |    27 -
 .../x/rpc/idl/exceptions/ExceptionType.java     |    26 -
 .../x/rpc/idl/exceptions/NodeExceptionType.java |    27 -
 .../x/rpc/idl/exceptions/RpcException.java      |    98 -
 .../idl/exceptions/ZooKeeperExceptionType.java  |    40 -
 .../idl/services/CuratorProjectionService.java  |   765 -
 .../x/rpc/idl/services/EventService.java        |    55 -
 .../curator/x/rpc/idl/structs/CreateSpec.java   |    66 -
 .../x/rpc/idl/structs/CuratorProjection.java    |    38 -
 .../curator/x/rpc/idl/structs/DeleteSpec.java   |    50 -
 .../curator/x/rpc/idl/structs/ExistsSpec.java   |    46 -
 .../x/rpc/idl/structs/GetChildrenSpec.java      |    46 -
 .../curator/x/rpc/idl/structs/GetDataSpec.java  |    50 -
 .../curator/x/rpc/idl/structs/LeaderEvent.java  |    46 -
 .../x/rpc/idl/structs/LeaderProjection.java     |    38 -
 .../curator/x/rpc/idl/structs/LeaderResult.java |    42 -
 .../x/rpc/idl/structs/LeaseProjection.java      |    38 -
 .../x/rpc/idl/structs/LockProjection.java       |    38 -
 .../x/rpc/idl/structs/NodeCacheProjection.java  |    38 -
 .../x/rpc/idl/structs/OptionalChildrenList.java |    39 -
 .../curator/x/rpc/idl/structs/OptionalData.java |    38 -
 .../rpc/idl/structs/OptionalLockProjection.java |    38 -
 .../curator/x/rpc/idl/structs/OptionalPath.java |    38 -
 .../x/rpc/idl/structs/OptionalRpcStat.java      |    38 -
 .../structs/PathChildrenCacheProjection.java    |    38 -
 .../idl/structs/PathChildrenCacheStartMode.java |    26 -
 .../PersistentEphemeralNodeProjection.java      |    38 -
 .../curator/x/rpc/idl/structs/RpcAcl.java       |    42 -
 .../curator/x/rpc/idl/structs/RpcChildData.java |    57 -
 .../x/rpc/idl/structs/RpcCreateMode.java        |    31 -
 .../x/rpc/idl/structs/RpcCuratorEvent.java      |   224 -
 .../x/rpc/idl/structs/RpcCuratorEventType.java  |    46 -
 .../curator/x/rpc/idl/structs/RpcEventType.java |    31 -
 .../apache/curator/x/rpc/idl/structs/RpcId.java |    42 -
 .../x/rpc/idl/structs/RpcKeeperState.java       |    34 -
 .../x/rpc/idl/structs/RpcParticipant.java       |    42 -
 .../idl/structs/RpcPathChildrenCacheEvent.java  |    54 -
 .../structs/RpcPathChildrenCacheEventType.java  |    33 -
 .../structs/RpcPersistentEphemeralNodeMode.java |    30 -
 .../curator/x/rpc/idl/structs/RpcStat.java      |    95 -
 .../x/rpc/idl/structs/RpcWatchedEvent.java      |    54 -
 .../curator/x/rpc/idl/structs/SetDataSpec.java  |    58 -
 .../curator/x/rpc/idl/structs/Version.java      |    38 -
 .../src/main/resources/curator/help.txt         |    72 -
 curator-x-rpc/src/main/scripts/apply-thrift.sh  |    36 -
 curator-x-rpc/src/main/scripts/generate.sh      |    56 -
 curator-x-rpc/src/main/thrift/curator.thrift    |   297 -
 .../site/confluence/configuration.confluence    |   143 -
 .../src/site/confluence/deploy.confluence       |    31 -
 .../src/site/confluence/events.confluence       |    90 -
 .../src/site/confluence/index.confluence        |    49 -
 .../src/site/confluence/reference.confluence    |   120 -
 .../src/site/confluence/usage.confluence        |   115 -
 .../src/site/resources/images/topology.png      |   Bin 135534 -> 0 bytes
 curator-x-rpc/src/site/site.xml                 |    33 -
 .../java/org/apache/curator/generated/Acl.java  |   491 -
 .../org/apache/curator/generated/ChildData.java |   604 -
 .../apache/curator/generated/CreateMode.java    |    51 -
 .../apache/curator/generated/CreateSpec.java    |  1001 -
 .../apache/curator/generated/CuratorEvent.java  |  1636 --
 .../curator/generated/CuratorEventType.java     |    99 -
 .../curator/generated/CuratorException.java     |   736 -
 .../curator/generated/CuratorProjection.java    |   388 -
 .../curator/generated/CuratorService.java       | 24123 -----------------
 .../apache/curator/generated/DeleteSpec.java    |   691 -
 .../curator/generated/DiscoveryInstance.java    |  1201 -
 .../generated/DiscoveryInstanceType.java        |    48 -
 .../curator/generated/DiscoveryProjection.java  |   388 -
 .../generated/DiscoveryProviderProjection.java  |   388 -
 .../curator/generated/DiscoveryService.java     |  7236 -----
 .../generated/DiscoveryServiceLowLevel.java     |  6966 -----
 .../apache/curator/generated/EventService.java  |  1082 -
 .../org/apache/curator/generated/EventType.java |    54 -
 .../apache/curator/generated/ExceptionType.java |    48 -
 .../apache/curator/generated/ExistsSpec.java    |   586 -
 .../curator/generated/GetChildrenSpec.java      |   586 -
 .../apache/curator/generated/GetDataSpec.java   |   680 -
 .../java/org/apache/curator/generated/Id.java   |   488 -
 .../apache/curator/generated/KeeperState.java   |    63 -
 .../apache/curator/generated/LeaderEvent.java   |   586 -
 .../curator/generated/LeaderProjection.java     |   388 -
 .../apache/curator/generated/LeaderResult.java  |   491 -
 .../curator/generated/LeaseProjection.java      |   388 -
 .../curator/generated/LockProjection.java       |   388 -
 .../curator/generated/NodeCacheProjection.java  |   388 -
 .../curator/generated/NodeExceptionType.java    |    51 -
 .../curator/generated/OptionalChildrenList.java |   437 -
 .../apache/curator/generated/OptionalData.java  |   399 -
 .../generated/OptionalLockProjection.java       |   393 -
 .../apache/curator/generated/OptionalPath.java  |   388 -
 .../apache/curator/generated/OptionalStat.java  |   393 -
 .../apache/curator/generated/Participant.java   |   486 -
 .../generated/PathChildrenCacheEvent.java       |   609 -
 .../generated/PathChildrenCacheEventType.java   |    60 -
 .../generated/PathChildrenCacheProjection.java  |   388 -
 .../generated/PathChildrenCacheStartMode.java   |    48 -
 .../generated/PersistentEphemeralNodeMode.java  |    51 -
 .../PersistentEphemeralNodeProjection.java      |   388 -
 .../curator/generated/ProviderStrategyType.java |    51 -
 .../apache/curator/generated/SetDataSpec.java   |   896 -
 .../java/org/apache/curator/generated/Stat.java |  1326 -
 .../org/apache/curator/generated/Version.java   |   386 -
 .../apache/curator/generated/WatchedEvent.java  |   620 -
 .../generated/ZooKeeperExceptionType.java       |    90 -
 .../java/org/apache/curator/x/rpc/RpcTests.java |   294 -
 .../org/apache/curator/x/rpc/TestClient.java    |   168 -
 .../org/apache/curator/x/rpc/TestServer.java    |    34 -
 .../resources/configuration/connections.json    |    27 -
 .../test/resources/configuration/logging.json   |    19 -
 .../test/resources/configuration/simple.json    |     7 -
 .../src/test/resources/configuration/test.json  |    14 -
 pom.xml                                         |     7 -
 src/site/confluence/index.confluence            |    23 +-
 src/site/site.xml                               |     7 +-
 135 files changed, 17 insertions(+), 66064 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/334950f6/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index b816d36,3aceb5f..a88d370
--- a/pom.xml
+++ b/pom.xml
@@@ -296,9 -296,7 +296,8 @@@
          <module>curator-examples</module>
          <module>curator-x-discovery</module>
          <module>curator-x-discovery-server</module>
-         <module>curator-x-rpc</module>
          <module>curator-x-async</module>
 +        <module>curator-test-zk34</module>
      </modules>
  
      <dependencyManagement>


[09/18] curator git commit: read me

Posted by ra...@apache.org.
read me


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

Branch: refs/heads/CURATOR-426
Commit: d1a650715ab955758369ff2db123bd97577cf826
Parents: 6b6f13f
Author: randgalt <ra...@apache.org>
Authored: Thu Jul 20 12:35:49 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Thu Jul 20 12:35:49 2017 -0500

----------------------------------------------------------------------
 curator-test-zk34/README.md | 11 +++++++++++
 1 file changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/d1a65071/curator-test-zk34/README.md
----------------------------------------------------------------------
diff --git a/curator-test-zk34/README.md b/curator-test-zk34/README.md
new file mode 100644
index 0000000..3615c0b
--- /dev/null
+++ b/curator-test-zk34/README.md
@@ -0,0 +1,11 @@
+# ZooKeeper 3.4 Compatibility
+
+Apache Curator 4.0 adds best-effort backward compatibility for ZooKeeper 3.4 ensembles and the ZooKeeper 3.4 library/JAR. This module, `curator-test-zk34`, exists to run the standard Curator tests using ZooKeeper 3.4. It does this via Maven. In the curator-test-zk34 pom.xml:
+
+- The Curator modules framework and recipes libraries are included - both main and test JARs - but the ZooKeeper dependency is excluded (otherwise ZooKeeper 3.5.x would be brought in)
+- The curator-test module is included but as version 2.12.0 which brings in ZooKeeper 3.4.8
+- The maven-surefire-plugin is configured to run the framework and recipes tests
+- The current version of the curator-test module includes new methods that didn't exist in version 2.12.0 in `Timing.java` and `KillSession.java`. Therefore, these classes are now soft-deprecated, reverted to their original implementations and there are new classes with the new methods: `Timing2.java` and `KillSession2.java`
+- A new test base class `CuratorTestBase` is started. Over time more common stuff should go in here but, for now, this defines a TestNG listener, Zk35MethodInterceptor, that allows for tests that are ZooKeeper 3.5 only to be marked by `@Test(groups = Zk35MethodInterceptor.zk35Group)`. These tests will not be run during the 3.4 compatibility check.
+- curator-test-zk34 needs some of the new classes from curator-test. Rather than have copies of the classes the maven-resources-plugin is used to copy from curator-test to the generated sources dir of curator-test-zk34. All classes in `curator-test/src/main/java/org/apache/curator/test/compatibility` are copied.
+


[10/18] curator git commit: More tests, refined tests and doc

Posted by ra...@apache.org.
More tests, refined tests and doc


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

Branch: refs/heads/CURATOR-426
Commit: 89182ed11a8f6ff4fa5ec9cc955ff68241076f5d
Parents: d1a6507
Author: randgalt <ra...@apache.org>
Authored: Thu Jul 20 13:40:30 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Thu Jul 20 13:40:30 2017 -0500

----------------------------------------------------------------------
 .../framework/imps/CreateBuilderImpl.java       |   2 +
 .../framework/imps/CuratorFrameworkImpl.java    |   3 +
 .../framework/imps/WatcherRemovalManager.java   |   5 +
 .../java/org/apache/curator/test/DummyTest.java |  30 ------
 .../apache/curator/test/TestCompatibility.java  |  97 +++++++++++++++++++
 src/site/confluence/compatibility.confluence    |  53 ++++++++--
 src/site/confluence/index.confluence            |  22 ++---
 src/site/resources/images/arrow.png             | Bin 0 -> 2159 bytes
 src/site/site.xml                               |   7 +-
 9 files changed, 164 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/89182ed1/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 1487d6b..11050f3 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
@@ -20,6 +20,7 @@
 package org.apache.curator.framework.imps;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 import org.apache.curator.RetryLoop;
@@ -119,6 +120,7 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
     @Override
     public CreateBuilderMain withTtl(long ttl)
     {
+        Preconditions.checkState(!client.isZk34CompatibilityMode(), "TTLs are not support when running in ZooKeeper 3.4 compatibility mode");
         this.ttl = ttl;
         return this;
     }

http://git-wip-us.apache.org/repos/asf/curator/blob/89182ed1/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java
index 00adb3e..d58c56b 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java
@@ -472,12 +472,14 @@ public class CuratorFrameworkImpl implements CuratorFramework
     @Override
     public ReconfigBuilder reconfig()
     {
+        Preconditions.checkState(!isZk34CompatibilityMode(), "reconfig/config APIs are not support when running in ZooKeeper 3.4 compatibility mode");
         return new ReconfigBuilderImpl(this);
     }
 
     @Override
     public GetConfigBuilder getConfig()
     {
+        Preconditions.checkState(!isZk34CompatibilityMode(), "reconfig/config APIs are not support when running in ZooKeeper 3.4 compatibility mode");
         return new GetConfigBuilderImpl(this);
     }
 
@@ -542,6 +544,7 @@ public class CuratorFrameworkImpl implements CuratorFramework
     @Override
     public RemoveWatchesBuilder watches()
     {
+        Preconditions.checkState(!isZk34CompatibilityMode(), "Remove watches APIs are not support when running in ZooKeeper 3.4 compatibility mode");
         return new RemoveWatchesBuilderImpl(this);
     }
 

http://git-wip-us.apache.org/repos/asf/curator/blob/89182ed1/curator-framework/src/main/java/org/apache/curator/framework/imps/WatcherRemovalManager.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/WatcherRemovalManager.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/WatcherRemovalManager.java
index b85675b..bdb5428 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/WatcherRemovalManager.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/WatcherRemovalManager.java
@@ -53,6 +53,11 @@ public class WatcherRemovalManager
 
     void removeWatchers()
     {
+        if ( client.isZk34CompatibilityMode() )
+        {
+            return;
+        }
+
         List<NamespaceWatcher> localEntries = Lists.newArrayList(entries);
         while ( localEntries.size() > 0 )
         {

http://git-wip-us.apache.org/repos/asf/curator/blob/89182ed1/curator-test-zk34/src/test/java/org/apache/curator/test/DummyTest.java
----------------------------------------------------------------------
diff --git a/curator-test-zk34/src/test/java/org/apache/curator/test/DummyTest.java b/curator-test-zk34/src/test/java/org/apache/curator/test/DummyTest.java
deleted file mode 100644
index f19a85b..0000000
--- a/curator-test-zk34/src/test/java/org/apache/curator/test/DummyTest.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * 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.test;
-
-import org.testng.annotations.Test;
-
-public class DummyTest
-{
-    @Test
-    public void dummyTest()
-    {
-        // NOP
-    }
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/89182ed1/curator-test-zk34/src/test/java/org/apache/curator/test/TestCompatibility.java
----------------------------------------------------------------------
diff --git a/curator-test-zk34/src/test/java/org/apache/curator/test/TestCompatibility.java b/curator-test-zk34/src/test/java/org/apache/curator/test/TestCompatibility.java
new file mode 100644
index 0000000..2dd543c
--- /dev/null
+++ b/curator-test-zk34/src/test/java/org/apache/curator/test/TestCompatibility.java
@@ -0,0 +1,97 @@
+/**
+ * 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.test;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.compatibility.CuratorTestBase;
+import org.apache.curator.utils.CloseableUtils;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestCompatibility extends CuratorTestBase
+{
+    @Test
+    public void testAutoState()
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        Assert.assertTrue(client.isZk34CompatibilityMode());
+    }
+
+    @Test(expectedExceptions = IllegalStateException.class)
+    public void testTtl() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        client.start();
+        try
+        {
+            client.create().withTtl(100).forPath("/foo");
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test(expectedExceptions = IllegalStateException.class)
+    public void testReconfig() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        client.start();
+        try
+        {
+            client.reconfig().withNewMembers("a", "b");
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test(expectedExceptions = IllegalStateException.class)
+    public void testGetConfig() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        client.start();
+        try
+        {
+            client.getConfig().forEnsemble();
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test(expectedExceptions = IllegalStateException.class)
+    public void testRemoveWatches() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        client.start();
+        try
+        {
+            client.watches().removeAll();
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/89182ed1/src/site/confluence/compatibility.confluence
----------------------------------------------------------------------
diff --git a/src/site/confluence/compatibility.confluence b/src/site/confluence/compatibility.confluence
index ef3324f..7dd1ce5 100644
--- a/src/site/confluence/compatibility.confluence
+++ b/src/site/confluence/compatibility.confluence
@@ -1,9 +1,48 @@
-h1. API Compatibility
+h1. ZooKeeper Version Compatibility
 
-A [[Clirr|http://clirr.sourceforge.net/]] report is generated for each Curator module:
+While ZooKeeper 3.5.x is still considered "beta" by the ZooKeeper development team, the reality is that it is
+used in production by many users. However, ZooKeeper 3.4.x is also used in production. Prior to Apache Curator
+4.0, both versions of ZooKeeper were supported via two versions of Apache Curator. Starting with Curator 4.0
+both versions of ZooKeeper are supported via the same Curator libraries.
+
+h2. ZooKeeper 3.5.x
+
+* Curator 4.0 has a hard dependency on ZooKeeper 3.5.x
+* If you are using ZooKeeper 3.5.x there's nothing additional to do \- just use Curator 4.0
+
+h2. ZooKeeper 3.4.x
+
+Curator 4.0 supports ZooKeeper 3.4.x ensembles in a soft\-compatibility mode. To use this mode
+you must exclude ZooKeeper when adding Curator to your dependency management tool.
+
+_Maven_
+
+{code}
+<dependency>
+    <groupId>org.apache.curator</groupId>
+    <artifactId>curator-recipes</artifactId>
+    <version>${curator-version}</version>
+    <exclusions>
+        <exclusion>
+            <groupId>org.apache.zookeeper</groupId>
+            <artifactId>zookeeper</artifactId>
+        </exclusion>
+    </exclusions>
+</dependency>
+{code}
+
+_Gradle_
+
+{code}
+compile('org.apache.curator:curator-recipes:$curatorVersion') {
+  exclude('org.apache.zookeeper:zookeeper')
+}
+{code}
+
+You must add a dependency on ZooKeeper 3.4.x also.
+
+Curator will detect which ZooKeeper library is in use and automatically set ZooKeeper 3.4 compatibility
+mode as needed. In this mode, all features not supported by 3.4 are disabled. It is up to your
+application code to "do the right thing" and not use these features. Use the `isZk34CompatibilityMode()`
+method to determine which mode Curator is using at runtime.
 
-* [[Curator Client Report|curator-client/clirr-report.html]]
-* [[Curator Framework Report|curator-framework/clirr-report.html]]
-* [[Curator Recipes Report|curator-recipes/clirr-report.html]]
-* [[Curator Discovery Report|curator-x-discovery/clirr-report.html]]
-* [[Curator Discovery Server Report|curator-x-discovery-server/clirr-report.html]]

http://git-wip-us.apache.org/repos/asf/curator/blob/89182ed1/src/site/confluence/index.confluence
----------------------------------------------------------------------
diff --git a/src/site/confluence/index.confluence b/src/site/confluence/index.confluence
index f9490b4..cdd9902 100644
--- a/src/site/confluence/index.confluence
+++ b/src/site/confluence/index.confluence
@@ -12,14 +12,9 @@ h2. Getting Started
 
 See the page for quick start: [[Getting Started|getting-started.html]].
 
-h2. Components
+!images/arrow.png!
 
-|[[Recipes|curator-recipes/index.html]]|Implementations of some of the common ZooKeeper "recipes". The implementations are built on top of the Curator Framework.|
-|[[Framework|curator-framework/index.html]]|The Curator Framework is a high\-level API that greatly simplifies using ZooKeeper. It adds many features that build on ZooKeeper and handles the complexity of managing connections to the ZooKeeper cluster and retrying operations.|
-|[[Utilities|utilities.html]]|Various utilities that are useful when using ZooKeeper.|
-|[[Client|curator-client/index.html]]|A replacement for the bundled {{ZooKeeper}} class that takes care of some low\-level housekeeping and provides some useful utilities.|
-|[[Errors|errors.html]]|How Curator deals with errors, connection issues, recoverable exceptions, etc.|
-|Extensions|The curator\-recipes package implements the common recipes that are described in the ZooKeeper documentation. To avoid bloating that package, recipes/applications that have a vertical appeal will be put in separate "extension" packages using the naming convention curator\-x\-name.|
+Use the navigation links in the left\-nav menu for detailed information.
 
 h2. Maven / Artifacts
 
@@ -28,20 +23,17 @@ most users, the only artifact you need is curator\-recipes.
 
 ||GroupID/Org||ArtifactID/Name||Description||
 |org.apache.curator|curator\-recipes|All of the recipes. Note: this artifact has dependencies on client and framework and, so, Maven (or whatever tool you're using) should pull those in automatically.|
+|org.apache.curator|curator\-async|Asynchronous DSL with O/R modeling, migrations and many other features.|
 |org.apache.curator|curator\-framework|The Curator Framework high level API. This is built on top of the client and should pull it in automatically.|
 |org.apache.curator|curator\-client|The Curator Client \- replacement for the ZooKeeper class in the ZK distribution.|
 |org.apache.curator|curator\-test|Contains the TestingServer, the TestingCluster and a few other tools useful for testing.|
 |org.apache.curator|curator\-examples|Example usages of various Curator features.|
 |org.apache.curator|curator\-x\-discovery|A Service Discovery implementation built on the Curator Framework.|
-|org.apache.curator|curator\-x\-discovery-server|A RESTful server that can be used with Curator Discovery.|
+|org.apache.curator|curator\-x\-discovery\-server|A RESTful server that can be used with Curator Discovery.|
 |org.apache.curator|curator\-x\-rpc|A proxy that bridges non\-java environments with the Curator framework and recipes.|
 
-h2. Versions
+h2. ZooKeeper Compatibility
 
-The are currently two released versions of Curator, 2.x.x and 3.x.x:
+Apache Curator is meant to be used with ZooKeeper 3.5\+. However, it is also compatible with ZooKeeper 3.4.x.
+See [[Compatibility|compatibility.html]] for details.
 
-* Curator 2.x.x \- compatible with both ZooKeeper 3.4.x and ZooKeeper 3.5.x
-* Curator 3.x.x \- compatible only with ZooKeeper 3.5.x and includes support for new features such as dynamic reconfiguration, etc.
-
-ZooKeeper 3.5.x adds a number of new features including dynamic reconfiguration, watcher removal and others. The Curator 3.x.x Jira
-issues tracking these features are here: [[https://issues.apache.org/jira/browse/CURATOR-159]].

http://git-wip-us.apache.org/repos/asf/curator/blob/89182ed1/src/site/resources/images/arrow.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/arrow.png b/src/site/resources/images/arrow.png
new file mode 100644
index 0000000..5de46be
Binary files /dev/null and b/src/site/resources/images/arrow.png differ

http://git-wip-us.apache.org/repos/asf/curator/blob/89182ed1/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
index 8136c9a..e5ae374 100644
--- a/src/site/site.xml
+++ b/src/site/site.xml
@@ -76,6 +76,10 @@
             <item name="Schema Support" href="curator-framework/schema.html"/>
         </menu>
 
+        <menu name="Compatibility" inherit="top">
+            <item name="ZooKeeper Versions" href="compatibility.html"/>
+        </menu>
+
         <menu name="Low Level" inherit="top">
             <item name="Framework" href="curator-framework/index.html"/>
             <item name="Utilities" href="utilities.html"/>
@@ -90,7 +94,6 @@
             <item name="Source Code" href="source-repository.html"/>
             <item name="Project Team" href="team-list.html"/>
             <item name="Project Information" href="project-info.html"/>
-            <item name="API Compatibility" href="compatibility.html"/>
             <item name="Javadoc" href="apidocs/index.html"/>
             <item name="Wiki" href="https://cwiki.apache.org/confluence/display/CURATOR"/>
             <item name="Releases" href="https://cwiki.apache.org/confluence/display/CURATOR/Releases"/>
@@ -99,8 +102,6 @@
         <menu name="Extensions" inherit="top">
             <item name="Service Discovery" href="curator-x-discovery/index.html"/>
             <item name="Service Discovery Server" href="curator-x-discovery-server/index.html"/>
-            <item name="Curator RPC Proxy" href="curator-x-rpc/index.html"/>
-            <item name="Curator Java 8/Async" href="curator-x-async/index.html"/>
         </menu>
 
         <menu name="Community" inherit="top">


[11/18] curator git commit: Add Curator RPC Proxy back - I'll remove it in the Issue/PR for that purpose

Posted by ra...@apache.org.
Add Curator RPC Proxy back - I'll remove it in the Issue/PR for that purpose


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

Branch: refs/heads/CURATOR-426
Commit: a21c31c59c5209bda75ad853bcf657c85753e99d
Parents: 89182ed
Author: randgalt <ra...@apache.org>
Authored: Thu Jul 20 13:47:08 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Thu Jul 20 13:47:08 2017 -0500

----------------------------------------------------------------------
 src/site/site.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/a21c31c5/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
index e5ae374..469e76a 100644
--- a/src/site/site.xml
+++ b/src/site/site.xml
@@ -102,6 +102,7 @@
         <menu name="Extensions" inherit="top">
             <item name="Service Discovery" href="curator-x-discovery/index.html"/>
             <item name="Service Discovery Server" href="curator-x-discovery-server/index.html"/>
+            <item name="Curator RPC Proxy" href="curator-x-rpc/index.html"/>
         </menu>
 
         <menu name="Community" inherit="top">


[02/18] curator git commit: Basic concept of zk 3.4.x compatibility proven. The Compatibility class checks for a well-known 3.5 class and sets a static that advertises whether the ZK lib is 3.4.x or 3.5.x. Then, the code "ifs" using this static. The majo

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-test/src/main/java/org/apache/curator/test/KillSession.java
----------------------------------------------------------------------
diff --git a/curator-test/src/main/java/org/apache/curator/test/KillSession.java b/curator-test/src/main/java/org/apache/curator/test/KillSession.java
deleted file mode 100644
index ce2b7e6..0000000
--- a/curator-test/src/main/java/org/apache/curator/test/KillSession.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * 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.test;
-
-import org.apache.zookeeper.ZooKeeper;
-
-/**
- * <p>
- *     Utility to simulate a ZK session dying.
- * </p>
- */
-public class KillSession
-{
-    /**
-     * Kill the given ZK session
-     *
-     * @param client the client to kill
-     * @since 3.0.0
-     */
-    public static void     kill(ZooKeeper client)
-    {
-        client.getTestable().injectSessionExpiration();
-    }
-
-    /**
-     * Kill the given ZK session
-     *
-     * @param client the client to kill
-     * @param connectString server connection string
-     * @throws Exception errors
-     * @deprecated use {@link #kill(ZooKeeper)} instead
-     */
-    public static void     kill(ZooKeeper client, String connectString) throws Exception
-    {
-        kill(client);
-    }
-
-    /**
-     * Kill the given ZK session
-     *
-     * @param client the client to kill
-     * @param connectString server connection string
-     * @param maxMs max time ms to wait for kill
-     * @throws Exception errors
-     * @deprecated use {@link #kill(ZooKeeper)} instead
-     */
-    public static void     kill(ZooKeeper client, String connectString, int maxMs) throws Exception
-    {
-        kill(client);
-    }
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-test/src/main/java/org/apache/curator/test/Timing.java
----------------------------------------------------------------------
diff --git a/curator-test/src/main/java/org/apache/curator/test/Timing.java b/curator-test/src/main/java/org/apache/curator/test/Timing.java
index 242aa50..f29b1c5 100644
--- a/curator-test/src/main/java/org/apache/curator/test/Timing.java
+++ b/curator-test/src/main/java/org/apache/curator/test/Timing.java
@@ -19,11 +19,9 @@
 
 package org.apache.curator.test;
 
-import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 
 /**
  * Utility to get various testing times
@@ -36,8 +34,7 @@ public class Timing
 
     private static final int DEFAULT_SECONDS = 10;
     private static final int DEFAULT_WAITING_MULTIPLE = 5;
-    private static final double SESSION_MULTIPLE = 1.5;
-    private static final double SESSION_SLEEP_MULTIPLE = SESSION_MULTIPLE * 1.75;  // has to be at least session + 2/3 of a session to account for missed heartbeat then session expiration
+    private static final double SESSION_MULTIPLE = .25;
 
     /**
      * Use the default base time
@@ -130,32 +127,6 @@ public class Timing
     }
 
     /**
-     * Try to take an item from the given queue
-     *
-     * @param queue queue
-     * @return item
-     * @throws Exception interrupted or timed out
-     */
-    public <T> T takeFromQueue(BlockingQueue<T> queue) throws Exception
-    {
-        Timing m = forWaiting();
-        try
-        {
-            T value = queue.poll(m.value, m.unit);
-            if ( value == null )
-            {
-                throw new TimeoutException("Timed out trying to take from queue");
-            }
-            return value;
-        }
-        catch ( InterruptedException e )
-        {
-            Thread.currentThread().interrupt();
-            throw e;
-        }
-    }
-
-    /**
      * Wait on the given semaphore
      *
      * @param semaphore the semaphore
@@ -208,18 +179,6 @@ public class Timing
     }
 
     /**
-     * Return a new timing that is a multiple of the this timing
-     *
-     * @param n the multiple
-     * @param waitingMultiple new waitingMultiple
-     * @return this timing times the multiple
-     */
-    public Timing multiple(double n, int waitingMultiple)
-    {
-        return new Timing((int)(value * n), unit, waitingMultiple);
-    }
-
-    /**
      * Return a new timing with the standard multiple for waiting on latches, etc.
      *
      * @return this timing multiplied
@@ -231,43 +190,13 @@ public class Timing
     }
 
     /**
-     * Return a new timing with a multiple that ensures a ZK session timeout
-     *
-     * @return this timing multiplied
-     */
-    public Timing forSessionSleep()
-    {
-        return multiple(SESSION_SLEEP_MULTIPLE, 1);
-    }
-
-    /**
-     * Return a new timing with a multiple for sleeping a smaller amount of time
-     *
-     * @return this timing multiplied
-     */
-    public Timing forSleepingABit()
-    {
-        return multiple(.25);
-    }
-
-    /**
      * Sleep for a small amount of time
      *
      * @throws InterruptedException if interrupted
      */
     public void sleepABit() throws InterruptedException
     {
-        forSleepingABit().sleep();
-    }
-
-    /**
-     * Sleep for a the full amount of time
-     *
-     * @throws InterruptedException if interrupted
-     */
-    public void sleep() throws InterruptedException
-    {
-        unit.sleep(value);
+        unit.sleep(value / 4);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-x-async/src/test/java/org/apache/curator/x/async/CompletableBaseClassForTests.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/test/java/org/apache/curator/x/async/CompletableBaseClassForTests.java b/curator-x-async/src/test/java/org/apache/curator/x/async/CompletableBaseClassForTests.java
index 4a964b1..28c9f11 100644
--- a/curator-x-async/src/test/java/org/apache/curator/x/async/CompletableBaseClassForTests.java
+++ b/curator-x-async/src/test/java/org/apache/curator/x/async/CompletableBaseClassForTests.java
@@ -21,6 +21,7 @@ package org.apache.curator.x.async;
 import com.google.common.base.Throwables;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.Timing;
+import org.apache.curator.test.Timing2;
 import org.testng.Assert;
 import java.util.concurrent.CompletionStage;
 import java.util.concurrent.ExecutionException;
@@ -30,7 +31,7 @@ import java.util.function.BiConsumer;
 
 public abstract class CompletableBaseClassForTests extends BaseClassForTests
 {
-    protected static final Timing timing = new Timing();
+    protected static final Timing2 timing = new Timing2();
 
     protected <T, U> void complete(CompletionStage<T> stage)
     {

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java
----------------------------------------------------------------------
diff --git a/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java b/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java
index 47c74d5..a2cf157 100644
--- a/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java
+++ b/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java
@@ -25,7 +25,7 @@ import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.KillSession;
+import org.apache.curator.test.KillSession2;
 import org.apache.curator.test.Timing;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.x.discovery.ServiceDiscovery;
@@ -79,7 +79,7 @@ public class TestServiceDiscovery extends BaseClassForTests
             timing.acquireSemaphore(semaphore, 2);
             Assert.assertEquals(discovery.queryForInstances("test").size(), 2);
 
-            KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString());
+            KillSession2.kill(client.getZookeeperClient().getZooKeeper());
             server.stop();
 
             server.restart();
@@ -121,7 +121,7 @@ public class TestServiceDiscovery extends BaseClassForTests
             timing.acquireSemaphore(semaphore);
             Assert.assertEquals(discovery.queryForInstances("test").size(), 1);
 
-            KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString());
+            KillSession2.kill(client.getZookeeperClient().getZooKeeper());
             server.stop();
 
             server.restart();
@@ -154,7 +154,7 @@ public class TestServiceDiscovery extends BaseClassForTests
 
             Assert.assertEquals(discovery.queryForInstances("test").size(), 1);
 
-            KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString());
+            KillSession2.kill(client.getZookeeperClient().getZooKeeper());
             Thread.sleep(timing.multiple(1.5).session());
 
             Assert.assertEquals(discovery.queryForInstances("test").size(), 1);

http://git-wip-us.apache.org/repos/asf/curator/blob/58bc969f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b816d36..5a8045e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -80,6 +80,7 @@
         <maven-shade-plugin-version>2.4.3</maven-shade-plugin-version>
         <slf4j-version>1.7.6</slf4j-version>
         <clirr-maven-plugin-version>2.8</clirr-maven-plugin-version>
+        <build-helper-maven-plugin-version>3.0.0</build-helper-maven-plugin-version>
 
         <!-- OSGi Properties -->
         <osgi.export.package />
@@ -867,6 +868,12 @@
                     </execution>
                 </executions>
             </plugin>
+
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>${build-helper-maven-plugin-version}</version>
+            </plugin>
         </plugins>
     </build>
 </project>


[05/18] curator git commit: TestTreeCache now uses the improved KillSession2 but that results in different event ordering

Posted by ra...@apache.org.
TestTreeCache now uses the improved KillSession2 but that results in different event ordering


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

Branch: refs/heads/CURATOR-426
Commit: 097083c4a2cd2e252c6f3b57e3e837e4a9ecabc9
Parents: 242b701
Author: randgalt <ra...@apache.org>
Authored: Thu Jul 20 10:53:52 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Thu Jul 20 10:53:52 2017 -0500

----------------------------------------------------------------------
 .../org/apache/curator/framework/recipes/cache/TestTreeCache.java | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/097083c4/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 409cc48..a53627a 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
@@ -424,11 +424,10 @@ public class TestTreeCache extends BaseTestTreeCache
         assertEvent(TreeCacheEvent.Type.NODE_ADDED, "/test/me");
 
         KillSession2.kill(client.getZookeeperClient().getZooKeeper());
-        assertEvent(TreeCacheEvent.Type.CONNECTION_SUSPENDED);
         assertEvent(TreeCacheEvent.Type.CONNECTION_LOST);
         assertEvent(TreeCacheEvent.Type.CONNECTION_RECONNECTED);
-        assertEvent(TreeCacheEvent.Type.NODE_REMOVED, "/test/me", "data".getBytes());
         assertEvent(TreeCacheEvent.Type.INITIALIZED);
+        assertEvent(TreeCacheEvent.Type.NODE_REMOVED, "/test/me", "data".getBytes());
 
         assertNoMoreEvents();
     }


[15/18] curator git commit: Merge branch 'master' into CURATOR-425

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


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

Branch: refs/heads/CURATOR-426
Commit: 63f83104d8b7fc086ad92ba166ce58e1897646be
Parents: 334950f 840b434
Author: randgalt <ra...@apache.org>
Authored: Fri Jul 21 12:49:39 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Fri Jul 21 12:49:39 2017 -0500

----------------------------------------------------------------------
 .../server/jetty_resteasy/TestStringsWithRestEasy.java |  6 +++---
 pom.xml                                                | 13 +++++++------
 2 files changed, 10 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/63f83104/pom.xml
----------------------------------------------------------------------