From commits-return-7462-archive-asf-public=cust-asf.ponee.io@zookeeper.apache.org Sat Dec 15 07:35:27 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 2FCE4180652 for ; Sat, 15 Dec 2018 07:35:25 +0100 (CET) Received: (qmail 4598 invoked by uid 500); 15 Dec 2018 06:35:24 -0000 Mailing-List: contact commits-help@zookeeper.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@zookeeper.apache.org Delivered-To: mailing list commits@zookeeper.apache.org Received: (qmail 4587 invoked by uid 99); 15 Dec 2018 06:35:24 -0000 Received: from ec2-52-202-80-70.compute-1.amazonaws.com (HELO gitbox.apache.org) (52.202.80.70) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 15 Dec 2018 06:35:24 +0000 Received: by gitbox.apache.org (ASF Mail Server at gitbox.apache.org, from userid 33) id 5046485280; Sat, 15 Dec 2018 06:35:23 +0000 (UTC) Date: Sat, 15 Dec 2018 06:35:23 +0000 To: "commits@zookeeper.apache.org" Subject: [zookeeper] branch master updated: ZOOKEEPER-3174: Quorum TLS - support reloading trust/key store MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 8bit Message-ID: <154485572320.26417.17222145266451147355@gitbox.apache.org> From: andor@apache.org X-Git-Host: gitbox.apache.org X-Git-Repo: zookeeper X-Git-Refname: refs/heads/master X-Git-Reftype: branch X-Git-Oldrev: 46fc819622bf08cbd0781dea279aff734b492902 X-Git-Newrev: e043c322f12d56da0fc88131628edf0731c0f8e4 X-Git-Rev: e043c322f12d56da0fc88131628edf0731c0f8e4 X-Git-NotificationType: ref_changed_plus_diff X-Git-Multimail-Version: 1.5.dev Auto-Submitted: auto-generated This is an automated email from the ASF dual-hosted git repository. andor pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/zookeeper.git The following commit(s) were added to refs/heads/master by this push: new e043c32 ZOOKEEPER-3174: Quorum TLS - support reloading trust/key store e043c32 is described below commit e043c322f12d56da0fc88131628edf0731c0f8e4 Author: Ilya Maykov AuthorDate: Sat Dec 15 07:34:54 2018 +0100 ZOOKEEPER-3174: Quorum TLS - support reloading trust/key store Allow reloading SSL trust stores and key stores from disk when the files on disk change. ## Added support for reloading key/trust stores when the file on disk changes - new property `sslQuorumReloadCertFiles` which controls the behavior for reloading the key and trust store files for `QuorumX509Util`. Reloading of key and trust store for `ClientX509Util` is not in this PR but could be added easily - this allows a ZK server to keep running on a machine that uses short-lived certs that refresh frequently without having to restart the ZK process. Author: Ilya Maykov Reviewers: andor@apache.org Closes #680 from ivmaykov/ZOOKEEPER-3174 --- .../apache/zookeeper/ClientCnxnSocketNetty.java | 9 +- .../zookeeper/client/FourLetterWordMain.java | 15 +- .../apache/zookeeper/common/FileChangeWatcher.java | 253 ++++++++++++++++++++ .../java/org/apache/zookeeper/common/X509Util.java | 115 ++++++++- .../java/org/apache/zookeeper/common/ZKConfig.java | 13 +- .../zookeeper/server/NettyServerCnxnFactory.java | 2 + .../server/auth/X509AuthenticationProvider.java | 74 +++--- .../org/apache/zookeeper/server/quorum/Leader.java | 18 +- .../zookeeper/server/quorum/QuorumCnxManager.java | 4 +- .../apache/zookeeper/server/quorum/QuorumPeer.java | 1 + .../zookeeper/server/quorum/QuorumPeerConfig.java | 22 +- .../zookeeper/server/quorum/QuorumPeerMain.java | 3 + .../zookeeper/common/FileChangeWatcherTest.java | 263 +++++++++++++++++++++ .../org/apache/zookeeper/common/X509UtilTest.java | 36 +-- .../server/quorum/QuorumPeerConfigTest.java | 23 +- .../zookeeper/server/quorum/QuorumSSLTest.java | 4 +- .../UnifiedServerSocketModeDetectionTest.java | 1 + .../server/quorum/UnifiedServerSocketTest.java | 1 + .../org/apache/zookeeper/test/ClientSSLTest.java | 4 +- .../org/apache/zookeeper/test/SSLAuthTest.java | 5 +- 20 files changed, 760 insertions(+), 106 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocketNetty.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocketNetty.java index 74d1283..c4a7301 100755 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocketNetty.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocketNetty.java @@ -54,6 +54,7 @@ import org.apache.zookeeper.ClientCnxn.Packet; import org.apache.zookeeper.client.ZKClientConfig; import org.apache.zookeeper.common.ClientX509Util; import org.apache.zookeeper.common.NettyUtils; +import org.apache.zookeeper.common.X509Util; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -436,9 +437,11 @@ public class ClientCnxnSocketNetty extends ClientCnxnSocket { // Basically we only need to create it once. private synchronized void initSSL(ChannelPipeline pipeline) throws SSLContextException { if (sslContext == null || sslEngine == null) { - sslContext = new ClientX509Util().createSSLContext(clientConfig); - sslEngine = sslContext.createSSLEngine(host,port); - sslEngine.setUseClientMode(true); + try (X509Util x509Util = new ClientX509Util()) { + sslContext = x509Util.createSSLContext(clientConfig); + sslEngine = sslContext.createSSLEngine(host, port); + sslEngine.setUseClientMode(true); + } } pipeline.addLast("ssl", new SslHandler(sslEngine)); LOG.info("SSL handler added for channel: {}", pipeline.channel()); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/client/FourLetterWordMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/client/FourLetterWordMain.java index 41f5e9d..a98953f 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/client/FourLetterWordMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/client/FourLetterWordMain.java @@ -34,6 +34,7 @@ import javax.net.ssl.SSLSocketFactory; import org.apache.zookeeper.common.ClientX509Util; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.common.X509Exception.SSLContextException; +import org.apache.zookeeper.common.X509Util; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -90,12 +91,14 @@ public class FourLetterWordMain { new InetSocketAddress(InetAddress.getByName(null), port); if (secure) { LOG.info("using secure socket"); - SSLContext sslContext = new ClientX509Util().getDefaultSSLContext(); - SSLSocketFactory socketFactory = sslContext.getSocketFactory(); - SSLSocket sslSock = (SSLSocket) socketFactory.createSocket(); - sslSock.connect(hostaddress, timeout); - sslSock.startHandshake(); - sock = sslSock; + try (X509Util x509Util = new ClientX509Util()) { + SSLContext sslContext = x509Util.getDefaultSSLContext(); + SSLSocketFactory socketFactory = sslContext.getSocketFactory(); + SSLSocket sslSock = (SSLSocket) socketFactory.createSocket(); + sslSock.connect(hostaddress, timeout); + sslSock.startHandshake(); + sock = sslSock; + } } else { sock = new Socket(); sock.connect(hostaddress, timeout); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java b/zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java new file mode 100644 index 0000000..8b49be9 --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java @@ -0,0 +1,253 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.common; + +import com.sun.nio.file.SensitivityWatchEventModifier; +import org.apache.zookeeper.server.ZooKeeperThread; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.ClosedWatchServiceException; +import java.nio.file.FileSystem; +import java.nio.file.Path; +import java.nio.file.StandardWatchEventKinds; +import java.nio.file.WatchEvent; +import java.nio.file.WatchKey; +import java.nio.file.WatchService; +import java.util.function.Consumer; + +/** + * Instances of this class can be used to watch a directory for file changes. When a file is added to, deleted from, + * or is modified in the given directory, the callback provided by the user will be called from a background thread. + * Some things to keep in mind: + *
    + *
  • The callback should be thread-safe.
  • + *
  • Changes that happen around the time the thread is started may be missed.
  • + *
  • There is a delay between a file changing and the callback firing.
  • + *
  • The watch is not recursive - changes to subdirectories will not trigger a callback.
  • + *
+ */ +public final class FileChangeWatcher { + private static final Logger LOG = LoggerFactory.getLogger(FileChangeWatcher.class); + + public enum State { + NEW, // object created but start() not called yet + STARTING, // start() called but background thread has not entered main loop + RUNNING, // background thread is running + STOPPING, // stop() called but background thread has not exited main loop + STOPPED // stop() called and background thread has exited, or background thread crashed + } + + private final WatcherThread watcherThread; + private State state; // protected by synchronized(this) + + /** + * Creates a watcher that watches dirPath and invokes callback on changes. + * + * @param dirPath the directory to watch. + * @param callback the callback to invoke with events. event.kind() will return the type of event, + * and event.context() will return the filename relative to dirPath. + * @throws IOException if there is an error creating the WatchService. + */ + public FileChangeWatcher(Path dirPath, Consumer> callback) throws IOException { + FileSystem fs = dirPath.getFileSystem(); + WatchService watchService = fs.newWatchService(); + if (LOG.isDebugEnabled()) { + LOG.debug("Registering with watch service: " + dirPath); + } + dirPath.register( + watchService, + new WatchEvent.Kind[]{ + StandardWatchEventKinds.ENTRY_CREATE, + StandardWatchEventKinds.ENTRY_DELETE, + StandardWatchEventKinds.ENTRY_MODIFY, + StandardWatchEventKinds.OVERFLOW}, + SensitivityWatchEventModifier.HIGH); + state = State.NEW; + this.watcherThread = new WatcherThread(watchService, callback); + this.watcherThread.setDaemon(true); + } + + /** + * Returns the current {@link FileChangeWatcher.State}. + * @return the current state. + */ + public synchronized State getState() { + return state; + } + + /** + * Blocks until the current state becomes desiredState. + * Currently only used by tests, thus package-private. + * @param desiredState the desired state. + * @throws InterruptedException if the current thread gets interrupted. + */ + synchronized void waitForState(State desiredState) throws InterruptedException { + while (this.state != desiredState) { + this.wait(); + } + } + + /** + * Sets the state to newState. + * @param newState the new state. + */ + private synchronized void setState(State newState) { + state = newState; + this.notifyAll(); + } + + /** + * Atomically sets the state to update if and only if the + * state is currently expected. + * @param expected the expected state. + * @param update the new state. + * @return true if the update succeeds, or false if the current state + * does not equal expected. + */ + private synchronized boolean compareAndSetState(State expected, State update) { + if (state == expected) { + setState(update); + return true; + } else { + return false; + } + } + + /** + * Atomically sets the state to update if and only if the + * state is currently one of expectedStates. + * @param expectedStates the expected states. + * @param update the new state. + * @return true if the update succeeds, or false if the current state + * does not equal any of the expectedStates. + */ + private synchronized boolean compareAndSetState(State[] expectedStates, State update) { + for (State expected : expectedStates) { + if (state == expected) { + setState(update); + return true; + } + } + return false; + } + + /** + * Tells the background thread to start. Does not wait for it to be running. + * Calling this method more than once has no effect. + */ + public void start() { + if (!compareAndSetState(State.NEW, State.STARTING)) { + // If previous state was not NEW, start() has already been called. + return; + } + this.watcherThread.start(); + } + + /** + * Tells the background thread to stop. Does not wait for it to exit. + */ + public void stop() { + if (compareAndSetState( + new State[]{State.RUNNING, State.STARTING}, + State.STOPPING)) { + watcherThread.interrupt(); + } + } + + /** + * Inner class that implements the watcher thread logic. + */ + private class WatcherThread extends ZooKeeperThread { + private static final String THREAD_NAME = "FileChangeWatcher"; + + final WatchService watchService; + final Consumer> callback; + + WatcherThread(WatchService watchService, Consumer> callback) { + super(THREAD_NAME); + this.watchService = watchService; + this.callback = callback; + } + + @Override + public void run() { + try { + LOG.info(getName() + " thread started"); + if (!compareAndSetState( + FileChangeWatcher.State.STARTING, + FileChangeWatcher.State.RUNNING)) { + // stop() called shortly after start(), before + // this thread started running. + FileChangeWatcher.State state = FileChangeWatcher.this.getState(); + if (state != FileChangeWatcher.State.STOPPING) { + throw new IllegalStateException("Unexpected state: " + state); + } + return; + } + runLoop(); + } catch (Exception e) { + LOG.warn("Error in runLoop()", e); + throw e; + } finally { + try { + watchService.close(); + } catch (IOException e) { + LOG.warn("Error closing watch service", e); + } + LOG.info(getName() + " thread finished"); + FileChangeWatcher.this.setState(FileChangeWatcher.State.STOPPED); + } + } + + private void runLoop() { + while (FileChangeWatcher.this.getState() == FileChangeWatcher.State.RUNNING) { + WatchKey key; + try { + key = watchService.take(); + } catch (InterruptedException|ClosedWatchServiceException e) { + if (LOG.isDebugEnabled()) { + LOG.debug(getName() + " was interrupted and is shutting down ..."); + } + break; + } + for (WatchEvent event : key.pollEvents()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Got file changed event: " + event.kind() + " with context: " + event.context()); + } + try { + callback.accept(event); + } catch (Throwable e) { + LOG.error("Error from callback", e); + } + } + boolean isKeyValid = key.reset(); + if (!isKeyValid) { + // This is likely a problem, it means that file reloading is broken, probably because the + // directory we are watching was deleted or otherwise became inaccessible (unmounted, permissions + // changed, ???). + // For now, we log an error and exit the watcher thread. + LOG.error("Watch key no longer valid, maybe the directory is inaccessible?"); + break; + } + } + } + } +} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java b/zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java index e3625a5..4ea105b 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java @@ -19,8 +19,13 @@ package org.apache.zookeeper.common; import java.io.ByteArrayInputStream; +import java.io.Closeable; import java.io.IOException; import java.net.Socket; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardWatchEventKinds; +import java.nio.file.WatchEvent; import java.security.GeneralSecurityException; import java.security.KeyManagementException; import java.security.KeyStore; @@ -58,7 +63,7 @@ import org.slf4j.LoggerFactory; * Performance testing done by Facebook engineers shows that on Intel x86_64 machines, Java9 performs better with * GCM and Java8 performs better with CBC, so these seem like reasonable defaults. */ -public abstract class X509Util { +public abstract class X509Util implements Closeable, AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(X509Util.class); static final String DEFAULT_PROTOCOL = "TLSv1.2"; @@ -93,6 +98,8 @@ public abstract class X509Util { private String[] cipherSuites; private AtomicReference defaultSSLContext = new AtomicReference<>(null); + private FileChangeWatcher keyStoreFileWatcher; + private FileChangeWatcher trustStoreFileWatcher; public X509Util() { String cipherSuitesInput = System.getProperty(cipherSuitesProperty); @@ -172,6 +179,11 @@ public abstract class X509Util { return result; } + private void resetDefaultSSLContext() throws X509Exception.SSLContextException { + SSLContext newContext = createSSLContext(); + defaultSSLContext.set(newContext); + } + private SSLContext createSSLContext() throws SSLContextException { /* * Since Configuration initializes the key store and trust store related @@ -446,4 +458,105 @@ public abstract class X509Util { LOG.debug("Using Java8-optimized cipher suites for Java version {}", javaVersion); return DEFAULT_CIPHERS_JAVA8; } + + private FileChangeWatcher newFileChangeWatcher(String fileLocation) throws IOException { + if (fileLocation == null || fileLocation.isEmpty()) { + return null; + } + final Path filePath = Paths.get(fileLocation).toAbsolutePath(); + Path parentPath = filePath.getParent(); + if (parentPath == null) { + throw new IOException( + "Key/trust store path does not have a parent: " + filePath); + } + return new FileChangeWatcher( + parentPath, + watchEvent -> { + handleWatchEvent(filePath, watchEvent); + }); + } + + /** + * Enables automatic reloading of the trust store and key store files when they change on disk. + * + * @throws IOException if creating the FileChangeWatcher objects fails. + */ + public void enableCertFileReloading() throws IOException { + LOG.info("enabling cert file reloading"); + ZKConfig config = new ZKConfig(); + FileChangeWatcher newKeyStoreFileWatcher = + newFileChangeWatcher(config.getProperty(sslKeystoreLocationProperty)); + if (newKeyStoreFileWatcher != null) { + // stop old watcher if there is one + if (keyStoreFileWatcher != null) { + keyStoreFileWatcher.stop(); + } + keyStoreFileWatcher = newKeyStoreFileWatcher; + keyStoreFileWatcher.start(); + } + FileChangeWatcher newTrustStoreFileWatcher = + newFileChangeWatcher(config.getProperty(sslTruststoreLocationProperty)); + if (newTrustStoreFileWatcher != null) { + // stop old watcher if there is one + if (trustStoreFileWatcher != null) { + trustStoreFileWatcher.stop(); + } + trustStoreFileWatcher = newTrustStoreFileWatcher; + trustStoreFileWatcher.start(); + } + } + + /** + * Disables automatic reloading of the trust store and key store files when they change on disk. + * Stops background threads and closes WatchService instances. + */ + @Override + public void close() { + if (keyStoreFileWatcher != null) { + keyStoreFileWatcher.stop(); + keyStoreFileWatcher = null; + } + if (trustStoreFileWatcher != null) { + trustStoreFileWatcher.stop(); + trustStoreFileWatcher = null; + } + } + + /** + * Handler for watch events that let us know a file we may care about has changed on disk. + * + * @param filePath the path to the file we are watching for changes. + * @param event the WatchEvent. + */ + private void handleWatchEvent(Path filePath, WatchEvent event) { + boolean shouldResetContext = false; + Path dirPath = filePath.getParent(); + if (event.kind().equals(StandardWatchEventKinds.OVERFLOW)) { + // If we get notified about possibly missed events, reload the key store / trust store just to be sure. + shouldResetContext = true; + } else if (event.kind().equals(StandardWatchEventKinds.ENTRY_MODIFY) || + event.kind().equals(StandardWatchEventKinds.ENTRY_CREATE)) { + Path eventFilePath = dirPath.resolve((Path) event.context()); + if (filePath.equals(eventFilePath)) { + shouldResetContext = true; + } + } + // Note: we don't care about delete events + if (shouldResetContext) { + if (LOG.isDebugEnabled()) { + LOG.debug("Attempting to reset default SSL context after receiving watch event: " + + event.kind() + " with context: " + event.context()); + } + try { + this.resetDefaultSSLContext(); + } catch (SSLContextException e) { + throw new RuntimeException(e); + } + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Ignoring watch event and keeping previous default SSL context. Event kind: " + + event.kind() + " with context: " + event.context()); + } + } + } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java b/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java index effc0d5..086c07e 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java @@ -103,12 +103,15 @@ public class ZKConfig { properties.put(KINIT_COMMAND, System.getProperty(KINIT_COMMAND)); properties.put(JGSS_NATIVE, System.getProperty(JGSS_NATIVE)); - ClientX509Util clientX509Util = new ClientX509Util(); - putSSLProperties(clientX509Util); - properties.put(clientX509Util.getSslAuthProviderProperty(), - System.getProperty(clientX509Util.getSslAuthProviderProperty())); + try (ClientX509Util clientX509Util = new ClientX509Util()) { + putSSLProperties(clientX509Util); + properties.put(clientX509Util.getSslAuthProviderProperty(), + System.getProperty(clientX509Util.getSslAuthProviderProperty())); + } - putSSLProperties(new QuorumX509Util()); + try (X509Util x509Util = new QuorumX509Util()) { + putSSLProperties(x509Util); + } } private void putSSLProperties(X509Util x509Util) { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxnFactory.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxnFactory.java index 99de0e6..d96f56d 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxnFactory.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxnFactory.java @@ -408,6 +408,8 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory { } LOG.info("shutdown called {}", localAddress); + x509Util.close(); + if (login != null) { login.shutdown(); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/X509AuthenticationProvider.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/X509AuthenticationProvider.java index d0ca079..22ad070 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/X509AuthenticationProvider.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/X509AuthenticationProvider.java @@ -68,46 +68,46 @@ public class X509AuthenticationProvider implements AuthenticationProvider { */ public X509AuthenticationProvider() throws X509Exception { ZKConfig config = new ZKConfig(); - X509Util x509Util = new ClientX509Util(); - - String keyStoreLocation = config.getProperty(x509Util.getSslKeystoreLocationProperty(), ""); - String keyStorePassword = config.getProperty(x509Util.getSslKeystorePasswdProperty(), ""); - String keyStoreTypeProp = config.getProperty(x509Util.getSslKeystoreTypeProperty()); - - boolean crlEnabled = Boolean.parseBoolean(config.getProperty(x509Util.getSslCrlEnabledProperty())); - boolean ocspEnabled = Boolean.parseBoolean(config.getProperty(x509Util.getSslOcspEnabledProperty())); - boolean hostnameVerificationEnabled = Boolean.parseBoolean( - config.getProperty(x509Util.getSslHostnameVerificationEnabledProperty())); - - X509KeyManager km = null; - X509TrustManager tm = null; - if (keyStoreLocation.isEmpty()) { - LOG.warn("keystore not specified for client connection"); - } else { - try { - km = X509Util.createKeyManager(keyStoreLocation, keyStorePassword, keyStoreTypeProp); - } catch (KeyManagerException e) { - LOG.error("Failed to create key manager", e); + try (X509Util x509Util = new ClientX509Util()) { + String keyStoreLocation = config.getProperty(x509Util.getSslKeystoreLocationProperty(), ""); + String keyStorePassword = config.getProperty(x509Util.getSslKeystorePasswdProperty(), ""); + String keyStoreTypeProp = config.getProperty(x509Util.getSslKeystoreTypeProperty()); + + boolean crlEnabled = Boolean.parseBoolean(config.getProperty(x509Util.getSslCrlEnabledProperty())); + boolean ocspEnabled = Boolean.parseBoolean(config.getProperty(x509Util.getSslOcspEnabledProperty())); + boolean hostnameVerificationEnabled = Boolean.parseBoolean( + config.getProperty(x509Util.getSslHostnameVerificationEnabledProperty())); + + X509KeyManager km = null; + X509TrustManager tm = null; + if (keyStoreLocation.isEmpty()) { + LOG.warn("keystore not specified for client connection"); + } else { + try { + km = X509Util.createKeyManager(keyStoreLocation, keyStorePassword, keyStoreTypeProp); + } catch (KeyManagerException e) { + LOG.error("Failed to create key manager", e); + } } - } - - String trustStoreLocation = config.getProperty(x509Util.getSslTruststoreLocationProperty(), ""); - String trustStorePassword = config.getProperty(x509Util.getSslTruststorePasswdProperty(), ""); - String trustStoreTypeProp = config.getProperty(x509Util.getSslTruststoreTypeProperty()); - - if (trustStoreLocation.isEmpty()) { - LOG.warn("Truststore not specified for client connection"); - } else { - try { - tm = X509Util.createTrustManager( - trustStoreLocation, trustStorePassword, trustStoreTypeProp, crlEnabled, ocspEnabled, - hostnameVerificationEnabled, false); - } catch (TrustManagerException e) { - LOG.error("Failed to create trust manager", e); + + String trustStoreLocation = config.getProperty(x509Util.getSslTruststoreLocationProperty(), ""); + String trustStorePassword = config.getProperty(x509Util.getSslTruststorePasswdProperty(), ""); + String trustStoreTypeProp = config.getProperty(x509Util.getSslTruststoreTypeProperty()); + + if (trustStoreLocation.isEmpty()) { + LOG.warn("Truststore not specified for client connection"); + } else { + try { + tm = X509Util.createTrustManager( + trustStoreLocation, trustStorePassword, trustStoreTypeProp, crlEnabled, ocspEnabled, + hostnameVerificationEnabled, false); + } catch (TrustManagerException e) { + LOG.error("Failed to create trust manager", e); + } } + this.keyManager = km; + this.trustManager = tm; } - this.keyManager = km; - this.trustManager = tm; } /** diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java index 721a1b4..397ea6d 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java @@ -247,21 +247,16 @@ public class Leader implements LearnerMaster { private final ServerSocket ss; - Leader(QuorumPeer self,LeaderZooKeeperServer zk) throws IOException, X509Exception { + Leader(QuorumPeer self,LeaderZooKeeperServer zk) throws IOException { this.self = self; this.proposalStats = new BufferStats(); try { - if (self.shouldUsePortUnification()) { + if (self.shouldUsePortUnification() || self.isSslQuorum()) { + boolean allowInsecureConnection = self.shouldUsePortUnification(); if (self.getQuorumListenOnAllIPs()) { - ss = new UnifiedServerSocket(self.getX509Util(), true, self.getQuorumAddress().getPort()); + ss = new UnifiedServerSocket(self.getX509Util(), allowInsecureConnection, self.getQuorumAddress().getPort()); } else { - ss = new UnifiedServerSocket(self.getX509Util(), true); - } - } else if (self.isSslQuorum()) { - if (self.getQuorumListenOnAllIPs()) { - ss = self.getX509Util().createSSLServerSocket(self.getQuorumAddress().getPort()); - } else { - ss = self.getX509Util().createSSLServerSocket(); + ss = new UnifiedServerSocket(self.getX509Util(), allowInsecureConnection); } } else { if (self.getQuorumListenOnAllIPs()) { @@ -274,9 +269,6 @@ public class Leader implements LearnerMaster { if (!self.getQuorumListenOnAllIPs()) { ss.bind(self.getQuorumAddress()); } - } catch (X509Exception e) { - LOG.error("Failed to setup ssl server socket", e); - throw e; } catch (BindException e) { if (self.getQuorumListenOnAllIPs()) { LOG.error("Couldn't bind to port " + self.getQuorumAddress().getPort(), e); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java index 4175f3c..704580d 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java @@ -872,7 +872,7 @@ public class QuorumCnxManager { if (self.shouldUsePortUnification()) { ss = new UnifiedServerSocket(self.getX509Util(), true); } else if (self.isSslQuorum()) { - ss = self.getX509Util().createSSLServerSocket(); + ss = new UnifiedServerSocket(self.getX509Util(), false); } else { ss = new ServerSocket(); } @@ -914,7 +914,7 @@ public class QuorumCnxManager { + "see ZOOKEEPER-2836"); } } - } catch (IOException|X509Exception e) { + } catch (IOException e) { if (shutdown) { break; } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java index 9d36fe2..a724903 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -1324,6 +1324,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider public void shutdown() { running = false; + x509Util.close(); if (leader != null) { leader.shutdown("quorum Peer shutdown"); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java index 2c4dd0d..8bdeb33 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java @@ -72,6 +72,7 @@ public class QuorumPeerConfig { protected boolean sslQuorum = false; protected boolean shouldUsePortUnification = false; protected int observerMasterPort; + protected boolean sslQuorumReloadCertFiles = false; protected File dataDir; protected File dataLogDir; protected String dynamicConfigFileStr = null; @@ -321,6 +322,8 @@ public class QuorumPeerConfig { sslQuorum = Boolean.parseBoolean(value); } else if (key.equals("portUnification")){ shouldUsePortUnification = Boolean.parseBoolean(value); + } else if (key.equals("sslQuorumReloadCertFiles")) { + sslQuorumReloadCertFiles = Boolean.parseBoolean(value); } else if ((key.startsWith("server.") || key.startsWith("group") || key.startsWith("weight")) && zkProp.containsKey("dynamicConfigFile")) { throw new ConfigException("parameter: " + key + " must be in a separate dynamic config file"); } else if (key.equals(QuorumAuth.QUORUM_SASL_AUTH_ENABLED)) { @@ -462,15 +465,16 @@ public class QuorumPeerConfig { * provider is not configured. */ private void configureSSLAuth() throws ConfigException { - ClientX509Util clientX509Util = new ClientX509Util(); - String sslAuthProp = "zookeeper.authProvider." + System.getProperty(clientX509Util.getSslAuthProviderProperty(), "x509"); - if (System.getProperty(sslAuthProp) == null) { - if ("zookeeper.authProvider.x509".equals(sslAuthProp)) { - System.setProperty("zookeeper.authProvider.x509", - "org.apache.zookeeper.server.auth.X509AuthenticationProvider"); - } else { - throw new ConfigException("No auth provider configured for the SSL authentication scheme '" - + System.getProperty(clientX509Util.getSslAuthProviderProperty()) + "'."); + try (ClientX509Util clientX509Util = new ClientX509Util()) { + String sslAuthProp = "zookeeper.authProvider." + System.getProperty(clientX509Util.getSslAuthProviderProperty(), "x509"); + if (System.getProperty(sslAuthProp) == null) { + if ("zookeeper.authProvider.x509".equals(sslAuthProp)) { + System.setProperty("zookeeper.authProvider.x509", + "org.apache.zookeeper.server.auth.X509AuthenticationProvider"); + } else { + throw new ConfigException("No auth provider configured for the SSL authentication scheme '" + + System.getProperty(clientX509Util.getSslAuthProviderProperty()) + "'."); + } } } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java index a2e2bfc..24c8758 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java @@ -203,6 +203,9 @@ public class QuorumPeerMain { quorumPeer.setLearnerType(config.getPeerType()); quorumPeer.setSyncEnabled(config.getSyncEnabled()); quorumPeer.setQuorumListenOnAllIPs(config.getQuorumListenOnAllIPs()); + if (config.sslQuorumReloadCertFiles) { + quorumPeer.getX509Util().enableCertFileReloading(); + } // sets quorum sasl authentication configurations quorumPeer.setQuorumSaslEnabled(config.quorumEnableSasl); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/common/FileChangeWatcherTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/common/FileChangeWatcherTest.java new file mode 100644 index 0000000..e4950f3 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/common/FileChangeWatcherTest.java @@ -0,0 +1,263 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.common; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.StandardWatchEventKinds; +import java.nio.file.WatchEvent; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.commons.io.FileUtils; +import org.apache.zookeeper.ZKTestCase; +import org.apache.zookeeper.test.ClientBase; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +public class FileChangeWatcherTest extends ZKTestCase { + private static File tempDir; + private static File tempFile; + + private static final Logger LOG = LoggerFactory.getLogger(FileChangeWatcherTest.class); + + @BeforeClass + public static void createTempFile() throws IOException { + tempDir = ClientBase.createEmptyTestDir(); + tempFile = File.createTempFile("zk_test_", "", tempDir); + tempFile.deleteOnExit(); + } + + @AfterClass + public static void cleanupTempDir() { + try { + FileUtils.deleteDirectory(tempDir); + } catch (IOException e) { + // ignore + } + } + + @Test + public void testCallbackWorksOnFileChanges() throws IOException, InterruptedException { + FileChangeWatcher watcher = null; + try { + final List> events = new ArrayList<>(); + watcher = new FileChangeWatcher( + tempDir.toPath(), + event -> { + LOG.info("Got an update: " + event.kind() + " " + event.context()); + synchronized (events) { + events.add(event); + events.notifyAll(); + } + }); + watcher.start(); + watcher.waitForState(FileChangeWatcher.State.RUNNING); + Thread.sleep(1000L); // XXX hack + for (int i = 0; i < 3; i++) { + LOG.info("Modifying file, attempt " + (i + 1)); + FileUtils.writeStringToFile( + tempFile, + "Hello world " + i + "\n", + StandardCharsets.UTF_8, + true); + synchronized (events) { + if (events.size() < i + 1) { + events.wait(3000L); + } + assertEquals("Wrong number of events", i + 1, events.size()); + WatchEvent event = events.get(i); + assertEquals(StandardWatchEventKinds.ENTRY_MODIFY, event.kind()); + assertEquals(tempFile.getName(), event.context().toString()); + } + } + } finally { + if (watcher != null) { + watcher.stop(); + watcher.waitForState(FileChangeWatcher.State.STOPPED); + } + } + } + + @Test + public void testCallbackWorksOnFileTouched() throws IOException, InterruptedException { + FileChangeWatcher watcher = null; + try { + final List> events = new ArrayList<>(); + watcher = new FileChangeWatcher( + tempDir.toPath(), + event -> { + LOG.info("Got an update: " + event.kind() + " " + event.context()); + synchronized (events) { + events.add(event); + events.notifyAll(); + } + }); + watcher.start(); + watcher.waitForState(FileChangeWatcher.State.RUNNING); + Thread.sleep(1000L); // XXX hack + LOG.info("Touching file"); + FileUtils.touch(tempFile); + synchronized (events) { + if (events.isEmpty()) { + events.wait(3000L); + } + assertFalse(events.isEmpty()); + WatchEvent event = events.get(0); + assertEquals(StandardWatchEventKinds.ENTRY_MODIFY, event.kind()); + assertEquals(tempFile.getName(), event.context().toString()); + } + } finally { + if (watcher != null) { + watcher.stop(); + watcher.waitForState(FileChangeWatcher.State.STOPPED); + } + } + } + + @Test + public void testCallbackWorksOnFileAdded() throws IOException, InterruptedException { + FileChangeWatcher watcher = null; + try { + final List> events = new ArrayList<>(); + watcher = new FileChangeWatcher( + tempDir.toPath(), + event -> { + LOG.info("Got an update: " + event.kind() + " " + event.context()); + synchronized (events) { + events.add(event); + events.notifyAll(); + } + }); + watcher.start(); + watcher.waitForState(FileChangeWatcher.State.RUNNING); + Thread.sleep(1000L); // XXX hack + File tempFile2 = File.createTempFile("zk_test_", "", tempDir); + tempFile2.deleteOnExit(); + synchronized (events) { + if (events.isEmpty()) { + events.wait(3000L); + } + assertFalse(events.isEmpty()); + WatchEvent event = events.get(0); + assertEquals(StandardWatchEventKinds.ENTRY_CREATE, event.kind()); + assertEquals(tempFile2.getName(), event.context().toString()); + } + } finally { + if (watcher != null) { + watcher.stop(); + watcher.waitForState(FileChangeWatcher.State.STOPPED); + } + } + } + + @Test + public void testCallbackWorksOnFileDeleted() throws IOException, InterruptedException { + FileChangeWatcher watcher = null; + try { + final List> events = new ArrayList<>(); + watcher = new FileChangeWatcher( + tempDir.toPath(), + event -> { + LOG.info("Got an update: " + event.kind() + " " + event.context()); + synchronized (events) { + events.add(event); + events.notifyAll(); + } + }); + watcher.start(); + watcher.waitForState(FileChangeWatcher.State.RUNNING); + Thread.sleep(1000L); // XXX hack + tempFile.delete(); + synchronized (events) { + if (events.isEmpty()) { + events.wait(3000L); + } + assertFalse(events.isEmpty()); + WatchEvent event = events.get(0); + assertEquals(StandardWatchEventKinds.ENTRY_DELETE, event.kind()); + assertEquals(tempFile.getName(), event.context().toString()); + } + } finally { + if (watcher != null) { + watcher.stop(); + watcher.waitForState(FileChangeWatcher.State.STOPPED); + } + } + } + + @Test + public void testCallbackErrorDoesNotCrashWatcherThread() throws IOException, InterruptedException { + FileChangeWatcher watcher = null; + try { + final List> events = new ArrayList<>(); + final AtomicInteger callCount = new AtomicInteger(0); + watcher = new FileChangeWatcher( + tempDir.toPath(), + event -> { + LOG.info("Got an update: " + event.kind() + " " + event.context()); + synchronized (callCount) { + if (callCount.getAndIncrement() == 0) { + callCount.notifyAll(); + throw new RuntimeException("This error should not crash the watcher thread"); + } + } + synchronized (events) { + events.add(event); + events.notifyAll(); + } + }); + watcher.start(); + watcher.waitForState(FileChangeWatcher.State.RUNNING); + Thread.sleep(1000L); // XXX hack + LOG.info("Modifying file"); + FileUtils.writeStringToFile(tempFile, "Hello world\n", StandardCharsets.UTF_8, true); + synchronized (callCount) { + while (callCount.get() == 0) { + callCount.wait(3000L); + } + } + LOG.info("Modifying file again"); + FileUtils.writeStringToFile(tempFile, "Hello world again\n", StandardCharsets.UTF_8, true); + synchronized (events) { + if (events.isEmpty()) { + events.wait(3000L); + } + assertEquals(2, callCount.get()); + assertFalse(events.isEmpty()); + WatchEvent event = events.get(0); + assertEquals(StandardWatchEventKinds.ENTRY_MODIFY, event.kind()); + assertEquals(tempFile.getName(), event.context().toString()); + } + } finally { + if (watcher != null) { + watcher.stop(); + watcher.waitForState(FileChangeWatcher.State.STOPPED); + } + } + } +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/common/X509UtilTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/common/X509UtilTest.java index 546cf55..1058010 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/common/X509UtilTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/common/X509UtilTest.java @@ -70,7 +70,9 @@ public class X509UtilTest extends BaseX509ParameterizedTestCase { @Before public void setUp() throws Exception { - x509TestContext.setSystemProperties(new ClientX509Util(), KeyStoreFileType.JKS, KeyStoreFileType.JKS); + try (X509Util x509util = new ClientX509Util()) { + x509TestContext.setSystemProperties(x509util, KeyStoreFileType.JKS, KeyStoreFileType.JKS); + } System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, "org.apache.zookeeper.server.NettyServerCnxnFactory"); System.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, "org.apache.zookeeper.ClientCnxnSocketNetty"); x509Util = new ClientX509Util(); @@ -83,12 +85,14 @@ public class X509UtilTest extends BaseX509ParameterizedTestCase { System.clearProperty(x509Util.getSslCrlEnabledProperty()); System.clearProperty(x509Util.getCipherSuitesProperty()); System.clearProperty(x509Util.getSslProtocolProperty()); + System.clearProperty(x509Util.getSslHandshakeDetectionTimeoutMillisProperty()); System.clearProperty("com.sun.net.ssl.checkRevocation"); System.clearProperty("com.sun.security.enableCRLDP"); Security.setProperty("ocsp.enable", Boolean.FALSE.toString()); Security.setProperty("com.sun.security.enableCRLDP", Boolean.FALSE.toString()); System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); System.clearProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET); + x509Util.close(); } @Test(timeout = 5000) @@ -358,35 +362,37 @@ public class X509UtilTest extends BaseX509ParameterizedTestCase { @Test public void testGetSslHandshakeDetectionTimeoutMillisProperty() { - X509Util x509Util = new ClientX509Util(); Assert.assertEquals( X509Util.DEFAULT_HANDSHAKE_DETECTION_TIMEOUT_MILLIS, x509Util.getSslHandshakeTimeoutMillis()); - try { - String newPropertyString = Integer.toString(X509Util.DEFAULT_HANDSHAKE_DETECTION_TIMEOUT_MILLIS + 1); - System.setProperty(x509Util.getSslHandshakeDetectionTimeoutMillisProperty(), newPropertyString); - // Note: need to create a new ClientX509Util to pick up modified property value + // Note: need to create a new ClientX509Util each time to pick up modified property value + String newPropertyString = Integer.toString(X509Util.DEFAULT_HANDSHAKE_DETECTION_TIMEOUT_MILLIS + 1); + System.setProperty(x509Util.getSslHandshakeDetectionTimeoutMillisProperty(), newPropertyString); + try (X509Util tempX509Util = new ClientX509Util()) { Assert.assertEquals( X509Util.DEFAULT_HANDSHAKE_DETECTION_TIMEOUT_MILLIS + 1, - new ClientX509Util().getSslHandshakeTimeoutMillis()); - // 0 value not allowed, will return the default - System.setProperty(x509Util.getSslHandshakeDetectionTimeoutMillisProperty(), "0"); + tempX509Util.getSslHandshakeTimeoutMillis()); + } + // 0 value not allowed, will return the default + System.setProperty(x509Util.getSslHandshakeDetectionTimeoutMillisProperty(), "0"); + try (X509Util tempX509Util = new ClientX509Util()) { Assert.assertEquals( X509Util.DEFAULT_HANDSHAKE_DETECTION_TIMEOUT_MILLIS, - new ClientX509Util().getSslHandshakeTimeoutMillis()); - // Negative value not allowed, will return the default - System.setProperty(x509Util.getSslHandshakeDetectionTimeoutMillisProperty(), "-1"); + tempX509Util.getSslHandshakeTimeoutMillis()); + } + // Negative value not allowed, will return the default + System.setProperty(x509Util.getSslHandshakeDetectionTimeoutMillisProperty(), "-1"); + try (X509Util tempX509Util = new ClientX509Util()) { Assert.assertEquals( X509Util.DEFAULT_HANDSHAKE_DETECTION_TIMEOUT_MILLIS, - new ClientX509Util().getSslHandshakeTimeoutMillis()); - } finally { - System.clearProperty(x509Util.getSslHandshakeDetectionTimeoutMillisProperty()); + tempX509Util.getSslHandshakeTimeoutMillis()); } } // Warning: this will reset the x509Util private void setCustomCipherSuites() { System.setProperty(x509Util.getCipherSuitesProperty(), customCipherSuites[0] + "," + customCipherSuites[1]); + x509Util.close(); // remember to close old instance before replacing it x509Util = new ClientX509Util(); } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerConfigTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerConfigTest.java index f4a8b4b..f480f07 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerConfigTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerConfigTest.java @@ -91,17 +91,18 @@ public class QuorumPeerConfigTest { * https://issues.apache.org/jira/browse/ZOOKEEPER-2297 */ @Test - public void testCustomSSLAuth() - throws IOException{ - System.setProperty(new ClientX509Util().getSslAuthProviderProperty(), "y509"); - QuorumPeerConfig quorumPeerConfig = new QuorumPeerConfig(); - try { - Properties zkProp = getDefaultZKProperties(); - zkProp.setProperty("secureClientPort", "12345"); - quorumPeerConfig.parseProperties(zkProp); - fail("ConfigException is expected"); - } catch (ConfigException e) { - assertNotNull(e.getMessage()); + public void testCustomSSLAuth() throws IOException { + try (ClientX509Util x509Util = new ClientX509Util()) { + System.setProperty(x509Util.getSslAuthProviderProperty(), "y509"); + QuorumPeerConfig quorumPeerConfig = new QuorumPeerConfig(); + try { + Properties zkProp = getDefaultZKProperties(); + zkProp.setProperty("secureClientPort", "12345"); + quorumPeerConfig.parseProperties(zkProp); + fail("ConfigException is expected"); + } catch (ConfigException e) { + assertNotNull(e.getMessage()); + } } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java index 67c15ad..e47b7ef 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java @@ -124,7 +124,7 @@ public class QuorumSSLTest extends QuorumPeerTestBase { private static final char[] PASSWORD = "testpass".toCharArray(); private static final String HOSTNAME = "localhost"; - private QuorumX509Util quorumX509Util = new QuorumX509Util(); + private QuorumX509Util quorumX509Util; private MainThread q1; private MainThread q2; @@ -156,6 +156,7 @@ public class QuorumSSLTest extends QuorumPeerTestBase { @Before public void setup() throws Exception { + quorumX509Util = new QuorumX509Util(); ClientBase.setupTestEnv(); tmpDir = createTmpDir().getAbsolutePath(); @@ -406,6 +407,7 @@ public class QuorumSSLTest extends QuorumPeerTestBase { } Security.removeProvider("BC"); + quorumX509Util.close(); } private void clearSSLSystemProperties() { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketModeDetectionTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketModeDetectionTest.java index 61862a4..e9267b9 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketModeDetectionTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketModeDetectionTest.java @@ -171,6 +171,7 @@ public class UnifiedServerSocketModeDetectionTest extends ZKTestCase { forceClose(clientSocket); workerPool.shutdown(); workerPool.awaitTermination(1000, TimeUnit.MILLISECONDS); + x509Util.close(); } @Test diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java index 5e4e619..ddc05dc 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java @@ -114,6 +114,7 @@ public class UnifiedServerSocketTest extends BaseX509ParameterizedTestCase { @After public void tearDown() throws Exception { x509TestContext.clearSystemProperties(x509Util); + x509Util.close(); } private static void forceClose(Socket s) { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java index 08ffb4e..b486077 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java @@ -37,10 +37,11 @@ import org.junit.Test; public class ClientSSLTest extends QuorumPeerTestBase { - private ClientX509Util clientX509Util = new ClientX509Util(); + private ClientX509Util clientX509Util; @Before public void setup() { + clientX509Util = new ClientX509Util(); 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"); @@ -60,6 +61,7 @@ public class ClientSSLTest extends QuorumPeerTestBase { System.clearProperty(clientX509Util.getSslKeystorePasswdProperty()); System.clearProperty(clientX509Util.getSslTruststoreLocationProperty()); System.clearProperty(clientX509Util.getSslTruststorePasswdProperty()); + clientX509Util.close(); } /** diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/SSLAuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SSLAuthTest.java index 8fd35bc..6d30343 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/SSLAuthTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SSLAuthTest.java @@ -24,7 +24,6 @@ import org.apache.zookeeper.PortAssignment; import org.apache.zookeeper.TestableZooKeeper; import org.apache.zookeeper.client.ZKClientConfig; import org.apache.zookeeper.common.ClientX509Util; -import org.apache.zookeeper.common.X509Util; import org.apache.zookeeper.server.ServerCnxnFactory; import org.junit.After; import org.junit.Assert; @@ -33,10 +32,11 @@ import org.junit.Test; public class SSLAuthTest extends ClientBase { - private ClientX509Util clientX509Util = new ClientX509Util(); + private ClientX509Util clientX509Util; @Before public void setUp() throws Exception { + clientX509Util = new ClientX509Util(); 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"); @@ -71,6 +71,7 @@ public class SSLAuthTest extends ClientBase { System.clearProperty(clientX509Util.getSslTruststorePasswdProperty()); System.clearProperty("javax.net.debug"); System.clearProperty("zookeeper.authProvider.x509"); + clientX509Util.close(); } @Test