You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by js...@apache.org on 2017/08/23 19:40:18 UTC

nifi git commit: NIFI-4255 - Added flag to allow migration of existing (source) acls to destination, update to documentation Added build-helper-maven-plugin to nifi-toolkit-zookeeper-migrator pom to build groovy test code Moved spock spec from src/test/j

Repository: nifi
Updated Branches:
  refs/heads/master cf5763939 -> acee2627c


NIFI-4255 - Added flag to allow migration of existing (source) acls to destination, update to documentation
Added build-helper-maven-plugin to nifi-toolkit-zookeeper-migrator pom to build groovy test code
Moved spock spec from src/test/java to src/test/groovy
Minor code formatting updates to ZooKeeperMigrator.java

This closes #2065


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

Branch: refs/heads/master
Commit: acee2627ca562a4dc94adfe9836ecd32a4666902
Parents: cf57639
Author: Yolanda M. Davis <yo...@gmail.com>
Authored: Tue Aug 8 00:03:42 2017 -0400
Committer: Jeff Storck <jt...@gmail.com>
Committed: Wed Aug 23 15:38:44 2017 -0400

----------------------------------------------------------------------
 .../src/main/asciidoc/administration-guide.adoc |   1 +
 .../nifi-toolkit-zookeeper-migrator/pom.xml     |  31 ++
 .../toolkit/zkmigrator/ZooKeeperMigrator.java   |  40 ++-
 .../zkmigrator/ZooKeeperMigratorMain.java       |   8 +-
 .../zkmigrator/ZooKeeperMigratorTest.groovy     | 290 +++++++++++++++++++
 .../zkmigrator/ZooKeeperMigratorTest.groovy     | 265 -----------------
 6 files changed, 354 insertions(+), 281 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/acee2627/nifi-docs/src/main/asciidoc/administration-guide.adoc
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index 9ccde29..4ad79ee 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -2272,6 +2272,7 @@ You can use the following command line options with the ZooKeeper Migrator:
 * `-k`,`--krb-conf <jaas-filename>`               Allows the specification of a JAAS configuration file to allow authentication with a ZooKeeper configured to use Kerberos.  This option is mutually exclusive with the `-a`,`--auth` option.
 * `-r`,`--receive`                                Receives data from ZooKeeper and writes to the given filename (if the `-f`,`--file` option is provided) or standard output. The data received will contain the full path to each node read from ZooKeeper. This option is mutually exclusive with the `-s`,`--send` option.
 * `-s`,`--send`                                   Sends data to ZooKeeper that is read from the given filename (if the `-f`,`--file` option is provided) or standard input. The paths for each node in the data being sent to ZooKeeper are absolute paths, and will be stored in ZooKeeper under the *path* portion of the `-z`,`--zookeeper` argument.  Typically, the *path* portion of the argument can be omitted, which will store the nodes at their absolute paths. This option is mutually exclusive with the `-r`,`--receive` option.
+* `--use-existing-acl`                            Allows the Zookeeper Migrator to write ACL values retrieved from the source Zookeeper server to destination server. Default action will apply Open rights for unsecured destinations or Creator Only rights for secured destinations.
 * `-z`,`--zookeeper <zookeeper-endpoint>`         The ZooKeeper server(s) to use, specified by a connect string, comprised of one or more comma-separated host:port pairs followed by a path, in the format of _host:port[,host2:port...,hostn:port]/znode/path_.
 
 [[migrating_between_source_destination_zookeepers]]

http://git-wip-us.apache.org/repos/asf/nifi/blob/acee2627/nifi-toolkit/nifi-toolkit-zookeeper-migrator/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-zookeeper-migrator/pom.xml b/nifi-toolkit/nifi-toolkit-zookeeper-migrator/pom.xml
index 4c9c887..75f32bf 100644
--- a/nifi-toolkit/nifi-toolkit-zookeeper-migrator/pom.xml
+++ b/nifi-toolkit/nifi-toolkit-zookeeper-migrator/pom.xml
@@ -89,6 +89,37 @@
                     </excludes>
                 </configuration>
             </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>1.5</version>
+                <executions>
+                    <execution>
+                        <id>add-source</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/main/groovy</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>add-test-source</id>
+                        <phase>generate-test-sources</phase>
+                        <goals>
+                            <goal>add-test-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/test/groovy</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
     </build>
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/acee2627/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigrator.java
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigrator.java b/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigrator.java
index fa71ae0..8d07555 100644
--- a/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigrator.java
+++ b/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigrator.java
@@ -25,6 +25,7 @@ import com.google.gson.GsonBuilder;
 import com.google.gson.JsonParser;
 import com.google.gson.stream.JsonReader;
 import com.google.gson.stream.JsonWriter;
+
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Watcher;
@@ -116,7 +117,7 @@ class ZooKeeperMigrator {
         closeZooKeeper(zooKeeper);
     }
 
-    void writeZooKeeper(InputStream zkData, AuthMode authMode, byte[] authData, boolean ignoreSource) throws IOException, ExecutionException, InterruptedException {
+    void writeZooKeeper(InputStream zkData, AuthMode authMode, byte[] authData, boolean ignoreSource, boolean useExistingACL) throws IOException, ExecutionException, InterruptedException {
         // ensure that the chroot path exists
         ZooKeeper zooKeeperRoot = getZooKeeper(Joiner.on(',').join(zooKeeperEndpointConfig.getServers()), authMode, authData);
         ensureNodeExists(zooKeeperRoot, zooKeeperEndpointConfig.getPath(), CreateMode.PERSISTENT);
@@ -132,7 +133,7 @@ class ZooKeeperMigrator {
         final ZooKeeperEndpointConfig sourceZooKeeperEndpointConfig = gson.fromJson(jsonReader, ZooKeeperEndpointConfig.class);
         LOGGER.info("Source data was obtained from ZooKeeper: {}", sourceZooKeeperEndpointConfig);
         Preconditions.checkArgument(!Strings.isNullOrEmpty(sourceZooKeeperEndpointConfig.getConnectString()) && !Strings.isNullOrEmpty(sourceZooKeeperEndpointConfig.getPath())
-                && sourceZooKeeperEndpointConfig.getServers() != null && sourceZooKeeperEndpointConfig.getServers().size() > 0, "Source ZooKeeper %s from %s is invalid",
+                        && sourceZooKeeperEndpointConfig.getServers() != null && sourceZooKeeperEndpointConfig.getServers().size() > 0, "Source ZooKeeper %s from %s is invalid",
                 sourceZooKeeperEndpointConfig, zkData);
         Preconditions.checkArgument(Collections.disjoint(zooKeeperEndpointConfig.getServers(), sourceZooKeeperEndpointConfig.getServers())
                         || !zooKeeperEndpointConfig.getPath().equals(sourceZooKeeperEndpointConfig.getPath()) || ignoreSource,
@@ -161,16 +162,21 @@ class ZooKeeperMigrator {
 
         final List<CompletableFuture<Stat>> writeFutures = stream.parallel().map(node -> {
             /*
-             * create stage to migrate paths and ACLs based on the migration parent path plus the node path and the given AuthMode,
-             * this stage must be run first
+             * create stage to determine the acls that should be applied to the node.
+             * this stage will be used to initialize the chain
+             */
+            final CompletableFuture<List<ACL>> determineACLStage = CompletableFuture.supplyAsync(() -> determineACLs(node, authMode, useExistingACL));
+            /*
+             * create stage to apply acls to nodes and transform node to DataStatAclNode object
              */
-            final CompletableFuture<DataStatAclNode> transformNodeStage = CompletableFuture.supplyAsync(() -> transformNode(node, authMode));
+            final Function<List<ACL>, CompletableFuture<DataStatAclNode>> transformNodeStage = acls -> CompletableFuture.supplyAsync(() -> transformNode(node, acls));
             /*
              * create stage to ensure that nodes exist for the entire path of the zookeeper node, must be invoked after the transformNode stage to
              * ensure that the node will exist after path migration
              */
-            final Function<DataStatAclNode, String> ensureNodeExistsStage = dataStatAclNode ->
-                    ensureNodeExists(zooKeeper, dataStatAclNode.getPath(), dataStatAclNode.getEphemeralOwner() == 0 ? CreateMode.PERSISTENT : CreateMode.EPHEMERAL);
+            final Function<DataStatAclNode, CompletionStage<String>> ensureNodeExistsStage = dataStatAclNode ->
+                    CompletableFuture.supplyAsync(() -> ensureNodeExists(zooKeeper, dataStatAclNode.getPath(),
+                            dataStatAclNode.getEphemeralOwner() == 0 ? CreateMode.PERSISTENT : CreateMode.EPHEMERAL));
             /*
              * create stage that waits for both the transformNode and ensureNodeExists stages complete, and also provides that the given transformed node is
              * available to the next stage
@@ -180,12 +186,14 @@ class ZooKeeperMigrator {
              * create stage to transmit the node to the destination zookeeper endpoint, must be invoked after the node has been transformed and its path
              * has been created (or already exists) in the destination zookeeper
              */
-            final Function<DataStatAclNode, CompletionStage<Stat>> transmitNodeStage = dataStatNode ->
-                    CompletableFuture.supplyAsync(() -> transmitNode(zooKeeper, dataStatNode));
+            final Function<DataStatAclNode, CompletionStage<Stat>> transmitNodeStage = dataStatNode -> CompletableFuture.supplyAsync(() -> transmitNode(zooKeeper, dataStatNode));
             /*
              * submit the stages chained together in the proper order to perform the processing on the given node
              */
-            return transformNodeStage.thenApply(ensureNodeExistsStage).thenCombine(transformNodeStage, combineEnsureNodeAndTransferNodeStage).thenCompose(transmitNodeStage);
+            final CompletableFuture<DataStatAclNode> dataStatAclNodeCompletableFuture = determineACLStage.thenCompose(transformNodeStage);
+            return dataStatAclNodeCompletableFuture.thenCompose(ensureNodeExistsStage)
+                    .thenCombine(dataStatAclNodeCompletableFuture, combineEnsureNodeAndTransferNodeStage)
+                    .thenCompose(transmitNodeStage);
         }).collect(Collectors.toList());
 
         CompletableFuture<Void> allWritesFuture = CompletableFuture.allOf(writeFutures.toArray(new CompletableFuture[writeFutures.size()]));
@@ -269,11 +277,13 @@ class ZooKeeperMigrator {
         }
     }
 
-    private DataStatAclNode transformNode(DataStatAclNode node, AuthMode destinationAuthMode) {
-        // For the NiFi use case, all nodes will be migrated to CREATOR_ALL_ACL
-        final DataStatAclNode migratedNode = new DataStatAclNode(node.getPath(), node.getData(), node.getStat(),
-                destinationAuthMode.equals(AuthMode.OPEN) ? ZooDefs.Ids.OPEN_ACL_UNSAFE : ZooDefs.Ids.CREATOR_ALL_ACL,
-                node.getEphemeralOwner());
+    private List<ACL> determineACLs(DataStatAclNode node, AuthMode authMode, Boolean useExistingACL) {
+        return useExistingACL ? node.getAcls() :
+                (authMode.equals(AuthMode.OPEN) ? ZooDefs.Ids.OPEN_ACL_UNSAFE : ZooDefs.Ids.CREATOR_ALL_ACL);
+    }
+
+    private DataStatAclNode transformNode(DataStatAclNode node, List<ACL> acls) {
+        final DataStatAclNode migratedNode = new DataStatAclNode(node.getPath(), node.getData(), node.getStat(), acls, node.getEphemeralOwner());
         LOGGER.info("transformed original node {} to {}", node, migratedNode);
         return migratedNode;
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/acee2627/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorMain.java
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorMain.java b/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorMain.java
index 8d58866..dd2f54b 100644
--- a/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorMain.java
+++ b/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorMain.java
@@ -90,6 +90,10 @@ public class ZooKeeperMigratorMain {
             .longOpt("ignore-source")
             .desc("ignores the source ZooKeeper endpoint specified in the exported data")
             .build();
+    private static final Option OPTION_USE_EXISTING_ACL = Option.builder()
+            .longOpt("use-existing-acl")
+            .desc("allow write of existing acl data to destination")
+            .build();
 
     private static Options createOptions() {
         final Options options = new Options();
@@ -98,6 +102,7 @@ public class ZooKeeperMigratorMain {
         options.addOption(OPTION_ZK_AUTH_INFO);
         options.addOption(OPTION_FILE);
         options.addOption(OPTION_IGNORE_SOURCE);
+        options.addOption(OPTION_USE_EXISTING_ACL);
         final OptionGroup optionGroupAuth = new OptionGroup().addOption(OPTION_ZK_AUTH_INFO).addOption(OPTION_ZK_KRB_CONF_FILE);
         optionGroupAuth.setRequired(false);
         options.addOptionGroup(optionGroupAuth);
@@ -136,6 +141,7 @@ public class ZooKeeperMigratorMain {
                 final String auth = commandLine.getOptionValue(OPTION_ZK_AUTH_INFO.getOpt());
                 final String jaasFilename = commandLine.getOptionValue(OPTION_ZK_KRB_CONF_FILE.getOpt());
                 final boolean ignoreSource = commandLine.hasOption(OPTION_IGNORE_SOURCE.getLongOpt());
+                final boolean useExistingACL = commandLine.hasOption(OPTION_USE_EXISTING_ACL.getLongOpt());
                 final AuthMode authMode;
                 final byte[] authData;
                 if (auth != null) {
@@ -157,7 +163,7 @@ public class ZooKeeperMigratorMain {
                     }
                 } else {
                     try (InputStream zkData = filename != null ? new FileInputStream(Paths.get(filename).toFile()) : System.in) {
-                        zookeeperMigrator.writeZooKeeper(zkData, authMode, authData, ignoreSource);
+                        zookeeperMigrator.writeZooKeeper(zkData, authMode, authData, ignoreSource, useExistingACL);
                     }
                 }
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/acee2627/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/test/groovy/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/test/groovy/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorTest.groovy b/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/test/groovy/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorTest.groovy
new file mode 100644
index 0000000..2b9bc00
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/test/groovy/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorTest.groovy
@@ -0,0 +1,290 @@
+/*
+ * 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.nifi.toolkit.zkmigrator
+
+import com.google.gson.Gson
+import com.google.gson.stream.JsonReader
+import org.apache.curator.test.TestingServer
+import org.apache.curator.utils.ZKPaths
+import org.apache.zookeeper.CreateMode
+import org.apache.zookeeper.WatchedEvent
+import org.apache.zookeeper.ZKUtil
+import org.apache.zookeeper.ZooDefs
+import org.apache.zookeeper.ZooKeeper
+import org.apache.zookeeper.data.Stat
+import org.apache.zookeeper.server.auth.DigestAuthenticationProvider
+import spock.lang.Ignore
+import spock.lang.Specification
+import spock.lang.Unroll
+
+import java.nio.charset.StandardCharsets
+
+@Unroll
+class ZooKeeperMigratorTest extends Specification {
+
+    def "Test auth and jaas usage simultaneously"() {
+        when:
+        ZooKeeperMigratorMain.main(['-r', '-z', 'localhost:2181/path', '-a', 'user:pass', '-k', 'jaas.conf'] as String[])
+
+        then:
+        noExceptionThrown()
+    }
+
+    @Ignore
+    def "Test jaas conf on command line"() {
+        when:
+        ZooKeeperMigratorMain.main(['-r', '-z', 'localhost:2181/path', '-k', 'jaas.conf'] as String[])
+
+        then:
+        noExceptionThrown()
+    }
+
+    def "Receive from open ZooKeeper"() {
+        given:
+        def server = new TestingServer()
+        def client = new ZooKeeper(server.connectString, 3000, { WatchedEvent watchedEvent ->
+        })
+        def migrationPathRoot = '/nifi/components'
+        ZKPaths.mkdirs(client, migrationPathRoot)
+        client.setData(migrationPathRoot, 'some data'.bytes, 0)
+        def componentName = '1'
+        client.create("$migrationPathRoot/$componentName", 'some data'.bytes, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT)
+        componentName = '1/a'
+        client.create("$migrationPathRoot/$componentName", 'some data'.bytes, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL)
+        componentName = '2'
+        client.create("$migrationPathRoot/$componentName", 'some data'.bytes, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT)
+        componentName = '3'
+        client.create("$migrationPathRoot/$componentName", 'some data'.bytes, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT)
+        def outputFilePath = 'target/test-data.json'
+
+        when:
+        ZooKeeperMigratorMain.main(['-r', '-z', "$server.connectString$migrationPathRoot", '-f', outputFilePath] as String[])
+
+        then:
+        noExceptionThrown()
+        def persistedData = new Gson().fromJson(new JsonReader(new FileReader(outputFilePath)), List) as List
+        persistedData.size() == 6
+    }
+
+    def "Send to open ZooKeeper without ACL migration"() {
+        given:
+        def server = new TestingServer()
+        def client = new ZooKeeper(server.connectString, 3000, { WatchedEvent watchedEvent ->
+        })
+        def migrationPathRoot = '/newParent'
+
+        when:
+        ZooKeeperMigratorMain.main(['-s', '-z', "$server.connectString$migrationPathRoot", '-f', 'src/test/resources/test-data.json'] as String[])
+
+        then:
+        noExceptionThrown()
+        def nodes = ZKPaths.getSortedChildren(client, '/').collect { ZKUtil.listSubTreeBFS(client, "/$it") }.flatten()
+        nodes.size() == 6
+    }
+
+    def "Send to open ZooKeeper without ACL migration with new multi-node parent"() {
+        given:
+        def server = new TestingServer()
+        def client = new ZooKeeper(server.connectString, 3000, { WatchedEvent watchedEvent ->
+        })
+        def migrationPathRoot = '/newParent/node'
+
+        when:
+        ZooKeeperMigratorMain.main(['-s', '-z', "$server.connectString$migrationPathRoot", '-f', 'src/test/resources/test-data.json'] as String[])
+
+        then:
+        noExceptionThrown()
+        def nodes = ZKPaths.getSortedChildren(client, '/').collect { ZKUtil.listSubTreeBFS(client, "/$it") }.flatten()
+        nodes.size() == 7
+    }
+
+    def "Receive all nodes from ZooKeeper root"() {
+        given:
+        def server = new TestingServer()
+        def client = new ZooKeeper(server.connectString, 3000, { WatchedEvent watchedEvent ->
+        })
+        def migrationPathRoot = '/'
+        def addedNodePath = 'nifi'
+        client.create("$migrationPathRoot$addedNodePath", 'some data'.bytes, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT)
+        def outputFilePath = 'target/test-data-root.json'
+
+        when:
+        ZooKeeperMigratorMain.main(['-r', '-z', "$server.connectString$migrationPathRoot", '-f', outputFilePath] as String[])
+
+        then:
+        noExceptionThrown()
+        def persistedData = new Gson().fromJson(new JsonReader(new FileReader(outputFilePath)), List) as List
+        persistedData.size() == 5
+    }
+
+    def "Receive Zookeeper node created with username and password"() {
+        given:
+        def server = new TestingServer()
+        def client = new ZooKeeper(server.connectString, 3000, { WatchedEvent watchedEvent ->
+        })
+        def username = 'nifi'
+        def password = 'nifi'
+        client.addAuthInfo('digest', "$username:$password".getBytes(StandardCharsets.UTF_8))
+        def migrationPathRoot = '/nifi'
+        client.create(migrationPathRoot, 'some data'.bytes, ZooDefs.Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT)
+        def outputFilePath = 'target/test-data-user-pass.json'
+
+        when:
+        ZooKeeperMigratorMain.main(['-r', '-z', "$server.connectString$migrationPathRoot", '-f', outputFilePath, '-a', "$username:$password"] as String[])
+
+        then:
+        noExceptionThrown()
+        def persistedData = new Gson().fromJson(new JsonReader(new FileReader(outputFilePath)), List) as List
+        persistedData.size() == 2
+    }
+
+    def "Send to Zookeeper a node created with username and password"() {
+        given:
+        def server = new TestingServer()
+        def client = new ZooKeeper(server.connectString, 3000, { WatchedEvent watchedEvent ->
+        })
+        def username = 'nifi'
+        def password = 'nifi'
+        client.addAuthInfo('digest', "$username:$password".getBytes(StandardCharsets.UTF_8))
+        def migrationPathRoot = '/newParent'
+
+        when:
+        ZooKeeperMigratorMain.main(['-s', '-z', "$server.connectString$migrationPathRoot", '-f', 'src/test/resources/test-data-user-pass.json', '-a', "$username:$password"] as String[])
+
+        then:
+        noExceptionThrown()
+        def nodes = ZKPaths.getSortedChildren(client, '/').collect { ZKUtil.listSubTreeBFS(client, "/$it") }.flatten()
+        nodes.size() == 3
+    }
+
+    def "Send to open Zookeeper with ACL migration"() {
+        given:
+        def server = new TestingServer()
+        def client = new ZooKeeper(server.connectString, 3000, { WatchedEvent watchedEvent ->
+        })
+        def migrationPathRoot = '/nifi-open'
+
+        when:
+        ZooKeeperMigratorMain.main(['-s', '-z', "$server.connectString$migrationPathRoot", '-f', 'src/test/resources/test-data-user-pass.json'] as String[])
+
+        then:
+        noExceptionThrown()
+        def nodes = ZKPaths.getSortedChildren(client, '/').collect { ZKUtil.listSubTreeBFS(client, "/$it") }.flatten()
+        nodes.size() == 3
+    }
+
+    def "Send to open Zookeeper using existing ACL"() {
+        given:
+        def server = new TestingServer()
+        def securedClient = new ZooKeeper(server.connectString, 3000, { WatchedEvent watchedEvent -> })
+        def userPass = "nifi:nifi"
+        securedClient.addAuthInfo("digest",userPass.getBytes(StandardCharsets.UTF_8))
+        def digest = DigestAuthenticationProvider.generateDigest(userPass)
+        def migrationPathRoot = '/nifi'
+        def stat = new Stat()
+
+        when:
+        ZooKeeperMigratorMain.main(['-s', '-z', "$server.connectString$migrationPathRoot", '-f', 'src/test/resources/test-data-user-pass.json','--use-existing-acl'] as String[])
+
+        then:
+        noExceptionThrown()
+        def acl = securedClient.getACL("/nifi",stat)
+        acl.get(0).id.scheme == "digest"
+        acl.get(0).id.id == digest
+        def nodes = ZKPaths.getSortedChildren(securedClient, '/nifi').collect { ZKUtil.listSubTreeBFS(securedClient, "/$it") }.flatten()
+        nodes.size() == 0
+    }
+
+
+    def "Parse Zookeeper connect string and path"() {
+        when:
+        def zooKeeperMigrator = new ZooKeeperMigrator("$connectString")
+
+        then:
+        zooKeeperMigrator.zooKeeperEndpointConfig.connectString == connectString
+        zooKeeperMigrator.zooKeeperEndpointConfig.servers == servers.split(',').collect()
+        zooKeeperMigrator.zooKeeperEndpointConfig.path == path
+
+        where:
+        connectString                                       | path         | servers                                   || _
+        '127.0.0.1'                                         | '/'          | '127.0.0.1'                               || _
+        '127.0.0.1,127.0.0.2'                               | '/'          | '127.0.0.1,127.0.0.2'                     || _
+        '127.0.0.1/'                                        | '/'          | '127.0.0.1'                               || _
+        '127.0.0.1,127.0.0.2/'                              | '/'          | '127.0.0.1,127.0.0.2'                     || _
+        '127.0.0.1:2181'                                    | '/'          | '127.0.0.1:2181'                          || _
+        '127.0.0.1,127.0.0.2:2181'                          | '/'          | '127.0.0.1,127.0.0.2:2181'                || _
+        '127.0.0.1:2181/'                                   | '/'          | '127.0.0.1:2181'                          || _
+        '127.0.0.1,127.0.0.2:2181/'                         | '/'          | '127.0.0.1,127.0.0.2:2181'                || _
+        '127.0.0.1/path'                                    | '/path'      | '127.0.0.1'                               || _
+        '127.0.0.1,127.0.0.2/path'                          | '/path'      | '127.0.0.1,127.0.0.2'                     || _
+        '127.0.0.1/path/node'                               | '/path/node' | '127.0.0.1'                               || _
+        '127.0.0.1,127.0.0.2/path/node'                     | '/path/node' | '127.0.0.1,127.0.0.2'                     || _
+        '127.0.0.1:2181/'                                   | '/'          | '127.0.0.1:2181'                          || _
+        '127.0.0.1,127.0.0.2:2181/'                         | '/'          | '127.0.0.1,127.0.0.2:2181'                || _
+        '127.0.0.1:2181/path'                               | '/path'      | '127.0.0.1:2181'                          || _
+        '127.0.0.1,127.0.0.2:2181/path'                     | '/path'      | '127.0.0.1,127.0.0.2:2181'                || _
+        '127.0.0.1:2181/path/node'                          | '/path/node' | '127.0.0.1:2181'                          || _
+        '127.0.0.1,127.0.0.2:2181/path/node'                | '/path/node' | '127.0.0.1,127.0.0.2:2181'                || _
+        '127.0.0.1,127.0.0.2:2182,127.0.0.3:2183'           | '/'          | '127.0.0.1,127.0.0.2:2182,127.0.0.3:2183' || _
+        '127.0.0.1,127.0.0.2:2182,127.0.0.3:2183/'          | '/'          | '127.0.0.1,127.0.0.2:2182,127.0.0.3:2183' || _
+        '127.0.0.1,127.0.0.2:2182,127.0.0.3:2183/path'      | '/path'      | '127.0.0.1,127.0.0.2:2182,127.0.0.3:2183' || _
+        '127.0.0.1,127.0.0.2:2182,127.0.0.3:2183/path/node' | '/path/node' | '127.0.0.1,127.0.0.2:2182,127.0.0.3:2183' || _
+    }
+
+    def "Test ignore source"() {
+        given:
+        def server = new TestingServer()
+        def connectString = "$server.connectString"
+        def dataPath = 'target/test-data-ignore-source.json'
+
+        when: "data is read from the source zookeeper"
+        ZooKeeperMigratorMain.main(['-r', '-z', connectString, '-f', dataPath] as String[])
+
+        then: "verify the data has been written to the output file"
+        new File(dataPath).exists()
+
+        when: "data is sent to the same zookeeper as the the source zookeeper without ignore source"
+        ZooKeeperMigratorMain.main(['-s', '-z', connectString, '-f', dataPath] as String[])
+
+        then: "verify that an illegal argument exception is thrown"
+        thrown(IllegalArgumentException)
+
+        when: "data is sent to the same zookeeper as the source zookeeper with ignore source option is set"
+        ZooKeeperMigratorMain.main(['-s', '-z', connectString, '-f', dataPath, '--ignore-source'] as String[])
+
+        then: "no exceptions are thrown"
+        noExceptionThrown()
+    }
+
+    def "Send to same ZooKeeper with different path"() {
+        def server = new TestingServer()
+        def connectString = "$server.connectString"
+        def dataPath = 'target/test-data-different-path.json'
+
+        when: "data is read from the source zookeeper"
+        ZooKeeperMigratorMain.main(['-r', '-z', connectString, '-f', dataPath] as String[])
+
+        then: "verify the data has been written to the output file"
+        new File(dataPath).exists()
+
+        when: "data is sent to the same zookeeper as the the source zookeeper with a different path"
+        ZooKeeperMigratorMain.main(['-s', '-z', "$connectString/new-path", '-f', dataPath] as String[])
+
+        then: "no exceptions are thrown"
+        noExceptionThrown()
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/acee2627/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/test/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/test/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorTest.groovy b/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/test/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorTest.groovy
deleted file mode 100644
index 299fda5..0000000
--- a/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/test/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorTest.groovy
+++ /dev/null
@@ -1,265 +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.nifi.toolkit.zkmigrator
-
-import com.google.gson.Gson
-import com.google.gson.stream.JsonReader
-import org.apache.curator.test.TestingServer
-import org.apache.curator.utils.ZKPaths
-import org.apache.zookeeper.CreateMode
-import org.apache.zookeeper.WatchedEvent
-import org.apache.zookeeper.ZKUtil
-import org.apache.zookeeper.ZooDefs
-import org.apache.zookeeper.ZooKeeper
-import spock.lang.Ignore
-import spock.lang.Specification
-import spock.lang.Unroll
-
-import java.nio.charset.StandardCharsets
-
-@Unroll
-class ZooKeeperMigratorTest extends Specification {
-
-    def "Test auth and jaas usage simultaneously"() {
-        when:
-        ZooKeeperMigratorMain.main(['-r', '-z', 'localhost:2181/path', '-a', 'user:pass', '-k', 'jaas.conf'] as String[])
-
-        then:
-        noExceptionThrown()
-    }
-
-    @Ignore
-    def "Test jaas conf on command line"() {
-        when:
-        ZooKeeperMigratorMain.main(['-r', '-z', 'localhost:2181/path', '-k', 'jaas.conf'] as String[])
-
-        then:
-        noExceptionThrown()
-    }
-
-    def "Receive from open ZooKeeper"() {
-        given:
-        def server = new TestingServer()
-        def client = new ZooKeeper(server.connectString, 3000, { WatchedEvent watchedEvent ->
-        })
-        def migrationPathRoot = '/nifi/components'
-        ZKPaths.mkdirs(client, migrationPathRoot)
-        client.setData(migrationPathRoot, 'some data'.bytes, 0)
-        def componentName = '1'
-        client.create("$migrationPathRoot/$componentName", 'some data'.bytes, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT)
-        componentName = '1/a'
-        client.create("$migrationPathRoot/$componentName", 'some data'.bytes, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL)
-        componentName = '2'
-        client.create("$migrationPathRoot/$componentName", 'some data'.bytes, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT)
-        componentName = '3'
-        client.create("$migrationPathRoot/$componentName", 'some data'.bytes, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT)
-        def outputFilePath = 'target/test-data.json'
-
-        when:
-        ZooKeeperMigratorMain.main(['-r', '-z', "$server.connectString$migrationPathRoot", '-f', outputFilePath] as String[])
-
-        then:
-        noExceptionThrown()
-        def persistedData = new Gson().fromJson(new JsonReader(new FileReader(outputFilePath)), List) as List
-        persistedData.size() == 6
-    }
-
-    def "Send to open ZooKeeper without ACL migration"() {
-        given:
-        def server = new TestingServer()
-        def client = new ZooKeeper(server.connectString, 3000, { WatchedEvent watchedEvent ->
-        })
-        def migrationPathRoot = '/newParent'
-
-        when:
-        ZooKeeperMigratorMain.main(['-s', '-z', "$server.connectString$migrationPathRoot", '-f', 'src/test/resources/test-data.json'] as String[])
-
-        then:
-        noExceptionThrown()
-        def nodes = ZKPaths.getSortedChildren(client, '/').collect { ZKUtil.listSubTreeBFS(client, "/$it") }.flatten()
-        nodes.size() == 6
-    }
-
-    def "Send to open ZooKeeper without ACL migration with new multi-node parent"() {
-        given:
-        def server = new TestingServer()
-        def client = new ZooKeeper(server.connectString, 3000, { WatchedEvent watchedEvent ->
-        })
-        def migrationPathRoot = '/newParent/node'
-
-        when:
-        ZooKeeperMigratorMain.main(['-s', '-z', "$server.connectString$migrationPathRoot", '-f', 'src/test/resources/test-data.json'] as String[])
-
-        then:
-        noExceptionThrown()
-        def nodes = ZKPaths.getSortedChildren(client, '/').collect { ZKUtil.listSubTreeBFS(client, "/$it") }.flatten()
-        nodes.size() == 7
-    }
-
-    def "Receive all nodes from ZooKeeper root"() {
-        given:
-        def server = new TestingServer()
-        def client = new ZooKeeper(server.connectString, 3000, { WatchedEvent watchedEvent ->
-        })
-        def migrationPathRoot = '/'
-        def addedNodePath = 'nifi'
-        client.create("$migrationPathRoot$addedNodePath", 'some data'.bytes, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT)
-        def outputFilePath = 'target/test-data-root.json'
-
-        when:
-        ZooKeeperMigratorMain.main(['-r', '-z', "$server.connectString$migrationPathRoot", '-f', outputFilePath] as String[])
-
-        then:
-        noExceptionThrown()
-        def persistedData = new Gson().fromJson(new JsonReader(new FileReader(outputFilePath)), List) as List
-        persistedData.size() == 5
-    }
-
-    def "Receive Zookeeper node created with username and password"() {
-        given:
-        def server = new TestingServer()
-        def client = new ZooKeeper(server.connectString, 3000, { WatchedEvent watchedEvent ->
-        })
-        def username = 'nifi'
-        def password = 'nifi'
-        client.addAuthInfo('digest', "$username:$password".getBytes(StandardCharsets.UTF_8))
-        def migrationPathRoot = '/nifi'
-        client.create(migrationPathRoot, 'some data'.bytes, ZooDefs.Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT)
-        def outputFilePath = 'target/test-data-user-pass.json'
-
-        when:
-        ZooKeeperMigratorMain.main(['-r', '-z', "$server.connectString$migrationPathRoot", '-f', outputFilePath, '-a', "$username:$password"] as String[])
-
-        then:
-        noExceptionThrown()
-        def persistedData = new Gson().fromJson(new JsonReader(new FileReader(outputFilePath)), List) as List
-        persistedData.size() == 2
-    }
-
-    def "Send to Zookeeper a node created with username and password"() {
-        given:
-        def server = new TestingServer()
-        def client = new ZooKeeper(server.connectString, 3000, { WatchedEvent watchedEvent ->
-        })
-        def username = 'nifi'
-        def password = 'nifi'
-        client.addAuthInfo('digest', "$username:$password".getBytes(StandardCharsets.UTF_8))
-        def migrationPathRoot = '/newParent'
-
-        when:
-        ZooKeeperMigratorMain.main(['-s', '-z', "$server.connectString$migrationPathRoot", '-f', 'src/test/resources/test-data-user-pass.json', '-a', "$username:$password"] as String[])
-
-        then:
-        noExceptionThrown()
-        def nodes = ZKPaths.getSortedChildren(client, '/').collect { ZKUtil.listSubTreeBFS(client, "/$it") }.flatten()
-        nodes.size() == 3
-    }
-
-    def "Send to open Zookeeper with ACL migration"() {
-        given:
-        def server = new TestingServer()
-        def client = new ZooKeeper(server.connectString, 3000, { WatchedEvent watchedEvent ->
-        })
-        def migrationPathRoot = '/nifi-open'
-
-        when:
-        ZooKeeperMigratorMain.main(['-s', '-z', "$server.connectString$migrationPathRoot", '-f', 'src/test/resources/test-data-user-pass.json'] as String[])
-
-        then:
-        noExceptionThrown()
-        def nodes = ZKPaths.getSortedChildren(client, '/').collect { ZKUtil.listSubTreeBFS(client, "/$it") }.flatten()
-        nodes.size() == 3
-    }
-
-    def "Parse Zookeeper connect string and path"() {
-        when:
-        def zooKeeperMigrator = new ZooKeeperMigrator("$connectString")
-
-        then:
-        zooKeeperMigrator.zooKeeperEndpointConfig.connectString == connectString
-        zooKeeperMigrator.zooKeeperEndpointConfig.servers == servers.split(',').collect()
-        zooKeeperMigrator.zooKeeperEndpointConfig.path == path
-
-        where:
-        connectString                                       | path         | servers                                   || _
-        '127.0.0.1'                                         | '/'          | '127.0.0.1'                               || _
-        '127.0.0.1,127.0.0.2'                               | '/'          | '127.0.0.1,127.0.0.2'                     || _
-        '127.0.0.1/'                                        | '/'          | '127.0.0.1'                               || _
-        '127.0.0.1,127.0.0.2/'                              | '/'          | '127.0.0.1,127.0.0.2'                     || _
-        '127.0.0.1:2181'                                    | '/'          | '127.0.0.1:2181'                          || _
-        '127.0.0.1,127.0.0.2:2181'                          | '/'          | '127.0.0.1,127.0.0.2:2181'                || _
-        '127.0.0.1:2181/'                                   | '/'          | '127.0.0.1:2181'                          || _
-        '127.0.0.1,127.0.0.2:2181/'                         | '/'          | '127.0.0.1,127.0.0.2:2181'                || _
-        '127.0.0.1/path'                                    | '/path'      | '127.0.0.1'                               || _
-        '127.0.0.1,127.0.0.2/path'                          | '/path'      | '127.0.0.1,127.0.0.2'                     || _
-        '127.0.0.1/path/node'                               | '/path/node' | '127.0.0.1'                               || _
-        '127.0.0.1,127.0.0.2/path/node'                     | '/path/node' | '127.0.0.1,127.0.0.2'                     || _
-        '127.0.0.1:2181/'                                   | '/'          | '127.0.0.1:2181'                          || _
-        '127.0.0.1,127.0.0.2:2181/'                         | '/'          | '127.0.0.1,127.0.0.2:2181'                || _
-        '127.0.0.1:2181/path'                               | '/path'      | '127.0.0.1:2181'                          || _
-        '127.0.0.1,127.0.0.2:2181/path'                     | '/path'      | '127.0.0.1,127.0.0.2:2181'                || _
-        '127.0.0.1:2181/path/node'                          | '/path/node' | '127.0.0.1:2181'                          || _
-        '127.0.0.1,127.0.0.2:2181/path/node'                | '/path/node' | '127.0.0.1,127.0.0.2:2181'                || _
-        '127.0.0.1,127.0.0.2:2182,127.0.0.3:2183'           | '/'          | '127.0.0.1,127.0.0.2:2182,127.0.0.3:2183' || _
-        '127.0.0.1,127.0.0.2:2182,127.0.0.3:2183/'          | '/'          | '127.0.0.1,127.0.0.2:2182,127.0.0.3:2183' || _
-        '127.0.0.1,127.0.0.2:2182,127.0.0.3:2183/path'      | '/path'      | '127.0.0.1,127.0.0.2:2182,127.0.0.3:2183' || _
-        '127.0.0.1,127.0.0.2:2182,127.0.0.3:2183/path/node' | '/path/node' | '127.0.0.1,127.0.0.2:2182,127.0.0.3:2183' || _
-    }
-
-    def "Test ignore source"() {
-        given:
-        def server = new TestingServer()
-        def connectString = "$server.connectString"
-        def dataPath = 'target/test-data-ignore-source.json'
-
-        when: "data is read from the source zookeeper"
-        ZooKeeperMigratorMain.main(['-r', '-z', connectString, '-f', dataPath] as String[])
-
-        then: "verify the data has been written to the output file"
-        new File(dataPath).exists()
-
-        when: "data is sent to the same zookeeper as the the source zookeeper without ignore source"
-        ZooKeeperMigratorMain.main(['-s', '-z', connectString, '-f', dataPath] as String[])
-
-        then: "verify that an illegal argument exception is thrown"
-        thrown(IllegalArgumentException)
-
-        when: "data is sent to the same zookeeper as the source zookeeper with ignore source option is set"
-        ZooKeeperMigratorMain.main(['-s', '-z', connectString, '-f', dataPath, '--ignore-source'] as String[])
-
-        then: "no exceptions are thrown"
-        noExceptionThrown()
-    }
-
-    def "Send to same ZooKeeper with different path"() {
-        def server = new TestingServer()
-        def connectString = "$server.connectString"
-        def dataPath = 'target/test-data-different-path.json'
-
-        when: "data is read from the source zookeeper"
-        ZooKeeperMigratorMain.main(['-r', '-z', connectString, '-f', dataPath] as String[])
-
-        then: "verify the data has been written to the output file"
-        new File(dataPath).exists()
-
-        when: "data is sent to the same zookeeper as the the source zookeeper with a different path"
-        ZooKeeperMigratorMain.main(['-s', '-z', "$connectString/new-path", '-f', dataPath] as String[])
-
-        then: "no exceptions are thrown"
-        noExceptionThrown()
-    }
-}