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

nifi git commit: NIFI-4022 - Initial update for SASL support for cluster management in Zookeeper

Repository: nifi
Updated Branches:
  refs/heads/master 969bbe654 -> afd4f9e03


NIFI-4022 - Initial update for SASL support for cluster management in Zookeeper

NIFI-4022 - adding sasl documentation update and update to test

This closes #2046.

Signed-off-by: Bryan Bende <bb...@apache.org>


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

Branch: refs/heads/master
Commit: afd4f9e034e70e996abcf7e31382ee5c42a72445
Parents: 969bbe6
Author: Yolanda M. Davis <yo...@gmail.com>
Authored: Mon Jul 31 13:27:48 2017 -0400
Committer: Bryan Bende <bb...@apache.org>
Committed: Fri Aug 4 14:19:35 2017 -0400

----------------------------------------------------------------------
 .../org/apache/nifi/util/NiFiProperties.java    |   6 +
 .../src/main/asciidoc/administration-guide.adoc |  20 +++-
 .../cluster/ZooKeeperClientConfig.java          |  38 +++++-
 .../election/CuratorACLProviderFactory.java     |  83 +++++++++++++
 .../election/CuratorLeaderElectionManager.java  |   2 +
 .../election/TestCuratorACLProviderFactory.java | 117 +++++++++++++++++++
 .../nifi-framework/nifi-resources/pom.xml       |   3 +
 .../src/main/resources/conf/nifi.properties     |  10 ++
 8 files changed, 275 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/afd4f9e0/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
index ad99955..2d14165 100644
--- a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
+++ b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
@@ -187,6 +187,9 @@ public abstract class NiFiProperties {
     public static final String ZOOKEEPER_CONNECT_TIMEOUT = "nifi.zookeeper.connect.timeout";
     public static final String ZOOKEEPER_SESSION_TIMEOUT = "nifi.zookeeper.session.timeout";
     public static final String ZOOKEEPER_ROOT_NODE = "nifi.zookeeper.root.node";
+    public static final String ZOOKEEPER_AUTH_TYPE = "nifi.zookeeper.auth.type";
+    public static final String ZOOKEEPER_KERBEROS_REMOVE_HOST_FROM_PRINCIPAL = "nifi.zookeeper.kerberos.removeHostFromPrincipal";
+    public static final String ZOOKEEPER_KERBEROS_REMOVE_REALM_FROM_PRINCIPAL = "nifi.zookeeper.kerberos.removeRealmFromPrincipal";
 
     // kerberos properties
     public static final String KERBEROS_KRB5_FILE = "nifi.kerberos.krb5.file";
@@ -234,6 +237,9 @@ public abstract class NiFiProperties {
     public static final String DEFAULT_ZOOKEEPER_CONNECT_TIMEOUT = "3 secs";
     public static final String DEFAULT_ZOOKEEPER_SESSION_TIMEOUT = "3 secs";
     public static final String DEFAULT_ZOOKEEPER_ROOT_NODE = "/nifi";
+    public static final String DEFAULT_ZOOKEEPER_AUTH_TYPE = "default";
+    public static final String DEFAULT_ZOOKEEPER_KERBEROS_REMOVE_HOST_FROM_PRINCIPAL  = "true";
+    public static final String DEFAULT_ZOOKEEPER_KERBEROS_REMOVE_REALM_FROM_PRINCIPAL  = "true";
     public static final String DEFAULT_SITE_TO_SITE_HTTP_TRANSACTION_TTL = "30 secs";
     public static final String DEFAULT_FLOW_CONFIGURATION_ARCHIVE_ENABLED = "true";
     public static final String DEFAULT_FLOW_CONFIGURATION_ARCHIVE_MAX_TIME = "30 days";

http://git-wip-us.apache.org/repos/asf/nifi/blob/afd4f9e0/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 ef17cff..c2e9f8a 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -2106,9 +2106,17 @@ lines:
 
 [source]
 authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider
+kerberos.removeHostFromPrincipal=true
+kerberos.removeRealmFromPrincipal=true
 jaasLoginRenew=3600000
 requireClientAuthScheme=sasl
 
+The kerberos.removeHostFromPrincipal and the kerberos.removeRealmFromPrincipal properties are used to normalize the user principal name before comparing an identity to acls
+applied on a Znode.  By default the full principal is used however setting the removeHostFromPrincipal and removeRealmFromPrincipal kerberos properties to true will instruct
+Zookeeper to remove the host and the realm from the logged in user's identity for comparison.  In cases where NiFi nodes (within the same cluster) use principals that
+have different host(s)/realm(s) values, these kerberos properties can be configured to ensure that the nodes' identity will be normalized and that the nodes will have
+appropriate access to shared Znodes in Zookeeper.
+
 The last line is optional but specifies that clients MUST use Kerberos to communicate with our ZooKeeper instance.
 
 Now, we can start NiFi, and the embedded ZooKeeper server will use Kerberos as the authentication mechanism.
@@ -2157,12 +2165,22 @@ Client {
 };
 
 
-Finally, we need to tell NiFi to use this as our JAAS configuration. This is done by setting a JVM System Property, so we will edit the _conf/bootstrap.conf_ file.
+We then need to tell NiFi to use this as our JAAS configuration. This is done by setting a JVM System Property, so we will edit the _conf/bootstrap.conf_ file.
 We add the following line anywhere in this file in order to tell the NiFi JVM to use this configuration:
 
 [source]
 java.arg.15=-Djava.security.auth.login.config=./conf/zookeeper-jaas.conf
 
+Finally we need to update `nifi.properties` to ensure that NiFi knows to apply SASL specific ACLs for the Znodes it will create in Zookeeper for cluster management.
+To enable this, in the `$NIFI_HOME/conf/nifi.properties` file and edit the following properties as shown below:
+
+[source]
+nifi.zookeeper.auth.type=sasl
+nifi.zookeeper.kerberos.removeHostFromPrincipal=true
+nifi.zookeeper.kerberos.removeRealmFromPrincipal=true
+
+Note: The kerberos.removeHostFromPrincipal and kerberos.removeRealmFromPrincipal should be consistent with what is set in Zookeeper configuration.
+
 We can initialize our Kerberos ticket by running the following command:
 
 [source]

http://git-wip-us.apache.org/repos/asf/nifi/blob/afd4f9e0/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ZooKeeperClientConfig.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ZooKeeperClientConfig.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ZooKeeperClientConfig.java
index ffd4046..ad9f195 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ZooKeeperClientConfig.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ZooKeeperClientConfig.java
@@ -37,12 +37,21 @@ public class ZooKeeperClientConfig {
     private final int sessionTimeoutMillis;
     private final int connectionTimeoutMillis;
     private final String rootPath;
+    private final String authType;
+    private final String authPrincipal;
+    private final String removeHostFromPrincipal;
+    private final String removeRealmFromPrincipal;
 
-    private ZooKeeperClientConfig(String connectString, int sessionTimeoutMillis, int connectionTimeoutMillis, String rootPath) {
+    private ZooKeeperClientConfig(String connectString, int sessionTimeoutMillis, int connectionTimeoutMillis, String rootPath,
+                                  String authType, String authPrincipal, String removeHostFromPrincipal, String removeRealmFromPrincipal) {
         this.connectString = connectString;
         this.sessionTimeoutMillis = sessionTimeoutMillis;
         this.connectionTimeoutMillis = connectionTimeoutMillis;
         this.rootPath = rootPath.endsWith("/") ? rootPath.substring(0, rootPath.length() - 1) : rootPath;
+        this.authType = authType;
+        this.authPrincipal = authPrincipal;
+        this.removeHostFromPrincipal = removeHostFromPrincipal;
+        this.removeRealmFromPrincipal = removeRealmFromPrincipal;
     }
 
     public String getConnectString() {
@@ -61,6 +70,22 @@ public class ZooKeeperClientConfig {
         return rootPath;
     }
 
+    public String getAuthType() {
+        return authType;
+    }
+
+    public String getAuthPrincipal() {
+        return authPrincipal;
+    }
+
+    public String getRemoveHostFromPrincipal() {
+        return removeHostFromPrincipal;
+    }
+
+    public String getRemoveRealmFromPrincipal() {
+        return removeRealmFromPrincipal;
+    }
+
     public String resolvePath(final String path) {
         if (path.startsWith("/")) {
             return rootPath + path;
@@ -76,11 +101,18 @@ public class ZooKeeperClientConfig {
         }
         final String cleanedConnectString = cleanConnectString(connectString);
         if (cleanedConnectString.isEmpty()) {
-            throw new IllegalStateException("The '" + NiFiProperties.ZOOKEEPER_CONNECT_STRING + "' property is set in nifi.properties but needs to be in pairs of host:port separated by commas");
+            throw new IllegalStateException("The '" + NiFiProperties.ZOOKEEPER_CONNECT_STRING +
+                    "' property is set in nifi.properties but needs to be in pairs of host:port separated by commas");
         }
         final long sessionTimeoutMs = getTimePeriod(nifiProperties, NiFiProperties.ZOOKEEPER_SESSION_TIMEOUT, NiFiProperties.DEFAULT_ZOOKEEPER_SESSION_TIMEOUT);
         final long connectionTimeoutMs = getTimePeriod(nifiProperties, NiFiProperties.ZOOKEEPER_CONNECT_TIMEOUT, NiFiProperties.DEFAULT_ZOOKEEPER_CONNECT_TIMEOUT);
         final String rootPath = nifiProperties.getProperty(NiFiProperties.ZOOKEEPER_ROOT_NODE, NiFiProperties.DEFAULT_ZOOKEEPER_ROOT_NODE);
+        final String authType = nifiProperties.getProperty(NiFiProperties.ZOOKEEPER_AUTH_TYPE,NiFiProperties.DEFAULT_ZOOKEEPER_AUTH_TYPE);
+        final String authPrincipal = nifiProperties.getKerberosServicePrincipal();
+        final String removeHostFromPrincipal = nifiProperties.getProperty(NiFiProperties.ZOOKEEPER_KERBEROS_REMOVE_HOST_FROM_PRINCIPAL,
+                NiFiProperties.DEFAULT_ZOOKEEPER_KERBEROS_REMOVE_HOST_FROM_PRINCIPAL);
+        final String removeRealmFromPrincipal = nifiProperties.getProperty(NiFiProperties.ZOOKEEPER_KERBEROS_REMOVE_REALM_FROM_PRINCIPAL,
+                NiFiProperties.DEFAULT_ZOOKEEPER_KERBEROS_REMOVE_REALM_FROM_PRINCIPAL);
 
         try {
             PathUtils.validatePath(rootPath);
@@ -88,7 +120,7 @@ public class ZooKeeperClientConfig {
             throw new IllegalArgumentException("The '" + NiFiProperties.ZOOKEEPER_ROOT_NODE + "' property in nifi.properties is set to an illegal value: " + rootPath);
         }
 
-        return new ZooKeeperClientConfig(cleanedConnectString, (int) sessionTimeoutMs, (int) connectionTimeoutMs, rootPath);
+        return new ZooKeeperClientConfig(cleanedConnectString, (int) sessionTimeoutMs, (int) connectionTimeoutMs, rootPath, authType, authPrincipal, removeHostFromPrincipal, removeRealmFromPrincipal);
     }
 
     private static int getTimePeriod(final NiFiProperties nifiProperties, final String propertyName, final String defaultValue) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/afd4f9e0/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorACLProviderFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorACLProviderFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorACLProviderFactory.java
new file mode 100644
index 0000000..4644d95
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorACLProviderFactory.java
@@ -0,0 +1,83 @@
+/*
+ * 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.controller.leader.election;
+
+import java.util.List;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.curator.framework.api.ACLProvider;
+import org.apache.curator.framework.imps.DefaultACLProvider;
+import org.apache.nifi.controller.cluster.ZooKeeperClientConfig;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+
+import com.google.common.collect.Lists;
+
+public class CuratorACLProviderFactory {
+
+    public static final String SASL_AUTH_SCHEME = "sasl";
+
+    public ACLProvider create(ZooKeeperClientConfig config){
+        return StringUtils.equalsIgnoreCase(config.getAuthType(),SASL_AUTH_SCHEME) ? new SaslACLProvider(config) : new DefaultACLProvider();
+    }
+
+    private class SaslACLProvider implements ACLProvider{
+
+        private final List<ACL> acls;
+
+        private SaslACLProvider(ZooKeeperClientConfig config) {
+
+            if(!StringUtils.isEmpty(config.getAuthPrincipal())) {
+
+                final String realm = config.getAuthPrincipal().substring(config.getAuthPrincipal().indexOf('@') + 1, config.getAuthPrincipal().length());
+                final String[] user = config.getAuthPrincipal().substring(0, config.getAuthPrincipal().indexOf('@')).split("/");
+                final String host = user.length == 2 ? user[1] : null;
+                final String instance = user[0];
+                final StringBuilder principal = new StringBuilder(instance);
+
+                if (!config.getRemoveHostFromPrincipal().equalsIgnoreCase("true")) {
+                    principal.append("/");
+                    principal.append(host);
+                }
+
+                if (!config.getRemoveRealmFromPrincipal().equalsIgnoreCase("true")) {
+                    principal.append("@");
+                    principal.append(realm);
+                }
+
+                this.acls = Lists.newArrayList(new ACL(ZooDefs.Perms.ALL, new Id(SASL_AUTH_SCHEME, principal.toString())));
+                this.acls.addAll(ZooDefs.Ids.READ_ACL_UNSAFE);
+
+            }else{
+                throw new IllegalArgumentException("No Kerberos Principal configured for use with SASL Authentication Scheme");
+            }
+        }
+
+        @Override
+        public List<ACL> getDefaultAcl() {
+            return acls;
+        }
+
+        @Override
+        public List<ACL> getAclForPath(String s) {
+            return acls;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/afd4f9e0/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorLeaderElectionManager.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorLeaderElectionManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorLeaderElectionManager.java
index 229617f..54ca257 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorLeaderElectionManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorLeaderElectionManager.java
@@ -294,12 +294,14 @@ public class CuratorLeaderElectionManager implements LeaderElectionManager {
     private CuratorFramework createClient() {
         // Create a new client because we don't want to try indefinitely for this to occur.
         final RetryPolicy retryPolicy = new RetryNTimes(1, 100);
+        final CuratorACLProviderFactory aclProviderFactory = new CuratorACLProviderFactory();
 
         final CuratorFramework client = CuratorFrameworkFactory.builder()
             .connectString(zkConfig.getConnectString())
             .sessionTimeoutMs(zkConfig.getSessionTimeoutMillis())
             .connectionTimeoutMs(zkConfig.getConnectionTimeoutMillis())
             .retryPolicy(retryPolicy)
+            .aclProvider(aclProviderFactory.create(zkConfig))
             .defaultData(new byte[0])
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/afd4f9e0/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/leader/election/TestCuratorACLProviderFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/leader/election/TestCuratorACLProviderFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/leader/election/TestCuratorACLProviderFactory.java
new file mode 100644
index 0000000..c4824c9
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/leader/election/TestCuratorACLProviderFactory.java
@@ -0,0 +1,117 @@
+/*
+ * 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.controller.leader.election;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.curator.framework.api.ACLProvider;
+import org.apache.curator.framework.imps.DefaultACLProvider;
+import org.apache.nifi.controller.cluster.ZooKeeperClientConfig;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.zookeeper.data.ACL;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+
+public class TestCuratorACLProviderFactory {
+
+    private volatile String propsFile = TestCuratorACLProviderFactory.class.getResource("/flowcontrollertest.nifi.properties").getFile();
+    final Map<String, String> otherProps = new HashMap<>();
+
+    @Before
+    public void setup(){
+        otherProps.put("nifi.zookeeper.connect.string", "local:1234");
+        otherProps.put("nifi.zookeeper.root.node", "/nifi");
+        otherProps.put("nifi.zookeeper.auth.type", "sasl");
+        otherProps.put("nifi.kerberos.service.principal","nifi/host@REALM.COM");
+    }
+
+    @Test
+    public void testSaslAuthSchemeNoHostNoRealm(){
+        final NiFiProperties nifiProperties;
+        final CuratorACLProviderFactory factory;
+        otherProps.put("nifi.zookeeper.kerberos.removeHostFromPrincipal", "true");
+        otherProps.put("nifi.zookeeper.kerberos.removeRealmFromPrincipal", "true");
+        nifiProperties = NiFiProperties.createBasicNiFiProperties(propsFile, otherProps);
+        factory = new CuratorACLProviderFactory();
+        ZooKeeperClientConfig config = ZooKeeperClientConfig.createConfig(nifiProperties);
+        ACLProvider provider = factory.create(config);
+        assertFalse(provider instanceof DefaultACLProvider);
+        List<ACL> acls = provider.getDefaultAcl();
+        assertNotNull(acls);
+        assertEquals(acls.get(0).getId().toString().trim(),"'sasl,'nifi");
+    }
+
+    @Test
+    public void testSaslAuthSchemeHeadless(){
+        final NiFiProperties nifiProperties;
+        final CuratorACLProviderFactory factory;
+        otherProps.put("nifi.zookeeper.kerberos.removeHostFromPrincipal", "true");
+        otherProps.put("nifi.zookeeper.kerberos.removeRealmFromPrincipal", "true");
+        otherProps.put("nifi.kerberos.service.principal","nifi@REALM.COM");
+        nifiProperties = NiFiProperties.createBasicNiFiProperties(propsFile, otherProps);
+        factory = new CuratorACLProviderFactory();
+        ZooKeeperClientConfig config = ZooKeeperClientConfig.createConfig(nifiProperties);
+        ACLProvider provider = factory.create(config);
+        assertFalse(provider instanceof DefaultACLProvider);
+        List<ACL> acls = provider.getDefaultAcl();
+        assertNotNull(acls);
+        assertEquals(acls.get(0).getId().toString().trim(),"'sasl,'nifi");
+    }
+
+    @Test
+    public void testSaslAuthSchemeNoHostWithRealm(){
+
+        final NiFiProperties nifiProperties;
+        final CuratorACLProviderFactory factory;
+        otherProps.put("nifi.zookeeper.kerberos.removeHostFromPrincipal", "true");
+        otherProps.put("nifi.zookeeper.kerberos.removeRealmFromPrincipal", "false");
+        nifiProperties = NiFiProperties.createBasicNiFiProperties(propsFile, otherProps);
+        factory = new CuratorACLProviderFactory();
+        ZooKeeperClientConfig config = ZooKeeperClientConfig.createConfig(nifiProperties);
+        ACLProvider provider = factory.create(config);
+        assertFalse(provider instanceof DefaultACLProvider);
+        List<ACL> acls = provider.getDefaultAcl();
+        assertNotNull(acls);
+        assertEquals(acls.get(0).getId().toString().trim(),"'sasl,'nifi@REALM.COM");
+
+    }
+
+    @Test
+    public void testSaslAuthSchemeWithHostNoRealm(){
+
+        final NiFiProperties nifiProperties;
+        final CuratorACLProviderFactory factory;
+        otherProps.put("nifi.zookeeper.kerberos.removeHostFromPrincipal", "false");
+        otherProps.put("nifi.zookeeper.kerberos.removeRealmFromPrincipal", "true");
+        nifiProperties = NiFiProperties.createBasicNiFiProperties(propsFile, otherProps);
+        factory = new CuratorACLProviderFactory();
+        ZooKeeperClientConfig config = ZooKeeperClientConfig.createConfig(nifiProperties);
+        ACLProvider provider = factory.create(config);
+        assertFalse(provider instanceof DefaultACLProvider);
+        List<ACL> acls = provider.getDefaultAcl();
+        assertNotNull(acls);
+        assertEquals(acls.get(0).getId().toString().trim(),"'sasl,'nifi/host");
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/afd4f9e0/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
index 5cfe17c..b1c4b9d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
@@ -172,6 +172,9 @@
         <nifi.zookeeper.connect.timeout>3 secs</nifi.zookeeper.connect.timeout>
         <nifi.zookeeper.session.timeout>3 secs</nifi.zookeeper.session.timeout>
         <nifi.zookeeper.root.node>/nifi</nifi.zookeeper.root.node>
+        <nifi.zookeeper.auth.type/>
+        <nifi.zookeeper.kerberos.removeHostFromPrincipal/>
+        <nifi.zookeeper.kerberos.removeRealmFromPrincipal/>
 
         <!-- nifi.properties: kerberos properties -->
         <nifi.kerberos.krb5.file> </nifi.kerberos.krb5.file>

http://git-wip-us.apache.org/repos/asf/nifi/blob/afd4f9e0/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
index f96d167..6e77f04 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
@@ -190,6 +190,16 @@ nifi.zookeeper.connect.timeout=${nifi.zookeeper.connect.timeout}
 nifi.zookeeper.session.timeout=${nifi.zookeeper.session.timeout}
 nifi.zookeeper.root.node=${nifi.zookeeper.root.node}
 
+# Zookeeper properties for the authentication scheme used when creating acls on znodes used for cluster management
+# Values supported for nifi.zookeeper.auth.type are "default", which will apply world/anyone rights on znodes
+# and "sasl" which will give rights to the sasl/kerberos identity used to authenticate the nifi node
+# The identity is determined using the value in nifi.kerberos.service.principal and the removeHostFromPrincipal
+# and removeRealmFromPrincipal values (which should align with the kerberos.removeHostFromPrincipal and kerberos.removeRealmFromPrincipal
+# values configured on the zookeeper server).
+nifi.zookeeper.auth.type=${nifi.zookeeper.auth.type}
+nifi.zookeeper.kerberos.removeHostFromPrincipal=${nifi.zookeeper.kerberos.removeHostFromPrincipal}
+nifi.zookeeper.kerberos.removeRealmFromPrincipal=${nifi.zookeeper.kerberos.removeRealmFromPrincipal}
+
 # kerberos #
 nifi.kerberos.krb5.file=${nifi.kerberos.krb5.file}