瀏覽代碼

ZOOKEEPER-3978: Adding additional security metrics to zookeeper

This is same as https://github.com/apache/zookeeper/pull/1513 for master branch with additional changes using junit5 jupiter lib.

The changes include adding 3 metrics for security for zookeeper for better monitoring

Monitoring all authN and AuthZ errors
Monitoring all local non mtls local connections counts
Monitoring all remote non mtls connection counts.

Author: Pooja Malik <pmalik@salesforce.com>

Reviewers: Michael Han <hanm@apache.org>, Enrico Olivelli <eolivelli@apache.org>, Damien Diederen <ddiederen@apache.org>

Closes #1523 from p-malik/zk_sec_master
Pooja Malik 4 年之前
父節點
當前提交
b842cd4b46

+ 15 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxnFactory.java

@@ -47,6 +47,7 @@ import io.netty.util.concurrent.GenericFutureListener;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
+import java.net.SocketAddress;
 import java.security.KeyManagementException;
 import java.security.KeyManagementException;
 import java.security.NoSuchAlgorithmException;
 import java.security.NoSuchAlgorithmException;
 import java.util.HashSet;
 import java.util.HashSet;
@@ -258,6 +259,16 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory {
                 allChannels.add(ctx.channel());
                 allChannels.add(ctx.channel());
                 addCnxn(cnxn);
                 addCnxn(cnxn);
             }
             }
+            if (ctx.channel().pipeline().get(SslHandler.class) == null) {
+                SocketAddress remoteAddress = cnxn.getChannel().remoteAddress();
+                if (remoteAddress != null
+                        && !((InetSocketAddress) remoteAddress).getAddress().isLoopbackAddress()) {
+                    LOG.trace("NettyChannelHandler channelActive: remote={} local={}", remoteAddress, cnxn.getChannel().localAddress());
+                    zkServer.serverStats().incrementNonMTLSRemoteConnCount();
+                } else {
+                    zkServer.serverStats().incrementNonMTLSLocalConnCount();
+                }
+            }
         }
         }
 
 
         @Override
         @Override
@@ -429,7 +440,9 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory {
                         return;
                         return;
                     }
                     }
 
 
-                    if (KeeperException.Code.OK != authProvider.handleAuthentication(cnxn, null)) {
+                    KeeperException.Code code = authProvider.handleAuthentication(cnxn, null);
+                    if (KeeperException.Code.OK != code) {
+                        zkServer.serverStats().incrementAuthFailedCount();
                         LOG.error("Authentication failed for session 0x{}", Long.toHexString(cnxn.getSessionId()));
                         LOG.error("Authentication failed for session 0x{}", Long.toHexString(cnxn.getSessionId()));
                         cnxn.close(ServerCnxn.DisconnectReason.SASL_AUTH_FAILURE);
                         cnxn.close(ServerCnxn.DisconnectReason.SASL_AUTH_FAILURE);
                         return;
                         return;
@@ -440,6 +453,7 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory {
                 allChannels.add(Objects.requireNonNull(futureChannel));
                 allChannels.add(Objects.requireNonNull(futureChannel));
                 addCnxn(cnxn);
                 addCnxn(cnxn);
             } else {
             } else {
+                zkServer.serverStats().incrementAuthFailedCount();
                 LOG.error("Unsuccessful handshake with session 0x{}", Long.toHexString(cnxn.getSessionId()));
                 LOG.error("Unsuccessful handshake with session 0x{}", Long.toHexString(cnxn.getSessionId()));
                 ServerMetrics.getMetrics().UNSUCCESSFUL_HANDSHAKE.add(1);
                 ServerMetrics.getMetrics().UNSUCCESSFUL_HANDSHAKE.add(1);
                 cnxn.close(ServerCnxn.DisconnectReason.FAILED_HANDSHAKE);
                 cnxn.close(ServerCnxn.DisconnectReason.FAILED_HANDSHAKE);

+ 42 - 0
zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerStats.java

@@ -41,6 +41,12 @@ public class ServerStats {
 
 
     private final BufferStats clientResponseStats = new BufferStats();
     private final BufferStats clientResponseStats = new BufferStats();
 
 
+    private AtomicLong nonMTLSRemoteConnCntr = new AtomicLong(0);
+
+    private AtomicLong nonMTLSLocalConnCntr = new AtomicLong(0);
+
+    private AtomicLong authFailedCntr = new AtomicLong(0);
+
     private final Provider provider;
     private final Provider provider;
     private final long startTime = Time.currentElapsedTime();
     private final long startTime = Time.currentElapsedTime();
 
 
@@ -181,6 +187,42 @@ public class ServerStats {
         fsyncThresholdExceedCount.set(0);
         fsyncThresholdExceedCount.set(0);
     }
     }
 
 
+    public long getNonMTLSLocalConnCount() {
+        return nonMTLSLocalConnCntr.get();
+    }
+
+    public void incrementNonMTLSLocalConnCount() {
+        nonMTLSLocalConnCntr.incrementAndGet();
+    }
+
+    public void resetNonMTLSLocalConnCount() {
+        nonMTLSLocalConnCntr.set(0);
+    }
+
+    public long getNonMTLSRemoteConnCount() {
+        return nonMTLSRemoteConnCntr.get();
+    }
+
+    public void incrementNonMTLSRemoteConnCount() {
+        nonMTLSRemoteConnCntr.incrementAndGet();
+    }
+
+    public void resetNonMTLSRemoteConnCount() {
+        nonMTLSRemoteConnCntr.set(0);
+    }
+
+    public long getAuthFailedCount() {
+        return authFailedCntr.get();
+    }
+
+    public void incrementAuthFailedCount() {
+        authFailedCntr.incrementAndGet();
+    }
+
+    public void resetAuthFailedCount() {
+        authFailedCntr.set(0);
+    }
+
     public void reset() {
     public void reset() {
         resetLatency();
         resetLatency();
         resetRequestCounters();
         resetRequestCounters();

+ 8 - 0
zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java

@@ -1894,6 +1894,9 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
         rootContext.registerGauge("min_client_response_size", stats.getClientResponseStats()::getMinBufferSize);
         rootContext.registerGauge("min_client_response_size", stats.getClientResponseStats()::getMinBufferSize);
 
 
         rootContext.registerGauge("outstanding_tls_handshake", this::getOutstandingHandshakeNum);
         rootContext.registerGauge("outstanding_tls_handshake", this::getOutstandingHandshakeNum);
+        rootContext.registerGauge("auth_failed_count", stats::getAuthFailedCount);
+        rootContext.registerGauge("non_mtls_remote_conn_count", stats::getNonMTLSRemoteConnCount);
+        rootContext.registerGauge("non_mtls_local_conn_count", stats::getNonMTLSLocalConnCount);
     }
     }
 
 
     protected void unregisterMetrics() {
     protected void unregisterMetrics() {
@@ -1929,6 +1932,11 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
         rootContext.unregisterGauge("max_client_response_size");
         rootContext.unregisterGauge("max_client_response_size");
         rootContext.unregisterGauge("min_client_response_size");
         rootContext.unregisterGauge("min_client_response_size");
 
 
+        rootContext.unregisterGauge("auth_failed_count");
+        rootContext.unregisterGauge("non_mtls_remote_conn_count");
+        rootContext.unregisterGauge("non_mtls_local_conn_count");
+
+
     }
     }
 
 
     /**
     /**

+ 24 - 0
zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerBean.java

@@ -133,6 +133,18 @@ public class ZooKeeperServerBean implements ZooKeeperServerMXBean, ZKMBeanInfo {
         return zks.serverStats().getFsyncThresholdExceedCount();
         return zks.serverStats().getFsyncThresholdExceedCount();
     }
     }
 
 
+    public long getAuthFailedCount() {
+        return zks.serverStats().getAuthFailedCount();
+    }
+
+    public long getNonMTLSRemoteConnCount() {
+        return zks.serverStats().getNonMTLSRemoteConnCount();
+    }
+
+    public long getNonMTLSLocalConnCount() {
+        return zks.serverStats().getNonMTLSLocalConnCount();
+    }
+
     public void resetLatency() {
     public void resetLatency() {
         zks.serverStats().resetLatency();
         zks.serverStats().resetLatency();
     }
     }
@@ -145,11 +157,23 @@ public class ZooKeeperServerBean implements ZooKeeperServerMXBean, ZKMBeanInfo {
         zks.serverStats().resetFsyncThresholdExceedCount();
         zks.serverStats().resetFsyncThresholdExceedCount();
     }
     }
 
 
+    public void resetAuthFailedCount() {
+        zks.serverStats().resetAuthFailedCount();
+    }
+
+    public void resetNonMTLSConnCount() {
+        zks.serverStats().resetNonMTLSRemoteConnCount();
+        zks.serverStats().resetNonMTLSLocalConnCount();
+    }
+
     public void resetStatistics() {
     public void resetStatistics() {
         ServerStats serverStats = zks.serverStats();
         ServerStats serverStats = zks.serverStats();
         serverStats.resetRequestCounters();
         serverStats.resetRequestCounters();
         serverStats.resetLatency();
         serverStats.resetLatency();
         serverStats.resetFsyncThresholdExceedCount();
         serverStats.resetFsyncThresholdExceedCount();
+        serverStats.resetAuthFailedCount();
+        serverStats.resetNonMTLSRemoteConnCount();
+        serverStats.resetNonMTLSLocalConnCount();
     }
     }
 
 
     public long getNumAliveConnections() {
     public long getNumAliveConnections() {

+ 20 - 0
zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMXBean.java

@@ -59,6 +59,18 @@ public interface ZooKeeperServerMXBean {
      * @return number of fsync threshold exceeds so far
      * @return number of fsync threshold exceeds so far
      */
      */
     long getFsyncThresholdExceedCount();
     long getFsyncThresholdExceedCount();
+    /**
+     * @return number of AuthFailedCount so far
+     */
+    long getAuthFailedCount();
+    /**
+     * @return number of NonMTLSLocalConnCount so far
+     */
+    long getNonMTLSLocalConnCount();
+    /**
+     * @return number of NonMTLSRemoteConnCount so far
+     */
+    long getNonMTLSRemoteConnCount();
     /**
     /**
      * @return number of outstanding requests.
      * @return number of outstanding requests.
      */
      */
@@ -167,6 +179,14 @@ public interface ZooKeeperServerMXBean {
      * Reset Fsync Threshold Exceed Count statistics only.
      * Reset Fsync Threshold Exceed Count statistics only.
      */
      */
     void resetFsyncThresholdExceedCount();
     void resetFsyncThresholdExceedCount();
+    /**
+     * Reset NonMTLS(Local+Remote)ConnCount statistics only.
+     */
+    void resetNonMTLSConnCount();
+    /**
+     * Reset AuthFailedCount statistics only.
+     */
+    void resetAuthFailedCount();
     /**
     /**
      * @return number of alive client connections
      * @return number of alive client connections
      */
      */

+ 5 - 0
zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMain.java

@@ -237,4 +237,9 @@ public class ZooKeeperServerMain {
         return cnxnFactory;
         return cnxnFactory;
     }
     }
 
 
+    // VisibleForTesting
+    ServerCnxnFactory getSecureCnxnFactory() {
+        return secureCnxnFactory;
+    }
+
 }
 }

+ 3 - 0
zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java

@@ -432,6 +432,9 @@ public class Commands {
      *   - "open_file_descriptor_count": Long (unix only)
      *   - "open_file_descriptor_count": Long (unix only)
      *   - "max_file_descriptor_count": Long (unix only)
      *   - "max_file_descriptor_count": Long (unix only)
      *   - "fsync_threshold_exceed_count": Long
      *   - "fsync_threshold_exceed_count": Long
+     *   - "non_mtls_conn_count": Long
+     *   - "non_mtls_remote_conn_count": Long
+     *   - "non_mtls_local_conn_count": Long
      *   - "followers": Integer (leader only)
      *   - "followers": Integer (leader only)
      *   - "synced_followers": Integer (leader only)
      *   - "synced_followers": Integer (leader only)
      *   - "pending_syncs": Integer (leader only)
      *   - "pending_syncs": Integer (leader only)

+ 0 - 1
zookeeper-server/src/test/java/org/apache/zookeeper/common/BCFKSFileLoaderTest.java

@@ -25,7 +25,6 @@ import java.security.KeyStore;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.MethodSource;
 import org.junit.jupiter.params.provider.MethodSource;
 
 
-
 public class BCFKSFileLoaderTest extends BaseX509ParameterizedTestCase {
 public class BCFKSFileLoaderTest extends BaseX509ParameterizedTestCase {
 
 
 
 

+ 59 - 0
zookeeper-server/src/test/java/org/apache/zookeeper/server/NettyServerCnxnTest.java

@@ -27,7 +27,17 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
 import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelId;
+import io.netty.channel.ChannelPipeline;
+import io.netty.util.Attribute;
 import java.io.IOException;
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.net.ProtocolException;
 import java.net.ProtocolException;
 import java.nio.charset.StandardCharsets;
 import java.nio.charset.StandardCharsets;
 import java.util.Random;
 import java.util.Random;
@@ -43,6 +53,7 @@ import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.common.ClientX509Util;
 import org.apache.zookeeper.common.ClientX509Util;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.server.quorum.BufferStats;
 import org.apache.zookeeper.server.quorum.BufferStats;
+import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer;
 import org.apache.zookeeper.test.ClientBase;
 import org.apache.zookeeper.test.ClientBase;
 import org.apache.zookeeper.test.SSLAuthTest;
 import org.apache.zookeeper.test.SSLAuthTest;
 import org.apache.zookeeper.test.TestByteBufAllocator;
 import org.apache.zookeeper.test.TestByteBufAllocator;
@@ -50,6 +61,7 @@ import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.Timeout;
 import org.junit.jupiter.api.Timeout;
+import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
@@ -152,6 +164,53 @@ public class NettyServerCnxnTest extends ClientBase {
         }
         }
     }
     }
 
 
+    @Test
+    public void testNonMTLSLocalConn() throws IOException, InterruptedException, KeeperException {
+        try (ZooKeeper zk = createClient()) {
+            ServerStats serverStats = serverFactory.getZooKeeperServer().serverStats();
+            //2 for local stat connection and this client
+            assertEquals(2, serverStats.getNonMTLSLocalConnCount());
+            assertEquals(0, serverStats.getNonMTLSRemoteConnCount());
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testNonMTLSRemoteConn() throws Exception {
+        Channel channel = mock(Channel.class);
+        ChannelId id = mock(ChannelId.class);
+        ChannelFuture success = mock(ChannelFuture.class);
+        ChannelHandlerContext context = mock(ChannelHandlerContext.class);
+        ChannelPipeline channelPipeline = mock(ChannelPipeline.class);
+
+        when(context.channel()).thenReturn(channel);
+        when(channel.pipeline()).thenReturn(channelPipeline);
+        when(success.channel()).thenReturn(channel);
+        when(channel.closeFuture()).thenReturn(success);
+
+        InetSocketAddress address = new InetSocketAddress(0);
+        when(channel.remoteAddress()).thenReturn(address);
+        when(channel.id()).thenReturn(id);
+        NettyServerCnxnFactory factory = new NettyServerCnxnFactory();
+        LeaderZooKeeperServer zks = mock(LeaderZooKeeperServer.class);
+        factory.setZooKeeperServer(zks);
+        Attribute atr = mock(Attribute.class);
+        Mockito.doReturn(atr).when(channel).attr(
+                Mockito.any()
+        );
+        doNothing().when(atr).set(Mockito.any());
+
+        when(zks.isRunning()).thenReturn(true);
+
+        ServerStats.Provider providerMock = mock(ServerStats.Provider.class);
+        when(zks.serverStats()).thenReturn(new ServerStats(providerMock));
+
+        factory.channelHandler.channelActive(context);
+
+        assertEquals(0, zks.serverStats().getNonMTLSLocalConnCount());
+        assertEquals(1, zks.serverStats().getNonMTLSRemoteConnCount());
+    }
+
     @Test
     @Test
     public void testServerSideThrottling() throws IOException, InterruptedException, KeeperException {
     public void testServerSideThrottling() throws IOException, InterruptedException, KeeperException {
         try (ZooKeeper zk = createClient()) {
         try (ZooKeeper zk = createClient()) {

+ 94 - 0
zookeeper-server/src/test/java/org/apache/zookeeper/server/X509AuthFailureTest.java

@@ -0,0 +1,94 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.zookeeper.server;
+
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.client.ZKClientConfig;
+import org.apache.zookeeper.common.ClientX509Util;
+import org.apache.zookeeper.test.ClientBase;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class X509AuthFailureTest {
+    protected static final Logger LOG = LoggerFactory.getLogger(X509AuthFailureTest.class);
+
+    private static ClientX509Util clientX509Util;
+    public static final int TIMEOUT = 5000;
+    public static int CONNECTION_TIMEOUT = 30000;
+
+    @BeforeEach
+    public void setup() throws Exception{
+        clientX509Util = new ClientX509Util();
+        String testDataPath = System.getProperty("test.data.dir", "src/test/resources/data");
+        System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, "org.apache.zookeeper.server.NettyServerCnxnFactory");
+        System.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, "org.apache.zookeeper.ClientCnxnSocketNetty");
+        System.setProperty(ZKClientConfig.SECURE_CLIENT, "true");
+        System.setProperty(clientX509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks");
+        System.setProperty(clientX509Util.getSslKeystorePasswdProperty(), "testpass");
+    }
+
+    @AfterEach
+    public void teardown() throws Exception {
+        System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
+        System.clearProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET);
+        System.clearProperty(ZKClientConfig.SECURE_CLIENT);
+        System.clearProperty(clientX509Util.getSslKeystoreLocationProperty());
+        System.clearProperty(clientX509Util.getSslKeystorePasswdProperty());
+        System.clearProperty(clientX509Util.getSslTruststoreLocationProperty());
+        System.clearProperty(clientX509Util.getSslTruststorePasswdProperty());
+        clientX509Util.close();
+    }
+
+    /**
+     * Developers might use standalone mode (which is the default for one server).
+     * This test checks metrics for authz failure in standalone server
+     */
+    @Test
+    public void testSecureStandaloneServerAuthNFailure() throws Exception {
+        final Integer CLIENT_PORT = PortAssignment.unique();
+        final Integer SECURE_CLIENT_PORT = PortAssignment.unique();
+
+        ZooKeeperServerMainTest.MainThread mt = new ZooKeeperServerMainTest.MainThread(CLIENT_PORT, SECURE_CLIENT_PORT, true, null);
+        mt.start();
+
+        try {
+            ZooKeeper zk = createZKClnt("127.0.0.1:" + SECURE_CLIENT_PORT);
+            fail("should not be reached");
+        } catch (Exception e){
+            //Expected
+        }
+        ServerStats serverStats = mt.getSecureCnxnFactory().getZooKeeperServer().serverStats();
+        assertTrue(serverStats.getAuthFailedCount() >= 1);
+        mt.shutdown();
+
+    }
+
+    private ZooKeeper createZKClnt(String cxnString) throws Exception {
+        ClientBase.CountdownWatcher watcher = new ClientBase.CountdownWatcher();
+        ZooKeeper zk = new ZooKeeper(cxnString, TIMEOUT, watcher);
+        watcher.waitForConnected(CONNECTION_TIMEOUT);
+        return zk;
+    }
+
+}

+ 18 - 4
zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerMainTest.java

@@ -72,10 +72,16 @@ public class ZooKeeperServerMainTest extends ZKTestCase implements Watcher {
         final File logDir;
         final File logDir;
 
 
         public MainThread(int clientPort, boolean preCreateDirs, String configs) throws IOException {
         public MainThread(int clientPort, boolean preCreateDirs, String configs) throws IOException {
-            this(clientPort, preCreateDirs, ClientBase.createTmpDir(), configs);
+            this(clientPort, null, preCreateDirs, ClientBase.createTmpDir(), configs);
         }
         }
 
 
-        public MainThread(int clientPort, boolean preCreateDirs, File tmpDir, String configs) throws IOException {
+        public MainThread(int clientPort, Integer secureClientPort, boolean preCreateDirs, String configs)
+                throws  IOException {
+            this(clientPort, secureClientPort,
+                    preCreateDirs, ClientBase.createTmpDir(), configs);
+        }
+
+        public MainThread(int clientPort, Integer secureClientPort, boolean preCreateDirs, File tmpDir, String configs) throws IOException {
             super("Standalone server with clientPort:" + clientPort);
             super("Standalone server with clientPort:" + clientPort);
             this.tmpDir = tmpDir;
             this.tmpDir = tmpDir;
             confFile = new File(tmpDir, "zoo.cfg");
             confFile = new File(tmpDir, "zoo.cfg");
@@ -105,6 +111,10 @@ public class ZooKeeperServerMainTest extends ZKTestCase implements Watcher {
             fwriter.write("dataDir=" + normalizedDataDir + "\n");
             fwriter.write("dataDir=" + normalizedDataDir + "\n");
             fwriter.write("dataLogDir=" + normalizedLogDir + "\n");
             fwriter.write("dataLogDir=" + normalizedLogDir + "\n");
             fwriter.write("clientPort=" + clientPort + "\n");
             fwriter.write("clientPort=" + clientPort + "\n");
+
+            if (secureClientPort != null) {
+                fwriter.write("secureClientPort=" + secureClientPort + "\n");
+            }
             fwriter.flush();
             fwriter.flush();
             fwriter.close();
             fwriter.close();
 
 
@@ -148,6 +158,10 @@ public class ZooKeeperServerMainTest extends ZKTestCase implements Watcher {
             return main.getCnxnFactory();
             return main.getCnxnFactory();
         }
         }
 
 
+        public ServerCnxnFactory getSecureCnxnFactory(){
+            return main.getSecureCnxnFactory();
+        }
+
     }
     }
 
 
     public static class TestZKSMain extends ZooKeeperServerMain {
     public static class TestZKSMain extends ZooKeeperServerMain {
@@ -234,7 +248,7 @@ public class ZooKeeperServerMainTest extends ZKTestCase implements Watcher {
         snapDir.setWritable(false);
         snapDir.setWritable(false);
 
 
         // Restart ZK and observe a failure
         // Restart ZK and observe a failure
-        main = new MainThread(CLIENT_PORT, false, tmpDir, null);
+        main = new MainThread(CLIENT_PORT, null, false, tmpDir, null);
         main.start();
         main.start();
 
 
         assertFalse(ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT, CONNECTION_TIMEOUT / 2),
         assertFalse(ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT, CONNECTION_TIMEOUT / 2),
@@ -273,7 +287,7 @@ public class ZooKeeperServerMainTest extends ZKTestCase implements Watcher {
         logDir.setWritable(false);
         logDir.setWritable(false);
 
 
         // Restart ZK and observe a failure
         // Restart ZK and observe a failure
-        main = new MainThread(CLIENT_PORT, false, tmpDir, null);
+        main = new MainThread(CLIENT_PORT, null, false, tmpDir, null);
         main.start();
         main.start();
 
 
         assertFalse(ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT, CONNECTION_TIMEOUT / 2),
         assertFalse(ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT, CONNECTION_TIMEOUT / 2),

+ 3 - 0
zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/CommandsTest.java

@@ -165,6 +165,9 @@ public class CommandsTest extends ClientBase {
                 new Field("last_client_response_size", Integer.class),
                 new Field("last_client_response_size", Integer.class),
                 new Field("max_client_response_size", Integer.class),
                 new Field("max_client_response_size", Integer.class),
                 new Field("min_client_response_size", Integer.class),
                 new Field("min_client_response_size", Integer.class),
+                new Field("auth_failed_count", Long.class),
+                new Field("non_mtls_remote_conn_count", Long.class),
+                new Field("non_mtls_local_conn_count", Long.class),
                 new Field("uptime", Long.class),
                 new Field("uptime", Long.class),
                 new Field("global_sessions", Long.class),
                 new Field("global_sessions", Long.class),
                 new Field("local_sessions", Long.class),
                 new Field("local_sessions", Long.class),