You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by ivmaykov <gi...@git.apache.org> on 2018/10/25 02:28:08 UTC

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

GitHub user ivmaykov opened a pull request:

    https://github.com/apache/zookeeper/pull/679

    ZOOKEEPER-3172: Quorum TLS - fix port unification to allow rolling upgrades

    Fix numerous problems with UnifiedServerSocket, such as hanging the accept() thread when the client doesn't send any data or crashing if less than 5 bytes are read from the socket in the initial read.
    
    Re-enable the "portUnification" config option.
    
    Note that this is stacked on top of #678 and thus includes it. Please only consider the ZOOKEEPER-3172 commit when reviewing.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ivmaykov/zookeeper ZOOKEEPER-3172

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/679.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #679
    
----
commit b8b687ae4dea912ef18ee2ee1ace406800f3fce7
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T00:41:48Z

    ZOOKEEPER-3173: Quorum TLS - support PEM trust/key stores
    ZOOKEEPER-3175: Quorum TLS - test improvements
    
    Add support for loading key and trust stores from PEM files.
    Also added test utils for testing X509-related code, because it
    was very difficult to untangle them from the PEM support code.

commit f9fb9c69f15f4d23acc714de75efe4592c6578b9
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T01:22:24Z

    ZOOKEEPER-3172: Quorum TLS - fix port unification to allow rolling upgrades

----


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2564/



---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    Don't use UnifiedServerSocket for the non-port-unification path in this PR, since cert reloading is not yet available here.


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    Thanks @ivmaykov !
    Committed to master branch.
    Please create separate PR for 3.5, because it has some merge conflicts.


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2598/



---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov closed the pull request at:

    https://github.com/apache/zookeeper/pull/679


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r233661917
  
    --- Diff: zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java ---
    @@ -17,156 +17,644 @@
      */
     package org.apache.zookeeper.server.quorum;
     
    +import java.io.BufferedInputStream;
    +import java.io.IOException;
    +import java.net.ConnectException;
    +import java.net.InetSocketAddress;
    +import java.net.Socket;
    +import java.net.SocketException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Random;
    +
    +import javax.net.ssl.HandshakeCompletedEvent;
    +import javax.net.ssl.HandshakeCompletedListener;
    +import javax.net.ssl.SSLSocket;
    +
     import org.apache.zookeeper.PortAssignment;
     import org.apache.zookeeper.client.ZKClientConfig;
    +import org.apache.zookeeper.common.BaseX509ParameterizedTestCase;
     import org.apache.zookeeper.common.ClientX509Util;
    -import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.common.KeyStoreFileType;
    +import org.apache.zookeeper.common.X509Exception;
    +import org.apache.zookeeper.common.X509KeyType;
    +import org.apache.zookeeper.common.X509TestContext;
     import org.apache.zookeeper.common.X509Util;
     import org.apache.zookeeper.server.ServerCnxnFactory;
     import org.junit.Assert;
     import org.junit.Before;
     import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
     
    -import javax.net.ssl.HandshakeCompletedEvent;
    -import javax.net.ssl.HandshakeCompletedListener;
    -import javax.net.ssl.SSLSocket;
    -import java.io.IOException;
    -import java.net.ConnectException;
    -import java.net.InetSocketAddress;
    -import java.net.Socket;
    -
    -import static org.hamcrest.CoreMatchers.equalTo;
    -import static org.junit.Assert.assertThat;
    +@RunWith(Parameterized.class)
    +public class UnifiedServerSocketTest extends BaseX509ParameterizedTestCase {
     
    -public class UnifiedServerSocketTest {
    +    @Parameterized.Parameters
    +    public static Collection<Object[]> params() {
    +        ArrayList<Object[]> result = new ArrayList<>();
    +        int paramIndex = 0;
    +        for (X509KeyType caKeyType : X509KeyType.values()) {
    +            for (X509KeyType certKeyType : X509KeyType.values()) {
    +                for (Boolean hostnameVerification : new Boolean[] { true, false  }) {
    +                    result.add(new Object[]{
    +                            caKeyType,
    +                            certKeyType,
    +                            hostnameVerification,
    +                            paramIndex++
    +                    });
    +                }
    +            }
    +        }
    +        return result;
    +    }
     
         private static final int MAX_RETRIES = 5;
         private static final int TIMEOUT = 1000;
    +    private static final byte[] DATA_TO_CLIENT = "hello client".getBytes();
    +    private static final byte[] DATA_FROM_CLIENT = "hello server".getBytes();
     
         private X509Util x509Util;
         private int port;
    -    private volatile boolean handshakeCompleted;
    +    private InetSocketAddress localServerAddress;
    +    private final Object handshakeCompletedLock = new Object();
    +    // access only inside synchronized(handshakeCompletedLock) { ... } blocks
    +    private boolean handshakeCompleted = false;
    +
    +    public UnifiedServerSocketTest(
    +            final X509KeyType caKeyType,
    +            final X509KeyType certKeyType,
    +            final Boolean hostnameVerification,
    +            final Integer paramIndex) {
    +        super(paramIndex, () -> {
    +            try {
    +                return X509TestContext.newBuilder()
    +                    .setTempDir(tempDir)
    +                    .setKeyStoreKeyType(certKeyType)
    +                    .setTrustStoreKeyType(caKeyType)
    +                    .setHostnameVerification(hostnameVerification)
    +                    .build();
    +            } catch (Exception e) {
    +                throw new RuntimeException(e);
    +            }
    +        });
    +    }
     
         @Before
         public void setUp() throws Exception {
    -        handshakeCompleted = false;
    -
             port = PortAssignment.unique();
    +        localServerAddress = new InetSocketAddress("localhost", port);
    --- End diff --
    
    Thanks.


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    @anmolnar renamed the tests and cleaned them up quite a bit too (the formerly-DoS tests don't actually need a background thread for the bad client socket, which simplifies things).


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov closed the pull request at:

    https://github.com/apache/zookeeper/pull/679


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r233657773
  
    --- Diff: zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java ---
    @@ -17,156 +17,644 @@
      */
     package org.apache.zookeeper.server.quorum;
     
    +import java.io.BufferedInputStream;
    +import java.io.IOException;
    +import java.net.ConnectException;
    +import java.net.InetSocketAddress;
    +import java.net.Socket;
    +import java.net.SocketException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Random;
    +
    +import javax.net.ssl.HandshakeCompletedEvent;
    +import javax.net.ssl.HandshakeCompletedListener;
    +import javax.net.ssl.SSLSocket;
    +
     import org.apache.zookeeper.PortAssignment;
     import org.apache.zookeeper.client.ZKClientConfig;
    +import org.apache.zookeeper.common.BaseX509ParameterizedTestCase;
     import org.apache.zookeeper.common.ClientX509Util;
    -import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.common.KeyStoreFileType;
    +import org.apache.zookeeper.common.X509Exception;
    +import org.apache.zookeeper.common.X509KeyType;
    +import org.apache.zookeeper.common.X509TestContext;
     import org.apache.zookeeper.common.X509Util;
     import org.apache.zookeeper.server.ServerCnxnFactory;
     import org.junit.Assert;
     import org.junit.Before;
     import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
     
    -import javax.net.ssl.HandshakeCompletedEvent;
    -import javax.net.ssl.HandshakeCompletedListener;
    -import javax.net.ssl.SSLSocket;
    -import java.io.IOException;
    -import java.net.ConnectException;
    -import java.net.InetSocketAddress;
    -import java.net.Socket;
    -
    -import static org.hamcrest.CoreMatchers.equalTo;
    -import static org.junit.Assert.assertThat;
    +@RunWith(Parameterized.class)
    +public class UnifiedServerSocketTest extends BaseX509ParameterizedTestCase {
     
    -public class UnifiedServerSocketTest {
    +    @Parameterized.Parameters
    +    public static Collection<Object[]> params() {
    +        ArrayList<Object[]> result = new ArrayList<>();
    +        int paramIndex = 0;
    +        for (X509KeyType caKeyType : X509KeyType.values()) {
    +            for (X509KeyType certKeyType : X509KeyType.values()) {
    +                for (Boolean hostnameVerification : new Boolean[] { true, false  }) {
    +                    result.add(new Object[]{
    +                            caKeyType,
    +                            certKeyType,
    +                            hostnameVerification,
    +                            paramIndex++
    +                    });
    +                }
    +            }
    +        }
    +        return result;
    +    }
     
         private static final int MAX_RETRIES = 5;
         private static final int TIMEOUT = 1000;
    +    private static final byte[] DATA_TO_CLIENT = "hello client".getBytes();
    +    private static final byte[] DATA_FROM_CLIENT = "hello server".getBytes();
     
         private X509Util x509Util;
         private int port;
    -    private volatile boolean handshakeCompleted;
    +    private InetSocketAddress localServerAddress;
    +    private final Object handshakeCompletedLock = new Object();
    +    // access only inside synchronized(handshakeCompletedLock) { ... } blocks
    +    private boolean handshakeCompleted = false;
    +
    +    public UnifiedServerSocketTest(
    +            final X509KeyType caKeyType,
    +            final X509KeyType certKeyType,
    +            final Boolean hostnameVerification,
    +            final Integer paramIndex) {
    +        super(paramIndex, () -> {
    +            try {
    +                return X509TestContext.newBuilder()
    +                    .setTempDir(tempDir)
    +                    .setKeyStoreKeyType(certKeyType)
    +                    .setTrustStoreKeyType(caKeyType)
    +                    .setHostnameVerification(hostnameVerification)
    +                    .build();
    +            } catch (Exception e) {
    +                throw new RuntimeException(e);
    +            }
    +        });
    +    }
     
         @Before
         public void setUp() throws Exception {
    -        handshakeCompleted = false;
    -
             port = PortAssignment.unique();
    +        localServerAddress = new InetSocketAddress("localhost", port);
     
    -        String testDataPath = System.getProperty("test.data.dir", "build/test/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");
     
             x509Util = new ClientX509Util();
     
    -        System.setProperty(x509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks");
    -        System.setProperty(x509Util.getSslKeystorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks");
    -        System.setProperty(x509Util.getSslTruststorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslHostnameVerificationEnabledProperty(), "false");
    +        x509TestContext.setSystemProperties(x509Util, KeyStoreFileType.JKS, KeyStoreFileType.JKS);
         }
     
    -    @Test
    -    public void testConnectWithSSL() throws Exception {
    -        class ServerThread extends Thread {
    -            public void run() {
    -                try {
    -                    Socket unifiedSocket = new UnifiedServerSocket(x509Util, port).accept();
    -                    ((SSLSocket)unifiedSocket).getSession(); // block until handshake completes
    -                } catch (IOException e) {
    -                    e.printStackTrace();
    +    private static void forceClose(java.io.Closeable s) {
    +        if (s == null) {
    +            return;
    +        }
    +        try {
    +            s.close();
    +        } catch (IOException e) {
    +            e.printStackTrace();
    +        }
    +    }
    +
    +    private static final class UnifiedServerThread extends Thread {
    +        private final byte[] dataToClient;
    +        private List<byte[]> dataFromClients;
    +        private List<Thread> workerThreads;
    +        private UnifiedServerSocket serverSocket;
    +
    +        UnifiedServerThread(X509Util x509Util,
    +                            InetSocketAddress bindAddress,
    +                            boolean allowInsecureConnection,
    +                            byte[] dataToClient) throws IOException {
    +            this.dataToClient = dataToClient;
    +            dataFromClients = new ArrayList<>();
    +            workerThreads = new ArrayList<>();
    +            serverSocket = new UnifiedServerSocket(x509Util, allowInsecureConnection);
    +            serverSocket.bind(bindAddress);
    +        }
    +
    +        @Override
    +        public void run() {
    +            try {
    +                Random rnd = new Random();
    +                while (true) {
    +                    final Socket unifiedSocket = serverSocket.accept();
    +                    final boolean tcpNoDelay = rnd.nextBoolean();
    +                    unifiedSocket.setTcpNoDelay(tcpNoDelay);
    +                    unifiedSocket.setSoTimeout(TIMEOUT);
    +                    final boolean keepAlive = rnd.nextBoolean();
    +                    unifiedSocket.setKeepAlive(keepAlive);
    +                    // Note: getting the input stream should not block the thread or trigger mode detection.
    +                    BufferedInputStream bis = new BufferedInputStream(unifiedSocket.getInputStream());
    +                    Thread t = new Thread(new Runnable() {
    +                        @Override
    +                        public void run() {
    +                            try {
    +                                byte[] buf = new byte[1024];
    +                                int bytesRead = unifiedSocket.getInputStream().read(buf, 0, 1024);
    +                                // Make sure the settings applied above before the socket was potentially upgraded to
    +                                // TLS still apply.
    +                                Assert.assertEquals(tcpNoDelay, unifiedSocket.getTcpNoDelay());
    +                                Assert.assertEquals(TIMEOUT, unifiedSocket.getSoTimeout());
    +                                Assert.assertEquals(keepAlive, unifiedSocket.getKeepAlive());
    +                                if (bytesRead > 0) {
    +                                    byte[] dataFromClient = new byte[bytesRead];
    +                                    System.arraycopy(buf, 0, dataFromClient, 0, bytesRead);
    +                                    synchronized (dataFromClients) {
    +                                        dataFromClients.add(dataFromClient);
    +                                    }
    +                                }
    +                                unifiedSocket.getOutputStream().write(dataToClient);
    +                                unifiedSocket.getOutputStream().flush();
    +                            } catch (IOException e) {
    +                                e.printStackTrace();
    +                                throw new RuntimeException(e);
    +                            } finally {
    +                                forceClose(unifiedSocket);
    +                            }
    +                        }
    +                    });
    +                    workerThreads.add(t);
    +                    t.start();
                     }
    +            } catch (IOException e) {
    +                e.printStackTrace();
    +                throw new RuntimeException(e);
    +            } finally {
    +                forceClose(serverSocket);
                 }
             }
    -        ServerThread serverThread = new ServerThread();
    -        serverThread.start();
     
    +        public void shutdown(long millis) throws InterruptedException {
    +            forceClose(serverSocket); // this should break the run() loop
    +            for (Thread t : workerThreads) {
    +                t.join(millis);
    +            }
    +            this.join(millis);
    +        }
    +
    +        synchronized byte[] getDataFromClient(int index) {
    +            return dataFromClients.get(index);
    +        }
    +    }
    +
    +    private SSLSocket connectWithSSL() throws IOException, X509Exception, InterruptedException {
             SSLSocket sslSocket = null;
             int retries = 0;
             while (retries < MAX_RETRIES) {
                 try {
                     sslSocket = x509Util.createSSLSocket();
    +                sslSocket.addHandshakeCompletedListener(new HandshakeCompletedListener() {
    +                    @Override
    +                    public void handshakeCompleted(HandshakeCompletedEvent handshakeCompletedEvent) {
    +                        synchronized (handshakeCompletedLock) {
    +                            handshakeCompleted = true;
    +                            handshakeCompletedLock.notifyAll();
    +                        }
    +                    }
    +                });
                     sslSocket.setSoTimeout(TIMEOUT);
    -                sslSocket.connect(new InetSocketAddress(port), TIMEOUT);
    +                sslSocket.connect(localServerAddress, TIMEOUT);
                     break;
                 } catch (ConnectException connectException) {
                     connectException.printStackTrace();
    +                forceClose(sslSocket);
    +                sslSocket = null;
                     Thread.sleep(TIMEOUT);
                 }
                 retries++;
             }
     
    -        sslSocket.addHandshakeCompletedListener(new HandshakeCompletedListener() {
    -            @Override
    -            public void handshakeCompleted(HandshakeCompletedEvent handshakeCompletedEvent) {
    -                completeHandshake();
    +        Assert.assertNotNull("Failed to connect to server with SSL", sslSocket);
    +        return sslSocket;
    +    }
    +
    +    private Socket connectWithoutSSL() throws IOException, InterruptedException {
    +        Socket socket = null;
    +        int retries = 0;
    +        while (retries < MAX_RETRIES) {
    +            try {
    +                socket = new Socket();
    +                socket.setSoTimeout(TIMEOUT);
    +                socket.connect(localServerAddress, TIMEOUT);
    +                break;
    +            } catch (ConnectException connectException) {
    +                connectException.printStackTrace();
    +                forceClose(socket);
    +                socket = null;
    +                Thread.sleep(TIMEOUT);
                 }
    -        });
    -        sslSocket.startHandshake();
    +            retries++;
    +        }
    +        Assert.assertNotNull("Failed to connect to server without SSL", socket);
    +        return socket;
    +    }
    +
    +    // In the tests below, a "Strict" server means a UnifiedServerSocket that
    +    // does not allow plaintext connections (in other words, it's SSL-only).
    +    // A "Non Strict" server means a UnifiedServerSocket that allows both
    +    // plaintext and SSL incoming connections.
    +
    +    /**
    +     * Attempting to connect to a SSL-or-plaintext server with SSL should work.
    +     */
    +    @Test
    +    public void testConnectWithSSLToNonStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, true, DATA_TO_CLIENT);
    +        serverThread.start();
     
    -        serverThread.join(TIMEOUT);
    +        Socket sslSocket = connectWithSSL();
    +        sslSocket.getOutputStream().write(DATA_FROM_CLIENT);
    +        sslSocket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = sslSocket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
     
    -        long start = Time.currentElapsedTime();
    -        while (Time.currentElapsedTime() < start + TIMEOUT) {
    -            if (handshakeCompleted) {
    -                return;
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(sslSocket);
    +
    +        synchronized (handshakeCompletedLock) {
    +            if (!handshakeCompleted) {
    +                handshakeCompletedLock.wait(TIMEOUT);
                 }
    +            Assert.assertTrue(handshakeCompleted);
             }
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
    +    }
    +
    +    /**
    +     * Attempting to connect to a SSL-only server with SSL should work.
    +     */
    +    @Test
    +    public void testConnectWithSSLToStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, false, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket sslSocket = connectWithSSL();
    +        sslSocket.getOutputStream().write(DATA_FROM_CLIENT);
    +        sslSocket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = sslSocket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
     
    -        Assert.fail("failed to complete handshake");
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(sslSocket);
    +
    +        synchronized (handshakeCompletedLock) {
    +            if (!handshakeCompleted) {
    +                handshakeCompletedLock.wait(TIMEOUT);
    +            }
    +            Assert.assertTrue(handshakeCompleted);
    +        }
    +
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
         }
     
    -    private void completeHandshake() {
    -        handshakeCompleted = true;
    +    /**
    +     * Attempting to connect to a SSL-or-plaintext server without SSL should work.
    +     */
    +    @Test
    +    public void testConnectWithoutSSLToNonStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, true, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket socket = connectWithoutSSL();
    +        socket.getOutputStream().write(DATA_FROM_CLIENT);
    +        socket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = socket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
    +
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(socket);
    +
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
         }
     
    +    /**
    +     * Attempting to connect to a SSL-or-plaintext server without SSL with a
    +     * small initial data write should work. This makes sure that sending
    +     * less than 5 bytes does not break the logic in the server's initial 5
    +     * byte read.
    +     */
         @Test
    -    public void testConnectWithoutSSL() throws Exception {
    -        final byte[] testData = "hello there".getBytes();
    -        final String[] dataReadFromClient = {null};
    +    public void testConnectWithoutSSLToNonStrictServerPartialWrite() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, true, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket socket = connectWithoutSSL();
    +        // Write only 2 bytes of the message, wait a bit, then write the rest.
    +        // This makes sure that writes smaller than 5 bytes don't break the plaintext mode on the server
    +        // once it decides that the input doesn't look like a TLS handshake.
    +        socket.getOutputStream().write(DATA_FROM_CLIENT, 0, 2);
    +        socket.getOutputStream().flush();
    +        Thread.sleep(TIMEOUT / 2);
    +        socket.getOutputStream().write(DATA_FROM_CLIENT, 2, DATA_FROM_CLIENT.length - 2);
    +        socket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = socket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
    +
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(socket);
     
    -        class ServerThread extends Thread {
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
    +    }
    +
    +    /**
    +     * Attempting to connect to a SSL-only server without SSL should fail.
    +     */
    +    @Test
    +    public void testConnectWithoutSSLToStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, false, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket socket = connectWithoutSSL();
    +        socket.getOutputStream().write(DATA_FROM_CLIENT);
    +        socket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        try {
    +            socket.getInputStream().read(buf, 0, buf.length);
    +        } catch (SocketException e) {
    +            // We expect the other end to hang up the connection
    +            return;
    +        } finally {
    +            serverThread.shutdown(TIMEOUT);
    +            forceClose(socket);
    +        }
    +        Assert.fail("Expected server to hang up the connection. Read from server succeeded unexpectedly.");
    +    }
    +
    +    /**
    +     * This test makes sure that UnifiedServerSocket used properly (a single thread accept()-ing connections and
    +     * handing the resulting sockets to other threads for processing) is not vulnerable to a simple denial-of-service
    +     * attack in which a client connects and never writes any bytes. This should not block the accepting thread, since
    +     * the read to determine if the client is sending a TLS handshake or not happens in the processing thread.
    +     *
    +     * This version of the test uses a non-strict server socket (i.e. it accepts both TLS and plaintext connections).
    +     */
    +    @Test
    +    public void testDenialOfServiceResistanceNonStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    --- End diff --
    
    This is true. I tried to make the threading behavior in this test as similar as possible to `Leader`. I don't think Leader should be involved in this unit test, but perhaps a similar "denial of service resistance" test case can be added to some quorum test. Do you know which unit test would be a good place for such a test case?


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    Rebase on latest master, no longer includes #678 as it has been merged upstream.


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2547/



---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    I like the old DoS tests, I'm happy to restore them. The new unit test I added can stay too.


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    @anmolnar use Executor and InetAddress.getLoopbackAddress()


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
GitHub user ivmaykov reopened a pull request:

    https://github.com/apache/zookeeper/pull/679

    ZOOKEEPER-3172: Quorum TLS - fix port unification to allow rolling upgrades

    Fix numerous problems with UnifiedServerSocket, such as hanging the accept() thread when the client doesn't send any data or crashing if less than 5 bytes are read from the socket in the initial read.
    
    Re-enable the "portUnification" config option.
    
    ## Fixed networking issues/bugs in UnifiedServerSocket
    
    - don't crash the `accept()` thread if the client closes the connection without sending any data
    - don't corrupt the connection if the client sends fewer than 5 bytes for the initial read
    - delay the detection of TLS vs. plaintext mode until a socket stream is read from or written to. This prevents the `accept()` thread from getting blocked on a `read()` operation from the newly connected socket.
    - prepending 5 bytes to `PrependableSocket` and then trying to read >5 bytes would only return the first 5 bytes, even if more bytes were available. This is fixed.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ivmaykov/zookeeper ZOOKEEPER-3172

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/679.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #679
    
----
commit 33f7aaab6fe16122b7e1faedbb408d739bbe8a30
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T01:22:24Z

    ZOOKEEPER-3172: Quorum TLS - fix port unification to allow rolling upgrades

----


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r233632959
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java ---
    @@ -350,14 +389,22 @@ public static X509TrustManager createTrustManager(
         public SSLSocket createSSLSocket() throws X509Exception, IOException {
             SSLSocket sslSocket = (SSLSocket) getDefaultSSLContext().getSocketFactory().createSocket();
             configureSSLSocket(sslSocket);
    -
    +        sslSocket.setUseClientMode(true);
             return sslSocket;
         }
     
    -    public SSLSocket createSSLSocket(Socket socket) throws X509Exception, IOException {
    -        SSLSocket sslSocket = (SSLSocket) getDefaultSSLContext().getSocketFactory().createSocket(socket, null, socket.getPort(), true);
    +    public SSLSocket createSSLSocket(Socket socket, byte[] pushbackBytes) throws X509Exception, IOException {
    +        SSLSocket sslSocket;
    +        if (pushbackBytes != null && pushbackBytes.length > 0) {
    +            sslSocket = (SSLSocket) getDefaultSSLContext().getSocketFactory().createSocket(
    +                    socket, new ByteArrayInputStream(pushbackBytes), true);
    +        } else {
    +            sslSocket = (SSLSocket) getDefaultSSLContext().getSocketFactory().createSocket(
    +                    socket, null, socket.getPort(), true);
    +        }
             configureSSLSocket(sslSocket);
    -
    +        sslSocket.setUseClientMode(false);
    --- End diff --
    
    Just to double check what you changed here:
    - setting the client mode explicitly on both client/server side,
    - requesting client authentication in TLS mode: so without client authentication, quorum TLS cannot be established anymore


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2560/



---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
GitHub user ivmaykov reopened a pull request:

    https://github.com/apache/zookeeper/pull/679

    ZOOKEEPER-3172: Quorum TLS - fix port unification to allow rolling upgrades

    Fix numerous problems with UnifiedServerSocket, such as hanging the accept() thread when the client doesn't send any data or crashing if less than 5 bytes are read from the socket in the initial read.
    
    Re-enable the "portUnification" config option.
    
    ## Fixed networking issues/bugs in UnifiedServerSocket
    
    - don't crash the `accept()` thread if the client closes the connection without sending any data
    - don't corrupt the connection if the client sends fewer than 5 bytes for the initial read
    - delay the detection of TLS vs. plaintext mode until a socket stream is read from or written to. This prevents the `accept()` thread from getting blocked on a `read()` operation from the newly connected socket.
    - prepending 5 bytes to `PrependableSocket` and then trying to read >5 bytes would only return the first 5 bytes, even if more bytes were available. This is fixed.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ivmaykov/zookeeper ZOOKEEPER-3172

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/679.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #679
    
----
commit 367bef0980193e2761c7008844c5e9fe029d8a66
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T01:22:24Z

    ZOOKEEPER-3172: Quorum TLS - fix port unification to allow rolling upgrades

----


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2501/



---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r234053630
  
    --- Diff: zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java ---
    @@ -17,156 +17,644 @@
      */
     package org.apache.zookeeper.server.quorum;
     
    +import java.io.BufferedInputStream;
    +import java.io.IOException;
    +import java.net.ConnectException;
    +import java.net.InetSocketAddress;
    +import java.net.Socket;
    +import java.net.SocketException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Random;
    +
    +import javax.net.ssl.HandshakeCompletedEvent;
    +import javax.net.ssl.HandshakeCompletedListener;
    +import javax.net.ssl.SSLSocket;
    +
     import org.apache.zookeeper.PortAssignment;
     import org.apache.zookeeper.client.ZKClientConfig;
    +import org.apache.zookeeper.common.BaseX509ParameterizedTestCase;
     import org.apache.zookeeper.common.ClientX509Util;
    -import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.common.KeyStoreFileType;
    +import org.apache.zookeeper.common.X509Exception;
    +import org.apache.zookeeper.common.X509KeyType;
    +import org.apache.zookeeper.common.X509TestContext;
     import org.apache.zookeeper.common.X509Util;
     import org.apache.zookeeper.server.ServerCnxnFactory;
     import org.junit.Assert;
     import org.junit.Before;
     import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
     
    -import javax.net.ssl.HandshakeCompletedEvent;
    -import javax.net.ssl.HandshakeCompletedListener;
    -import javax.net.ssl.SSLSocket;
    -import java.io.IOException;
    -import java.net.ConnectException;
    -import java.net.InetSocketAddress;
    -import java.net.Socket;
    -
    -import static org.hamcrest.CoreMatchers.equalTo;
    -import static org.junit.Assert.assertThat;
    +@RunWith(Parameterized.class)
    +public class UnifiedServerSocketTest extends BaseX509ParameterizedTestCase {
     
    -public class UnifiedServerSocketTest {
    +    @Parameterized.Parameters
    +    public static Collection<Object[]> params() {
    +        ArrayList<Object[]> result = new ArrayList<>();
    +        int paramIndex = 0;
    +        for (X509KeyType caKeyType : X509KeyType.values()) {
    +            for (X509KeyType certKeyType : X509KeyType.values()) {
    +                for (Boolean hostnameVerification : new Boolean[] { true, false  }) {
    +                    result.add(new Object[]{
    +                            caKeyType,
    +                            certKeyType,
    +                            hostnameVerification,
    +                            paramIndex++
    +                    });
    +                }
    +            }
    +        }
    +        return result;
    +    }
     
         private static final int MAX_RETRIES = 5;
         private static final int TIMEOUT = 1000;
    +    private static final byte[] DATA_TO_CLIENT = "hello client".getBytes();
    +    private static final byte[] DATA_FROM_CLIENT = "hello server".getBytes();
     
         private X509Util x509Util;
         private int port;
    -    private volatile boolean handshakeCompleted;
    +    private InetSocketAddress localServerAddress;
    +    private final Object handshakeCompletedLock = new Object();
    +    // access only inside synchronized(handshakeCompletedLock) { ... } blocks
    +    private boolean handshakeCompleted = false;
    +
    +    public UnifiedServerSocketTest(
    +            final X509KeyType caKeyType,
    +            final X509KeyType certKeyType,
    +            final Boolean hostnameVerification,
    +            final Integer paramIndex) {
    +        super(paramIndex, () -> {
    +            try {
    +                return X509TestContext.newBuilder()
    +                    .setTempDir(tempDir)
    +                    .setKeyStoreKeyType(certKeyType)
    +                    .setTrustStoreKeyType(caKeyType)
    +                    .setHostnameVerification(hostnameVerification)
    +                    .build();
    +            } catch (Exception e) {
    +                throw new RuntimeException(e);
    +            }
    +        });
    +    }
     
         @Before
         public void setUp() throws Exception {
    -        handshakeCompleted = false;
    -
             port = PortAssignment.unique();
    +        localServerAddress = new InetSocketAddress("localhost", port);
     
    -        String testDataPath = System.getProperty("test.data.dir", "build/test/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");
     
             x509Util = new ClientX509Util();
     
    -        System.setProperty(x509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks");
    -        System.setProperty(x509Util.getSslKeystorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks");
    -        System.setProperty(x509Util.getSslTruststorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslHostnameVerificationEnabledProperty(), "false");
    +        x509TestContext.setSystemProperties(x509Util, KeyStoreFileType.JKS, KeyStoreFileType.JKS);
         }
     
    -    @Test
    -    public void testConnectWithSSL() throws Exception {
    -        class ServerThread extends Thread {
    -            public void run() {
    -                try {
    -                    Socket unifiedSocket = new UnifiedServerSocket(x509Util, port).accept();
    -                    ((SSLSocket)unifiedSocket).getSession(); // block until handshake completes
    -                } catch (IOException e) {
    -                    e.printStackTrace();
    +    private static void forceClose(java.io.Closeable s) {
    +        if (s == null) {
    +            return;
    +        }
    +        try {
    +            s.close();
    +        } catch (IOException e) {
    +            e.printStackTrace();
    +        }
    +    }
    +
    +    private static final class UnifiedServerThread extends Thread {
    +        private final byte[] dataToClient;
    +        private List<byte[]> dataFromClients;
    +        private List<Thread> workerThreads;
    --- End diff --
    
    Done


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    @anmolnar removed DoS tests from `UnifiedServerSocketTest.java`. Added a new unit test `UnifiedServerSocketModeDetectionTest.java` which checks that calling certain methods on the server side of a unified socket connection does not trigger mode detection. Also rebased on latest master to fix merge conflict.


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2522/



---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r233643417
  
    --- Diff: zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java ---
    @@ -17,156 +17,644 @@
      */
     package org.apache.zookeeper.server.quorum;
     
    +import java.io.BufferedInputStream;
    +import java.io.IOException;
    +import java.net.ConnectException;
    +import java.net.InetSocketAddress;
    +import java.net.Socket;
    +import java.net.SocketException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Random;
    +
    +import javax.net.ssl.HandshakeCompletedEvent;
    +import javax.net.ssl.HandshakeCompletedListener;
    +import javax.net.ssl.SSLSocket;
    +
     import org.apache.zookeeper.PortAssignment;
     import org.apache.zookeeper.client.ZKClientConfig;
    +import org.apache.zookeeper.common.BaseX509ParameterizedTestCase;
     import org.apache.zookeeper.common.ClientX509Util;
    -import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.common.KeyStoreFileType;
    +import org.apache.zookeeper.common.X509Exception;
    +import org.apache.zookeeper.common.X509KeyType;
    +import org.apache.zookeeper.common.X509TestContext;
     import org.apache.zookeeper.common.X509Util;
     import org.apache.zookeeper.server.ServerCnxnFactory;
     import org.junit.Assert;
     import org.junit.Before;
     import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
     
    -import javax.net.ssl.HandshakeCompletedEvent;
    -import javax.net.ssl.HandshakeCompletedListener;
    -import javax.net.ssl.SSLSocket;
    -import java.io.IOException;
    -import java.net.ConnectException;
    -import java.net.InetSocketAddress;
    -import java.net.Socket;
    -
    -import static org.hamcrest.CoreMatchers.equalTo;
    -import static org.junit.Assert.assertThat;
    +@RunWith(Parameterized.class)
    +public class UnifiedServerSocketTest extends BaseX509ParameterizedTestCase {
     
    -public class UnifiedServerSocketTest {
    +    @Parameterized.Parameters
    +    public static Collection<Object[]> params() {
    +        ArrayList<Object[]> result = new ArrayList<>();
    +        int paramIndex = 0;
    +        for (X509KeyType caKeyType : X509KeyType.values()) {
    +            for (X509KeyType certKeyType : X509KeyType.values()) {
    +                for (Boolean hostnameVerification : new Boolean[] { true, false  }) {
    +                    result.add(new Object[]{
    +                            caKeyType,
    +                            certKeyType,
    +                            hostnameVerification,
    +                            paramIndex++
    +                    });
    +                }
    +            }
    +        }
    +        return result;
    +    }
     
         private static final int MAX_RETRIES = 5;
         private static final int TIMEOUT = 1000;
    +    private static final byte[] DATA_TO_CLIENT = "hello client".getBytes();
    +    private static final byte[] DATA_FROM_CLIENT = "hello server".getBytes();
     
         private X509Util x509Util;
         private int port;
    -    private volatile boolean handshakeCompleted;
    +    private InetSocketAddress localServerAddress;
    +    private final Object handshakeCompletedLock = new Object();
    +    // access only inside synchronized(handshakeCompletedLock) { ... } blocks
    +    private boolean handshakeCompleted = false;
    +
    +    public UnifiedServerSocketTest(
    +            final X509KeyType caKeyType,
    +            final X509KeyType certKeyType,
    +            final Boolean hostnameVerification,
    +            final Integer paramIndex) {
    +        super(paramIndex, () -> {
    +            try {
    +                return X509TestContext.newBuilder()
    +                    .setTempDir(tempDir)
    +                    .setKeyStoreKeyType(certKeyType)
    +                    .setTrustStoreKeyType(caKeyType)
    +                    .setHostnameVerification(hostnameVerification)
    +                    .build();
    +            } catch (Exception e) {
    +                throw new RuntimeException(e);
    +            }
    +        });
    +    }
     
         @Before
         public void setUp() throws Exception {
    -        handshakeCompleted = false;
    -
             port = PortAssignment.unique();
    +        localServerAddress = new InetSocketAddress("localhost", port);
     
    -        String testDataPath = System.getProperty("test.data.dir", "build/test/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");
     
             x509Util = new ClientX509Util();
     
    -        System.setProperty(x509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks");
    -        System.setProperty(x509Util.getSslKeystorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks");
    -        System.setProperty(x509Util.getSslTruststorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslHostnameVerificationEnabledProperty(), "false");
    +        x509TestContext.setSystemProperties(x509Util, KeyStoreFileType.JKS, KeyStoreFileType.JKS);
         }
     
    -    @Test
    -    public void testConnectWithSSL() throws Exception {
    -        class ServerThread extends Thread {
    -            public void run() {
    -                try {
    -                    Socket unifiedSocket = new UnifiedServerSocket(x509Util, port).accept();
    -                    ((SSLSocket)unifiedSocket).getSession(); // block until handshake completes
    -                } catch (IOException e) {
    -                    e.printStackTrace();
    +    private static void forceClose(java.io.Closeable s) {
    +        if (s == null) {
    +            return;
    +        }
    +        try {
    +            s.close();
    +        } catch (IOException e) {
    +            e.printStackTrace();
    +        }
    +    }
    +
    +    private static final class UnifiedServerThread extends Thread {
    +        private final byte[] dataToClient;
    +        private List<byte[]> dataFromClients;
    +        private List<Thread> workerThreads;
    --- End diff --
    
    Use some `Executor` instead?


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r233657163
  
    --- Diff: zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java ---
    @@ -17,156 +17,644 @@
      */
     package org.apache.zookeeper.server.quorum;
     
    +import java.io.BufferedInputStream;
    +import java.io.IOException;
    +import java.net.ConnectException;
    +import java.net.InetSocketAddress;
    +import java.net.Socket;
    +import java.net.SocketException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Random;
    +
    +import javax.net.ssl.HandshakeCompletedEvent;
    +import javax.net.ssl.HandshakeCompletedListener;
    +import javax.net.ssl.SSLSocket;
    +
     import org.apache.zookeeper.PortAssignment;
     import org.apache.zookeeper.client.ZKClientConfig;
    +import org.apache.zookeeper.common.BaseX509ParameterizedTestCase;
     import org.apache.zookeeper.common.ClientX509Util;
    -import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.common.KeyStoreFileType;
    +import org.apache.zookeeper.common.X509Exception;
    +import org.apache.zookeeper.common.X509KeyType;
    +import org.apache.zookeeper.common.X509TestContext;
     import org.apache.zookeeper.common.X509Util;
     import org.apache.zookeeper.server.ServerCnxnFactory;
     import org.junit.Assert;
     import org.junit.Before;
     import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
     
    -import javax.net.ssl.HandshakeCompletedEvent;
    -import javax.net.ssl.HandshakeCompletedListener;
    -import javax.net.ssl.SSLSocket;
    -import java.io.IOException;
    -import java.net.ConnectException;
    -import java.net.InetSocketAddress;
    -import java.net.Socket;
    -
    -import static org.hamcrest.CoreMatchers.equalTo;
    -import static org.junit.Assert.assertThat;
    +@RunWith(Parameterized.class)
    +public class UnifiedServerSocketTest extends BaseX509ParameterizedTestCase {
     
    -public class UnifiedServerSocketTest {
    +    @Parameterized.Parameters
    +    public static Collection<Object[]> params() {
    +        ArrayList<Object[]> result = new ArrayList<>();
    +        int paramIndex = 0;
    +        for (X509KeyType caKeyType : X509KeyType.values()) {
    +            for (X509KeyType certKeyType : X509KeyType.values()) {
    +                for (Boolean hostnameVerification : new Boolean[] { true, false  }) {
    +                    result.add(new Object[]{
    +                            caKeyType,
    +                            certKeyType,
    +                            hostnameVerification,
    +                            paramIndex++
    +                    });
    +                }
    +            }
    +        }
    +        return result;
    +    }
     
         private static final int MAX_RETRIES = 5;
         private static final int TIMEOUT = 1000;
    +    private static final byte[] DATA_TO_CLIENT = "hello client".getBytes();
    +    private static final byte[] DATA_FROM_CLIENT = "hello server".getBytes();
     
         private X509Util x509Util;
         private int port;
    -    private volatile boolean handshakeCompleted;
    +    private InetSocketAddress localServerAddress;
    +    private final Object handshakeCompletedLock = new Object();
    +    // access only inside synchronized(handshakeCompletedLock) { ... } blocks
    +    private boolean handshakeCompleted = false;
    +
    +    public UnifiedServerSocketTest(
    +            final X509KeyType caKeyType,
    +            final X509KeyType certKeyType,
    +            final Boolean hostnameVerification,
    +            final Integer paramIndex) {
    +        super(paramIndex, () -> {
    +            try {
    +                return X509TestContext.newBuilder()
    +                    .setTempDir(tempDir)
    +                    .setKeyStoreKeyType(certKeyType)
    +                    .setTrustStoreKeyType(caKeyType)
    +                    .setHostnameVerification(hostnameVerification)
    +                    .build();
    +            } catch (Exception e) {
    +                throw new RuntimeException(e);
    +            }
    +        });
    +    }
     
         @Before
         public void setUp() throws Exception {
    -        handshakeCompleted = false;
    -
             port = PortAssignment.unique();
    +        localServerAddress = new InetSocketAddress("localhost", port);
     
    -        String testDataPath = System.getProperty("test.data.dir", "build/test/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");
     
             x509Util = new ClientX509Util();
     
    -        System.setProperty(x509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks");
    -        System.setProperty(x509Util.getSslKeystorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks");
    -        System.setProperty(x509Util.getSslTruststorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslHostnameVerificationEnabledProperty(), "false");
    +        x509TestContext.setSystemProperties(x509Util, KeyStoreFileType.JKS, KeyStoreFileType.JKS);
         }
     
    -    @Test
    -    public void testConnectWithSSL() throws Exception {
    -        class ServerThread extends Thread {
    -            public void run() {
    -                try {
    -                    Socket unifiedSocket = new UnifiedServerSocket(x509Util, port).accept();
    -                    ((SSLSocket)unifiedSocket).getSession(); // block until handshake completes
    -                } catch (IOException e) {
    -                    e.printStackTrace();
    +    private static void forceClose(java.io.Closeable s) {
    +        if (s == null) {
    +            return;
    +        }
    +        try {
    +            s.close();
    +        } catch (IOException e) {
    +            e.printStackTrace();
    +        }
    +    }
    +
    +    private static final class UnifiedServerThread extends Thread {
    +        private final byte[] dataToClient;
    +        private List<byte[]> dataFromClients;
    +        private List<Thread> workerThreads;
    --- End diff --
    
    I used explicit threads because that's what we do in Leader.java - create a dedicated Thread per connection. I wanted the threading in the test to be as close as possible to the real use case. I could change it to an Executor if you want, but don't see that much value in such a change.


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2592/



---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r233656740
  
    --- Diff: zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java ---
    @@ -17,156 +17,644 @@
      */
     package org.apache.zookeeper.server.quorum;
     
    +import java.io.BufferedInputStream;
    +import java.io.IOException;
    +import java.net.ConnectException;
    +import java.net.InetSocketAddress;
    +import java.net.Socket;
    +import java.net.SocketException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Random;
    +
    +import javax.net.ssl.HandshakeCompletedEvent;
    +import javax.net.ssl.HandshakeCompletedListener;
    +import javax.net.ssl.SSLSocket;
    +
     import org.apache.zookeeper.PortAssignment;
     import org.apache.zookeeper.client.ZKClientConfig;
    +import org.apache.zookeeper.common.BaseX509ParameterizedTestCase;
     import org.apache.zookeeper.common.ClientX509Util;
    -import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.common.KeyStoreFileType;
    +import org.apache.zookeeper.common.X509Exception;
    +import org.apache.zookeeper.common.X509KeyType;
    +import org.apache.zookeeper.common.X509TestContext;
     import org.apache.zookeeper.common.X509Util;
     import org.apache.zookeeper.server.ServerCnxnFactory;
     import org.junit.Assert;
     import org.junit.Before;
     import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
     
    -import javax.net.ssl.HandshakeCompletedEvent;
    -import javax.net.ssl.HandshakeCompletedListener;
    -import javax.net.ssl.SSLSocket;
    -import java.io.IOException;
    -import java.net.ConnectException;
    -import java.net.InetSocketAddress;
    -import java.net.Socket;
    -
    -import static org.hamcrest.CoreMatchers.equalTo;
    -import static org.junit.Assert.assertThat;
    +@RunWith(Parameterized.class)
    +public class UnifiedServerSocketTest extends BaseX509ParameterizedTestCase {
     
    -public class UnifiedServerSocketTest {
    +    @Parameterized.Parameters
    +    public static Collection<Object[]> params() {
    +        ArrayList<Object[]> result = new ArrayList<>();
    +        int paramIndex = 0;
    +        for (X509KeyType caKeyType : X509KeyType.values()) {
    +            for (X509KeyType certKeyType : X509KeyType.values()) {
    +                for (Boolean hostnameVerification : new Boolean[] { true, false  }) {
    +                    result.add(new Object[]{
    +                            caKeyType,
    +                            certKeyType,
    +                            hostnameVerification,
    +                            paramIndex++
    +                    });
    +                }
    +            }
    +        }
    +        return result;
    +    }
     
         private static final int MAX_RETRIES = 5;
         private static final int TIMEOUT = 1000;
    +    private static final byte[] DATA_TO_CLIENT = "hello client".getBytes();
    +    private static final byte[] DATA_FROM_CLIENT = "hello server".getBytes();
     
         private X509Util x509Util;
         private int port;
    -    private volatile boolean handshakeCompleted;
    +    private InetSocketAddress localServerAddress;
    +    private final Object handshakeCompletedLock = new Object();
    +    // access only inside synchronized(handshakeCompletedLock) { ... } blocks
    +    private boolean handshakeCompleted = false;
    +
    +    public UnifiedServerSocketTest(
    +            final X509KeyType caKeyType,
    +            final X509KeyType certKeyType,
    +            final Boolean hostnameVerification,
    +            final Integer paramIndex) {
    +        super(paramIndex, () -> {
    +            try {
    +                return X509TestContext.newBuilder()
    +                    .setTempDir(tempDir)
    +                    .setKeyStoreKeyType(certKeyType)
    +                    .setTrustStoreKeyType(caKeyType)
    +                    .setHostnameVerification(hostnameVerification)
    +                    .build();
    +            } catch (Exception e) {
    +                throw new RuntimeException(e);
    +            }
    +        });
    +    }
     
         @Before
         public void setUp() throws Exception {
    -        handshakeCompleted = false;
    -
             port = PortAssignment.unique();
    +        localServerAddress = new InetSocketAddress("localhost", port);
    --- End diff --
    
    Sure, I could do that if you prefer.


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov closed the pull request at:

    https://github.com/apache/zookeeper/pull/679


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2556/



---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r233661888
  
    --- Diff: zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java ---
    @@ -17,156 +17,644 @@
      */
     package org.apache.zookeeper.server.quorum;
     
    +import java.io.BufferedInputStream;
    +import java.io.IOException;
    +import java.net.ConnectException;
    +import java.net.InetSocketAddress;
    +import java.net.Socket;
    +import java.net.SocketException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Random;
    +
    +import javax.net.ssl.HandshakeCompletedEvent;
    +import javax.net.ssl.HandshakeCompletedListener;
    +import javax.net.ssl.SSLSocket;
    +
     import org.apache.zookeeper.PortAssignment;
     import org.apache.zookeeper.client.ZKClientConfig;
    +import org.apache.zookeeper.common.BaseX509ParameterizedTestCase;
     import org.apache.zookeeper.common.ClientX509Util;
    -import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.common.KeyStoreFileType;
    +import org.apache.zookeeper.common.X509Exception;
    +import org.apache.zookeeper.common.X509KeyType;
    +import org.apache.zookeeper.common.X509TestContext;
     import org.apache.zookeeper.common.X509Util;
     import org.apache.zookeeper.server.ServerCnxnFactory;
     import org.junit.Assert;
     import org.junit.Before;
     import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
     
    -import javax.net.ssl.HandshakeCompletedEvent;
    -import javax.net.ssl.HandshakeCompletedListener;
    -import javax.net.ssl.SSLSocket;
    -import java.io.IOException;
    -import java.net.ConnectException;
    -import java.net.InetSocketAddress;
    -import java.net.Socket;
    -
    -import static org.hamcrest.CoreMatchers.equalTo;
    -import static org.junit.Assert.assertThat;
    +@RunWith(Parameterized.class)
    +public class UnifiedServerSocketTest extends BaseX509ParameterizedTestCase {
     
    -public class UnifiedServerSocketTest {
    +    @Parameterized.Parameters
    +    public static Collection<Object[]> params() {
    +        ArrayList<Object[]> result = new ArrayList<>();
    +        int paramIndex = 0;
    +        for (X509KeyType caKeyType : X509KeyType.values()) {
    +            for (X509KeyType certKeyType : X509KeyType.values()) {
    +                for (Boolean hostnameVerification : new Boolean[] { true, false  }) {
    +                    result.add(new Object[]{
    +                            caKeyType,
    +                            certKeyType,
    +                            hostnameVerification,
    +                            paramIndex++
    +                    });
    +                }
    +            }
    +        }
    +        return result;
    +    }
     
         private static final int MAX_RETRIES = 5;
         private static final int TIMEOUT = 1000;
    +    private static final byte[] DATA_TO_CLIENT = "hello client".getBytes();
    +    private static final byte[] DATA_FROM_CLIENT = "hello server".getBytes();
     
         private X509Util x509Util;
         private int port;
    -    private volatile boolean handshakeCompleted;
    +    private InetSocketAddress localServerAddress;
    +    private final Object handshakeCompletedLock = new Object();
    +    // access only inside synchronized(handshakeCompletedLock) { ... } blocks
    +    private boolean handshakeCompleted = false;
    +
    +    public UnifiedServerSocketTest(
    +            final X509KeyType caKeyType,
    +            final X509KeyType certKeyType,
    +            final Boolean hostnameVerification,
    +            final Integer paramIndex) {
    +        super(paramIndex, () -> {
    +            try {
    +                return X509TestContext.newBuilder()
    +                    .setTempDir(tempDir)
    +                    .setKeyStoreKeyType(certKeyType)
    +                    .setTrustStoreKeyType(caKeyType)
    +                    .setHostnameVerification(hostnameVerification)
    +                    .build();
    +            } catch (Exception e) {
    +                throw new RuntimeException(e);
    +            }
    +        });
    +    }
     
         @Before
         public void setUp() throws Exception {
    -        handshakeCompleted = false;
    -
             port = PortAssignment.unique();
    +        localServerAddress = new InetSocketAddress("localhost", port);
     
    -        String testDataPath = System.getProperty("test.data.dir", "build/test/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");
     
             x509Util = new ClientX509Util();
     
    -        System.setProperty(x509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks");
    -        System.setProperty(x509Util.getSslKeystorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks");
    -        System.setProperty(x509Util.getSslTruststorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslHostnameVerificationEnabledProperty(), "false");
    +        x509TestContext.setSystemProperties(x509Util, KeyStoreFileType.JKS, KeyStoreFileType.JKS);
         }
     
    -    @Test
    -    public void testConnectWithSSL() throws Exception {
    -        class ServerThread extends Thread {
    -            public void run() {
    -                try {
    -                    Socket unifiedSocket = new UnifiedServerSocket(x509Util, port).accept();
    -                    ((SSLSocket)unifiedSocket).getSession(); // block until handshake completes
    -                } catch (IOException e) {
    -                    e.printStackTrace();
    +    private static void forceClose(java.io.Closeable s) {
    +        if (s == null) {
    +            return;
    +        }
    +        try {
    +            s.close();
    +        } catch (IOException e) {
    +            e.printStackTrace();
    +        }
    +    }
    +
    +    private static final class UnifiedServerThread extends Thread {
    +        private final byte[] dataToClient;
    +        private List<byte[]> dataFromClients;
    +        private List<Thread> workerThreads;
    --- End diff --
    
    Creating tests as close as possible to the original impl is nice, but you're still testing a different thing anyway. Executors are good for dealing with threads and I think they're preferred over manual thread creation if acceptable. There're various different kinds of them: you can control the thread creation behaviour, number of threads, etc. And I think it leads to more readable code. 


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
GitHub user ivmaykov reopened a pull request:

    https://github.com/apache/zookeeper/pull/679

    ZOOKEEPER-3172: Quorum TLS - fix port unification to allow rolling upgrades

    Fix numerous problems with UnifiedServerSocket, such as hanging the accept() thread when the client doesn't send any data or crashing if less than 5 bytes are read from the socket in the initial read.
    
    Re-enable the "portUnification" config option.
    
    Note that this is stacked on top of #678 and thus includes it. Please only consider the ZOOKEEPER-3172 commit when reviewing. Once the other PR is merged upstream, I will rebase this so it only contains one commit.
    
    ## Fixed networking issues/bugs in UnifiedServerSocket
    
    - don't crash the `accept()` thread if the client closes the connection without sending any data
    - don't corrupt the connection if the client sends fewer than 5 bytes for the initial read
    - delay the detection of TLS vs. plaintext mode until a socket stream is read from or written to. This prevents the `accept()` thread from getting blocked on a `read()` operation from the newly connected socket.
    - prepending 5 bytes to `PrependableSocket` and then trying to read >5 bytes would only return the first 5 bytes, even if more bytes were available. This is fixed.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ivmaykov/zookeeper ZOOKEEPER-3172

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/679.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #679
    
----
commit 2122c8c23a0dbb27f9b2aff55e800e48d253f943
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T00:41:48Z

    ZOOKEEPER-3173: Quorum TLS - support PEM trust/key stores
    ZOOKEEPER-3175: Quorum TLS - test improvements
    
    Add support for loading key and trust stores from PEM files.
    Also added test utils for testing X509-related code, because it
    was very difficult to untangle them from the PEM support code.

commit 69f5185c8c14720e94c81f0147ee9cbc2ae42f89
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T01:22:24Z

    ZOOKEEPER-3172: Quorum TLS - fix port unification to allow rolling upgrades

----


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    @anmolnar the benefit of the current DOS tests is to verify that certain actions on a UnifiedSocket (getting input stream, setting socket options, etc) do not trigger the blocking read for TLS / plaintext mode detection. I think there is benefit to having such tests `UnifiedServerSocketTest`, since they test behavior that's implemented in `UnifiedServerSocket.java`. What do you think?


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r233656201
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java ---
    @@ -350,14 +389,22 @@ public static X509TrustManager createTrustManager(
         public SSLSocket createSSLSocket() throws X509Exception, IOException {
             SSLSocket sslSocket = (SSLSocket) getDefaultSSLContext().getSocketFactory().createSocket();
             configureSSLSocket(sslSocket);
    -
    +        sslSocket.setUseClientMode(true);
             return sslSocket;
         }
     
    -    public SSLSocket createSSLSocket(Socket socket) throws X509Exception, IOException {
    -        SSLSocket sslSocket = (SSLSocket) getDefaultSSLContext().getSocketFactory().createSocket(socket, null, socket.getPort(), true);
    +    public SSLSocket createSSLSocket(Socket socket, byte[] pushbackBytes) throws X509Exception, IOException {
    +        SSLSocket sslSocket;
    +        if (pushbackBytes != null && pushbackBytes.length > 0) {
    +            sslSocket = (SSLSocket) getDefaultSSLContext().getSocketFactory().createSocket(
    +                    socket, new ByteArrayInputStream(pushbackBytes), true);
    +        } else {
    +            sslSocket = (SSLSocket) getDefaultSSLContext().getSocketFactory().createSocket(
    +                    socket, null, socket.getPort(), true);
    +        }
             configureSSLSocket(sslSocket);
    -
    +        sslSocket.setUseClientMode(false);
    --- End diff --
    
    Yes and yes. In #681 I make the client auth setting configurable.


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    @ivmaykov I did another cycle of review the unit tests, sorry I still not see value in denial-of-service tests, but maybe I don't see something important. Let's put it this way:
    
    Use case of `UnifiedServerSocket` is the following:
    > Extend standard Socket class to catch all read events in order to trigger TLS/Plaintext mode detection in a lazy fashion (catching the last chance of doing so).
    
    In order to do that it captures calls to the underlying input/output streams and initiates detection of channel type on the first read/write operation. So far so good.
    
    The important bit here is that `UnifiedServerSocket` **doesn't contain anything related to threading**. As a consequence if we write purely unit tests for this class we won't have to verify anything which is related to threading.
    
    DOS test comment says:
    > This test makes sure that UnifiedServerSocket used properly (a single thread accept()-ing connections and handing the resulting sockets to other threads for processing) is not vulnerable to a simple denial-of-service attack in which a client connects and never writes any bytes. **This should not block the accepting thread, since the read to determine if the client is sending a TLS handshake or not happens in the processing thread.**
    
    And here's the thing: this test is testing the proper implementation of the server (e.g. dealing with threads in the right way), which is implemented in the test itself: `UnifiedServerThread`.
    
    My 2 cents here is that if you think you haven't covered a user scenario or an edge with the existing tests, you need to rewrite these tests to be more strict and test that-and-only-that particular case which is missing.
    
    I think the coverage is acceptable (what you mentioned in your latest comment has already been covered), but there's no harm in adding more. I just don't want tests which don't add value **or** adds value in an unreasonably cumbersome way.
    
    Please correct me if I'm wrong.
    
    
    



---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r235070776
  
    --- Diff: zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java ---
    @@ -17,156 +17,644 @@
      */
     package org.apache.zookeeper.server.quorum;
     
    +import java.io.BufferedInputStream;
    +import java.io.IOException;
    +import java.net.ConnectException;
    +import java.net.InetSocketAddress;
    +import java.net.Socket;
    +import java.net.SocketException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Random;
    +
    +import javax.net.ssl.HandshakeCompletedEvent;
    +import javax.net.ssl.HandshakeCompletedListener;
    +import javax.net.ssl.SSLSocket;
    +
     import org.apache.zookeeper.PortAssignment;
     import org.apache.zookeeper.client.ZKClientConfig;
    +import org.apache.zookeeper.common.BaseX509ParameterizedTestCase;
     import org.apache.zookeeper.common.ClientX509Util;
    -import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.common.KeyStoreFileType;
    +import org.apache.zookeeper.common.X509Exception;
    +import org.apache.zookeeper.common.X509KeyType;
    +import org.apache.zookeeper.common.X509TestContext;
     import org.apache.zookeeper.common.X509Util;
     import org.apache.zookeeper.server.ServerCnxnFactory;
     import org.junit.Assert;
     import org.junit.Before;
     import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
     
    -import javax.net.ssl.HandshakeCompletedEvent;
    -import javax.net.ssl.HandshakeCompletedListener;
    -import javax.net.ssl.SSLSocket;
    -import java.io.IOException;
    -import java.net.ConnectException;
    -import java.net.InetSocketAddress;
    -import java.net.Socket;
    -
    -import static org.hamcrest.CoreMatchers.equalTo;
    -import static org.junit.Assert.assertThat;
    +@RunWith(Parameterized.class)
    +public class UnifiedServerSocketTest extends BaseX509ParameterizedTestCase {
     
    -public class UnifiedServerSocketTest {
    +    @Parameterized.Parameters
    +    public static Collection<Object[]> params() {
    +        ArrayList<Object[]> result = new ArrayList<>();
    +        int paramIndex = 0;
    +        for (X509KeyType caKeyType : X509KeyType.values()) {
    +            for (X509KeyType certKeyType : X509KeyType.values()) {
    +                for (Boolean hostnameVerification : new Boolean[] { true, false  }) {
    +                    result.add(new Object[]{
    +                            caKeyType,
    +                            certKeyType,
    +                            hostnameVerification,
    +                            paramIndex++
    +                    });
    +                }
    +            }
    +        }
    +        return result;
    +    }
     
         private static final int MAX_RETRIES = 5;
         private static final int TIMEOUT = 1000;
    +    private static final byte[] DATA_TO_CLIENT = "hello client".getBytes();
    +    private static final byte[] DATA_FROM_CLIENT = "hello server".getBytes();
     
         private X509Util x509Util;
         private int port;
    -    private volatile boolean handshakeCompleted;
    +    private InetSocketAddress localServerAddress;
    +    private final Object handshakeCompletedLock = new Object();
    +    // access only inside synchronized(handshakeCompletedLock) { ... } blocks
    +    private boolean handshakeCompleted = false;
    +
    +    public UnifiedServerSocketTest(
    +            final X509KeyType caKeyType,
    +            final X509KeyType certKeyType,
    +            final Boolean hostnameVerification,
    +            final Integer paramIndex) {
    +        super(paramIndex, () -> {
    +            try {
    +                return X509TestContext.newBuilder()
    +                    .setTempDir(tempDir)
    +                    .setKeyStoreKeyType(certKeyType)
    +                    .setTrustStoreKeyType(caKeyType)
    +                    .setHostnameVerification(hostnameVerification)
    +                    .build();
    +            } catch (Exception e) {
    +                throw new RuntimeException(e);
    +            }
    +        });
    +    }
     
         @Before
         public void setUp() throws Exception {
    -        handshakeCompleted = false;
    -
             port = PortAssignment.unique();
    +        localServerAddress = new InetSocketAddress("localhost", port);
     
    -        String testDataPath = System.getProperty("test.data.dir", "build/test/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");
     
             x509Util = new ClientX509Util();
     
    -        System.setProperty(x509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks");
    -        System.setProperty(x509Util.getSslKeystorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks");
    -        System.setProperty(x509Util.getSslTruststorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslHostnameVerificationEnabledProperty(), "false");
    +        x509TestContext.setSystemProperties(x509Util, KeyStoreFileType.JKS, KeyStoreFileType.JKS);
         }
     
    -    @Test
    -    public void testConnectWithSSL() throws Exception {
    -        class ServerThread extends Thread {
    -            public void run() {
    -                try {
    -                    Socket unifiedSocket = new UnifiedServerSocket(x509Util, port).accept();
    -                    ((SSLSocket)unifiedSocket).getSession(); // block until handshake completes
    -                } catch (IOException e) {
    -                    e.printStackTrace();
    +    private static void forceClose(java.io.Closeable s) {
    +        if (s == null) {
    +            return;
    +        }
    +        try {
    +            s.close();
    +        } catch (IOException e) {
    +            e.printStackTrace();
    +        }
    +    }
    +
    +    private static final class UnifiedServerThread extends Thread {
    +        private final byte[] dataToClient;
    +        private List<byte[]> dataFromClients;
    +        private List<Thread> workerThreads;
    +        private UnifiedServerSocket serverSocket;
    +
    +        UnifiedServerThread(X509Util x509Util,
    +                            InetSocketAddress bindAddress,
    +                            boolean allowInsecureConnection,
    +                            byte[] dataToClient) throws IOException {
    +            this.dataToClient = dataToClient;
    +            dataFromClients = new ArrayList<>();
    +            workerThreads = new ArrayList<>();
    +            serverSocket = new UnifiedServerSocket(x509Util, allowInsecureConnection);
    +            serverSocket.bind(bindAddress);
    +        }
    +
    +        @Override
    +        public void run() {
    +            try {
    +                Random rnd = new Random();
    +                while (true) {
    +                    final Socket unifiedSocket = serverSocket.accept();
    +                    final boolean tcpNoDelay = rnd.nextBoolean();
    +                    unifiedSocket.setTcpNoDelay(tcpNoDelay);
    +                    unifiedSocket.setSoTimeout(TIMEOUT);
    +                    final boolean keepAlive = rnd.nextBoolean();
    +                    unifiedSocket.setKeepAlive(keepAlive);
    +                    // Note: getting the input stream should not block the thread or trigger mode detection.
    +                    BufferedInputStream bis = new BufferedInputStream(unifiedSocket.getInputStream());
    +                    Thread t = new Thread(new Runnable() {
    +                        @Override
    +                        public void run() {
    +                            try {
    +                                byte[] buf = new byte[1024];
    +                                int bytesRead = unifiedSocket.getInputStream().read(buf, 0, 1024);
    +                                // Make sure the settings applied above before the socket was potentially upgraded to
    +                                // TLS still apply.
    +                                Assert.assertEquals(tcpNoDelay, unifiedSocket.getTcpNoDelay());
    +                                Assert.assertEquals(TIMEOUT, unifiedSocket.getSoTimeout());
    +                                Assert.assertEquals(keepAlive, unifiedSocket.getKeepAlive());
    +                                if (bytesRead > 0) {
    +                                    byte[] dataFromClient = new byte[bytesRead];
    +                                    System.arraycopy(buf, 0, dataFromClient, 0, bytesRead);
    +                                    synchronized (dataFromClients) {
    +                                        dataFromClients.add(dataFromClient);
    +                                    }
    +                                }
    +                                unifiedSocket.getOutputStream().write(dataToClient);
    +                                unifiedSocket.getOutputStream().flush();
    +                            } catch (IOException e) {
    +                                e.printStackTrace();
    +                                throw new RuntimeException(e);
    +                            } finally {
    +                                forceClose(unifiedSocket);
    +                            }
    +                        }
    +                    });
    +                    workerThreads.add(t);
    +                    t.start();
                     }
    +            } catch (IOException e) {
    +                e.printStackTrace();
    +                throw new RuntimeException(e);
    +            } finally {
    +                forceClose(serverSocket);
                 }
             }
    -        ServerThread serverThread = new ServerThread();
    -        serverThread.start();
     
    +        public void shutdown(long millis) throws InterruptedException {
    +            forceClose(serverSocket); // this should break the run() loop
    +            for (Thread t : workerThreads) {
    +                t.join(millis);
    +            }
    +            this.join(millis);
    +        }
    +
    +        synchronized byte[] getDataFromClient(int index) {
    +            return dataFromClients.get(index);
    +        }
    +    }
    +
    +    private SSLSocket connectWithSSL() throws IOException, X509Exception, InterruptedException {
             SSLSocket sslSocket = null;
             int retries = 0;
             while (retries < MAX_RETRIES) {
                 try {
                     sslSocket = x509Util.createSSLSocket();
    +                sslSocket.addHandshakeCompletedListener(new HandshakeCompletedListener() {
    +                    @Override
    +                    public void handshakeCompleted(HandshakeCompletedEvent handshakeCompletedEvent) {
    +                        synchronized (handshakeCompletedLock) {
    +                            handshakeCompleted = true;
    +                            handshakeCompletedLock.notifyAll();
    +                        }
    +                    }
    +                });
                     sslSocket.setSoTimeout(TIMEOUT);
    -                sslSocket.connect(new InetSocketAddress(port), TIMEOUT);
    +                sslSocket.connect(localServerAddress, TIMEOUT);
                     break;
                 } catch (ConnectException connectException) {
                     connectException.printStackTrace();
    +                forceClose(sslSocket);
    +                sslSocket = null;
                     Thread.sleep(TIMEOUT);
                 }
                 retries++;
             }
     
    -        sslSocket.addHandshakeCompletedListener(new HandshakeCompletedListener() {
    -            @Override
    -            public void handshakeCompleted(HandshakeCompletedEvent handshakeCompletedEvent) {
    -                completeHandshake();
    +        Assert.assertNotNull("Failed to connect to server with SSL", sslSocket);
    +        return sslSocket;
    +    }
    +
    +    private Socket connectWithoutSSL() throws IOException, InterruptedException {
    +        Socket socket = null;
    +        int retries = 0;
    +        while (retries < MAX_RETRIES) {
    +            try {
    +                socket = new Socket();
    +                socket.setSoTimeout(TIMEOUT);
    +                socket.connect(localServerAddress, TIMEOUT);
    +                break;
    +            } catch (ConnectException connectException) {
    +                connectException.printStackTrace();
    +                forceClose(socket);
    +                socket = null;
    +                Thread.sleep(TIMEOUT);
                 }
    -        });
    -        sslSocket.startHandshake();
    +            retries++;
    +        }
    +        Assert.assertNotNull("Failed to connect to server without SSL", socket);
    +        return socket;
    +    }
    +
    +    // In the tests below, a "Strict" server means a UnifiedServerSocket that
    +    // does not allow plaintext connections (in other words, it's SSL-only).
    +    // A "Non Strict" server means a UnifiedServerSocket that allows both
    +    // plaintext and SSL incoming connections.
    +
    +    /**
    +     * Attempting to connect to a SSL-or-plaintext server with SSL should work.
    +     */
    +    @Test
    +    public void testConnectWithSSLToNonStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, true, DATA_TO_CLIENT);
    +        serverThread.start();
     
    -        serverThread.join(TIMEOUT);
    +        Socket sslSocket = connectWithSSL();
    +        sslSocket.getOutputStream().write(DATA_FROM_CLIENT);
    +        sslSocket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = sslSocket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
     
    -        long start = Time.currentElapsedTime();
    -        while (Time.currentElapsedTime() < start + TIMEOUT) {
    -            if (handshakeCompleted) {
    -                return;
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(sslSocket);
    +
    +        synchronized (handshakeCompletedLock) {
    +            if (!handshakeCompleted) {
    +                handshakeCompletedLock.wait(TIMEOUT);
                 }
    +            Assert.assertTrue(handshakeCompleted);
             }
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
    +    }
    +
    +    /**
    +     * Attempting to connect to a SSL-only server with SSL should work.
    +     */
    +    @Test
    +    public void testConnectWithSSLToStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, false, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket sslSocket = connectWithSSL();
    +        sslSocket.getOutputStream().write(DATA_FROM_CLIENT);
    +        sslSocket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = sslSocket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
     
    -        Assert.fail("failed to complete handshake");
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(sslSocket);
    +
    +        synchronized (handshakeCompletedLock) {
    +            if (!handshakeCompleted) {
    +                handshakeCompletedLock.wait(TIMEOUT);
    +            }
    +            Assert.assertTrue(handshakeCompleted);
    +        }
    +
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
         }
     
    -    private void completeHandshake() {
    -        handshakeCompleted = true;
    +    /**
    +     * Attempting to connect to a SSL-or-plaintext server without SSL should work.
    +     */
    +    @Test
    +    public void testConnectWithoutSSLToNonStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, true, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket socket = connectWithoutSSL();
    +        socket.getOutputStream().write(DATA_FROM_CLIENT);
    +        socket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = socket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
    +
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(socket);
    +
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
         }
     
    +    /**
    +     * Attempting to connect to a SSL-or-plaintext server without SSL with a
    +     * small initial data write should work. This makes sure that sending
    +     * less than 5 bytes does not break the logic in the server's initial 5
    +     * byte read.
    +     */
         @Test
    -    public void testConnectWithoutSSL() throws Exception {
    -        final byte[] testData = "hello there".getBytes();
    -        final String[] dataReadFromClient = {null};
    +    public void testConnectWithoutSSLToNonStrictServerPartialWrite() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, true, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket socket = connectWithoutSSL();
    +        // Write only 2 bytes of the message, wait a bit, then write the rest.
    +        // This makes sure that writes smaller than 5 bytes don't break the plaintext mode on the server
    +        // once it decides that the input doesn't look like a TLS handshake.
    +        socket.getOutputStream().write(DATA_FROM_CLIENT, 0, 2);
    +        socket.getOutputStream().flush();
    +        Thread.sleep(TIMEOUT / 2);
    +        socket.getOutputStream().write(DATA_FROM_CLIENT, 2, DATA_FROM_CLIENT.length - 2);
    +        socket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = socket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
    +
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(socket);
     
    -        class ServerThread extends Thread {
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
    +    }
    +
    +    /**
    +     * Attempting to connect to a SSL-only server without SSL should fail.
    +     */
    +    @Test
    +    public void testConnectWithoutSSLToStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, false, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket socket = connectWithoutSSL();
    +        socket.getOutputStream().write(DATA_FROM_CLIENT);
    +        socket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        try {
    +            socket.getInputStream().read(buf, 0, buf.length);
    +        } catch (SocketException e) {
    +            // We expect the other end to hang up the connection
    +            return;
    +        } finally {
    +            serverThread.shutdown(TIMEOUT);
    +            forceClose(socket);
    +        }
    +        Assert.fail("Expected server to hang up the connection. Read from server succeeded unexpectedly.");
    +    }
    +
    +    /**
    +     * This test makes sure that UnifiedServerSocket used properly (a single thread accept()-ing connections and
    +     * handing the resulting sockets to other threads for processing) is not vulnerable to a simple denial-of-service
    +     * attack in which a client connects and never writes any bytes. This should not block the accepting thread, since
    +     * the read to determine if the client is sending a TLS handshake or not happens in the processing thread.
    +     *
    +     * This version of the test uses a non-strict server socket (i.e. it accepts both TLS and plaintext connections).
    +     */
    +    @Test
    +    public void testDenialOfServiceResistanceNonStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    --- End diff --
    
    @ivmaykov That's pretty much the same as I suggested: keep all unit tests except the ones which contains the word "dos". Also please open a new jira about adding DOS tests.


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r233645255
  
    --- Diff: zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java ---
    @@ -17,156 +17,644 @@
      */
     package org.apache.zookeeper.server.quorum;
     
    +import java.io.BufferedInputStream;
    +import java.io.IOException;
    +import java.net.ConnectException;
    +import java.net.InetSocketAddress;
    +import java.net.Socket;
    +import java.net.SocketException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Random;
    +
    +import javax.net.ssl.HandshakeCompletedEvent;
    +import javax.net.ssl.HandshakeCompletedListener;
    +import javax.net.ssl.SSLSocket;
    +
     import org.apache.zookeeper.PortAssignment;
     import org.apache.zookeeper.client.ZKClientConfig;
    +import org.apache.zookeeper.common.BaseX509ParameterizedTestCase;
     import org.apache.zookeeper.common.ClientX509Util;
    -import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.common.KeyStoreFileType;
    +import org.apache.zookeeper.common.X509Exception;
    +import org.apache.zookeeper.common.X509KeyType;
    +import org.apache.zookeeper.common.X509TestContext;
     import org.apache.zookeeper.common.X509Util;
     import org.apache.zookeeper.server.ServerCnxnFactory;
     import org.junit.Assert;
     import org.junit.Before;
     import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
     
    -import javax.net.ssl.HandshakeCompletedEvent;
    -import javax.net.ssl.HandshakeCompletedListener;
    -import javax.net.ssl.SSLSocket;
    -import java.io.IOException;
    -import java.net.ConnectException;
    -import java.net.InetSocketAddress;
    -import java.net.Socket;
    -
    -import static org.hamcrest.CoreMatchers.equalTo;
    -import static org.junit.Assert.assertThat;
    +@RunWith(Parameterized.class)
    +public class UnifiedServerSocketTest extends BaseX509ParameterizedTestCase {
     
    -public class UnifiedServerSocketTest {
    +    @Parameterized.Parameters
    +    public static Collection<Object[]> params() {
    +        ArrayList<Object[]> result = new ArrayList<>();
    +        int paramIndex = 0;
    +        for (X509KeyType caKeyType : X509KeyType.values()) {
    +            for (X509KeyType certKeyType : X509KeyType.values()) {
    +                for (Boolean hostnameVerification : new Boolean[] { true, false  }) {
    +                    result.add(new Object[]{
    +                            caKeyType,
    +                            certKeyType,
    +                            hostnameVerification,
    +                            paramIndex++
    +                    });
    +                }
    +            }
    +        }
    +        return result;
    +    }
     
         private static final int MAX_RETRIES = 5;
         private static final int TIMEOUT = 1000;
    +    private static final byte[] DATA_TO_CLIENT = "hello client".getBytes();
    +    private static final byte[] DATA_FROM_CLIENT = "hello server".getBytes();
     
         private X509Util x509Util;
         private int port;
    -    private volatile boolean handshakeCompleted;
    +    private InetSocketAddress localServerAddress;
    +    private final Object handshakeCompletedLock = new Object();
    +    // access only inside synchronized(handshakeCompletedLock) { ... } blocks
    +    private boolean handshakeCompleted = false;
    +
    +    public UnifiedServerSocketTest(
    +            final X509KeyType caKeyType,
    +            final X509KeyType certKeyType,
    +            final Boolean hostnameVerification,
    +            final Integer paramIndex) {
    +        super(paramIndex, () -> {
    +            try {
    +                return X509TestContext.newBuilder()
    +                    .setTempDir(tempDir)
    +                    .setKeyStoreKeyType(certKeyType)
    +                    .setTrustStoreKeyType(caKeyType)
    +                    .setHostnameVerification(hostnameVerification)
    +                    .build();
    +            } catch (Exception e) {
    +                throw new RuntimeException(e);
    +            }
    +        });
    +    }
     
         @Before
         public void setUp() throws Exception {
    -        handshakeCompleted = false;
    -
             port = PortAssignment.unique();
    +        localServerAddress = new InetSocketAddress("localhost", port);
    --- End diff --
    
    Using `InetAddress.getLoopbackAddress()` ?


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
GitHub user ivmaykov reopened a pull request:

    https://github.com/apache/zookeeper/pull/679

    ZOOKEEPER-3172: Quorum TLS - fix port unification to allow rolling upgrades

    Fix numerous problems with UnifiedServerSocket, such as hanging the accept() thread when the client doesn't send any data or crashing if less than 5 bytes are read from the socket in the initial read.
    
    Re-enable the "portUnification" config option.
    
    Note that this is stacked on top of #678 and thus includes it. Please only consider the ZOOKEEPER-3172 commit when reviewing. Once the other PR is merged upstream, I will rebase this so it only contains one commit.
    
    ## Fixed networking issues/bugs in UnifiedServerSocket
    
    - don't crash the `accept()` thread if the client closes the connection without sending any data
    - don't corrupt the connection if the client sends fewer than 5 bytes for the initial read
    - delay the detection of TLS vs. plaintext mode until a socket stream is read from or written to. This prevents the `accept()` thread from getting blocked on a `read()` operation from the newly connected socket.
    - prepending 5 bytes to `PrependableSocket` and then trying to read >5 bytes would only return the first 5 bytes, even if more bytes were available. This is fixed.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ivmaykov/zookeeper ZOOKEEPER-3172

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/679.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #679
    
----
commit 2122c8c23a0dbb27f9b2aff55e800e48d253f943
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T00:41:48Z

    ZOOKEEPER-3173: Quorum TLS - support PEM trust/key stores
    ZOOKEEPER-3175: Quorum TLS - test improvements
    
    Add support for loading key and trust stores from PEM files.
    Also added test utils for testing X509-related code, because it
    was very difficult to untangle them from the PEM support code.

commit 514d48a26aeeca37290ad14ff8f0cdae69b53eb2
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T01:22:24Z

    ZOOKEEPER-3172: Quorum TLS - fix port unification to allow rolling upgrades

----


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2511/



---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
GitHub user ivmaykov reopened a pull request:

    https://github.com/apache/zookeeper/pull/679

    ZOOKEEPER-3172: Quorum TLS - fix port unification to allow rolling upgrades

    Fix numerous problems with UnifiedServerSocket, such as hanging the accept() thread when the client doesn't send any data or crashing if less than 5 bytes are read from the socket in the initial read.
    
    Re-enable the "portUnification" config option.
    
    ## Fixed networking issues/bugs in UnifiedServerSocket
    
    - don't crash the `accept()` thread if the client closes the connection without sending any data
    - don't corrupt the connection if the client sends fewer than 5 bytes for the initial read
    - delay the detection of TLS vs. plaintext mode until a socket stream is read from or written to. This prevents the `accept()` thread from getting blocked on a `read()` operation from the newly connected socket.
    - prepending 5 bytes to `PrependableSocket` and then trying to read >5 bytes would only return the first 5 bytes, even if more bytes were available. This is fixed.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ivmaykov/zookeeper ZOOKEEPER-3172

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/679.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #679
    
----

----


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov closed the pull request at:

    https://github.com/apache/zookeeper/pull/679


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r233633340
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/PrependableSocket.java ---
    @@ -18,32 +18,47 @@
     
     package org.apache.zookeeper.server.quorum;
     
    -import java.io.ByteArrayInputStream;
     import java.io.IOException;
     import java.io.InputStream;
    -import java.io.SequenceInputStream;
    +import java.io.PushbackInputStream;
     import java.net.Socket;
     import java.net.SocketImpl;
     
     public class PrependableSocket extends Socket {
     
    -  private SequenceInputStream sequenceInputStream;
    +  private PushbackInputStream pushbackInputStream;
    --- End diff --
    
    Please explain again what exactly was the problem with `SequenceInputStream`?
    Why did it return only the first 5 bytes after it had been prepended?


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r233656579
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/PrependableSocket.java ---
    @@ -18,32 +18,47 @@
     
     package org.apache.zookeeper.server.quorum;
     
    -import java.io.ByteArrayInputStream;
     import java.io.IOException;
     import java.io.InputStream;
    -import java.io.SequenceInputStream;
    +import java.io.PushbackInputStream;
     import java.net.Socket;
     import java.net.SocketImpl;
     
     public class PrependableSocket extends Socket {
     
    -  private SequenceInputStream sequenceInputStream;
    +  private PushbackInputStream pushbackInputStream;
    --- End diff --
    
    I don't know why it worked like that, but that is what I observed in tests. It looks like SequenceInputStream does not join returned data across boundaries of the underlying streams when the first stream gets to EOF. I don't think this is desired behavior since it would cause the stream to return 5 bytes when more than 5 bytes are actually available.


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r234052921
  
    --- Diff: zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java ---
    @@ -17,156 +17,644 @@
      */
     package org.apache.zookeeper.server.quorum;
     
    +import java.io.BufferedInputStream;
    +import java.io.IOException;
    +import java.net.ConnectException;
    +import java.net.InetSocketAddress;
    +import java.net.Socket;
    +import java.net.SocketException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Random;
    +
    +import javax.net.ssl.HandshakeCompletedEvent;
    +import javax.net.ssl.HandshakeCompletedListener;
    +import javax.net.ssl.SSLSocket;
    +
     import org.apache.zookeeper.PortAssignment;
     import org.apache.zookeeper.client.ZKClientConfig;
    +import org.apache.zookeeper.common.BaseX509ParameterizedTestCase;
     import org.apache.zookeeper.common.ClientX509Util;
    -import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.common.KeyStoreFileType;
    +import org.apache.zookeeper.common.X509Exception;
    +import org.apache.zookeeper.common.X509KeyType;
    +import org.apache.zookeeper.common.X509TestContext;
     import org.apache.zookeeper.common.X509Util;
     import org.apache.zookeeper.server.ServerCnxnFactory;
     import org.junit.Assert;
     import org.junit.Before;
     import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
     
    -import javax.net.ssl.HandshakeCompletedEvent;
    -import javax.net.ssl.HandshakeCompletedListener;
    -import javax.net.ssl.SSLSocket;
    -import java.io.IOException;
    -import java.net.ConnectException;
    -import java.net.InetSocketAddress;
    -import java.net.Socket;
    -
    -import static org.hamcrest.CoreMatchers.equalTo;
    -import static org.junit.Assert.assertThat;
    +@RunWith(Parameterized.class)
    +public class UnifiedServerSocketTest extends BaseX509ParameterizedTestCase {
     
    -public class UnifiedServerSocketTest {
    +    @Parameterized.Parameters
    +    public static Collection<Object[]> params() {
    +        ArrayList<Object[]> result = new ArrayList<>();
    +        int paramIndex = 0;
    +        for (X509KeyType caKeyType : X509KeyType.values()) {
    +            for (X509KeyType certKeyType : X509KeyType.values()) {
    +                for (Boolean hostnameVerification : new Boolean[] { true, false  }) {
    +                    result.add(new Object[]{
    +                            caKeyType,
    +                            certKeyType,
    +                            hostnameVerification,
    +                            paramIndex++
    +                    });
    +                }
    +            }
    +        }
    +        return result;
    +    }
     
         private static final int MAX_RETRIES = 5;
         private static final int TIMEOUT = 1000;
    +    private static final byte[] DATA_TO_CLIENT = "hello client".getBytes();
    +    private static final byte[] DATA_FROM_CLIENT = "hello server".getBytes();
     
         private X509Util x509Util;
         private int port;
    -    private volatile boolean handshakeCompleted;
    +    private InetSocketAddress localServerAddress;
    +    private final Object handshakeCompletedLock = new Object();
    +    // access only inside synchronized(handshakeCompletedLock) { ... } blocks
    +    private boolean handshakeCompleted = false;
    +
    +    public UnifiedServerSocketTest(
    +            final X509KeyType caKeyType,
    +            final X509KeyType certKeyType,
    +            final Boolean hostnameVerification,
    +            final Integer paramIndex) {
    +        super(paramIndex, () -> {
    +            try {
    +                return X509TestContext.newBuilder()
    +                    .setTempDir(tempDir)
    +                    .setKeyStoreKeyType(certKeyType)
    +                    .setTrustStoreKeyType(caKeyType)
    +                    .setHostnameVerification(hostnameVerification)
    +                    .build();
    +            } catch (Exception e) {
    +                throw new RuntimeException(e);
    +            }
    +        });
    +    }
     
         @Before
         public void setUp() throws Exception {
    -        handshakeCompleted = false;
    -
             port = PortAssignment.unique();
    +        localServerAddress = new InetSocketAddress("localhost", port);
     
    -        String testDataPath = System.getProperty("test.data.dir", "build/test/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");
     
             x509Util = new ClientX509Util();
     
    -        System.setProperty(x509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks");
    -        System.setProperty(x509Util.getSslKeystorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks");
    -        System.setProperty(x509Util.getSslTruststorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslHostnameVerificationEnabledProperty(), "false");
    +        x509TestContext.setSystemProperties(x509Util, KeyStoreFileType.JKS, KeyStoreFileType.JKS);
         }
     
    -    @Test
    -    public void testConnectWithSSL() throws Exception {
    -        class ServerThread extends Thread {
    -            public void run() {
    -                try {
    -                    Socket unifiedSocket = new UnifiedServerSocket(x509Util, port).accept();
    -                    ((SSLSocket)unifiedSocket).getSession(); // block until handshake completes
    -                } catch (IOException e) {
    -                    e.printStackTrace();
    +    private static void forceClose(java.io.Closeable s) {
    +        if (s == null) {
    +            return;
    +        }
    +        try {
    +            s.close();
    +        } catch (IOException e) {
    +            e.printStackTrace();
    +        }
    +    }
    +
    +    private static final class UnifiedServerThread extends Thread {
    +        private final byte[] dataToClient;
    +        private List<byte[]> dataFromClients;
    +        private List<Thread> workerThreads;
    --- End diff --
    
    Sure, will change it to an Executor


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/zookeeper/pull/679


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2600/



---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov closed the pull request at:

    https://github.com/apache/zookeeper/pull/679


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r233647195
  
    --- Diff: zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java ---
    @@ -17,156 +17,644 @@
      */
     package org.apache.zookeeper.server.quorum;
     
    +import java.io.BufferedInputStream;
    +import java.io.IOException;
    +import java.net.ConnectException;
    +import java.net.InetSocketAddress;
    +import java.net.Socket;
    +import java.net.SocketException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Random;
    +
    +import javax.net.ssl.HandshakeCompletedEvent;
    +import javax.net.ssl.HandshakeCompletedListener;
    +import javax.net.ssl.SSLSocket;
    +
     import org.apache.zookeeper.PortAssignment;
     import org.apache.zookeeper.client.ZKClientConfig;
    +import org.apache.zookeeper.common.BaseX509ParameterizedTestCase;
     import org.apache.zookeeper.common.ClientX509Util;
    -import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.common.KeyStoreFileType;
    +import org.apache.zookeeper.common.X509Exception;
    +import org.apache.zookeeper.common.X509KeyType;
    +import org.apache.zookeeper.common.X509TestContext;
     import org.apache.zookeeper.common.X509Util;
     import org.apache.zookeeper.server.ServerCnxnFactory;
     import org.junit.Assert;
     import org.junit.Before;
     import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
     
    -import javax.net.ssl.HandshakeCompletedEvent;
    -import javax.net.ssl.HandshakeCompletedListener;
    -import javax.net.ssl.SSLSocket;
    -import java.io.IOException;
    -import java.net.ConnectException;
    -import java.net.InetSocketAddress;
    -import java.net.Socket;
    -
    -import static org.hamcrest.CoreMatchers.equalTo;
    -import static org.junit.Assert.assertThat;
    +@RunWith(Parameterized.class)
    +public class UnifiedServerSocketTest extends BaseX509ParameterizedTestCase {
     
    -public class UnifiedServerSocketTest {
    +    @Parameterized.Parameters
    +    public static Collection<Object[]> params() {
    +        ArrayList<Object[]> result = new ArrayList<>();
    +        int paramIndex = 0;
    +        for (X509KeyType caKeyType : X509KeyType.values()) {
    +            for (X509KeyType certKeyType : X509KeyType.values()) {
    +                for (Boolean hostnameVerification : new Boolean[] { true, false  }) {
    +                    result.add(new Object[]{
    +                            caKeyType,
    +                            certKeyType,
    +                            hostnameVerification,
    +                            paramIndex++
    +                    });
    +                }
    +            }
    +        }
    +        return result;
    +    }
     
         private static final int MAX_RETRIES = 5;
         private static final int TIMEOUT = 1000;
    +    private static final byte[] DATA_TO_CLIENT = "hello client".getBytes();
    +    private static final byte[] DATA_FROM_CLIENT = "hello server".getBytes();
     
         private X509Util x509Util;
         private int port;
    -    private volatile boolean handshakeCompleted;
    +    private InetSocketAddress localServerAddress;
    +    private final Object handshakeCompletedLock = new Object();
    +    // access only inside synchronized(handshakeCompletedLock) { ... } blocks
    +    private boolean handshakeCompleted = false;
    +
    +    public UnifiedServerSocketTest(
    +            final X509KeyType caKeyType,
    +            final X509KeyType certKeyType,
    +            final Boolean hostnameVerification,
    +            final Integer paramIndex) {
    +        super(paramIndex, () -> {
    +            try {
    +                return X509TestContext.newBuilder()
    +                    .setTempDir(tempDir)
    +                    .setKeyStoreKeyType(certKeyType)
    +                    .setTrustStoreKeyType(caKeyType)
    +                    .setHostnameVerification(hostnameVerification)
    +                    .build();
    +            } catch (Exception e) {
    +                throw new RuntimeException(e);
    +            }
    +        });
    +    }
     
         @Before
         public void setUp() throws Exception {
    -        handshakeCompleted = false;
    -
             port = PortAssignment.unique();
    +        localServerAddress = new InetSocketAddress("localhost", port);
     
    -        String testDataPath = System.getProperty("test.data.dir", "build/test/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");
     
             x509Util = new ClientX509Util();
     
    -        System.setProperty(x509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks");
    -        System.setProperty(x509Util.getSslKeystorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks");
    -        System.setProperty(x509Util.getSslTruststorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslHostnameVerificationEnabledProperty(), "false");
    +        x509TestContext.setSystemProperties(x509Util, KeyStoreFileType.JKS, KeyStoreFileType.JKS);
         }
     
    -    @Test
    -    public void testConnectWithSSL() throws Exception {
    -        class ServerThread extends Thread {
    -            public void run() {
    -                try {
    -                    Socket unifiedSocket = new UnifiedServerSocket(x509Util, port).accept();
    -                    ((SSLSocket)unifiedSocket).getSession(); // block until handshake completes
    -                } catch (IOException e) {
    -                    e.printStackTrace();
    +    private static void forceClose(java.io.Closeable s) {
    +        if (s == null) {
    +            return;
    +        }
    +        try {
    +            s.close();
    +        } catch (IOException e) {
    +            e.printStackTrace();
    +        }
    +    }
    +
    +    private static final class UnifiedServerThread extends Thread {
    +        private final byte[] dataToClient;
    +        private List<byte[]> dataFromClients;
    +        private List<Thread> workerThreads;
    +        private UnifiedServerSocket serverSocket;
    +
    +        UnifiedServerThread(X509Util x509Util,
    +                            InetSocketAddress bindAddress,
    +                            boolean allowInsecureConnection,
    +                            byte[] dataToClient) throws IOException {
    +            this.dataToClient = dataToClient;
    +            dataFromClients = new ArrayList<>();
    +            workerThreads = new ArrayList<>();
    +            serverSocket = new UnifiedServerSocket(x509Util, allowInsecureConnection);
    +            serverSocket.bind(bindAddress);
    +        }
    +
    +        @Override
    +        public void run() {
    +            try {
    +                Random rnd = new Random();
    +                while (true) {
    +                    final Socket unifiedSocket = serverSocket.accept();
    +                    final boolean tcpNoDelay = rnd.nextBoolean();
    +                    unifiedSocket.setTcpNoDelay(tcpNoDelay);
    +                    unifiedSocket.setSoTimeout(TIMEOUT);
    +                    final boolean keepAlive = rnd.nextBoolean();
    +                    unifiedSocket.setKeepAlive(keepAlive);
    +                    // Note: getting the input stream should not block the thread or trigger mode detection.
    +                    BufferedInputStream bis = new BufferedInputStream(unifiedSocket.getInputStream());
    +                    Thread t = new Thread(new Runnable() {
    +                        @Override
    +                        public void run() {
    +                            try {
    +                                byte[] buf = new byte[1024];
    +                                int bytesRead = unifiedSocket.getInputStream().read(buf, 0, 1024);
    +                                // Make sure the settings applied above before the socket was potentially upgraded to
    +                                // TLS still apply.
    +                                Assert.assertEquals(tcpNoDelay, unifiedSocket.getTcpNoDelay());
    +                                Assert.assertEquals(TIMEOUT, unifiedSocket.getSoTimeout());
    +                                Assert.assertEquals(keepAlive, unifiedSocket.getKeepAlive());
    +                                if (bytesRead > 0) {
    +                                    byte[] dataFromClient = new byte[bytesRead];
    +                                    System.arraycopy(buf, 0, dataFromClient, 0, bytesRead);
    +                                    synchronized (dataFromClients) {
    +                                        dataFromClients.add(dataFromClient);
    +                                    }
    +                                }
    +                                unifiedSocket.getOutputStream().write(dataToClient);
    +                                unifiedSocket.getOutputStream().flush();
    +                            } catch (IOException e) {
    +                                e.printStackTrace();
    +                                throw new RuntimeException(e);
    +                            } finally {
    +                                forceClose(unifiedSocket);
    +                            }
    +                        }
    +                    });
    +                    workerThreads.add(t);
    +                    t.start();
                     }
    +            } catch (IOException e) {
    +                e.printStackTrace();
    +                throw new RuntimeException(e);
    +            } finally {
    +                forceClose(serverSocket);
                 }
             }
    -        ServerThread serverThread = new ServerThread();
    -        serverThread.start();
     
    +        public void shutdown(long millis) throws InterruptedException {
    +            forceClose(serverSocket); // this should break the run() loop
    +            for (Thread t : workerThreads) {
    +                t.join(millis);
    +            }
    +            this.join(millis);
    +        }
    +
    +        synchronized byte[] getDataFromClient(int index) {
    +            return dataFromClients.get(index);
    +        }
    +    }
    +
    +    private SSLSocket connectWithSSL() throws IOException, X509Exception, InterruptedException {
             SSLSocket sslSocket = null;
             int retries = 0;
             while (retries < MAX_RETRIES) {
                 try {
                     sslSocket = x509Util.createSSLSocket();
    +                sslSocket.addHandshakeCompletedListener(new HandshakeCompletedListener() {
    +                    @Override
    +                    public void handshakeCompleted(HandshakeCompletedEvent handshakeCompletedEvent) {
    +                        synchronized (handshakeCompletedLock) {
    +                            handshakeCompleted = true;
    +                            handshakeCompletedLock.notifyAll();
    +                        }
    +                    }
    +                });
                     sslSocket.setSoTimeout(TIMEOUT);
    -                sslSocket.connect(new InetSocketAddress(port), TIMEOUT);
    +                sslSocket.connect(localServerAddress, TIMEOUT);
                     break;
                 } catch (ConnectException connectException) {
                     connectException.printStackTrace();
    +                forceClose(sslSocket);
    +                sslSocket = null;
                     Thread.sleep(TIMEOUT);
                 }
                 retries++;
             }
     
    -        sslSocket.addHandshakeCompletedListener(new HandshakeCompletedListener() {
    -            @Override
    -            public void handshakeCompleted(HandshakeCompletedEvent handshakeCompletedEvent) {
    -                completeHandshake();
    +        Assert.assertNotNull("Failed to connect to server with SSL", sslSocket);
    +        return sslSocket;
    +    }
    +
    +    private Socket connectWithoutSSL() throws IOException, InterruptedException {
    +        Socket socket = null;
    +        int retries = 0;
    +        while (retries < MAX_RETRIES) {
    +            try {
    +                socket = new Socket();
    +                socket.setSoTimeout(TIMEOUT);
    +                socket.connect(localServerAddress, TIMEOUT);
    +                break;
    +            } catch (ConnectException connectException) {
    +                connectException.printStackTrace();
    +                forceClose(socket);
    +                socket = null;
    +                Thread.sleep(TIMEOUT);
                 }
    -        });
    -        sslSocket.startHandshake();
    +            retries++;
    +        }
    +        Assert.assertNotNull("Failed to connect to server without SSL", socket);
    +        return socket;
    +    }
    +
    +    // In the tests below, a "Strict" server means a UnifiedServerSocket that
    +    // does not allow plaintext connections (in other words, it's SSL-only).
    +    // A "Non Strict" server means a UnifiedServerSocket that allows both
    +    // plaintext and SSL incoming connections.
    +
    +    /**
    +     * Attempting to connect to a SSL-or-plaintext server with SSL should work.
    +     */
    +    @Test
    +    public void testConnectWithSSLToNonStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, true, DATA_TO_CLIENT);
    +        serverThread.start();
     
    -        serverThread.join(TIMEOUT);
    +        Socket sslSocket = connectWithSSL();
    +        sslSocket.getOutputStream().write(DATA_FROM_CLIENT);
    +        sslSocket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = sslSocket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
     
    -        long start = Time.currentElapsedTime();
    -        while (Time.currentElapsedTime() < start + TIMEOUT) {
    -            if (handshakeCompleted) {
    -                return;
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(sslSocket);
    +
    +        synchronized (handshakeCompletedLock) {
    +            if (!handshakeCompleted) {
    +                handshakeCompletedLock.wait(TIMEOUT);
                 }
    +            Assert.assertTrue(handshakeCompleted);
             }
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
    +    }
    +
    +    /**
    +     * Attempting to connect to a SSL-only server with SSL should work.
    +     */
    +    @Test
    +    public void testConnectWithSSLToStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, false, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket sslSocket = connectWithSSL();
    +        sslSocket.getOutputStream().write(DATA_FROM_CLIENT);
    +        sslSocket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = sslSocket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
     
    -        Assert.fail("failed to complete handshake");
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(sslSocket);
    +
    +        synchronized (handshakeCompletedLock) {
    +            if (!handshakeCompleted) {
    +                handshakeCompletedLock.wait(TIMEOUT);
    +            }
    +            Assert.assertTrue(handshakeCompleted);
    +        }
    +
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
         }
     
    -    private void completeHandshake() {
    -        handshakeCompleted = true;
    +    /**
    +     * Attempting to connect to a SSL-or-plaintext server without SSL should work.
    +     */
    +    @Test
    +    public void testConnectWithoutSSLToNonStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, true, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket socket = connectWithoutSSL();
    +        socket.getOutputStream().write(DATA_FROM_CLIENT);
    +        socket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = socket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
    +
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(socket);
    +
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
         }
     
    +    /**
    +     * Attempting to connect to a SSL-or-plaintext server without SSL with a
    +     * small initial data write should work. This makes sure that sending
    +     * less than 5 bytes does not break the logic in the server's initial 5
    +     * byte read.
    +     */
         @Test
    -    public void testConnectWithoutSSL() throws Exception {
    -        final byte[] testData = "hello there".getBytes();
    -        final String[] dataReadFromClient = {null};
    +    public void testConnectWithoutSSLToNonStrictServerPartialWrite() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, true, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket socket = connectWithoutSSL();
    +        // Write only 2 bytes of the message, wait a bit, then write the rest.
    +        // This makes sure that writes smaller than 5 bytes don't break the plaintext mode on the server
    +        // once it decides that the input doesn't look like a TLS handshake.
    +        socket.getOutputStream().write(DATA_FROM_CLIENT, 0, 2);
    +        socket.getOutputStream().flush();
    +        Thread.sleep(TIMEOUT / 2);
    +        socket.getOutputStream().write(DATA_FROM_CLIENT, 2, DATA_FROM_CLIENT.length - 2);
    +        socket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = socket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
    +
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(socket);
     
    -        class ServerThread extends Thread {
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
    +    }
    +
    +    /**
    +     * Attempting to connect to a SSL-only server without SSL should fail.
    +     */
    +    @Test
    +    public void testConnectWithoutSSLToStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, false, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket socket = connectWithoutSSL();
    +        socket.getOutputStream().write(DATA_FROM_CLIENT);
    +        socket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        try {
    +            socket.getInputStream().read(buf, 0, buf.length);
    +        } catch (SocketException e) {
    +            // We expect the other end to hang up the connection
    +            return;
    +        } finally {
    +            serverThread.shutdown(TIMEOUT);
    +            forceClose(socket);
    +        }
    +        Assert.fail("Expected server to hang up the connection. Read from server succeeded unexpectedly.");
    +    }
    +
    +    /**
    +     * This test makes sure that UnifiedServerSocket used properly (a single thread accept()-ing connections and
    +     * handing the resulting sockets to other threads for processing) is not vulnerable to a simple denial-of-service
    +     * attack in which a client connects and never writes any bytes. This should not block the accepting thread, since
    +     * the read to determine if the client is sending a TLS handshake or not happens in the processing thread.
    +     *
    +     * This version of the test uses a non-strict server socket (i.e. it accepts both TLS and plaintext connections).
    +     */
    +    @Test
    +    public void testDenialOfServiceResistanceNonStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    --- End diff --
    
    In these DOS tests you're actually testing the `UnifiedServerThread` implementation which is a test-only thing. Does it make sense or am I missing something here?
    
    In "reality" the accept thread is basically the `Leader` and the one which does TLS handshake is the `LearnerHandler`.


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r233668370
  
    --- Diff: zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java ---
    @@ -17,156 +17,644 @@
      */
     package org.apache.zookeeper.server.quorum;
     
    +import java.io.BufferedInputStream;
    +import java.io.IOException;
    +import java.net.ConnectException;
    +import java.net.InetSocketAddress;
    +import java.net.Socket;
    +import java.net.SocketException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Random;
    +
    +import javax.net.ssl.HandshakeCompletedEvent;
    +import javax.net.ssl.HandshakeCompletedListener;
    +import javax.net.ssl.SSLSocket;
    +
     import org.apache.zookeeper.PortAssignment;
     import org.apache.zookeeper.client.ZKClientConfig;
    +import org.apache.zookeeper.common.BaseX509ParameterizedTestCase;
     import org.apache.zookeeper.common.ClientX509Util;
    -import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.common.KeyStoreFileType;
    +import org.apache.zookeeper.common.X509Exception;
    +import org.apache.zookeeper.common.X509KeyType;
    +import org.apache.zookeeper.common.X509TestContext;
     import org.apache.zookeeper.common.X509Util;
     import org.apache.zookeeper.server.ServerCnxnFactory;
     import org.junit.Assert;
     import org.junit.Before;
     import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
     
    -import javax.net.ssl.HandshakeCompletedEvent;
    -import javax.net.ssl.HandshakeCompletedListener;
    -import javax.net.ssl.SSLSocket;
    -import java.io.IOException;
    -import java.net.ConnectException;
    -import java.net.InetSocketAddress;
    -import java.net.Socket;
    -
    -import static org.hamcrest.CoreMatchers.equalTo;
    -import static org.junit.Assert.assertThat;
    +@RunWith(Parameterized.class)
    +public class UnifiedServerSocketTest extends BaseX509ParameterizedTestCase {
     
    -public class UnifiedServerSocketTest {
    +    @Parameterized.Parameters
    +    public static Collection<Object[]> params() {
    +        ArrayList<Object[]> result = new ArrayList<>();
    +        int paramIndex = 0;
    +        for (X509KeyType caKeyType : X509KeyType.values()) {
    +            for (X509KeyType certKeyType : X509KeyType.values()) {
    +                for (Boolean hostnameVerification : new Boolean[] { true, false  }) {
    +                    result.add(new Object[]{
    +                            caKeyType,
    +                            certKeyType,
    +                            hostnameVerification,
    +                            paramIndex++
    +                    });
    +                }
    +            }
    +        }
    +        return result;
    +    }
     
         private static final int MAX_RETRIES = 5;
         private static final int TIMEOUT = 1000;
    +    private static final byte[] DATA_TO_CLIENT = "hello client".getBytes();
    +    private static final byte[] DATA_FROM_CLIENT = "hello server".getBytes();
     
         private X509Util x509Util;
         private int port;
    -    private volatile boolean handshakeCompleted;
    +    private InetSocketAddress localServerAddress;
    +    private final Object handshakeCompletedLock = new Object();
    +    // access only inside synchronized(handshakeCompletedLock) { ... } blocks
    +    private boolean handshakeCompleted = false;
    +
    +    public UnifiedServerSocketTest(
    +            final X509KeyType caKeyType,
    +            final X509KeyType certKeyType,
    +            final Boolean hostnameVerification,
    +            final Integer paramIndex) {
    +        super(paramIndex, () -> {
    +            try {
    +                return X509TestContext.newBuilder()
    +                    .setTempDir(tempDir)
    +                    .setKeyStoreKeyType(certKeyType)
    +                    .setTrustStoreKeyType(caKeyType)
    +                    .setHostnameVerification(hostnameVerification)
    +                    .build();
    +            } catch (Exception e) {
    +                throw new RuntimeException(e);
    +            }
    +        });
    +    }
     
         @Before
         public void setUp() throws Exception {
    -        handshakeCompleted = false;
    -
             port = PortAssignment.unique();
    +        localServerAddress = new InetSocketAddress("localhost", port);
     
    -        String testDataPath = System.getProperty("test.data.dir", "build/test/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");
     
             x509Util = new ClientX509Util();
     
    -        System.setProperty(x509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks");
    -        System.setProperty(x509Util.getSslKeystorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks");
    -        System.setProperty(x509Util.getSslTruststorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslHostnameVerificationEnabledProperty(), "false");
    +        x509TestContext.setSystemProperties(x509Util, KeyStoreFileType.JKS, KeyStoreFileType.JKS);
         }
     
    -    @Test
    -    public void testConnectWithSSL() throws Exception {
    -        class ServerThread extends Thread {
    -            public void run() {
    -                try {
    -                    Socket unifiedSocket = new UnifiedServerSocket(x509Util, port).accept();
    -                    ((SSLSocket)unifiedSocket).getSession(); // block until handshake completes
    -                } catch (IOException e) {
    -                    e.printStackTrace();
    +    private static void forceClose(java.io.Closeable s) {
    +        if (s == null) {
    +            return;
    +        }
    +        try {
    +            s.close();
    +        } catch (IOException e) {
    +            e.printStackTrace();
    +        }
    +    }
    +
    +    private static final class UnifiedServerThread extends Thread {
    +        private final byte[] dataToClient;
    +        private List<byte[]> dataFromClients;
    +        private List<Thread> workerThreads;
    +        private UnifiedServerSocket serverSocket;
    +
    +        UnifiedServerThread(X509Util x509Util,
    +                            InetSocketAddress bindAddress,
    +                            boolean allowInsecureConnection,
    +                            byte[] dataToClient) throws IOException {
    +            this.dataToClient = dataToClient;
    +            dataFromClients = new ArrayList<>();
    +            workerThreads = new ArrayList<>();
    +            serverSocket = new UnifiedServerSocket(x509Util, allowInsecureConnection);
    +            serverSocket.bind(bindAddress);
    +        }
    +
    +        @Override
    +        public void run() {
    +            try {
    +                Random rnd = new Random();
    +                while (true) {
    +                    final Socket unifiedSocket = serverSocket.accept();
    +                    final boolean tcpNoDelay = rnd.nextBoolean();
    +                    unifiedSocket.setTcpNoDelay(tcpNoDelay);
    +                    unifiedSocket.setSoTimeout(TIMEOUT);
    +                    final boolean keepAlive = rnd.nextBoolean();
    +                    unifiedSocket.setKeepAlive(keepAlive);
    +                    // Note: getting the input stream should not block the thread or trigger mode detection.
    +                    BufferedInputStream bis = new BufferedInputStream(unifiedSocket.getInputStream());
    +                    Thread t = new Thread(new Runnable() {
    +                        @Override
    +                        public void run() {
    +                            try {
    +                                byte[] buf = new byte[1024];
    +                                int bytesRead = unifiedSocket.getInputStream().read(buf, 0, 1024);
    +                                // Make sure the settings applied above before the socket was potentially upgraded to
    +                                // TLS still apply.
    +                                Assert.assertEquals(tcpNoDelay, unifiedSocket.getTcpNoDelay());
    +                                Assert.assertEquals(TIMEOUT, unifiedSocket.getSoTimeout());
    +                                Assert.assertEquals(keepAlive, unifiedSocket.getKeepAlive());
    +                                if (bytesRead > 0) {
    +                                    byte[] dataFromClient = new byte[bytesRead];
    +                                    System.arraycopy(buf, 0, dataFromClient, 0, bytesRead);
    +                                    synchronized (dataFromClients) {
    +                                        dataFromClients.add(dataFromClient);
    +                                    }
    +                                }
    +                                unifiedSocket.getOutputStream().write(dataToClient);
    +                                unifiedSocket.getOutputStream().flush();
    +                            } catch (IOException e) {
    +                                e.printStackTrace();
    +                                throw new RuntimeException(e);
    +                            } finally {
    +                                forceClose(unifiedSocket);
    +                            }
    +                        }
    +                    });
    +                    workerThreads.add(t);
    +                    t.start();
                     }
    +            } catch (IOException e) {
    +                e.printStackTrace();
    +                throw new RuntimeException(e);
    +            } finally {
    +                forceClose(serverSocket);
                 }
             }
    -        ServerThread serverThread = new ServerThread();
    -        serverThread.start();
     
    +        public void shutdown(long millis) throws InterruptedException {
    +            forceClose(serverSocket); // this should break the run() loop
    +            for (Thread t : workerThreads) {
    +                t.join(millis);
    +            }
    +            this.join(millis);
    +        }
    +
    +        synchronized byte[] getDataFromClient(int index) {
    +            return dataFromClients.get(index);
    +        }
    +    }
    +
    +    private SSLSocket connectWithSSL() throws IOException, X509Exception, InterruptedException {
             SSLSocket sslSocket = null;
             int retries = 0;
             while (retries < MAX_RETRIES) {
                 try {
                     sslSocket = x509Util.createSSLSocket();
    +                sslSocket.addHandshakeCompletedListener(new HandshakeCompletedListener() {
    +                    @Override
    +                    public void handshakeCompleted(HandshakeCompletedEvent handshakeCompletedEvent) {
    +                        synchronized (handshakeCompletedLock) {
    +                            handshakeCompleted = true;
    +                            handshakeCompletedLock.notifyAll();
    +                        }
    +                    }
    +                });
                     sslSocket.setSoTimeout(TIMEOUT);
    -                sslSocket.connect(new InetSocketAddress(port), TIMEOUT);
    +                sslSocket.connect(localServerAddress, TIMEOUT);
                     break;
                 } catch (ConnectException connectException) {
                     connectException.printStackTrace();
    +                forceClose(sslSocket);
    +                sslSocket = null;
                     Thread.sleep(TIMEOUT);
                 }
                 retries++;
             }
     
    -        sslSocket.addHandshakeCompletedListener(new HandshakeCompletedListener() {
    -            @Override
    -            public void handshakeCompleted(HandshakeCompletedEvent handshakeCompletedEvent) {
    -                completeHandshake();
    +        Assert.assertNotNull("Failed to connect to server with SSL", sslSocket);
    +        return sslSocket;
    +    }
    +
    +    private Socket connectWithoutSSL() throws IOException, InterruptedException {
    +        Socket socket = null;
    +        int retries = 0;
    +        while (retries < MAX_RETRIES) {
    +            try {
    +                socket = new Socket();
    +                socket.setSoTimeout(TIMEOUT);
    +                socket.connect(localServerAddress, TIMEOUT);
    +                break;
    +            } catch (ConnectException connectException) {
    +                connectException.printStackTrace();
    +                forceClose(socket);
    +                socket = null;
    +                Thread.sleep(TIMEOUT);
                 }
    -        });
    -        sslSocket.startHandshake();
    +            retries++;
    +        }
    +        Assert.assertNotNull("Failed to connect to server without SSL", socket);
    +        return socket;
    +    }
    +
    +    // In the tests below, a "Strict" server means a UnifiedServerSocket that
    +    // does not allow plaintext connections (in other words, it's SSL-only).
    +    // A "Non Strict" server means a UnifiedServerSocket that allows both
    +    // plaintext and SSL incoming connections.
    +
    +    /**
    +     * Attempting to connect to a SSL-or-plaintext server with SSL should work.
    +     */
    +    @Test
    +    public void testConnectWithSSLToNonStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, true, DATA_TO_CLIENT);
    +        serverThread.start();
     
    -        serverThread.join(TIMEOUT);
    +        Socket sslSocket = connectWithSSL();
    +        sslSocket.getOutputStream().write(DATA_FROM_CLIENT);
    +        sslSocket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = sslSocket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
     
    -        long start = Time.currentElapsedTime();
    -        while (Time.currentElapsedTime() < start + TIMEOUT) {
    -            if (handshakeCompleted) {
    -                return;
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(sslSocket);
    +
    +        synchronized (handshakeCompletedLock) {
    +            if (!handshakeCompleted) {
    +                handshakeCompletedLock.wait(TIMEOUT);
                 }
    +            Assert.assertTrue(handshakeCompleted);
             }
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
    +    }
    +
    +    /**
    +     * Attempting to connect to a SSL-only server with SSL should work.
    +     */
    +    @Test
    +    public void testConnectWithSSLToStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, false, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket sslSocket = connectWithSSL();
    +        sslSocket.getOutputStream().write(DATA_FROM_CLIENT);
    +        sslSocket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = sslSocket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
     
    -        Assert.fail("failed to complete handshake");
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(sslSocket);
    +
    +        synchronized (handshakeCompletedLock) {
    +            if (!handshakeCompleted) {
    +                handshakeCompletedLock.wait(TIMEOUT);
    +            }
    +            Assert.assertTrue(handshakeCompleted);
    +        }
    +
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
         }
     
    -    private void completeHandshake() {
    -        handshakeCompleted = true;
    +    /**
    +     * Attempting to connect to a SSL-or-plaintext server without SSL should work.
    +     */
    +    @Test
    +    public void testConnectWithoutSSLToNonStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, true, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket socket = connectWithoutSSL();
    +        socket.getOutputStream().write(DATA_FROM_CLIENT);
    +        socket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = socket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
    +
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(socket);
    +
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
         }
     
    +    /**
    +     * Attempting to connect to a SSL-or-plaintext server without SSL with a
    +     * small initial data write should work. This makes sure that sending
    +     * less than 5 bytes does not break the logic in the server's initial 5
    +     * byte read.
    +     */
         @Test
    -    public void testConnectWithoutSSL() throws Exception {
    -        final byte[] testData = "hello there".getBytes();
    -        final String[] dataReadFromClient = {null};
    +    public void testConnectWithoutSSLToNonStrictServerPartialWrite() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, true, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket socket = connectWithoutSSL();
    +        // Write only 2 bytes of the message, wait a bit, then write the rest.
    +        // This makes sure that writes smaller than 5 bytes don't break the plaintext mode on the server
    +        // once it decides that the input doesn't look like a TLS handshake.
    +        socket.getOutputStream().write(DATA_FROM_CLIENT, 0, 2);
    +        socket.getOutputStream().flush();
    +        Thread.sleep(TIMEOUT / 2);
    +        socket.getOutputStream().write(DATA_FROM_CLIENT, 2, DATA_FROM_CLIENT.length - 2);
    +        socket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = socket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
    +
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(socket);
     
    -        class ServerThread extends Thread {
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
    +    }
    +
    +    /**
    +     * Attempting to connect to a SSL-only server without SSL should fail.
    +     */
    +    @Test
    +    public void testConnectWithoutSSLToStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, false, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket socket = connectWithoutSSL();
    +        socket.getOutputStream().write(DATA_FROM_CLIENT);
    +        socket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        try {
    +            socket.getInputStream().read(buf, 0, buf.length);
    +        } catch (SocketException e) {
    +            // We expect the other end to hang up the connection
    +            return;
    +        } finally {
    +            serverThread.shutdown(TIMEOUT);
    +            forceClose(socket);
    +        }
    +        Assert.fail("Expected server to hang up the connection. Read from server succeeded unexpectedly.");
    +    }
    +
    +    /**
    +     * This test makes sure that UnifiedServerSocket used properly (a single thread accept()-ing connections and
    +     * handing the resulting sockets to other threads for processing) is not vulnerable to a simple denial-of-service
    +     * attack in which a client connects and never writes any bytes. This should not block the accepting thread, since
    +     * the read to determine if the client is sending a TLS handshake or not happens in the processing thread.
    +     *
    +     * This version of the test uses a non-strict server socket (i.e. it accepts both TLS and plaintext connections).
    +     */
    +    @Test
    +    public void testDenialOfServiceResistanceNonStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    --- End diff --
    
    How about we keep the unit test that's focused strictly on UnifiedServerSocket in this PR, and I'll open a separate PR/Jira for adding a new unit test that tests `Leader`? There is some value in the existing tests, since they did catch issues with the original UnifiedServerSocket implementation.


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    @anmolnar put the DoS tests back in. If you decide that you're ok with keeping them, let me know what you'd like me to rename / comment.


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2543/



---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2533/



---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2597/



---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r233661093
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/PrependableSocket.java ---
    @@ -18,32 +18,47 @@
     
     package org.apache.zookeeper.server.quorum;
     
    -import java.io.ByteArrayInputStream;
     import java.io.IOException;
     import java.io.InputStream;
    -import java.io.SequenceInputStream;
    +import java.io.PushbackInputStream;
     import java.net.Socket;
     import java.net.SocketImpl;
     
     public class PrependableSocket extends Socket {
     
    -  private SequenceInputStream sequenceInputStream;
    +  private PushbackInputStream pushbackInputStream;
    --- End diff --
    
    But I suspect the second read() will read from the other stream of the Sequence. Which is okay, because client has to initiate another read call if it hasn't received everything it needed.
    Anyway I wouldn't say go back to that impl, just speculating. I believe handling the 2 streams seamlessly is desirable.


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r233662723
  
    --- Diff: zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java ---
    @@ -17,156 +17,644 @@
      */
     package org.apache.zookeeper.server.quorum;
     
    +import java.io.BufferedInputStream;
    +import java.io.IOException;
    +import java.net.ConnectException;
    +import java.net.InetSocketAddress;
    +import java.net.Socket;
    +import java.net.SocketException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Random;
    +
    +import javax.net.ssl.HandshakeCompletedEvent;
    +import javax.net.ssl.HandshakeCompletedListener;
    +import javax.net.ssl.SSLSocket;
    +
     import org.apache.zookeeper.PortAssignment;
     import org.apache.zookeeper.client.ZKClientConfig;
    +import org.apache.zookeeper.common.BaseX509ParameterizedTestCase;
     import org.apache.zookeeper.common.ClientX509Util;
    -import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.common.KeyStoreFileType;
    +import org.apache.zookeeper.common.X509Exception;
    +import org.apache.zookeeper.common.X509KeyType;
    +import org.apache.zookeeper.common.X509TestContext;
     import org.apache.zookeeper.common.X509Util;
     import org.apache.zookeeper.server.ServerCnxnFactory;
     import org.junit.Assert;
     import org.junit.Before;
     import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
     
    -import javax.net.ssl.HandshakeCompletedEvent;
    -import javax.net.ssl.HandshakeCompletedListener;
    -import javax.net.ssl.SSLSocket;
    -import java.io.IOException;
    -import java.net.ConnectException;
    -import java.net.InetSocketAddress;
    -import java.net.Socket;
    -
    -import static org.hamcrest.CoreMatchers.equalTo;
    -import static org.junit.Assert.assertThat;
    +@RunWith(Parameterized.class)
    +public class UnifiedServerSocketTest extends BaseX509ParameterizedTestCase {
     
    -public class UnifiedServerSocketTest {
    +    @Parameterized.Parameters
    +    public static Collection<Object[]> params() {
    +        ArrayList<Object[]> result = new ArrayList<>();
    +        int paramIndex = 0;
    +        for (X509KeyType caKeyType : X509KeyType.values()) {
    +            for (X509KeyType certKeyType : X509KeyType.values()) {
    +                for (Boolean hostnameVerification : new Boolean[] { true, false  }) {
    +                    result.add(new Object[]{
    +                            caKeyType,
    +                            certKeyType,
    +                            hostnameVerification,
    +                            paramIndex++
    +                    });
    +                }
    +            }
    +        }
    +        return result;
    +    }
     
         private static final int MAX_RETRIES = 5;
         private static final int TIMEOUT = 1000;
    +    private static final byte[] DATA_TO_CLIENT = "hello client".getBytes();
    +    private static final byte[] DATA_FROM_CLIENT = "hello server".getBytes();
     
         private X509Util x509Util;
         private int port;
    -    private volatile boolean handshakeCompleted;
    +    private InetSocketAddress localServerAddress;
    +    private final Object handshakeCompletedLock = new Object();
    +    // access only inside synchronized(handshakeCompletedLock) { ... } blocks
    +    private boolean handshakeCompleted = false;
    +
    +    public UnifiedServerSocketTest(
    +            final X509KeyType caKeyType,
    +            final X509KeyType certKeyType,
    +            final Boolean hostnameVerification,
    +            final Integer paramIndex) {
    +        super(paramIndex, () -> {
    +            try {
    +                return X509TestContext.newBuilder()
    +                    .setTempDir(tempDir)
    +                    .setKeyStoreKeyType(certKeyType)
    +                    .setTrustStoreKeyType(caKeyType)
    +                    .setHostnameVerification(hostnameVerification)
    +                    .build();
    +            } catch (Exception e) {
    +                throw new RuntimeException(e);
    +            }
    +        });
    +    }
     
         @Before
         public void setUp() throws Exception {
    -        handshakeCompleted = false;
    -
             port = PortAssignment.unique();
    +        localServerAddress = new InetSocketAddress("localhost", port);
     
    -        String testDataPath = System.getProperty("test.data.dir", "build/test/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");
     
             x509Util = new ClientX509Util();
     
    -        System.setProperty(x509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks");
    -        System.setProperty(x509Util.getSslKeystorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks");
    -        System.setProperty(x509Util.getSslTruststorePasswdProperty(), "testpass");
    -        System.setProperty(x509Util.getSslHostnameVerificationEnabledProperty(), "false");
    +        x509TestContext.setSystemProperties(x509Util, KeyStoreFileType.JKS, KeyStoreFileType.JKS);
         }
     
    -    @Test
    -    public void testConnectWithSSL() throws Exception {
    -        class ServerThread extends Thread {
    -            public void run() {
    -                try {
    -                    Socket unifiedSocket = new UnifiedServerSocket(x509Util, port).accept();
    -                    ((SSLSocket)unifiedSocket).getSession(); // block until handshake completes
    -                } catch (IOException e) {
    -                    e.printStackTrace();
    +    private static void forceClose(java.io.Closeable s) {
    +        if (s == null) {
    +            return;
    +        }
    +        try {
    +            s.close();
    +        } catch (IOException e) {
    +            e.printStackTrace();
    +        }
    +    }
    +
    +    private static final class UnifiedServerThread extends Thread {
    +        private final byte[] dataToClient;
    +        private List<byte[]> dataFromClients;
    +        private List<Thread> workerThreads;
    +        private UnifiedServerSocket serverSocket;
    +
    +        UnifiedServerThread(X509Util x509Util,
    +                            InetSocketAddress bindAddress,
    +                            boolean allowInsecureConnection,
    +                            byte[] dataToClient) throws IOException {
    +            this.dataToClient = dataToClient;
    +            dataFromClients = new ArrayList<>();
    +            workerThreads = new ArrayList<>();
    +            serverSocket = new UnifiedServerSocket(x509Util, allowInsecureConnection);
    +            serverSocket.bind(bindAddress);
    +        }
    +
    +        @Override
    +        public void run() {
    +            try {
    +                Random rnd = new Random();
    +                while (true) {
    +                    final Socket unifiedSocket = serverSocket.accept();
    +                    final boolean tcpNoDelay = rnd.nextBoolean();
    +                    unifiedSocket.setTcpNoDelay(tcpNoDelay);
    +                    unifiedSocket.setSoTimeout(TIMEOUT);
    +                    final boolean keepAlive = rnd.nextBoolean();
    +                    unifiedSocket.setKeepAlive(keepAlive);
    +                    // Note: getting the input stream should not block the thread or trigger mode detection.
    +                    BufferedInputStream bis = new BufferedInputStream(unifiedSocket.getInputStream());
    +                    Thread t = new Thread(new Runnable() {
    +                        @Override
    +                        public void run() {
    +                            try {
    +                                byte[] buf = new byte[1024];
    +                                int bytesRead = unifiedSocket.getInputStream().read(buf, 0, 1024);
    +                                // Make sure the settings applied above before the socket was potentially upgraded to
    +                                // TLS still apply.
    +                                Assert.assertEquals(tcpNoDelay, unifiedSocket.getTcpNoDelay());
    +                                Assert.assertEquals(TIMEOUT, unifiedSocket.getSoTimeout());
    +                                Assert.assertEquals(keepAlive, unifiedSocket.getKeepAlive());
    +                                if (bytesRead > 0) {
    +                                    byte[] dataFromClient = new byte[bytesRead];
    +                                    System.arraycopy(buf, 0, dataFromClient, 0, bytesRead);
    +                                    synchronized (dataFromClients) {
    +                                        dataFromClients.add(dataFromClient);
    +                                    }
    +                                }
    +                                unifiedSocket.getOutputStream().write(dataToClient);
    +                                unifiedSocket.getOutputStream().flush();
    +                            } catch (IOException e) {
    +                                e.printStackTrace();
    +                                throw new RuntimeException(e);
    +                            } finally {
    +                                forceClose(unifiedSocket);
    +                            }
    +                        }
    +                    });
    +                    workerThreads.add(t);
    +                    t.start();
                     }
    +            } catch (IOException e) {
    +                e.printStackTrace();
    +                throw new RuntimeException(e);
    +            } finally {
    +                forceClose(serverSocket);
                 }
             }
    -        ServerThread serverThread = new ServerThread();
    -        serverThread.start();
     
    +        public void shutdown(long millis) throws InterruptedException {
    +            forceClose(serverSocket); // this should break the run() loop
    +            for (Thread t : workerThreads) {
    +                t.join(millis);
    +            }
    +            this.join(millis);
    +        }
    +
    +        synchronized byte[] getDataFromClient(int index) {
    +            return dataFromClients.get(index);
    +        }
    +    }
    +
    +    private SSLSocket connectWithSSL() throws IOException, X509Exception, InterruptedException {
             SSLSocket sslSocket = null;
             int retries = 0;
             while (retries < MAX_RETRIES) {
                 try {
                     sslSocket = x509Util.createSSLSocket();
    +                sslSocket.addHandshakeCompletedListener(new HandshakeCompletedListener() {
    +                    @Override
    +                    public void handshakeCompleted(HandshakeCompletedEvent handshakeCompletedEvent) {
    +                        synchronized (handshakeCompletedLock) {
    +                            handshakeCompleted = true;
    +                            handshakeCompletedLock.notifyAll();
    +                        }
    +                    }
    +                });
                     sslSocket.setSoTimeout(TIMEOUT);
    -                sslSocket.connect(new InetSocketAddress(port), TIMEOUT);
    +                sslSocket.connect(localServerAddress, TIMEOUT);
                     break;
                 } catch (ConnectException connectException) {
                     connectException.printStackTrace();
    +                forceClose(sslSocket);
    +                sslSocket = null;
                     Thread.sleep(TIMEOUT);
                 }
                 retries++;
             }
     
    -        sslSocket.addHandshakeCompletedListener(new HandshakeCompletedListener() {
    -            @Override
    -            public void handshakeCompleted(HandshakeCompletedEvent handshakeCompletedEvent) {
    -                completeHandshake();
    +        Assert.assertNotNull("Failed to connect to server with SSL", sslSocket);
    +        return sslSocket;
    +    }
    +
    +    private Socket connectWithoutSSL() throws IOException, InterruptedException {
    +        Socket socket = null;
    +        int retries = 0;
    +        while (retries < MAX_RETRIES) {
    +            try {
    +                socket = new Socket();
    +                socket.setSoTimeout(TIMEOUT);
    +                socket.connect(localServerAddress, TIMEOUT);
    +                break;
    +            } catch (ConnectException connectException) {
    +                connectException.printStackTrace();
    +                forceClose(socket);
    +                socket = null;
    +                Thread.sleep(TIMEOUT);
                 }
    -        });
    -        sslSocket.startHandshake();
    +            retries++;
    +        }
    +        Assert.assertNotNull("Failed to connect to server without SSL", socket);
    +        return socket;
    +    }
    +
    +    // In the tests below, a "Strict" server means a UnifiedServerSocket that
    +    // does not allow plaintext connections (in other words, it's SSL-only).
    +    // A "Non Strict" server means a UnifiedServerSocket that allows both
    +    // plaintext and SSL incoming connections.
    +
    +    /**
    +     * Attempting to connect to a SSL-or-plaintext server with SSL should work.
    +     */
    +    @Test
    +    public void testConnectWithSSLToNonStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, true, DATA_TO_CLIENT);
    +        serverThread.start();
     
    -        serverThread.join(TIMEOUT);
    +        Socket sslSocket = connectWithSSL();
    +        sslSocket.getOutputStream().write(DATA_FROM_CLIENT);
    +        sslSocket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = sslSocket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
     
    -        long start = Time.currentElapsedTime();
    -        while (Time.currentElapsedTime() < start + TIMEOUT) {
    -            if (handshakeCompleted) {
    -                return;
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(sslSocket);
    +
    +        synchronized (handshakeCompletedLock) {
    +            if (!handshakeCompleted) {
    +                handshakeCompletedLock.wait(TIMEOUT);
                 }
    +            Assert.assertTrue(handshakeCompleted);
             }
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
    +    }
    +
    +    /**
    +     * Attempting to connect to a SSL-only server with SSL should work.
    +     */
    +    @Test
    +    public void testConnectWithSSLToStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, false, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket sslSocket = connectWithSSL();
    +        sslSocket.getOutputStream().write(DATA_FROM_CLIENT);
    +        sslSocket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = sslSocket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
     
    -        Assert.fail("failed to complete handshake");
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(sslSocket);
    +
    +        synchronized (handshakeCompletedLock) {
    +            if (!handshakeCompleted) {
    +                handshakeCompletedLock.wait(TIMEOUT);
    +            }
    +            Assert.assertTrue(handshakeCompleted);
    +        }
    +
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
         }
     
    -    private void completeHandshake() {
    -        handshakeCompleted = true;
    +    /**
    +     * Attempting to connect to a SSL-or-plaintext server without SSL should work.
    +     */
    +    @Test
    +    public void testConnectWithoutSSLToNonStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, true, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket socket = connectWithoutSSL();
    +        socket.getOutputStream().write(DATA_FROM_CLIENT);
    +        socket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = socket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
    +
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(socket);
    +
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
         }
     
    +    /**
    +     * Attempting to connect to a SSL-or-plaintext server without SSL with a
    +     * small initial data write should work. This makes sure that sending
    +     * less than 5 bytes does not break the logic in the server's initial 5
    +     * byte read.
    +     */
         @Test
    -    public void testConnectWithoutSSL() throws Exception {
    -        final byte[] testData = "hello there".getBytes();
    -        final String[] dataReadFromClient = {null};
    +    public void testConnectWithoutSSLToNonStrictServerPartialWrite() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, true, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket socket = connectWithoutSSL();
    +        // Write only 2 bytes of the message, wait a bit, then write the rest.
    +        // This makes sure that writes smaller than 5 bytes don't break the plaintext mode on the server
    +        // once it decides that the input doesn't look like a TLS handshake.
    +        socket.getOutputStream().write(DATA_FROM_CLIENT, 0, 2);
    +        socket.getOutputStream().flush();
    +        Thread.sleep(TIMEOUT / 2);
    +        socket.getOutputStream().write(DATA_FROM_CLIENT, 2, DATA_FROM_CLIENT.length - 2);
    +        socket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        int bytesRead = socket.getInputStream().read(buf, 0, buf.length);
    +        Assert.assertEquals(buf.length, bytesRead);
    +        Assert.assertArrayEquals(DATA_TO_CLIENT, buf);
    +
    +        serverThread.shutdown(TIMEOUT);
    +        forceClose(socket);
     
    -        class ServerThread extends Thread {
    +        Assert.assertArrayEquals(DATA_FROM_CLIENT, serverThread.getDataFromClient(0));
    +    }
    +
    +    /**
    +     * Attempting to connect to a SSL-only server without SSL should fail.
    +     */
    +    @Test
    +    public void testConnectWithoutSSLToStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    +                x509Util, localServerAddress, false, DATA_TO_CLIENT);
    +        serverThread.start();
    +
    +        Socket socket = connectWithoutSSL();
    +        socket.getOutputStream().write(DATA_FROM_CLIENT);
    +        socket.getOutputStream().flush();
    +        byte[] buf = new byte[DATA_TO_CLIENT.length];
    +        try {
    +            socket.getInputStream().read(buf, 0, buf.length);
    +        } catch (SocketException e) {
    +            // We expect the other end to hang up the connection
    +            return;
    +        } finally {
    +            serverThread.shutdown(TIMEOUT);
    +            forceClose(socket);
    +        }
    +        Assert.fail("Expected server to hang up the connection. Read from server succeeded unexpectedly.");
    +    }
    +
    +    /**
    +     * This test makes sure that UnifiedServerSocket used properly (a single thread accept()-ing connections and
    +     * handing the resulting sockets to other threads for processing) is not vulnerable to a simple denial-of-service
    +     * attack in which a client connects and never writes any bytes. This should not block the accepting thread, since
    +     * the read to determine if the client is sending a TLS handshake or not happens in the processing thread.
    +     *
    +     * This version of the test uses a non-strict server socket (i.e. it accepts both TLS and plaintext connections).
    +     */
    +    @Test
    +    public void testDenialOfServiceResistanceNonStrictServer() throws Exception {
    +        UnifiedServerThread serverThread = new UnifiedServerThread(
    --- End diff --
    
    I would create new test class inherited from `QuorumPeerTestBase` dedicated to DOS tests, but I think this topic leads to another: stress testing ZooKeeper. I highly support the effort, but please remove them from this PR and create separate Jira for that.


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2537/



---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r234053611
  
    --- Diff: zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java ---
    @@ -17,156 +17,644 @@
      */
     package org.apache.zookeeper.server.quorum;
     
    +import java.io.BufferedInputStream;
    +import java.io.IOException;
    +import java.net.ConnectException;
    +import java.net.InetSocketAddress;
    +import java.net.Socket;
    +import java.net.SocketException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.List;
    +import java.util.Random;
    +
    +import javax.net.ssl.HandshakeCompletedEvent;
    +import javax.net.ssl.HandshakeCompletedListener;
    +import javax.net.ssl.SSLSocket;
    +
     import org.apache.zookeeper.PortAssignment;
     import org.apache.zookeeper.client.ZKClientConfig;
    +import org.apache.zookeeper.common.BaseX509ParameterizedTestCase;
     import org.apache.zookeeper.common.ClientX509Util;
    -import org.apache.zookeeper.common.Time;
    +import org.apache.zookeeper.common.KeyStoreFileType;
    +import org.apache.zookeeper.common.X509Exception;
    +import org.apache.zookeeper.common.X509KeyType;
    +import org.apache.zookeeper.common.X509TestContext;
     import org.apache.zookeeper.common.X509Util;
     import org.apache.zookeeper.server.ServerCnxnFactory;
     import org.junit.Assert;
     import org.junit.Before;
     import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
     
    -import javax.net.ssl.HandshakeCompletedEvent;
    -import javax.net.ssl.HandshakeCompletedListener;
    -import javax.net.ssl.SSLSocket;
    -import java.io.IOException;
    -import java.net.ConnectException;
    -import java.net.InetSocketAddress;
    -import java.net.Socket;
    -
    -import static org.hamcrest.CoreMatchers.equalTo;
    -import static org.junit.Assert.assertThat;
    +@RunWith(Parameterized.class)
    +public class UnifiedServerSocketTest extends BaseX509ParameterizedTestCase {
     
    -public class UnifiedServerSocketTest {
    +    @Parameterized.Parameters
    +    public static Collection<Object[]> params() {
    +        ArrayList<Object[]> result = new ArrayList<>();
    +        int paramIndex = 0;
    +        for (X509KeyType caKeyType : X509KeyType.values()) {
    +            for (X509KeyType certKeyType : X509KeyType.values()) {
    +                for (Boolean hostnameVerification : new Boolean[] { true, false  }) {
    +                    result.add(new Object[]{
    +                            caKeyType,
    +                            certKeyType,
    +                            hostnameVerification,
    +                            paramIndex++
    +                    });
    +                }
    +            }
    +        }
    +        return result;
    +    }
     
         private static final int MAX_RETRIES = 5;
         private static final int TIMEOUT = 1000;
    +    private static final byte[] DATA_TO_CLIENT = "hello client".getBytes();
    +    private static final byte[] DATA_FROM_CLIENT = "hello server".getBytes();
     
         private X509Util x509Util;
         private int port;
    -    private volatile boolean handshakeCompleted;
    +    private InetSocketAddress localServerAddress;
    +    private final Object handshakeCompletedLock = new Object();
    +    // access only inside synchronized(handshakeCompletedLock) { ... } blocks
    +    private boolean handshakeCompleted = false;
    +
    +    public UnifiedServerSocketTest(
    +            final X509KeyType caKeyType,
    +            final X509KeyType certKeyType,
    +            final Boolean hostnameVerification,
    +            final Integer paramIndex) {
    +        super(paramIndex, () -> {
    +            try {
    +                return X509TestContext.newBuilder()
    +                    .setTempDir(tempDir)
    +                    .setKeyStoreKeyType(certKeyType)
    +                    .setTrustStoreKeyType(caKeyType)
    +                    .setHostnameVerification(hostnameVerification)
    +                    .build();
    +            } catch (Exception e) {
    +                throw new RuntimeException(e);
    +            }
    +        });
    +    }
     
         @Before
         public void setUp() throws Exception {
    -        handshakeCompleted = false;
    -
             port = PortAssignment.unique();
    +        localServerAddress = new InetSocketAddress("localhost", port);
    --- End diff --
    
    Done


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    I should mention that this code has been internally reviewed at Facebook, has been landed on our internal fork, and has been running in production for weeks. 


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2594/



---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    @anmolnar I understand your point about the DoS tests, and agree that they are not very focused and do end up testing the structure of the test code itself in (i.e. that the implementation of `UnifiedServerSocketTest$UnifiedServerThread` plays nicely with `UnifiedServerSocket`). Let me explain really briefly how that test came about, since the history behind it is missing in this discussion.
    
    Initially when I rewrote `UnifiedServerSocket` and added the `UnifiedSocket` inner class, I didn't have the `UnifiedInputStream` and `UnifiedOutputStream` inner classes. I was careful to avoid any reads inside `accept()`, and thought that was good enough. However, with this version of the code we still saw problems on our test cluster during a stress disconnect/reconnect test. Investigation showed evidence that the Leader's `accept()` thread was getting stuck, and then I found the code in `Leader$LearnerCnxAcceptor#run()` that calls `getInputStream()` on the accepted socket and wraps the result in a `BufferedInputStream`. The only way at the time to get the input stream was to resolve the type of socket (by doing a read of first 5 bytes and then TLS/plaintext mode detection), and get the real socket's input stream.
    
    My fix involved adding the `UnifiedInputStream` and `UnifiedOutputStream` inner classes, which allowed me to delay the mode detection. Now, instead of triggering mode detection at the time of `getInputStream()`, we can trigger it the first time the stream is used for I/O. I wrote the DoS test as part of the same diff, and was careful to replicate the threading behavior of `Leader$LearnerCnxAcceptor.run()` in `UnifiedServerThread`. I also verified that the new tests failed on a version of the code that didn't have the unified stream wrappers.
    
    If I remove the DoS tests, the behavior of delaying the mode detection's read to the point of first I/O operation would not be tested properly. I think that code should be tested! However, if you prefer I could probably test it in a more focused way (i.e. just test each method that's meant to be non-blocking by itself). Would that be an acceptable compromise?


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/679#discussion_r233661570
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/PrependableSocket.java ---
    @@ -18,32 +18,47 @@
     
     package org.apache.zookeeper.server.quorum;
     
    -import java.io.ByteArrayInputStream;
     import java.io.IOException;
     import java.io.InputStream;
    -import java.io.SequenceInputStream;
    +import java.io.PushbackInputStream;
     import java.net.Socket;
     import java.net.SocketImpl;
     
     public class PrependableSocket extends Socket {
     
    -  private SequenceInputStream sequenceInputStream;
    +  private PushbackInputStream pushbackInputStream;
    --- End diff --
    
    Yes, SequenceInputStream likely still works in practice, but exposes the discontinuity between the underlying streams to the caller. Using PushbackInputStream hides it and I think is preferable.


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
Github user ivmaykov closed the pull request at:

    https://github.com/apache/zookeeper/pull/679


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2529/



---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2585/



---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    @ivmaykov I'm not a fan of them, but understand the intention and can't figure out a more straightforward way to test the behaviour. So let's go with them.
    
    What do you think of renaming them to something like `testTLSDetectionNonBlocking(Non)StrictServer`?
    I would also rephrase the comment to emphasize that the tests are intended to validate that TLS detection is non-blocking, so it's safe to getInputStream() on the accept thread.
    
    These are not really denial-of-service tests IMHO. Thoughts?


---

[GitHub] zookeeper issue #679: ZOOKEEPER-3172: Quorum TLS - fix port unification to a...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on the issue:

    https://github.com/apache/zookeeper/pull/679
  
    @ivmaykov Gosh! I just did another round of review with your tests today and was tending to accept the DoS tests with just renaming and amending the comment. :)
    Sorry for not responding earlier, I'll check your new code tomorrow and give feedback.


---

[GitHub] zookeeper pull request #679: ZOOKEEPER-3172: Quorum TLS - fix port unificati...

Posted by ivmaykov <gi...@git.apache.org>.
GitHub user ivmaykov reopened a pull request:

    https://github.com/apache/zookeeper/pull/679

    ZOOKEEPER-3172: Quorum TLS - fix port unification to allow rolling upgrades

    Fix numerous problems with UnifiedServerSocket, such as hanging the accept() thread when the client doesn't send any data or crashing if less than 5 bytes are read from the socket in the initial read.
    
    Re-enable the "portUnification" config option.
    
    Note that this is stacked on top of #678 and thus includes it. Please only consider the ZOOKEEPER-3172 commit when reviewing. Once the other PR is merged upstream, I will rebase this so it only contains one commit.
    
    ## Fixed networking issues/bugs in UnifiedServerSocket
    
    - don't crash the `accept()` thread if the client closes the connection without sending any data
    - don't corrupt the connection if the client sends fewer than 5 bytes for the initial read
    - delay the detection of TLS vs. plaintext mode until a socket stream is read from or written to. This prevents the `accept()` thread from getting blocked on a `read()` operation from the newly connected socket.
    - prepending 5 bytes to `PrependableSocket` and then trying to read >5 bytes would only return the first 5 bytes, even if more bytes were available. This is fixed.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ivmaykov/zookeeper ZOOKEEPER-3172

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/679.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #679
    
----
commit 2122c8c23a0dbb27f9b2aff55e800e48d253f943
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T00:41:48Z

    ZOOKEEPER-3173: Quorum TLS - support PEM trust/key stores
    ZOOKEEPER-3175: Quorum TLS - test improvements
    
    Add support for loading key and trust stores from PEM files.
    Also added test utils for testing X509-related code, because it
    was very difficult to untangle them from the PEM support code.

commit 514d48a26aeeca37290ad14ff8f0cdae69b53eb2
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T01:22:24Z

    ZOOKEEPER-3172: Quorum TLS - fix port unification to allow rolling upgrades

----


---