You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by sy...@apache.org on 2020/08/12 06:49:47 UTC

[zookeeper] branch branch-3.5 updated: ZOOKEEPER-3905: Race condition causes sessions to be created for clients even though their certificate authentication has failed (3.5)

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

symat pushed a commit to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/zookeeper.git


The following commit(s) were added to refs/heads/branch-3.5 by this push:
     new f177161  ZOOKEEPER-3905: Race condition causes sessions to be created for clients even though their certificate authentication has failed (3.5)
f177161 is described below

commit f177161172cddb6c771b7b766eb42db5ce9dc445
Author: Andor Molnar <an...@apache.org>
AuthorDate: Wed Aug 12 06:23:06 2020 +0000

    ZOOKEEPER-3905: Race condition causes sessions to be created for clients even though their certificate authentication has failed (3.5)
    
    Backport to branch-3.5
    
    Netty channel doesn't get closed if authentication fails after a successful SSL handshake. We need a custom authentication provider in order to trigger this, because the default implementation does the same check as for the SSL handshake. Hence it never fails.
    
    Unit test added to make sure client is not able to connect.
    
    Author: Andor Molnar <an...@apache.org>
    
    Reviewers: Enrico Olivelli <eo...@apache.org>, Mate Szalay-Beko <sy...@apache.org>, Norbert Kalmar <nk...@apache.org>
    
    Closes #1427 from anmolnar/ZOOKEEPER-3905_36
    
    (cherry picked from commit 8bcaf7bb3cfa6470e1660e2b36964ae2284197df)
    Signed-off-by: Mate Szalay-Beko <sy...@apache.org>
---
 .../apache/zookeeper/server/NettyServerCnxn.java   |  3 +
 .../zookeeper/server/auth/ProviderRegistry.java    | 13 ++--
 .../zookeeper/server/quorum/QuorumPeerConfig.java  | 76 +++++++++++-----------
 .../test/AuthFailX509AuthenticationProvider.java   | 51 +++++++++++++++
 .../java/org/apache/zookeeper/test/ClientBase.java |  6 +-
 .../org/apache/zookeeper/test/ClientSSLTest.java   | 30 ++++++++-
 6 files changed, 135 insertions(+), 44 deletions(-)

diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxn.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxn.java
index e6f3495..623f6b1 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxn.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxn.java
@@ -102,6 +102,9 @@ public class NettyServerCnxn extends ServerCnxn {
             if (LOG.isDebugEnabled()) {
                 LOG.debug("cnxns size:{}", factory.cnxns.size());
             }
+            if (channel.isOpen()) {
+                channel.close();
+            }
             return;
         }
         if (LOG.isDebugEnabled()) {
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/ProviderRegistry.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/ProviderRegistry.java
index 9bce71e..a8ecd58 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/ProviderRegistry.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/ProviderRegistry.java
@@ -20,6 +20,7 @@ package org.apache.zookeeper.server.auth;
 
 import java.util.Enumeration;
 import java.util.HashMap;
+import java.util.Map;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -29,9 +30,11 @@ import org.apache.zookeeper.server.ZooKeeperServer;
 public class ProviderRegistry {
     private static final Logger LOG = LoggerFactory.getLogger(ProviderRegistry.class);
 
+    public static final String AUTHPROVIDER_PROPERTY_PREFIX = "zookeeper.authProvider.";
+
     private static boolean initialized = false;
-    private static HashMap<String, AuthenticationProvider> authenticationProviders =
-        new HashMap<String, AuthenticationProvider>();
+    private static final Map<String, AuthenticationProvider> authenticationProviders = new HashMap<>();
+
 
     public static void initialize() {
         synchronized (ProviderRegistry.class) {
@@ -44,7 +47,7 @@ public class ProviderRegistry {
             Enumeration<Object> en = System.getProperties().keys();
             while (en.hasMoreElements()) {
                 String k = (String) en.nextElement();
-                if (k.startsWith("zookeeper.authProvider.")) {
+                if (k.startsWith(AUTHPROVIDER_PROPERTY_PREFIX)) {
                     String className = System.getProperty(k);
                     try {
                         Class<?> c = ZooKeeperServer.class.getClassLoader()
@@ -70,8 +73,8 @@ public class ProviderRegistry {
     public static String listProviders() {
         StringBuilder sb = new StringBuilder();
         for(String s: authenticationProviders.keySet()) {
-        sb.append(s + " ");
-}
+            sb.append(s + " ");
+        }
         return sb.toString();
     }
 }
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
index d32806f..f05ab3f 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
@@ -47,6 +47,7 @@ import org.apache.zookeeper.common.AtomicFileWritingIdiom.OutputStreamStatement;
 import org.apache.zookeeper.common.AtomicFileWritingIdiom.WriterStatement;
 import org.apache.zookeeper.common.PathUtils;
 import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.zookeeper.server.auth.ProviderRegistry;
 import org.apache.zookeeper.server.quorum.QuorumPeer.LearnerType;
 import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
 import org.apache.zookeeper.server.quorum.auth.QuorumAuth;
@@ -133,13 +134,13 @@ public class QuorumPeerConfig {
      */
     public void parse(String path) throws ConfigException {
         LOG.info("Reading configuration from: " + path);
-       
+
         try {
             File configFile = (new VerifyingFileFactory.Builder(LOG)
                 .warnForRelativePath()
                 .failForNonExistingPath()
                 .build()).create(path);
-                
+
             Properties cfg = new Properties();
             FileInputStream in = new FileInputStream(configFile);
             try {
@@ -148,16 +149,16 @@ public class QuorumPeerConfig {
             } finally {
                 in.close();
             }
-            
+
             parseProperties(cfg);
         } catch (IOException e) {
             throw new ConfigException("Error processing " + path, e);
         } catch (IllegalArgumentException e) {
             throw new ConfigException("Error processing " + path, e);
-        }   
-        
+        }
+
         if (dynamicConfigFileStr!=null) {
-           try {           
+           try {
                Properties dynamicCfg = new Properties();
                FileInputStream inConfig = new FileInputStream(dynamicConfigFileStr);
                try {
@@ -181,12 +182,12 @@ public class QuorumPeerConfig {
                throw new ConfigException("Error processing " + dynamicConfigFileStr, e);
            } catch (IllegalArgumentException e) {
                throw new ConfigException("Error processing " + dynamicConfigFileStr, e);
-           }        
+           }
            File nextDynamicConfigFile = new File(configFileStr + nextDynamicConfigFileSuffix);
            if (nextDynamicConfigFile.exists()) {
-               try {           
+               try {
                    Properties dynamicConfigNextCfg = new Properties();
-                   FileInputStream inConfigNext = new FileInputStream(nextDynamicConfigFile);       
+                   FileInputStream inConfigNext = new FileInputStream(nextDynamicConfigFile);
                    try {
                        dynamicConfigNextCfg.load(inConfigNext);
                    } finally {
@@ -194,7 +195,7 @@ public class QuorumPeerConfig {
                    }
                    boolean isHierarchical = false;
                    for (Entry<Object, Object> entry : dynamicConfigNextCfg.entrySet()) {
-                       String key = entry.getKey().toString().trim();  
+                       String key = entry.getKey().toString().trim();
                        if (key.startsWith("group") || key.startsWith("weight")) {
                            isHierarchical = true;
                            break;
@@ -414,7 +415,7 @@ public class QuorumPeerConfig {
         if (minSessionTimeout > maxSessionTimeout) {
             throw new IllegalArgumentException(
                     "minSessionTimeout must not be larger than maxSessionTimeout");
-        }          
+        }
 
         // backward compatibility - dynamic configuration in the same file as
         // static configuration params see writeDynamicConfig()
@@ -430,18 +431,19 @@ public class QuorumPeerConfig {
 
     /**
      * Configure SSL authentication only if it is not configured.
-     * 
+     *
      * @throws ConfigException
      *             If authentication scheme is configured but authentication
      *             provider is not configured.
      */
     public static void configureSSLAuth() throws ConfigException {
         try (ClientX509Util clientX509Util = new ClientX509Util()) {
-            String sslAuthProp = "zookeeper.authProvider." + System.getProperty(clientX509Util.getSslAuthProviderProperty(), "x509");
+            String sslAuthProp = ProviderRegistry.AUTHPROVIDER_PROPERTY_PREFIX
+                                 + System.getProperty(clientX509Util.getSslAuthProviderProperty(), "x509");
             if (System.getProperty(sslAuthProp) == null) {
-                if ("zookeeper.authProvider.x509".equals(sslAuthProp)) {
-                    System.setProperty("zookeeper.authProvider.x509",
-                            "org.apache.zookeeper.server.auth.X509AuthenticationProvider");
+                if ((ProviderRegistry.AUTHPROVIDER_PROPERTY_PREFIX + "x509").equals(sslAuthProp)) {
+                    System.setProperty(ProviderRegistry.AUTHPROVIDER_PROPERTY_PREFIX + "x509",
+                        "org.apache.zookeeper.server.auth.X509AuthenticationProvider");
                 } else {
                     throw new ConfigException("No auth provider configured for the SSL authentication scheme '"
                             + System.getProperty(clientX509Util.getSslAuthProviderProperty()) + "'.");
@@ -533,7 +535,7 @@ public class QuorumPeerConfig {
                 .warnForRelativePath()
                 .failForNonExistingPath()
                 .build()).create(dynamicFileStr);
-        
+
         final Properties cfg = new Properties();
         FileInputStream in = new FileInputStream(configFile);
         try {
@@ -577,25 +579,25 @@ public class QuorumPeerConfig {
        if (filename == null) return;
        File f = new File(filename);
        if (f.exists()) {
-           try{ 
+           try{
                f.delete();
            } catch (Exception e) {
                LOG.warn("deleting " + filename + " failed");
            }
-       }                   
+       }
     }
-    
-    
+
+
     private static QuorumVerifier createQuorumVerifier(Properties dynamicConfigProp, boolean isHierarchical) throws ConfigException{
        if(isHierarchical){
             return new QuorumHierarchical(dynamicConfigProp);
         } else {
            /*
              * The default QuorumVerifier is QuorumMaj
-             */        
+             */
             //LOG.info("Defaulting to majority quorums");
-            return new QuorumMaj(dynamicConfigProp);            
-        }          
+            return new QuorumMaj(dynamicConfigProp);
+        }
     }
 
     void setupQuorumPeerConfig(Properties prop, boolean configBackwardCompatibilityMode)
@@ -618,17 +620,17 @@ public class QuorumPeerConfig {
 	   boolean configBackwardCompatibilityMode) throws IOException, ConfigException {
        boolean isHierarchical = false;
         for (Entry<Object, Object> entry : dynamicConfigProp.entrySet()) {
-            String key = entry.getKey().toString().trim();                    
+            String key = entry.getKey().toString().trim();
             if (key.startsWith("group") || key.startsWith("weight")) {
                isHierarchical = true;
-            } else if (!configBackwardCompatibilityMode && !key.startsWith("server.") && !key.equals("version")){ 
+            } else if (!configBackwardCompatibilityMode && !key.startsWith("server.") && !key.equals("version")){
                LOG.info(dynamicConfigProp.toString());
-               throw new ConfigException("Unrecognised parameter: " + key);                
+               throw new ConfigException("Unrecognised parameter: " + key);
             }
         }
-        
+
         QuorumVerifier qv = createQuorumVerifier(dynamicConfigProp, isHierarchical);
-               
+
         int numParticipators = qv.getVotingMembers().size();
         int numObservers = qv.getObservingMembers().size();
         if (numParticipators == 0) {
@@ -660,14 +662,14 @@ public class QuorumPeerConfig {
             /*
              * If using FLE, then every server requires a separate election
              * port.
-             */            
+             */
            if (eAlg != 0) {
                for (QuorumServer s : qv.getVotingMembers().values()) {
                    if (s.electionAddr == null)
                        throw new IllegalArgumentException(
                                "Missing election port for server: " + s.id);
                }
-           }   
+           }
         }
         return qv;
     }
@@ -770,7 +772,7 @@ public class QuorumPeerConfig {
     public int getPurgeInterval() {
         return purgeInterval;
     }
-    
+
     public boolean getSyncEnabled() {
         return syncEnabled;
     }
@@ -778,8 +780,8 @@ public class QuorumPeerConfig {
     public QuorumVerifier getQuorumVerifier() {
         return quorumVerifier;
     }
-    
-    public QuorumVerifier getLastSeenQuorumVerifier() {   
+
+    public QuorumVerifier getLastSeenQuorumVerifier() {
         return lastSeenQuorumVerifier;
     }
 
@@ -801,15 +803,15 @@ public class QuorumPeerConfig {
     public String getConfigFilename(){
         return configFileStr;
     }
-    
+
     public Boolean getQuorumListenOnAllIPs() {
         return quorumListenOnAllIPs;
     }
- 
+
     public static boolean isStandaloneEnabled() {
 	return standaloneEnabled;
     }
-    
+
     public static void setStandaloneEnabled(boolean enabled) {
         standaloneEnabled = enabled;
     }
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/AuthFailX509AuthenticationProvider.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/AuthFailX509AuthenticationProvider.java
new file mode 100644
index 0000000..32121e6
--- /dev/null
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/AuthFailX509AuthenticationProvider.java
@@ -0,0 +1,51 @@
+/*
+ * 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.zookeeper.test;
+
+import javax.net.ssl.X509KeyManager;
+import javax.net.ssl.X509TrustManager;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.common.X509Exception;
+import org.apache.zookeeper.server.ServerCnxn;
+import org.apache.zookeeper.server.auth.X509AuthenticationProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AuthFailX509AuthenticationProvider extends X509AuthenticationProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(AuthFailX509AuthenticationProvider.class);
+
+  public AuthFailX509AuthenticationProvider() throws X509Exception {
+    super();
+  }
+
+  public AuthFailX509AuthenticationProvider(X509TrustManager trustManager, X509KeyManager keyManager) {
+    super(trustManager, keyManager);
+  }
+
+  @Override
+  public KeeperException.Code handleAuthentication(ServerCnxn cnxn, byte[] authData) {
+    LOG.info("Authentication failed");
+    return KeeperException.Code.AUTHFAILED;
+  }
+
+  @Override
+  public String getScheme() {
+    return "authfail";
+  }
+}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientBase.java
index 8111099..7a5311d 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientBase.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientBase.java
@@ -763,10 +763,14 @@ public abstract class ClientBase extends ZKTestCase {
      *             in cases of network failure
      */
     public static ZooKeeper createZKClient(String cxnString, int sessionTimeout) throws IOException {
+        return createZKClient(cxnString, sessionTimeout, CONNECTION_TIMEOUT);
+    }
+
+    public static ZooKeeper createZKClient(String cxnString, int sessionTimeout, long connectionTimeout) throws IOException {
         CountdownWatcher watcher = new CountdownWatcher();
         ZooKeeper zk = new ZooKeeper(cxnString, sessionTimeout, watcher);
         try {
-            watcher.waitForConnected(CONNECTION_TIMEOUT);
+            watcher.waitForConnected(connectionTimeout);
         } catch (InterruptedException | TimeoutException e) {
             Assert.fail("ZooKeeper client can not connect to " + cxnString);
         }
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java
index 019ec5f..0d50bc9 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java
@@ -21,7 +21,8 @@
  */
 package org.apache.zookeeper.test;
 
-
+import static org.junit.Assert.fail;
+import java.io.IOException;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.ZooDefs;
@@ -30,16 +31,22 @@ import org.apache.zookeeper.client.ZKClientConfig;
 import org.apache.zookeeper.common.ClientX509Util;
 import org.apache.zookeeper.server.NettyServerCnxnFactory;
 import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.auth.ProviderRegistry;
 import org.apache.zookeeper.server.quorum.QuorumPeerTestBase;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 
 public class ClientSSLTest extends QuorumPeerTestBase {
 
     private ClientX509Util clientX509Util;
 
+    @Rule
+    public ExpectedException exceptionRule = ExpectedException.none();
+
     @Before
     public void setup() {
         System.setProperty(NettyServerCnxnFactory.PORT_UNIFICATION_KEY, Boolean.TRUE.toString());
@@ -158,4 +165,25 @@ public class ClientSSLTest extends QuorumPeerTestBase {
         zk.close();
         mt.shutdown();
     }
+
+    @Test
+    public void testSecureStandaloneServerAuthFail() throws IOException {
+        exceptionRule.expect(AssertionError.class);
+        exceptionRule.expectMessage("ZooKeeper client can not connect");
+        try {
+            System.setProperty(ProviderRegistry.AUTHPROVIDER_PROPERTY_PREFIX + "authfail",
+                "org.apache.zookeeper.test.AuthFailX509AuthenticationProvider");
+            System.setProperty(clientX509Util.getSslAuthProviderProperty(), "authfail");
+
+            Integer secureClientPort = PortAssignment.unique();
+            MainThread mt = new MainThread(MainThread.UNSET_MYID, "", secureClientPort, false);
+            mt.start();
+
+            ClientBase.createZKClient("localhost:" + secureClientPort, TIMEOUT, 3000);
+            fail("Client should not able to connect to this server, because auth failed");
+        } finally {
+            System.clearProperty(ProviderRegistry.AUTHPROVIDER_PROPERTY_PREFIX + "authfail");
+            System.clearProperty(clientX509Util.getSslAuthProviderProperty());
+        }
+    }
 }