Skip to content

Commit

Permalink
ZOOKEEPER-3174: Quorum TLS - support reloading trust/key store
Browse files Browse the repository at this point in the history
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.

This is the branch-3.5 version of #680

Author: Ilya Maykov <ilyam@fb.com>

Reviewers: fangmin@apache.org, andor@apache.org

Closes #737 from ivmaykov/ZOOKEEPER-3174-branch3.5 and squashes the following commits:

6cc1d62 [Ilya Maykov] ZOOKEEPER-3219: Fix flaky FileChangeWatcherTest
df72944 [Ilya Maykov] ZOOKEEPER-3174: Quorum TLS - support reloading trust/key store
  • Loading branch information
ivmaykov authored and anmolnar committed Dec 19, 2018
1 parent 193a6ed commit 1730774
Show file tree
Hide file tree
Showing 20 changed files with 791 additions and 122 deletions.
Expand Up @@ -18,10 +18,27 @@

package org.apache.zookeeper;

import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.SocketAddress;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Executors;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;

import javax.net.ssl.SSLContext;
import javax.net.ssl.SSLEngine;

import org.apache.zookeeper.ClientCnxn.EndOfStreamException;
import org.apache.zookeeper.ClientCnxn.Packet;
import org.apache.zookeeper.client.ZKClientConfig;
import org.apache.zookeeper.common.ClientX509Util;
import org.apache.zookeeper.common.X509Util;
import org.jboss.netty.bootstrap.ClientBootstrap;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.buffer.ChannelBuffers;
Expand All @@ -42,22 +59,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.net.ssl.SSLContext;
import javax.net.ssl.SSLEngine;

import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.SocketAddress;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Executors;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;

import static org.apache.zookeeper.common.X509Exception.SSLContextException;

/**
Expand Down Expand Up @@ -370,9 +371,11 @@ public ChannelPipeline getPipeline() throws Exception {
// 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.getChannel());
Expand Down
Expand Up @@ -34,6 +34,7 @@
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;

Expand Down Expand Up @@ -90,12 +91,14 @@ public static String send4LetterWord(String host, int port, String cmd, boolean
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);
Expand Down
@@ -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:
* <ul>
* <li>The callback should be thread-safe.</li>
* <li>Changes that happen around the time the thread is started may be missed.</li>
* <li>There is a delay between a file changing and the callback firing.</li>
* <li>The watch is not recursive - changes to subdirectories will not trigger a callback.</li>
* </ul>
*/
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 <code>dirPath</code> and invokes <code>callback</code> on changes.
*
* @param dirPath the directory to watch.
* @param callback the callback to invoke with events. <code>event.kind()</code> will return the type of event,
* and <code>event.context()</code> will return the filename relative to <code>dirPath</code>.
* @throws IOException if there is an error creating the WatchService.
*/
public FileChangeWatcher(Path dirPath, Consumer<WatchEvent<?>> 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 <code>desiredState</code>.
* 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 <code>newState</code>.
* @param newState the new state.
*/
private synchronized void setState(State newState) {
state = newState;
this.notifyAll();
}

/**
* Atomically sets the state to <code>update</code> if and only if the
* state is currently <code>expected</code>.
* @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 <code>expected</code>.
*/
private synchronized boolean compareAndSetState(State expected, State update) {
if (state == expected) {
setState(update);
return true;
} else {
return false;
}
}

/**
* Atomically sets the state to <code>update</code> if and only if the
* state is currently one of <code>expectedStates</code>.
* @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 <code>expectedStates</code>.
*/
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<WatchEvent<?>> callback;

WatcherThread(WatchService watchService, Consumer<WatchEvent<?>> 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;
}
}
}
}
}

0 comments on commit 1730774

Please sign in to comment.