-
Notifications
You must be signed in to change notification settings - Fork 3.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
HBASE-27347 Port FileWatcher from ZK to autodetect keystore/truststore changes in TLS connections #4869
Merged
Merged
HBASE-27347 Port FileWatcher from ZK to autodetect keystore/truststore changes in TLS connections #4869
Changes from all commits
Commits
Show all changes
6 commits
Select commit
Hold shift + click to select a range
34a23c1
HBASE-27347. Copied FileChangeWatcher class and tests
anmolnar 910b197
HBASE-27347. Add file change watcher for cert file reloading
anmolnar be6aaa3
HBASE-27347. Keep state of file watchers in client/server
anmolnar dd88c3b
HBASE-27347. Finish modification on test context class
anmolnar a7e4d63
HBASE-27347. Make tests parameterized, spotless apply
anmolnar 5aa518a
HBASE-27347. Address review comments
anmolnar File filter
Filter by extension
Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
251 changes: 251 additions & 0 deletions
251
hbase-common/src/main/java/org/apache/hadoop/hbase/io/FileChangeWatcher.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,251 @@ | ||
/* | ||
* 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.hadoop.hbase.io; | ||
|
||
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; | ||
import org.apache.yetus.audience.InterfaceAudience; | ||
import org.apache.zookeeper.server.ZooKeeperThread; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
/** | ||
* 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> | ||
* <p/> | ||
* This file has been copied from the Apache ZooKeeper project. | ||
* @see <a href= | ||
* "https://github.com/apache/zookeeper/blob/8148f966947d3ecf3db0b756d93c9ffa88174af9/zookeeper-server/src/main/java/org/apache/zookeeper/common/FileChangeWatcher.java">Base | ||
* revision</a> | ||
*/ | ||
@InterfaceAudience.Private | ||
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(); | ||
|
||
LOG.debug("Registering with watch service: {}", dirPath); | ||
|
||
dirPath.register(watchService, | ||
new WatchEvent.Kind<?>[] { StandardWatchEventKinds.ENTRY_CREATE, | ||
StandardWatchEventKinds.ENTRY_DELETE, StandardWatchEventKinds.ENTRY_MODIFY, | ||
StandardWatchEventKinds.OVERFLOW }); | ||
state = State.NEW; | ||
this.watcherThread = new WatcherThread(watchService, callback); | ||
this.watcherThread.setDaemon(true); | ||
} | ||
|
||
/** | ||
* Returns the current {@link FileChangeWatcher.State}. | ||
* @return the current state. | ||
*/ | ||
public synchronized State getState() { | ||
return state; | ||
} | ||
|
||
/** | ||
* Blocks until the current state becomes <code>desiredState</code>. Currently only used by tests, | ||
* thus package-private. | ||
* @param desiredState the desired state. | ||
* @throws InterruptedException if the current thread gets interrupted. | ||
*/ | ||
synchronized void waitForState(State desiredState) throws InterruptedException { | ||
while (this.state != desiredState) { | ||
this.wait(); | ||
} | ||
} | ||
|
||
/** | ||
* Sets the state to <code>newState</code>. | ||
* @param newState the new state. | ||
*/ | ||
private synchronized void setState(State newState) { | ||
state = newState; | ||
this.notifyAll(); | ||
} | ||
|
||
/** | ||
* Atomically sets the state to <code>update</code> if and only if the state is currently | ||
* <code>expected</code>. | ||
* @param expected the expected state. | ||
* @param update the new state. | ||
* @return true if the update succeeds, or false if the current state does not equal | ||
* <code>expected</code>. | ||
*/ | ||
private synchronized boolean compareAndSetState(State expected, State update) { | ||
if (state == expected) { | ||
setState(update); | ||
return true; | ||
} else { | ||
return false; | ||
} | ||
} | ||
|
||
/** | ||
* Atomically sets the state to <code>update</code> if and only if the state is currently one of | ||
* <code>expectedStates</code>. | ||
* @param expectedStates the expected states. | ||
* @param update the new state. | ||
* @return true if the update succeeds, or false if the current state does not equal any of the | ||
* <code>expectedStates</code>. | ||
*/ | ||
private synchronized boolean compareAndSetState(State[] expectedStates, State update) { | ||
for (State expected : expectedStates) { | ||
if (state == expected) { | ||
setState(update); | ||
return true; | ||
} | ||
} | ||
return false; | ||
} | ||
|
||
/** | ||
* Tells the background thread to start. Does not wait for it to be running. Calling this method | ||
* more than once has no effect. | ||
*/ | ||
public void start() { | ||
if (!compareAndSetState(State.NEW, State.STARTING)) { | ||
// If previous state was not NEW, start() has already been called. | ||
return; | ||
} | ||
this.watcherThread.start(); | ||
} | ||
|
||
/** | ||
* Tells the background thread to stop. Does not wait for it to exit. | ||
*/ | ||
public void stop() { | ||
if (compareAndSetState(new State[] { State.RUNNING, State.STARTING }, State.STOPPING)) { | ||
watcherThread.interrupt(); | ||
} | ||
} | ||
|
||
/** | ||
* Inner class that implements the watcher thread logic. | ||
*/ | ||
private class WatcherThread extends ZooKeeperThread { | ||
|
||
private static final String THREAD_NAME = "FileChangeWatcher"; | ||
|
||
final WatchService watchService; | ||
final Consumer<WatchEvent<?>> callback; | ||
|
||
WatcherThread(WatchService watchService, Consumer<WatchEvent<?>> callback) { | ||
super(THREAD_NAME); | ||
this.watchService = watchService; | ||
this.callback = callback; | ||
} | ||
|
||
@Override | ||
public void run() { | ||
try { | ||
LOG.info("{} thread started", getName()); | ||
if ( | ||
!compareAndSetState(FileChangeWatcher.State.STARTING, FileChangeWatcher.State.RUNNING) | ||
) { | ||
// stop() called shortly after start(), before | ||
// this thread started running. | ||
FileChangeWatcher.State state = FileChangeWatcher.this.getState(); | ||
if (state != FileChangeWatcher.State.STOPPING) { | ||
throw new IllegalStateException("Unexpected state: " + state); | ||
} | ||
return; | ||
} | ||
runLoop(); | ||
} catch (Exception e) { | ||
LOG.warn("Error in runLoop()", e); | ||
throw e; | ||
} finally { | ||
try { | ||
watchService.close(); | ||
} catch (IOException e) { | ||
LOG.warn("Error closing watch service", e); | ||
} | ||
LOG.info("{} thread finished", getName()); | ||
FileChangeWatcher.this.setState(FileChangeWatcher.State.STOPPED); | ||
} | ||
} | ||
|
||
private void runLoop() { | ||
while (FileChangeWatcher.this.getState() == FileChangeWatcher.State.RUNNING) { | ||
WatchKey key; | ||
try { | ||
key = watchService.take(); | ||
} catch (InterruptedException | ClosedWatchServiceException e) { | ||
LOG.debug("{} was interrupted and is shutting down...", getName()); | ||
break; | ||
} | ||
for (WatchEvent<?> event : key.pollEvents()) { | ||
LOG.debug("Got file changed event: {} with context: {}", event.kind(), event.context()); | ||
try { | ||
callback.accept(event); | ||
} catch (Throwable e) { | ||
LOG.error("Error from callback", e); | ||
} | ||
} | ||
boolean isKeyValid = key.reset(); | ||
if (!isKeyValid) { | ||
// This is likely a problem, it means that file reloading is broken, probably because the | ||
// directory we are watching was deleted or otherwise became inaccessible (unmounted, | ||
// permissions | ||
// changed, ???). | ||
// For now, we log an error and exit the watcher thread. | ||
LOG.error("Watch key no longer valid, maybe the directory is inaccessible?"); | ||
break; | ||
} | ||
} | ||
} | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is this a straight copy from Zookeeper? Couldn't we just reuse the ZK impl directly, as we already have ZK as a dependency?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Urm... I don't have a strong opinion. We were following this pattern in the entire implementation of TLS. It's probably better to stay on the safe side and avoid sideeffects of ZK non-backward compatible changes. They're quite unlikely though.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for clarifying. Not a big deal for me. And on the flip side, it would be actually a bit weird to depend on ZK for a functionality that isn't really ZK specific, so I'm ok with this copy approach.