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:30:18 UTC

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

GitHub user ivmaykov opened a pull request:

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

    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.
    
    Note that this is stacked on top of #678 and #679 and thus includes them. Please only consider the ZOOKEEPER-3174 commit when reviewing.

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

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

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

    https://github.com/apache/zookeeper/pull/680.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 #680
    
----
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

commit 65edf69084bebfc50613daafefe7ebb3afbb6e36
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T01:54:06Z

    ZOOKEEPER-3174: Quorum TLS - support reloading trust/key store

----


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230560444
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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);
    +    }
    +
    +    public synchronized State getState() {
    +        return state;
    +    }
    +
    +    private synchronized void setState(State newState) {
    --- End diff --
    
    See above.


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230574877
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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);
    +    }
    +
    +    public synchronized State getState() {
    +        return state;
    +    }
    +
    +    private synchronized void setState(State newState) {
    +        state = newState;
    +        this.notifyAll();
    +    }
    +
    +    /**
    +     * Tells the background thread to start. Does not wait for it to be running.
    +     * Calling this method more than once has no effect.
    +     */
    +    public synchronized void start() {
    +        if (state != State.NEW) {
    +            return;
    +        }
    +        setState(State.STARTING);
    +        this.watcherThread.start();
    +    }
    +
    +    /**
    +     * Tells the background thread to stop. Does not wait for it to exit.
    +     */
    +    public synchronized void stop() {
    +        if (state == State.STARTING || state == State.RUNNING) {
    +            setState(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");
    +                synchronized (FileChangeWatcher.this) {
    --- End diff --
    
    Yes.Kind of state matrix->(current state,new state)->derived state


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

    https://github.com/apache/zookeeper/pull/680
  
    @anmolnar looks like FileMonitor only exists in our fork for now and is not upstreamed yet, so never mind about that comment! I'll work with other FB engineers to unify the two classes.


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r231969309
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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);
    +    }
    +
    +    public synchronized State getState() {
    +        return state;
    +    }
    +
    +    private synchronized void setState(State newState) {
    +        state = newState;
    +        this.notifyAll();
    +    }
    +
    +    /**
    +     * Tells the background thread to start. Does not wait for it to be running.
    +     * Calling this method more than once has no effect.
    +     */
    +    public synchronized void start() {
    +        if (state != State.NEW) {
    +            return;
    +        }
    +        setState(State.STARTING);
    +        this.watcherThread.start();
    +    }
    +
    +    /**
    +     * Tells the background thread to stop. Does not wait for it to exit.
    +     */
    +    public synchronized void stop() {
    +        if (state == State.STARTING || state == State.RUNNING) {
    +            setState(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");
    +                synchronized (FileChangeWatcher.this) {
    --- End diff --
    
    Please see the latest version.


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r231994796
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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);
    +    }
    +
    +    public synchronized State getState() {
    +        return state;
    +    }
    +
    +    private synchronized void setState(State newState) {
    +        state = newState;
    +        this.notifyAll();
    +    }
    +
    +    /**
    +     * Tells the background thread to start. Does not wait for it to be running.
    +     * Calling this method more than once has no effect.
    +     */
    +    public synchronized void start() {
    +        if (state != State.NEW) {
    +            return;
    +        }
    +        setState(State.STARTING);
    +        this.watcherThread.start();
    +    }
    +
    +    /**
    +     * Tells the background thread to stop. Does not wait for it to exit.
    +     */
    +    public synchronized void stop() {
    +        if (state == State.STARTING || state == State.RUNNING) {
    +            setState(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");
    +                synchronized (FileChangeWatcher.this) {
    --- End diff --
    
    I see it. Thanks for the change.


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

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


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230461099
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,180 @@
    +/**
    + * 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);
    +
    +    private final WatcherThread watcherThread;
    +
    +    /**
    +     * 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);
    +        this.watcherThread = new WatcherThread(watchService, callback);
    +        this.watcherThread.setDaemon(true);
    +        this.watcherThread.start();
    --- End diff --
    
    How about defining the life cycle for file watcher like start, stop etc.   and also define the state of the file watcher like starting, running, not started and stopping etc. The locking is not required if we define the states. The clients can leverage these states if there are any tasks depending on file watcher state


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r239138642
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java ---
    @@ -446,4 +458,119 @@ private void configureSSLServerSocket(SSLServerSocket sslServerSocket) {
             LOG.debug("Using Java8-optimized cipher suites for Java version {}", javaVersion);
             return DEFAULT_CIPHERS_JAVA8;
         }
    +
    +    /**
    +     * 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();
    +        String keyStoreLocation = config.getProperty(sslKeystoreLocationProperty);
    +        if (keyStoreLocation != null && !keyStoreLocation.isEmpty()) {
    +            final Path filePath = Paths.get(keyStoreLocation).toAbsolutePath();
    +            Path parentPath = filePath.getParent();
    +            if (parentPath == null) {
    +                throw new IOException(
    +                        "Key store path does not have a parent: " + filePath);
    +            }
    +            FileChangeWatcher newKeyStoreFileWatcher = new FileChangeWatcher(
    +                    parentPath,
    +                    watchEvent -> {
    +                        handleWatchEvent(filePath, watchEvent);
    +                    });
    +            // stop old watcher if there is one
    +            if (keyStoreFileWatcher != null) {
    +                keyStoreFileWatcher.stop();
    +            }
    +            keyStoreFileWatcher = newKeyStoreFileWatcher;
    +            keyStoreFileWatcher.start();
    +        }
    +        String trustStoreLocation = config.getProperty(sslTruststoreLocationProperty);
    +        if (trustStoreLocation != null && !trustStoreLocation.isEmpty()) {
    +            final Path filePath = Paths.get(trustStoreLocation).toAbsolutePath();
    +            Path parentPath = filePath.getParent();
    +            if (parentPath == null) {
    +                throw new IOException(
    +                        "Trust store path does not have a parent: " + filePath);
    +            }
    +            FileChangeWatcher newTrustStoreFileWatcher = new FileChangeWatcher(
    +                    parentPath,
    +                    watchEvent -> {
    +                        handleWatchEvent(filePath, watchEvent);
    +                    });
    +            // 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.
    +     */
    +    public void disableCertFileReloading() {
    +        if (keyStoreFileWatcher != null) {
    +            keyStoreFileWatcher.stop();
    +            keyStoreFileWatcher = null;
    +        }
    +        if (trustStoreFileWatcher != null) {
    +            trustStoreFileWatcher.stop();
    +            trustStoreFileWatcher = null;
    +        }
    +    }
    +
    +    // Finalizer guardian object, see Effective Java item 7
    +    // TODO: finalize() is deprecated starting with Java 10. This needs to be
    +    // replaced with an explicit shutdown call.
    +    @SuppressWarnings("unused")
    +    private final Object finalizerGuardian = new Object() {
    --- End diff --
    
    Reading the referenced literature about this, I believe it should be better to avoid using finalizer like this. We might even be better avoid using finalizers entirely. There's no guarantee when finalizer gets executed, not even guarantee to be executed at all, it has a huge performance penalty, etc.
    
    We should rather implement an explicit `close()` method with the `AutoClosable` interface and call it from QuorumPeer's shutdown() method.


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

    https://github.com/apache/zookeeper/pull/680
  
    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 #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

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


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r239143301
  
    --- Diff: 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:
    + * <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 {
    --- End diff --
    
    Maybe it doesn't make sense, but I'm thinking of whether it would be better to make `FileChangeWatcher` class the thread itself instead using a "wrapped" Thread instance. In which case you might not need to forward the `stop()` call, but can call it directly.


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

    https://github.com/apache/zookeeper/pull/680
  
    @ivmaykov No, I think it's fine to keep `FileChangeWatcher`. I keep looking...


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230559840
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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);
    +    }
    +
    +    public synchronized State getState() {
    --- End diff --
    
    Can we use ReentrantReadWriteLock  and get read lock here? The readers  will not be  blocked if  there is no any write lock 


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r239144917
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java ---
    @@ -446,4 +458,119 @@ private void configureSSLServerSocket(SSLServerSocket sslServerSocket) {
             LOG.debug("Using Java8-optimized cipher suites for Java version {}", javaVersion);
             return DEFAULT_CIPHERS_JAVA8;
         }
    +
    +    /**
    +     * 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();
    +        String keyStoreLocation = config.getProperty(sslKeystoreLocationProperty);
    +        if (keyStoreLocation != null && !keyStoreLocation.isEmpty()) {
    +            final Path filePath = Paths.get(keyStoreLocation).toAbsolutePath();
    +            Path parentPath = filePath.getParent();
    +            if (parentPath == null) {
    +                throw new IOException(
    +                        "Key store path does not have a parent: " + filePath);
    +            }
    +            FileChangeWatcher newKeyStoreFileWatcher = new FileChangeWatcher(
    +                    parentPath,
    +                    watchEvent -> {
    +                        handleWatchEvent(filePath, watchEvent);
    +                    });
    +            // stop old watcher if there is one
    +            if (keyStoreFileWatcher != null) {
    +                keyStoreFileWatcher.stop();
    +            }
    +            keyStoreFileWatcher = newKeyStoreFileWatcher;
    +            keyStoreFileWatcher.start();
    +        }
    +        String trustStoreLocation = config.getProperty(sslTruststoreLocationProperty);
    +        if (trustStoreLocation != null && !trustStoreLocation.isEmpty()) {
    --- End diff --
    
    This logic is redundant and can be extracted in a separate method.


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r239593442
  
    --- Diff: 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:
    + * <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 {
    --- End diff --
    
    I prefer to use composition over inheritance in cases where inheritance is not clearly better - it tends to avoid problems. If FileChangeWatcher extends Thread, then it will have a "is a" relationship with Thread, and can be used anywhere a Thread is used. Since it's not a generic Thread, it's not clear to me that this would be correct. But I don't feel too strongly about it.


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

    https://github.com/apache/zookeeper/pull/680
  
    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 pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

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


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

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


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

    https://github.com/apache/zookeeper/pull/680
  
    @anmolnar is there anything I can do to help move this PR and #681 along?


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r239593417
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java ---
    @@ -446,4 +458,119 @@ private void configureSSLServerSocket(SSLServerSocket sslServerSocket) {
             LOG.debug("Using Java8-optimized cipher suites for Java version {}", javaVersion);
             return DEFAULT_CIPHERS_JAVA8;
         }
    +
    +    /**
    +     * 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();
    +        String keyStoreLocation = config.getProperty(sslKeystoreLocationProperty);
    +        if (keyStoreLocation != null && !keyStoreLocation.isEmpty()) {
    +            final Path filePath = Paths.get(keyStoreLocation).toAbsolutePath();
    +            Path parentPath = filePath.getParent();
    +            if (parentPath == null) {
    +                throw new IOException(
    +                        "Key store path does not have a parent: " + filePath);
    +            }
    +            FileChangeWatcher newKeyStoreFileWatcher = new FileChangeWatcher(
    +                    parentPath,
    +                    watchEvent -> {
    +                        handleWatchEvent(filePath, watchEvent);
    +                    });
    +            // stop old watcher if there is one
    +            if (keyStoreFileWatcher != null) {
    +                keyStoreFileWatcher.stop();
    +            }
    +            keyStoreFileWatcher = newKeyStoreFileWatcher;
    +            keyStoreFileWatcher.start();
    +        }
    +        String trustStoreLocation = config.getProperty(sslTruststoreLocationProperty);
    +        if (trustStoreLocation != null && !trustStoreLocation.isEmpty()) {
    +            final Path filePath = Paths.get(trustStoreLocation).toAbsolutePath();
    +            Path parentPath = filePath.getParent();
    +            if (parentPath == null) {
    +                throw new IOException(
    +                        "Trust store path does not have a parent: " + filePath);
    +            }
    +            FileChangeWatcher newTrustStoreFileWatcher = new FileChangeWatcher(
    +                    parentPath,
    +                    watchEvent -> {
    +                        handleWatchEvent(filePath, watchEvent);
    +                    });
    +            // 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.
    +     */
    +    public void disableCertFileReloading() {
    +        if (keyStoreFileWatcher != null) {
    +            keyStoreFileWatcher.stop();
    +            keyStoreFileWatcher = null;
    +        }
    +        if (trustStoreFileWatcher != null) {
    +            trustStoreFileWatcher.stop();
    +            trustStoreFileWatcher = null;
    +        }
    +    }
    +
    +    // Finalizer guardian object, see Effective Java item 7
    +    // TODO: finalize() is deprecated starting with Java 10. This needs to be
    +    // replaced with an explicit shutdown call.
    +    @SuppressWarnings("unused")
    +    private final Object finalizerGuardian = new Object() {
    --- End diff --
    
    I'm worried about forgetting to call `close()` and leaking the background threads. X509Util is created in other places besides QuorumPeer. But I'll see what I can do about it, we only need to call close() if we called `enableCertFileReloading()` so it might be ok ...


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

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


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r239593424
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java ---
    @@ -446,4 +458,119 @@ private void configureSSLServerSocket(SSLServerSocket sslServerSocket) {
             LOG.debug("Using Java8-optimized cipher suites for Java version {}", javaVersion);
             return DEFAULT_CIPHERS_JAVA8;
         }
    +
    +    /**
    +     * 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();
    +        String keyStoreLocation = config.getProperty(sslKeystoreLocationProperty);
    +        if (keyStoreLocation != null && !keyStoreLocation.isEmpty()) {
    +            final Path filePath = Paths.get(keyStoreLocation).toAbsolutePath();
    +            Path parentPath = filePath.getParent();
    +            if (parentPath == null) {
    +                throw new IOException(
    +                        "Key store path does not have a parent: " + filePath);
    +            }
    +            FileChangeWatcher newKeyStoreFileWatcher = new FileChangeWatcher(
    +                    parentPath,
    +                    watchEvent -> {
    +                        handleWatchEvent(filePath, watchEvent);
    +                    });
    +            // stop old watcher if there is one
    +            if (keyStoreFileWatcher != null) {
    +                keyStoreFileWatcher.stop();
    +            }
    +            keyStoreFileWatcher = newKeyStoreFileWatcher;
    +            keyStoreFileWatcher.start();
    +        }
    +        String trustStoreLocation = config.getProperty(sslTruststoreLocationProperty);
    +        if (trustStoreLocation != null && !trustStoreLocation.isEmpty()) {
    +            final Path filePath = Paths.get(trustStoreLocation).toAbsolutePath();
    +            Path parentPath = filePath.getParent();
    +            if (parentPath == null) {
    +                throw new IOException(
    +                        "Trust store path does not have a parent: " + filePath);
    +            }
    +            FileChangeWatcher newTrustStoreFileWatcher = new FileChangeWatcher(
    +                    parentPath,
    +                    watchEvent -> {
    +                        handleWatchEvent(filePath, watchEvent);
    +                    });
    +            // 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.
    +     */
    +    public void disableCertFileReloading() {
    --- End diff --
    
    Will do.


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r239614732
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java ---
    @@ -446,4 +458,119 @@ private void configureSSLServerSocket(SSLServerSocket sslServerSocket) {
             LOG.debug("Using Java8-optimized cipher suites for Java version {}", javaVersion);
             return DEFAULT_CIPHERS_JAVA8;
         }
    +
    +    /**
    +     * 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();
    +        String keyStoreLocation = config.getProperty(sslKeystoreLocationProperty);
    +        if (keyStoreLocation != null && !keyStoreLocation.isEmpty()) {
    +            final Path filePath = Paths.get(keyStoreLocation).toAbsolutePath();
    +            Path parentPath = filePath.getParent();
    +            if (parentPath == null) {
    +                throw new IOException(
    +                        "Key store path does not have a parent: " + filePath);
    +            }
    +            FileChangeWatcher newKeyStoreFileWatcher = new FileChangeWatcher(
    +                    parentPath,
    +                    watchEvent -> {
    +                        handleWatchEvent(filePath, watchEvent);
    +                    });
    +            // stop old watcher if there is one
    +            if (keyStoreFileWatcher != null) {
    +                keyStoreFileWatcher.stop();
    +            }
    +            keyStoreFileWatcher = newKeyStoreFileWatcher;
    +            keyStoreFileWatcher.start();
    +        }
    +        String trustStoreLocation = config.getProperty(sslTruststoreLocationProperty);
    +        if (trustStoreLocation != null && !trustStoreLocation.isEmpty()) {
    +            final Path filePath = Paths.get(trustStoreLocation).toAbsolutePath();
    +            Path parentPath = filePath.getParent();
    +            if (parentPath == null) {
    +                throw new IOException(
    +                        "Trust store path does not have a parent: " + filePath);
    +            }
    +            FileChangeWatcher newTrustStoreFileWatcher = new FileChangeWatcher(
    +                    parentPath,
    +                    watchEvent -> {
    +                        handleWatchEvent(filePath, watchEvent);
    +                    });
    +            // 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.
    +     */
    +    public void disableCertFileReloading() {
    +        if (keyStoreFileWatcher != null) {
    +            keyStoreFileWatcher.stop();
    +            keyStoreFileWatcher = null;
    +        }
    +        if (trustStoreFileWatcher != null) {
    +            trustStoreFileWatcher.stop();
    +            trustStoreFileWatcher = null;
    +        }
    +    }
    +
    +    // Finalizer guardian object, see Effective Java item 7
    +    // TODO: finalize() is deprecated starting with Java 10. This needs to be
    +    // replaced with an explicit shutdown call.
    +    @SuppressWarnings("unused")
    +    private final Object finalizerGuardian = new Object() {
    --- End diff --
    
    I got your point, but in my understanding this is not the recommended way to do destructor-like behaviour in Java. The article advice on using the guardian pattern only for a safety net besides the `close()` method, logging a warning message if the class has not been properly closed by the user. This could be a valid case for us too, but given that this is purely our codebase, not a public API, I think it should be preferrable to expect that `X509Util` is closed properly everywhere.


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230560912
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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);
    +    }
    +
    +    public synchronized State getState() {
    --- End diff --
    
    And here is relevant a quote from _Java Concurrency in Practice_ by Brian Goetz et. al:
    
    > Read-write locks are a performance optimization designed to allow greater concurrency in certain situations. In practice, read-write locks can improve performance for frequently accessed read-mostly data structures on multiprocessor systems; under other conditions they perform slightly worse than exclusive locks due to their greater complexity. Whether they are an improvement in any given situation is best determined via profiling"


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230572268
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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);
    +    }
    +
    +    public synchronized State getState() {
    +        return state;
    +    }
    +
    +    private synchronized void setState(State newState) {
    +        state = newState;
    +        this.notifyAll();
    +    }
    +
    +    /**
    +     * Tells the background thread to start. Does not wait for it to be running.
    +     * Calling this method more than once has no effect.
    +     */
    +    public synchronized void start() {
    +        if (state != State.NEW) {
    +            return;
    +        }
    +        setState(State.STARTING);
    +        this.watcherThread.start();
    +    }
    +
    +    /**
    +     * Tells the background thread to stop. Does not wait for it to exit.
    +     */
    +    public synchronized void stop() {
    +        if (state == State.STARTING || state == State.RUNNING) {
    +            setState(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");
    +                synchronized (FileChangeWatcher.this) {
    --- End diff --
    
    My intention was, checking and state transition logic can be moved to setState as It is the entry point to change, state transition needs to be managed here.
    
    The below can be moved to setState
    
    if (FileChangeWatcher.this.getState() == FileChangeWatcher.State.STARTING) {
     state = FileChangeWatcher.State.RUNNING;
    }
    
    we can simply check state is RUNNING or not to proceed further in the run method. 
    
    
    



---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230574754
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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);
    +    }
    +
    +    public synchronized State getState() {
    +        return state;
    +    }
    +
    +    private synchronized void setState(State newState) {
    +        state = newState;
    +        this.notifyAll();
    +    }
    +
    +    /**
    +     * Tells the background thread to start. Does not wait for it to be running.
    +     * Calling this method more than once has no effect.
    +     */
    +    public synchronized void start() {
    +        if (state != State.NEW) {
    +            return;
    +        }
    +        setState(State.STARTING);
    +        this.watcherThread.start();
    +    }
    +
    +    /**
    +     * Tells the background thread to stop. Does not wait for it to exit.
    +     */
    +    public synchronized void stop() {
    +        if (state == State.STARTING || state == State.RUNNING) {
    +            setState(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");
    +                synchronized (FileChangeWatcher.this) {
    --- End diff --
    
    I don't think that logic belongs in `setState()` since it's just a simple setter method. Are you proposing I add a new method that's something like "compareAndSetState(State expected, State newState)"?


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

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


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

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

    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.
    
    Note that this is stacked on top of #678 and #679 and thus includes them. Please only consider the ZOOKEEPER-3174 commit when reviewing. Once the other PRs are merged upstream, I will rebase this so it only contains one commit.
    
    ## 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.

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

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

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

    https://github.com/apache/zookeeper/pull/680.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 #680
    
----
commit 33f7aaab6fe16122b7e1faedbb408d739bbe8a30
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T01:22:24Z

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

commit 30adde0fa951d5d99b6b33370eca9736e370a952
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T01:54:06Z

    ZOOKEEPER-3174: Quorum TLS - support reloading trust/key store

----


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230559850
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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);
    +    }
    +
    +    public synchronized State getState() {
    +        return state;
    +    }
    +
    +    private synchronized void setState(State newState) {
    --- End diff --
    
    Can we use ReentrantReadWriteLock  and get write lock here? 


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230560515
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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);
    +    }
    +
    +    public synchronized State getState() {
    +        return state;
    +    }
    +
    +    private synchronized void setState(State newState) {
    +        state = newState;
    +        this.notifyAll();
    +    }
    +
    +    /**
    +     * Tells the background thread to start. Does not wait for it to be running.
    +     * Calling this method more than once has no effect.
    +     */
    +    public synchronized void start() {
    +        if (state != State.NEW) {
    +            return;
    +        }
    +        setState(State.STARTING);
    +        this.watcherThread.start();
    +    }
    +
    +    /**
    +     * Tells the background thread to stop. Does not wait for it to exit.
    +     */
    +    public synchronized void stop() {
    +        if (state == State.STARTING || state == State.RUNNING) {
    +            setState(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");
    +                synchronized (FileChangeWatcher.this) {
    --- End diff --
    
    We have to check the state and modify it within one critical section. If I change the code to this:
    
    ```
    if (FileChangeWatcher.this.getState() == FileChangeWatcher.State.STARTING) {
      FileChangeWatcher.this.setState(FileChangeWatcher.State.RUNNING);
    }
    ```
    
    then there is a race between me checking the state and changing it. During this time, another thread could call `stop()` and change the state to `STOPPING`. Then I would overwrite `STOPPING` with `RUNNING` and suppress the `stop()` command.
    
    I should probably document this in a comment so it doesn't get refactored into incorrect code later by someone else.


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230539608
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,180 @@
    +/**
    + * 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);
    +
    +    private final WatcherThread watcherThread;
    +
    +    /**
    +     * 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);
    +        this.watcherThread = new WatcherThread(watchService, callback);
    +        this.watcherThread.setDaemon(true);
    +        this.watcherThread.start();
    --- End diff --
    
    Thanks for defining states. 


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

    https://github.com/apache/zookeeper/pull/680
  
    @eolivelli switched to lambdas, kept the finalizer in for now but added a TODO to remove it.


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

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

    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.

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

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

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

    https://github.com/apache/zookeeper/pull/680.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 #680
    
----
commit b373878e45151ea736055cd0a1cf51181a0f0e0d
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T01:54:06Z

    ZOOKEEPER-3174: Quorum TLS - support reloading trust/key store

----


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

    https://github.com/apache/zookeeper/pull/680
  
    @ivmaykov For the future, I think it's more convenient for reviewers if you submit your commits separately instead of squashing them. Especially if you provide some feedback on code review, it's hard to locate new changes if everything is in a single patch. Commit script will squash all them eventually, so it doesn't really matter in PRs.


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

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

    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.
    
    Note that this is stacked on top of #678 and #679 and thus includes them. Please only consider the ZOOKEEPER-3174 commit when reviewing. Once the other PRs are merged upstream, I will rebase this so it only contains one commit.
    
    ## 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.

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

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

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

    https://github.com/apache/zookeeper/pull/680.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 #680
    
----
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

commit d9e09dc73a42be079a6bd28b51c09635fff32e95
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T01:54:06Z

    ZOOKEEPER-3174: Quorum TLS - support reloading trust/key store

----


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230560416
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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);
    +    }
    +
    +    public synchronized State getState() {
    --- End diff --
    
    Read/Write locks are slower than exclusive locks (i.e. in Facebook's folly C++ library the locking operations on a `SharedMutex` are about 40x slower than the locking operations on a regular `Mutex` if I remember correctly. I suspect the difference in performance is of a similar order between Java's object monitor lock and `ReentrantReadWriteLock`). So, I would not switch to a R/W lock unless profiling data shows lots of lock contention in read paths. We also can't use `wait()/notify()/notifyAll()` for synchronization in tests unless we use the object monitor lock.


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r228719526
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java ---
    @@ -546,4 +557,109 @@ public static StoreFileType detectStoreFileTypeFromFileExtension(File filename)
             }
             throw new IOException("Unable to auto-detect store file type from file name: " + filename);
         }
    +
    +    /**
    +     * 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 {
    +        ZKConfig config = new ZKConfig();
    +        String keyStoreLocation = config.getProperty(sslKeystoreLocationProperty);
    +        if (keyStoreLocation != null && !keyStoreLocation.isEmpty()) {
    +            final Path filePath = Paths.get(keyStoreLocation).toAbsolutePath();
    +            FileChangeWatcher newKeyStoreFileWatcher = new FileChangeWatcher(
    +                    filePath.getParent(),
    +                    new Consumer<WatchEvent<?>>() {
    +                        @Override
    +                        public void accept(WatchEvent<?> watchEvent) {
    +                            handleWatchEvent(filePath, watchEvent);
    +                        }
    +                    });
    +            // stop old watcher if there is one
    +            if (keyStoreFileWatcher != null) {
    +                keyStoreFileWatcher.stop();
    +                keyStoreFileWatcher = newKeyStoreFileWatcher;
    --- End diff --
    
    oops, this has a bug that I introduced while refactoring. Will fix.


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230572615
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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);
    +    }
    +
    +    public synchronized State getState() {
    --- End diff --
    
    Thanks for the info. ReentrantReadWriteLock may not be relevant in this use case. It will give good results when there are more readers, example reading data from Caching layer etc..


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

    https://github.com/apache/zookeeper/pull/680
  
    @anmolnar I just stumbled across the FileMonitor class, which has similar functionality to my FileChangeWatcher. The main difference is that it polls for changes and sleeps the thread in between, rather than using WatchService. It also ignores mtime changes if the file contents have not changed. Maybe I should use FileMonitor and remove the FileChangeWatcher class?


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230523105
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,180 @@
    +/**
    + * 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);
    +
    +    private final WatcherThread watcherThread;
    +
    +    /**
    +     * 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);
    +        this.watcherThread = new WatcherThread(watchService, callback);
    +        this.watcherThread.setDaemon(true);
    +        this.watcherThread.start();
    +    }
    +
    +    /**
    +     * Waits for the background thread to enter the main loop before returning. This method exists mostly to make
    +     * the unit tests simpler, which is why it is package private.
    +     *
    +     * @throws InterruptedException if this thread is interrupted while waiting for the background thread to start.
    +     */
    +    void waitForBackgroundThreadToStart() throws InterruptedException {
    --- End diff --
    
    I'll see what I can do about this.


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r239641391
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java ---
    @@ -446,4 +458,119 @@ private void configureSSLServerSocket(SSLServerSocket sslServerSocket) {
             LOG.debug("Using Java8-optimized cipher suites for Java version {}", javaVersion);
             return DEFAULT_CIPHERS_JAVA8;
         }
    +
    +    /**
    +     * 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();
    +        String keyStoreLocation = config.getProperty(sslKeystoreLocationProperty);
    +        if (keyStoreLocation != null && !keyStoreLocation.isEmpty()) {
    +            final Path filePath = Paths.get(keyStoreLocation).toAbsolutePath();
    +            Path parentPath = filePath.getParent();
    +            if (parentPath == null) {
    +                throw new IOException(
    +                        "Key store path does not have a parent: " + filePath);
    +            }
    +            FileChangeWatcher newKeyStoreFileWatcher = new FileChangeWatcher(
    +                    parentPath,
    +                    watchEvent -> {
    +                        handleWatchEvent(filePath, watchEvent);
    +                    });
    +            // stop old watcher if there is one
    +            if (keyStoreFileWatcher != null) {
    +                keyStoreFileWatcher.stop();
    +            }
    +            keyStoreFileWatcher = newKeyStoreFileWatcher;
    +            keyStoreFileWatcher.start();
    +        }
    +        String trustStoreLocation = config.getProperty(sslTruststoreLocationProperty);
    +        if (trustStoreLocation != null && !trustStoreLocation.isEmpty()) {
    +            final Path filePath = Paths.get(trustStoreLocation).toAbsolutePath();
    +            Path parentPath = filePath.getParent();
    +            if (parentPath == null) {
    +                throw new IOException(
    +                        "Trust store path does not have a parent: " + filePath);
    +            }
    +            FileChangeWatcher newTrustStoreFileWatcher = new FileChangeWatcher(
    +                    parentPath,
    +                    watchEvent -> {
    +                        handleWatchEvent(filePath, watchEvent);
    +                    });
    +            // 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.
    +     */
    +    public void disableCertFileReloading() {
    +        if (keyStoreFileWatcher != null) {
    +            keyStoreFileWatcher.stop();
    +            keyStoreFileWatcher = null;
    +        }
    +        if (trustStoreFileWatcher != null) {
    +            trustStoreFileWatcher.stop();
    +            trustStoreFileWatcher = null;
    +        }
    +    }
    +
    +    // Finalizer guardian object, see Effective Java item 7
    +    // TODO: finalize() is deprecated starting with Java 10. This needs to be
    +    // replaced with an explicit shutdown call.
    +    @SuppressWarnings("unused")
    +    private final Object finalizerGuardian = new Object() {
    --- End diff --
    
    Done.


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

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

    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.

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

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

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

    https://github.com/apache/zookeeper/pull/680.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 #680
    
----
commit cc72c083c0b70409d78da11507ca5e80e726bb69
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T01:54:06Z

    ZOOKEEPER-3174: Quorum TLS - support reloading trust/key store

----


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

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

    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.

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

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

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

    https://github.com/apache/zookeeper/pull/680.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 #680
    
----
commit 6c492bbad6021c12f3e1bc0b57a3b35bab771696
Author: Ilya Maykov <il...@...>
Date:   2018-10-25T01:54:06Z

    ZOOKEEPER-3174: Quorum TLS - support reloading trust/key store

----


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230523204
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,180 @@
    +/**
    + * 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);
    +
    +    private final WatcherThread watcherThread;
    +
    +    /**
    +     * 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);
    +        this.watcherThread = new WatcherThread(watchService, callback);
    +        this.watcherThread.setDaemon(true);
    +        this.watcherThread.start();
    +    }
    +
    +    /**
    +     * Waits for the background thread to enter the main loop before returning. This method exists mostly to make
    +     * the unit tests simpler, which is why it is package private.
    +     *
    +     * @throws InterruptedException if this thread is interrupted while waiting for the background thread to start.
    +     */
    +    void waitForBackgroundThreadToStart() throws InterruptedException {
    +        synchronized (watcherThread) {
    +            while (!watcherThread.started) {
    +                watcherThread.wait();
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Tells the background thread to stop. Does not wait for it to exit.
    +     */
    +    public void stop() {
    +        watcherThread.shouldStop = true;
    +        watcherThread.interrupt();
    +    }
    +
    +    /**
    +     * Tells the background thread to stop and waits for it to exit. Only used by unit tests, which is why it is package
    +     * private.
    +     */
    +    void stopAndJoinBackgroundThread() throws InterruptedException {
    +        stop();
    +        watcherThread.join();
    +    }
    +
    +    /**
    +     * Inner class that implements the watcher thread logic.
    +     */
    +    private static class WatcherThread extends ZooKeeperThread {
    +        private static final String THREAD_NAME = "FileChangeWatcher";
    +
    +        volatile boolean shouldStop;
    +        volatile boolean started;
    +        final WatchService watchService;
    +        final Consumer<WatchEvent<?>> callback;
    +
    +        WatcherThread(WatchService watchService, Consumer<WatchEvent<?>> callback) {
    +            super(THREAD_NAME);
    +            this.shouldStop = this.started = false;
    +            this.watchService = watchService;
    +            this.callback = callback;
    +        }
    +
    +        @Override
    +        public void run() {
    +            LOG.info(getName() + " thread started");
    +            synchronized (this) {
    --- End diff --
    
    I don't think we can, because we have to synchronize between the background thread that polls the watch service and main thread.


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r239593458
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java ---
    @@ -446,4 +458,119 @@ private void configureSSLServerSocket(SSLServerSocket sslServerSocket) {
             LOG.debug("Using Java8-optimized cipher suites for Java version {}", javaVersion);
             return DEFAULT_CIPHERS_JAVA8;
         }
    +
    +    /**
    +     * 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();
    +        String keyStoreLocation = config.getProperty(sslKeystoreLocationProperty);
    +        if (keyStoreLocation != null && !keyStoreLocation.isEmpty()) {
    +            final Path filePath = Paths.get(keyStoreLocation).toAbsolutePath();
    +            Path parentPath = filePath.getParent();
    +            if (parentPath == null) {
    +                throw new IOException(
    +                        "Key store path does not have a parent: " + filePath);
    +            }
    +            FileChangeWatcher newKeyStoreFileWatcher = new FileChangeWatcher(
    +                    parentPath,
    +                    watchEvent -> {
    +                        handleWatchEvent(filePath, watchEvent);
    +                    });
    +            // stop old watcher if there is one
    +            if (keyStoreFileWatcher != null) {
    +                keyStoreFileWatcher.stop();
    +            }
    +            keyStoreFileWatcher = newKeyStoreFileWatcher;
    +            keyStoreFileWatcher.start();
    +        }
    +        String trustStoreLocation = config.getProperty(sslTruststoreLocationProperty);
    +        if (trustStoreLocation != null && !trustStoreLocation.isEmpty()) {
    --- End diff --
    
    Will do.


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230538126
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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);
    +    }
    +
    +    public synchronized State getState() {
    +        return state;
    +    }
    +
    +    private synchronized void setState(State newState) {
    +        state = newState;
    +        this.notifyAll();
    +    }
    +
    +    /**
    +     * Tells the background thread to start. Does not wait for it to be running.
    +     * Calling this method more than once has no effect.
    +     */
    +    public synchronized void start() {
    +        if (state != State.NEW) {
    +            return;
    +        }
    +        setState(State.STARTING);
    +        this.watcherThread.start();
    +    }
    +
    +    /**
    +     * Tells the background thread to stop. Does not wait for it to exit.
    +     */
    +    public synchronized void stop() {
    +        if (state == State.STARTING || state == State.RUNNING) {
    +            setState(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");
    +                synchronized (FileChangeWatcher.this) {
    --- End diff --
    
    the setState is already synchronized. Can we move this logic to setState as it is more  of state management?N o further locking is required here We can simply check the state after setState, proceed if the state is running


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230539129
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,180 @@
    +/**
    + * 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);
    +
    +    private final WatcherThread watcherThread;
    +
    +    /**
    +     * 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);
    +        this.watcherThread = new WatcherThread(watchService, callback);
    +        this.watcherThread.setDaemon(true);
    +        this.watcherThread.start();
    +    }
    +
    +    /**
    +     * Waits for the background thread to enter the main loop before returning. This method exists mostly to make
    +     * the unit tests simpler, which is why it is package private.
    +     *
    +     * @throws InterruptedException if this thread is interrupted while waiting for the background thread to start.
    +     */
    +    void waitForBackgroundThreadToStart() throws InterruptedException {
    +        synchronized (watcherThread) {
    +            while (!watcherThread.started) {
    +                watcherThread.wait();
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Tells the background thread to stop. Does not wait for it to exit.
    +     */
    +    public void stop() {
    +        watcherThread.shouldStop = true;
    +        watcherThread.interrupt();
    +    }
    +
    +    /**
    +     * Tells the background thread to stop and waits for it to exit. Only used by unit tests, which is why it is package
    +     * private.
    +     */
    +    void stopAndJoinBackgroundThread() throws InterruptedException {
    +        stop();
    +        watcherThread.join();
    +    }
    +
    +    /**
    +     * Inner class that implements the watcher thread logic.
    +     */
    +    private static class WatcherThread extends ZooKeeperThread {
    +        private static final String THREAD_NAME = "FileChangeWatcher";
    +
    +        volatile boolean shouldStop;
    +        volatile boolean started;
    +        final WatchService watchService;
    +        final Consumer<WatchEvent<?>> callback;
    +
    +        WatcherThread(WatchService watchService, Consumer<WatchEvent<?>> callback) {
    +            super(THREAD_NAME);
    +            this.shouldStop = this.started = false;
    +            this.watchService = watchService;
    +            this.callback = callback;
    +        }
    +
    +        @Override
    +        public void run() {
    +            LOG.info(getName() + " thread started");
    +            synchronized (this) {
    --- End diff --
    
    the setState is already synchronized. Can we move this logic to setState as it is more of state management?N o further locking is required here We can simply check the state after setState, proceed if the state is running


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r239130075
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/X509Util.java ---
    @@ -446,4 +458,119 @@ private void configureSSLServerSocket(SSLServerSocket sslServerSocket) {
             LOG.debug("Using Java8-optimized cipher suites for Java version {}", javaVersion);
             return DEFAULT_CIPHERS_JAVA8;
         }
    +
    +    /**
    +     * 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();
    +        String keyStoreLocation = config.getProperty(sslKeystoreLocationProperty);
    +        if (keyStoreLocation != null && !keyStoreLocation.isEmpty()) {
    +            final Path filePath = Paths.get(keyStoreLocation).toAbsolutePath();
    +            Path parentPath = filePath.getParent();
    +            if (parentPath == null) {
    +                throw new IOException(
    +                        "Key store path does not have a parent: " + filePath);
    +            }
    +            FileChangeWatcher newKeyStoreFileWatcher = new FileChangeWatcher(
    +                    parentPath,
    +                    watchEvent -> {
    +                        handleWatchEvent(filePath, watchEvent);
    +                    });
    +            // stop old watcher if there is one
    +            if (keyStoreFileWatcher != null) {
    +                keyStoreFileWatcher.stop();
    +            }
    +            keyStoreFileWatcher = newKeyStoreFileWatcher;
    +            keyStoreFileWatcher.start();
    +        }
    +        String trustStoreLocation = config.getProperty(sslTruststoreLocationProperty);
    +        if (trustStoreLocation != null && !trustStoreLocation.isEmpty()) {
    +            final Path filePath = Paths.get(trustStoreLocation).toAbsolutePath();
    +            Path parentPath = filePath.getParent();
    +            if (parentPath == null) {
    +                throw new IOException(
    +                        "Trust store path does not have a parent: " + filePath);
    +            }
    +            FileChangeWatcher newTrustStoreFileWatcher = new FileChangeWatcher(
    +                    parentPath,
    +                    watchEvent -> {
    +                        handleWatchEvent(filePath, watchEvent);
    +                    });
    +            // 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.
    +     */
    +    public void disableCertFileReloading() {
    --- End diff --
    
    This method can be private.


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230574766
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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);
    +    }
    +
    +    public synchronized State getState() {
    --- End diff --
    
    I agree, there are use cases where a R/W lock is a great choice, but I don't think this is one of them.


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230463216
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,180 @@
    +/**
    + * 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);
    +
    +    private final WatcherThread watcherThread;
    +
    +    /**
    +     * 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);
    +        this.watcherThread = new WatcherThread(watchService, callback);
    +        this.watcherThread.setDaemon(true);
    +        this.watcherThread.start();
    +    }
    +
    +    /**
    +     * Waits for the background thread to enter the main loop before returning. This method exists mostly to make
    +     * the unit tests simpler, which is why it is package private.
    +     *
    +     * @throws InterruptedException if this thread is interrupted while waiting for the background thread to start.
    +     */
    +    void waitForBackgroundThreadToStart() throws InterruptedException {
    +        synchronized (watcherThread) {
    +            while (!watcherThread.started) {
    +                watcherThread.wait();
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Tells the background thread to stop. Does not wait for it to exit.
    +     */
    +    public void stop() {
    +        watcherThread.shouldStop = true;
    +        watcherThread.interrupt();
    +    }
    +
    +    /**
    +     * Tells the background thread to stop and waits for it to exit. Only used by unit tests, which is why it is package
    +     * private.
    +     */
    +    void stopAndJoinBackgroundThread() throws InterruptedException {
    --- End diff --
    
    same like above comment


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

    https://github.com/apache/zookeeper/pull/680
  
    @anmolnar is there anything blocking this from being merged at this point?


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r239615050
  
    --- Diff: 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:
    + * <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 {
    --- End diff --
    
    Leave as it is.


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

    https://github.com/apache/zookeeper/pull/680
  
    @anmolnar removed finalizer, use explicit close()


---

[GitHub] zookeeper issue #680: ZOOKEEPER-3174: Quorum TLS - support reloading trust/k...

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

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



---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230463114
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,180 @@
    +/**
    + * 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);
    +
    +    private final WatcherThread watcherThread;
    +
    +    /**
    +     * 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);
    +        this.watcherThread = new WatcherThread(watchService, callback);
    +        this.watcherThread.setDaemon(true);
    +        this.watcherThread.start();
    +    }
    +
    +    /**
    +     * Waits for the background thread to enter the main loop before returning. This method exists mostly to make
    +     * the unit tests simpler, which is why it is package private.
    +     *
    +     * @throws InterruptedException if this thread is interrupted while waiting for the background thread to start.
    +     */
    +    void waitForBackgroundThreadToStart() throws InterruptedException {
    --- End diff --
    
    not able to mock or manage the file watcher generally smell some design improvement.The same logic implemented in test case if we expose the states of file watcher, locking also not required if we expose the state of  file watcher.It is good to avoid adding code for testing purpose


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230522968
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,180 @@
    +/**
    + * 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);
    +
    +    private final WatcherThread watcherThread;
    +
    +    /**
    +     * 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);
    +        this.watcherThread = new WatcherThread(watchService, callback);
    +        this.watcherThread.setDaemon(true);
    +        this.watcherThread.start();
    --- End diff --
    
    I think we might still need locking because there is a background thread that's polling the WatchService for events. And starting/stopping the watcher requires starting/stopping the background thread, so some synchronization between threads is needed. Do you still think it makes sense to define life cycle stages if we still need locking?


---

[GitHub] zookeeper pull request #680: ZOOKEEPER-3174: Quorum TLS - support reloading ...

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

    https://github.com/apache/zookeeper/pull/680#discussion_r230463665
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java ---
    @@ -0,0 +1,180 @@
    +/**
    + * 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);
    +
    +    private final WatcherThread watcherThread;
    +
    +    /**
    +     * 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);
    +        this.watcherThread = new WatcherThread(watchService, callback);
    +        this.watcherThread.setDaemon(true);
    +        this.watcherThread.start();
    +    }
    +
    +    /**
    +     * Waits for the background thread to enter the main loop before returning. This method exists mostly to make
    +     * the unit tests simpler, which is why it is package private.
    +     *
    +     * @throws InterruptedException if this thread is interrupted while waiting for the background thread to start.
    +     */
    +    void waitForBackgroundThreadToStart() throws InterruptedException {
    +        synchronized (watcherThread) {
    +            while (!watcherThread.started) {
    +                watcherThread.wait();
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Tells the background thread to stop. Does not wait for it to exit.
    +     */
    +    public void stop() {
    +        watcherThread.shouldStop = true;
    +        watcherThread.interrupt();
    +    }
    +
    +    /**
    +     * Tells the background thread to stop and waits for it to exit. Only used by unit tests, which is why it is package
    +     * private.
    +     */
    +    void stopAndJoinBackgroundThread() throws InterruptedException {
    +        stop();
    +        watcherThread.join();
    +    }
    +
    +    /**
    +     * Inner class that implements the watcher thread logic.
    +     */
    +    private static class WatcherThread extends ZooKeeperThread {
    +        private static final String THREAD_NAME = "FileChangeWatcher";
    +
    +        volatile boolean shouldStop;
    +        volatile boolean started;
    +        final WatchService watchService;
    +        final Consumer<WatchEvent<?>> callback;
    +
    +        WatcherThread(WatchService watchService, Consumer<WatchEvent<?>> callback) {
    +            super(THREAD_NAME);
    +            this.shouldStop = this.started = false;
    +            this.watchService = watchService;
    +            this.callback = callback;
    +        }
    +
    +        @Override
    +        public void run() {
    +            LOG.info(getName() + " thread started");
    +            synchronized (this) {
    --- End diff --
    
    We can eliminate the locking if we expose the  states


---