You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by de...@apache.org on 2021/08/26 19:12:45 UTC

[nifi] branch main updated: NIFI-9082 Add nifi.zookeeper.jute.maxbuffer property (#5335)

This is an automated email from the ASF dual-hosted git repository.

denes pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/main by this push:
     new 0cf0984  NIFI-9082 Add nifi.zookeeper.jute.maxbuffer property (#5335)
0cf0984 is described below

commit 0cf09840eabd6131be8a9a928c1b2e7df03eb0ee
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Thu Aug 26 14:12:34 2021 -0500

    NIFI-9082 Add nifi.zookeeper.jute.maxbuffer property (#5335)
    
    * NIFI-9082 Added nifi.zookeeper.jute.maxbuffer property
    
    - Updated ZooKeeperStateProvider to check jute.maxbuffer instead of hard-coded number
    - Updated internal ZooKeeperClientConfig class to include new juteMaxBuffer property
    
    * NIFI-9082 Updated jute.maxbuffer documentation and added hexadecimal representation
    
    - Renamed DefaultWatcher to NoOpWatcher
    - Removed unnecessary validateDataSize() from createNode()
---
 .../java/org/apache/nifi/util/NiFiProperties.java  |  3 +
 .../src/main/asciidoc/administration-guide.adoc    |  5 ++
 .../controller/cluster/ZooKeeperClientConfig.java  | 13 +++-
 .../election/CuratorLeaderElectionManager.java     |  8 ++-
 .../zookeeper/ZooKeeperStateProvider.java          | 73 +++++++++++-----------
 .../zookeeper/TestZooKeeperStateProvider.java      | 51 ++++++++++++---
 .../nifi-framework/nifi-resources/pom.xml          |  1 +
 .../src/main/resources/conf/nifi.properties        |  1 +
 8 files changed, 107 insertions(+), 48 deletions(-)

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 bf7f6a4..d63f259 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
@@ -274,6 +274,7 @@ public class NiFiProperties extends ApplicationProperties {
     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";
+    public static final String ZOOKEEPER_JUTE_MAXBUFFER = "nifi.zookeeper.jute.maxbuffer";
 
     // kerberos properties
     public static final String KERBEROS_KRB5_FILE = "nifi.kerberos.krb5.file";
@@ -350,6 +351,8 @@ public class NiFiProperties extends ApplicationProperties {
     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";
+    // Based on org.apache.jute.BinaryInputArchive.maxBuffer hexadecimal 0xfffff from ZooKeeper NIOServerCnxn
+    public static final int DEFAULT_ZOOKEEPER_JUTE_MAXBUFFER = 1048575;
     public static final String DEFAULT_SECURITY_AUTO_RELOAD_INTERVAL = "10 secs";
     public static final String DEFAULT_SITE_TO_SITE_HTTP_TRANSACTION_TTL = "30 secs";
     public static final String DEFAULT_FLOW_CONFIGURATION_ARCHIVE_ENABLED = "true";
diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index 72e636f..1bb54a5 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -3915,6 +3915,11 @@ that is specified.
 |`nifi.zookeeper.security.truststore`|Filename of the Truststore that will be used to verify the ZooKeeper server(s).
 |`nifi.zookeeper.security.truststoreType`|Optional. The type of the Truststore. Must be `PKCS12`, `JKS`, or `PEM`. If not specified the type will be determined from the file extension (`.p12`, `.jks`, `.pem`).
 |`nifi.zookeeper.security.truststorePasswd`|The password for the Truststore.
+|`nifi.zookeeper.jute.maxbuffer`|Maximum buffer size in bytes for packets sent to and received from ZooKeeper.
+Defaults to `1048575` bytes (`0xfffff` in hexadecimal) following ZooKeeper default `jute.maxbuffer` property.
+
+The link:http://zookeeper.apache.org/doc/current/zookeeperAdmin.html#Unsafe+Options[ZooKeeper Administrator's Guide] categorizes this property as an *unsafe option*.
+Changing this property *requires* setting `jute.maxbuffer` on ZooKeeper servers.
 |====
 
 [[kerberos_properties]]
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 73c114b..7a09a8e 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
@@ -54,11 +54,13 @@ public class ZooKeeperClientConfig {
     private final String authPrincipal;
     private final String removeHostFromPrincipal;
     private final String removeRealmFromPrincipal;
+    private final int juteMaxbuffer;
 
     private ZooKeeperClientConfig(String connectString, int sessionTimeoutMillis, int connectionTimeoutMillis,
                                   String rootPath, String authType, String authPrincipal, String removeHostFromPrincipal,
                                   String removeRealmFromPrincipal, boolean clientSecure, String keyStore, String keyStoreType,
-                                  String keyStorePassword, String trustStore, String trustStoreType, String trustStorePassword) {
+                                  String keyStorePassword, String trustStore, String trustStoreType, String trustStorePassword,
+                                  final int juteMaxbuffer) {
         this.connectString = connectString;
         this.sessionTimeoutMillis = sessionTimeoutMillis;
         this.connectionTimeoutMillis = connectionTimeoutMillis;
@@ -74,6 +76,7 @@ public class ZooKeeperClientConfig {
         this.authPrincipal = authPrincipal;
         this.removeHostFromPrincipal = removeHostFromPrincipal;
         this.removeRealmFromPrincipal = removeRealmFromPrincipal;
+        this.juteMaxbuffer = juteMaxbuffer;
     }
 
     public String getConnectString() {
@@ -140,6 +143,10 @@ public class ZooKeeperClientConfig {
         return removeRealmFromPrincipal;
     }
 
+    public int getJuteMaxbuffer() {
+        return juteMaxbuffer;
+    }
+
     public String resolvePath(final String path) {
         if (path.startsWith("/")) {
             return rootPath + path;
@@ -174,6 +181,7 @@ public class ZooKeeperClientConfig {
                 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);
+        final int juteMaxbuffer = nifiProperties.getIntegerProperty(NiFiProperties.ZOOKEEPER_JUTE_MAXBUFFER, NiFiProperties.DEFAULT_ZOOKEEPER_JUTE_MAXBUFFER);
 
         try {
             PathUtils.validatePath(rootPath);
@@ -196,7 +204,8 @@ public class ZooKeeperClientConfig {
             keyStorePassword,
             trustStore,
             trustStoreType,
-            trustStorePassword
+            trustStorePassword,
+            juteMaxbuffer
         );
     }
 
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 a8f5790..99ac506 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
@@ -43,6 +43,7 @@ import org.apache.zookeeper.client.ZKClientConfig;
 import org.apache.zookeeper.common.ClientX509Util;
 import org.apache.zookeeper.common.PathUtils;
 import org.apache.zookeeper.common.X509Util;
+import org.apache.zookeeper.common.ZKConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -374,7 +375,7 @@ public class CuratorLeaderElectionManager implements LeaderElectionManager {
                 }
 
                 @Override
-                public void takeLeadership(CuratorFramework client) throws Exception {
+                public void takeLeadership(CuratorFramework client) {
                 }
             };
 
@@ -645,7 +646,7 @@ public class CuratorLeaderElectionManager implements LeaderElectionManager {
         public static final String NETTY_CLIENT_CNXN_SOCKET =
             org.apache.zookeeper.ClientCnxnSocketNetty.class.getName();
 
-        private ZKClientConfig zkSecureClientConfig;
+        private final ZKClientConfig zkSecureClientConfig;
 
         public SecureClientZooKeeperFactory(final ZooKeeperClientConfig zkConfig) {
             this.zkSecureClientConfig = new ZKClientConfig();
@@ -653,7 +654,7 @@ public class CuratorLeaderElectionManager implements LeaderElectionManager {
             // Netty is required for the secure client config.
             final String cnxnSocket = zkConfig.getConnectionSocket();
             if (!NETTY_CLIENT_CNXN_SOCKET.equals(cnxnSocket)) {
-                throw new IllegalArgumentException(String.format("connection factory set to '%s', %s required", String.valueOf(cnxnSocket), NETTY_CLIENT_CNXN_SOCKET));
+                throw new IllegalArgumentException(String.format("connection factory set to '%s', %s required", cnxnSocket, NETTY_CLIENT_CNXN_SOCKET));
             }
             zkSecureClientConfig.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, cnxnSocket);
 
@@ -671,6 +672,7 @@ public class CuratorLeaderElectionManager implements LeaderElectionManager {
             zkSecureClientConfig.setProperty(clientX509util.getSslTruststoreLocationProperty(), zkConfig.getTrustStore());
             zkSecureClientConfig.setProperty(clientX509util.getSslTruststoreTypeProperty(), zkConfig.getTrustStoreType());
             zkSecureClientConfig.setProperty(clientX509util.getSslTruststorePasswdProperty(), zkConfig.getTrustStorePassword());
+            zkSecureClientConfig.setProperty(ZKConfig.JUTE_MAXBUFFER, Integer.toString(zkConfig.getJuteMaxbuffer()));
         }
 
         @Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/providers/zookeeper/ZooKeeperStateProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/providers/zookeeper/ZooKeeperStateProvider.java
index 49bcb7f..e4c3be4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/providers/zookeeper/ZooKeeperStateProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/providers/zookeeper/ZooKeeperStateProvider.java
@@ -44,6 +44,8 @@ import org.apache.zookeeper.ZKUtil;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.client.ConnectStringParser;
+import org.apache.zookeeper.client.ZKClientConfig;
+import org.apache.zookeeper.common.ZKConfig;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
@@ -71,7 +73,6 @@ import java.util.stream.Collectors;
  */
 public class ZooKeeperStateProvider extends AbstractStateProvider {
     private static final Logger logger = LoggerFactory.getLogger(ZooKeeperStateProvider.class);
-    private static final int ONE_MB = 1024 * 1024;
     private NiFiProperties nifiProperties;
 
     static final AllowableValue OPEN_TO_WORLD = new AllowableValue("Open", "Open", "ZNodes will be open to any ZooKeeper client.");
@@ -212,25 +213,22 @@ public class ZooKeeperStateProvider extends AbstractStateProvider {
         }
 
         if (zooKeeper == null) {
-            if(clientConfig != null && clientConfig.isClientSecure()) {
+            if (clientConfig != null && clientConfig.isClientSecure()) {
                 SecureClientZooKeeperFactory factory = new SecureClientZooKeeperFactory(clientConfig);
                 try {
-                    zooKeeper = factory.newZooKeeper(connectionString, timeoutMillis, new Watcher() {
-                        @Override
-                        public void process(WatchedEvent event) {
-                        }
-                    }, true);
-                    logger.info("Secure Zookeeper client initialized successfully.");
-                } catch (Exception e) {
-                    logger.error("Secure Zookeeper configuration failed!", e);
+                    zooKeeper = factory.newZooKeeper(connectionString, timeoutMillis, new NoOpWatcher(), true);
+                    logger.debug("Secure ZooKeeper Client connection [{}] created", connectionString);
+                } catch (final Exception e) {
+                    logger.error("Secure ZooKeeper Client connection [{}] failed", connectionString, e);
                     invalidateClient();
                 }
             } else {
-                zooKeeper = new ZooKeeper(connectionString, timeoutMillis, new Watcher() {
-                    @Override
-                    public void process(WatchedEvent event) {
-                    }
-                });
+                final ZKClientConfig zkClientConfig = new ZKClientConfig();
+                if (clientConfig != null) {
+                    zkClientConfig.setProperty(ZKConfig.JUTE_MAXBUFFER, Integer.toString(clientConfig.getJuteMaxbuffer()));
+                }
+                zooKeeper = new ZooKeeper(connectionString, timeoutMillis, new NoOpWatcher(), zkClientConfig);
+                logger.debug("Standard ZooKeeper Client connection [{}] created", connectionString);
             }
 
             if (auth != null) {
@@ -242,17 +240,15 @@ public class ZooKeeperStateProvider extends AbstractStateProvider {
     }
 
     private ZooKeeperClientConfig getZooKeeperConfig() {
-        if(zooKeeperClientConfig != null) {
-            return zooKeeperClientConfig;
-        } else {
+        if (zooKeeperClientConfig == null) {
             Properties stateProviderProperties = new Properties();
             stateProviderProperties.setProperty(NiFiProperties.ZOOKEEPER_SESSION_TIMEOUT, timeoutMillis + " millis");
             stateProviderProperties.setProperty(NiFiProperties.ZOOKEEPER_CONNECT_TIMEOUT, timeoutMillis + " millis");
             stateProviderProperties.setProperty(NiFiProperties.ZOOKEEPER_ROOT_NODE, rootNode);
             stateProviderProperties.setProperty(NiFiProperties.ZOOKEEPER_CONNECT_STRING, connectionString);
             zooKeeperClientConfig = ZooKeeperClientConfig.createConfig(combineProperties(nifiProperties, stateProviderProperties));
-            return zooKeeperClientConfig;
         }
+        return zooKeeperClientConfig;
     }
 
     private synchronized void invalidateClient() {
@@ -373,7 +369,7 @@ public class ZooKeeperStateProvider extends AbstractStateProvider {
      *
      * @throws IOException if unable to communicate with ZooKeeper
      * @throws NoNodeException if the corresponding ZNode does not exist in ZooKeeper and allowNodeCreation is set to <code>false</code>
-     * @throws StateTooLargeException if the state to be stored exceeds the maximum size allowed by ZooKeeper (1 MB, after serialization)
+     * @throws StateTooLargeException if the state to be stored exceeds the maximum size allowed by ZooKeeper (Based on jute.maxbuffer property, after serialization)
      */
     private void setState(final Map<String, String> stateValues, final int version, final String componentId, final boolean allowNodeCreation) throws IOException, NoNodeException {
         verifyEnabled();
@@ -381,13 +377,8 @@ public class ZooKeeperStateProvider extends AbstractStateProvider {
         try {
             final String path = getComponentPath(componentId);
             final byte[] data = serialize(stateValues);
-            if (data.length > ONE_MB) {
-                throw new StateTooLargeException("Failed to set cluster-wide state in ZooKeeper for component with ID " + componentId
-                    + " because the state had " + stateValues.size() + " values, which serialized to " + data.length
-                    + " bytes, and the maximum allowed by ZooKeeper is 1 MB (" + ONE_MB + " bytes)");
-            }
-
             final ZooKeeper keeper = getZooKeeper();
+            validateDataSize(keeper.getClientConfig(), data, componentId, stateValues.size());
             try {
                 keeper.setData(path, data, version);
             } catch (final NoNodeException nne) {
@@ -425,13 +416,8 @@ public class ZooKeeperStateProvider extends AbstractStateProvider {
 
     private void createNode(final String path, final byte[] data, final String componentId, final Map<String, String> stateValues, final List<ACL> acls) throws IOException, KeeperException {
         try {
-            if (data != null && data.length > ONE_MB) {
-                throw new StateTooLargeException("Failed to set cluster-wide state in ZooKeeper for component with ID " + componentId
-                    + " because the state had " + stateValues.size() + " values, which serialized to " + data.length
-                    + " bytes, and the maximum allowed by ZooKeeper is 1 MB (" + ONE_MB + " bytes)");
-            }
-
-            getZooKeeper().create(path, data, acls, CreateMode.PERSISTENT);
+            final ZooKeeper zooKeeper = getZooKeeper();
+            zooKeeper.create(path, data, acls, CreateMode.PERSISTENT);
         } catch (final InterruptedException ie) {
             throw new IOException("Failed to update cluster-wide state due to interruption", ie);
         } catch (final KeeperException ke) {
@@ -509,7 +495,7 @@ public class ZooKeeperStateProvider extends AbstractStateProvider {
             return false;
         } catch (final IOException ioe) {
             final Throwable cause = ioe.getCause();
-            if (cause != null && cause instanceof KeeperException) {
+            if (cause instanceof KeeperException) {
                 final KeeperException ke = (KeeperException) cause;
                 if (Code.BADVERSION == ke.code()) {
                     return false;
@@ -524,6 +510,23 @@ public class ZooKeeperStateProvider extends AbstractStateProvider {
     @Override
     public void clear(final String componentId) throws IOException {
         verifyEnabled();
-        setState(Collections.<String, String>emptyMap(), componentId);
+        setState(Collections.emptyMap(), componentId);
+    }
+
+    private void validateDataSize(final ZKClientConfig clientConfig, final byte[] data, final String componentId, final int totalStateValues) throws StateTooLargeException {
+        final int maximumSize = clientConfig.getInt(ZKConfig.JUTE_MAXBUFFER, NiFiProperties.DEFAULT_ZOOKEEPER_JUTE_MAXBUFFER);
+        if (data != null && data.length > maximumSize) {
+            final String message = String.format("Component [%s] State Values [%d] Data Size [%d B] exceeds nifi.zookeeper.jute.maxbuffer size [%d B]",
+                    componentId, totalStateValues, data.length, maximumSize);
+            throw new StateTooLargeException(message);
+        }
+    }
+
+    private static final class NoOpWatcher implements Watcher {
+
+        @Override
+        public void process(final WatchedEvent watchedEvent) {
+
+        }
     }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/zookeeper/TestZooKeeperStateProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/zookeeper/TestZooKeeperStateProvider.java
index 455e54b..ebe2844 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/zookeeper/TestZooKeeperStateProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/zookeeper/TestZooKeeperStateProvider.java
@@ -38,8 +38,10 @@ import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Properties;
+import java.util.UUID;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
 
 public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
 
@@ -188,7 +190,7 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
     }
 
     @Test(timeout = 30000)
-    public void testStateTooLargeExceptionThrownOnReplace() throws IOException, InterruptedException {
+    public void testStateTooLargeExceptionThrownOnReplace() throws InterruptedException {
         final Map<String, String> state = new HashMap<>();
         final StringBuilder sb = new StringBuilder();
 
@@ -218,15 +220,48 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
             }
         }
 
+        assertThrows(StateTooLargeException.class, () -> getProvider().replace(getProvider().getState(componentId), state, componentId));
+    }
+
+    @Test(timeout = 5000)
+    public void testStateTooLargeExceptionThrownOnReplaceSmallJuteMaxbuffer() throws Exception {
+        final Map<String, String> initialState = new HashMap<>();
+        final String stateValue = UUID.randomUUID().toString();
+        final int maxBufferSize = 100;
+        initialState.put("1", stateValue);
+
+        final Map<PropertyDescriptor, String> properties = new HashMap<>(defaultProperties);
+        properties.put(ZooKeeperStateProvider.CONNECTION_STRING, zkServer.getConnectString());
+
+        final ZooKeeperStateProvider stateProvider = new ZooKeeperStateProvider();
+        final Properties applicationProperties = new Properties();
+        applicationProperties.setProperty(NiFiProperties.ZOOKEEPER_JUTE_MAXBUFFER, Integer.toString(maxBufferSize));
+        final NiFiProperties providerProperties = NiFiProperties.createBasicNiFiProperties(null, applicationProperties);
+        stateProvider.setNiFiProperties(providerProperties);
+        initializeProvider(stateProvider, properties);
+
         try {
-            getProvider().replace(getProvider().getState(componentId), state, componentId);
-            Assert.fail("Expected StateTooLargeException");
-        } catch (final StateTooLargeException stle) {
-            // expected behavior.
-        } catch (final Exception e) {
-            Assert.fail("Expected StateTooLargeException", e);
-        }
+            stateProvider.enable();
+
+            while (true) {
+                try {
+                    stateProvider.setState(initialState, componentId);
+                    break;
+                } catch (final IOException ioe) {
+                    // Retry initial connection
+                    Thread.sleep(1000L);
+                }
+            }
 
+            final Map<String, String> state = new HashMap<>();
+            state.put("1", stateValue);
+            state.put("2", stateValue);
+            state.put("3", stateValue);
+            state.put("4", stateValue);
+            assertThrows(StateTooLargeException.class, () -> stateProvider.replace(getProvider().getState(componentId), state, componentId));
+        } finally {
+            stateProvider.shutdown();
+        }
     }
 
     @Test
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 0a00789..af92897 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
@@ -233,6 +233,7 @@
         <nifi.zookeeper.session.timeout>10 secs</nifi.zookeeper.session.timeout>
         <nifi.zookeeper.root.node>/nifi</nifi.zookeeper.root.node>
         <nifi.zookeeper.client.secure>false</nifi.zookeeper.client.secure>
+        <nifi.zookeeper.jute.maxbuffer />
         <nifi.zookeeper.security.keystore />
         <nifi.zookeeper.security.keystoreType />
         <nifi.zookeeper.security.keystorePasswd />
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 18a00a4..c854b2a 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
@@ -290,6 +290,7 @@ nifi.zookeeper.security.keystorePasswd=${nifi.zookeeper.security.keystorePasswd}
 nifi.zookeeper.security.truststore=${nifi.zookeeper.security.truststore}
 nifi.zookeeper.security.truststoreType=${nifi.zookeeper.security.truststoreType}
 nifi.zookeeper.security.truststorePasswd=${nifi.zookeeper.security.truststorePasswd}
+nifi.zookeeper.jute.maxbuffer=${nifi.zookeeper.jute.maxbuffer}
 
 # 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