-
Notifications
You must be signed in to change notification settings - Fork 9.2k
HDFS-11114. Support for running async disk checks in DataNode. #153
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
Closed
Closed
Changes from 1 commit
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
62 changes: 62 additions & 0 deletions
62
...adoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/AsyncChecker.java
This file contains hidden or 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,62 @@ | ||
| /** | ||
| * 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 | ||
| * <p> | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * <p> | ||
| * 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.hdfs.server.datanode.checker; | ||
|
|
||
| import com.google.common.util.concurrent.ListenableFuture; | ||
| import org.apache.hadoop.classification.InterfaceAudience; | ||
| import org.apache.hadoop.classification.InterfaceStability; | ||
|
|
||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.TimeUnit; | ||
|
|
||
| /** | ||
| * A class that can be used to schedule an asynchronous check on a given | ||
| * {@link Checkable}. If the check is successfully scheduled then a | ||
| * {@link ListenableFuture} is returned. | ||
| * | ||
| */ | ||
| @InterfaceAudience.Private | ||
| @InterfaceStability.Unstable | ||
| public interface AsyncChecker<K, V> { | ||
|
|
||
| /** | ||
| * Schedule an asynchronous check for the given object. | ||
| * | ||
| * @param target object to be checked. | ||
| * | ||
| * @param context the interpretation of the context depends on the | ||
| * target. | ||
| * | ||
| * @return returns a {@link ListenableFuture} that can be used to | ||
| * retrieve the result of the asynchronous check. | ||
| */ | ||
| ListenableFuture<V> schedule(Checkable<K, V> target, K context); | ||
|
|
||
| /** | ||
| * Cancel all executing checks and wait for them to complete. | ||
| * First attempts a graceful cancellation, then cancels forcefully. | ||
| * Waits for the supplied timeout after both attempts. | ||
| * | ||
| * See {@link ExecutorService#awaitTermination} for a description of | ||
| * the parameters. | ||
| * | ||
| * @throws InterruptedException | ||
| */ | ||
| void join(long timeout, TimeUnit timeUnit) throws InterruptedException; | ||
| } | ||
49 changes: 49 additions & 0 deletions
49
...t/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/Checkable.java
This file contains hidden or 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,49 @@ | ||
| /** | ||
| * 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.hdfs.server.datanode.checker; | ||
|
|
||
| import org.apache.hadoop.classification.InterfaceAudience; | ||
| import org.apache.hadoop.classification.InterfaceStability; | ||
|
|
||
|
|
||
| /** | ||
| * A Checkable is an object whose health can be probed by invoking its | ||
| * {@link #check} method. | ||
| * | ||
| * e.g. a {@link Checkable} instance may represent a single hardware | ||
| * resource. | ||
| */ | ||
| @InterfaceAudience.Private | ||
| @InterfaceStability.Unstable | ||
| public interface Checkable<K, V> { | ||
|
|
||
| /** | ||
| * Query the health of this object. This method may hang | ||
| * indefinitely depending on the status of the target resource. | ||
| * | ||
| * @param context for the probe operation. May be null depending | ||
| * on the implementation. | ||
| * | ||
| * @return result of the check operation. | ||
| * | ||
| * @throws Exception encountered during the check operation. An | ||
| * exception indicates that the check failed. | ||
| */ | ||
| V check(K context) throws Exception; | ||
| } |
224 changes: 224 additions & 0 deletions
224
...s/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/ThrottledAsyncChecker.java
This file contains hidden or 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,224 @@ | ||
| /** | ||
| * 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 | ||
| * <p> | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * <p> | ||
| * 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.hdfs.server.datanode.checker; | ||
|
|
||
| import com.google.common.util.concurrent.FutureCallback; | ||
| import com.google.common.util.concurrent.Futures; | ||
| import com.google.common.util.concurrent.ListenableFuture; | ||
| import com.google.common.util.concurrent.ListeningExecutorService; | ||
| import com.google.common.util.concurrent.MoreExecutors; | ||
| import org.apache.hadoop.classification.InterfaceAudience; | ||
| import org.apache.hadoop.classification.InterfaceStability; | ||
| import org.apache.hadoop.util.Timer; | ||
|
|
||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import javax.annotation.Nonnull; | ||
| import javax.annotation.Nullable; | ||
| import java.util.HashMap; | ||
| import java.util.Map; | ||
| import java.util.WeakHashMap; | ||
| import java.util.concurrent.Callable; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.TimeUnit; | ||
|
|
||
| /** | ||
| * An implementation of {@link AsyncChecker} that skips checking recently | ||
| * checked objects. It will enforce at least {@link minMsBetweenChecks} | ||
| * milliseconds between two successive checks of any one object. | ||
| * | ||
| * It is assumed that the total number of Checkable objects in the system | ||
| * is small, (not more than a few dozen) since the checker uses O(Checkables) | ||
| * storage and also potentially O(Checkables) threads. | ||
| * | ||
| * {@link minMsBetweenChecks} should be configured reasonably | ||
| * by the caller to avoid spinning up too many threads frequently. | ||
| */ | ||
| @InterfaceAudience.Private | ||
| @InterfaceStability.Unstable | ||
| public class ThrottledAsyncChecker<K, V> implements AsyncChecker<K, V> { | ||
| public static final Logger LOG = | ||
| LoggerFactory.getLogger(ThrottledAsyncChecker.class); | ||
|
|
||
| private final Timer timer; | ||
|
|
||
| /** | ||
| * The ExecutorService used to schedule asynchronous checks. | ||
| */ | ||
| private final ListeningExecutorService executorService; | ||
|
|
||
| /** | ||
| * The minimum gap in milliseconds between two successive checks | ||
| * of the same object. This is the throttle. | ||
| */ | ||
| private final long minMsBetweenChecks; | ||
|
|
||
| /** | ||
| * Map of checks that are currently in progress. Protected by the object | ||
| * lock. | ||
| */ | ||
| private final Map<Checkable, ListenableFuture<V>> checksInProgress; | ||
|
|
||
| /** | ||
| * Maps Checkable objects to a future that can be used to retrieve | ||
| * the results of the operation. | ||
| * Protected by the object lock. | ||
| */ | ||
| private final Map<Checkable, LastCheckResult<V>> completedChecks; | ||
|
|
||
| ThrottledAsyncChecker(final Timer timer, | ||
| final long minMsBetweenChecks, | ||
| final ExecutorService executorService) { | ||
| this.timer = timer; | ||
| this.minMsBetweenChecks = minMsBetweenChecks; | ||
| this.executorService = MoreExecutors.listeningDecorator(executorService); | ||
| this.checksInProgress = new HashMap<>(); | ||
| this.completedChecks = new WeakHashMap<>(); | ||
| } | ||
|
|
||
| /** | ||
| * See {@link AsyncChecker#schedule} | ||
| * | ||
| * If the object has been checked recently then the check will | ||
| * be skipped. Multiple concurrent checks for the same object | ||
| * will receive the same Future. | ||
| */ | ||
| @Override | ||
| public synchronized ListenableFuture<V> schedule( | ||
| final Checkable<K, V> target, | ||
| final K context) { | ||
| LOG.debug("Scheduling a check of {}", target); | ||
|
|
||
| if (checksInProgress.containsKey(target)) { | ||
| return checksInProgress.get(target); | ||
| } | ||
|
|
||
| if (completedChecks.containsKey(target)) { | ||
| final LastCheckResult<V> result = completedChecks.get(target); | ||
| final long msSinceLastCheck = timer.monotonicNow() - result.completedAt; | ||
| if (msSinceLastCheck < minMsBetweenChecks) { | ||
| LOG.debug("Skipped checking {}. Time since last check {}ms " + | ||
| "is less than the min gap {}ms.", | ||
| target, msSinceLastCheck, minMsBetweenChecks); | ||
| return result.result != null ? | ||
| Futures.immediateFuture(result.result) : | ||
| Futures.immediateFailedFuture(result.exception); | ||
| } | ||
| } | ||
|
|
||
| final ListenableFuture<V> lf = executorService.submit( | ||
| new Callable<V>() { | ||
| @Override | ||
| public V call() throws Exception { | ||
| return target.check(context); | ||
| } | ||
| }); | ||
| checksInProgress.put(target, lf); | ||
| addResultCachingCallback(target, lf); | ||
| return lf; | ||
| } | ||
|
|
||
| /** | ||
| * Register a callback to cache the result of a check. | ||
| * @param target | ||
| * @param lf | ||
| */ | ||
| private void addResultCachingCallback( | ||
| Checkable<K, V> target, ListenableFuture<V> lf) { | ||
| Futures.addCallback(lf, new FutureCallback<V>() { | ||
| @Override | ||
| public void onSuccess(@Nullable V result) { | ||
| synchronized (ThrottledAsyncChecker.this) { | ||
| checksInProgress.remove(target); | ||
| completedChecks.put(target, new LastCheckResult<>( | ||
| result, timer.monotonicNow())); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void onFailure(@Nonnull Throwable t) { | ||
| synchronized (ThrottledAsyncChecker.this) { | ||
| checksInProgress.remove(target); | ||
| completedChecks.put(target, new LastCheckResult<>( | ||
| t, timer.monotonicNow())); | ||
| } | ||
| } | ||
| }); | ||
| } | ||
|
|
||
| /** | ||
| * See {@link AsyncChecker#join}. | ||
| */ | ||
| @Override | ||
| public void join(long timeout, TimeUnit timeUnit) | ||
| throws InterruptedException { | ||
| // Try orderly shutdown. | ||
| executorService.shutdown(); | ||
|
|
||
| if (!executorService.awaitTermination(timeout, timeUnit)) { | ||
| // Interrupt executing tasks and wait again. | ||
| executorService.shutdownNow(); | ||
| executorService.awaitTermination(timeout, timeUnit); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Status of running a check. It can either be a result or an | ||
| * exception, depending on whether the check completed or threw. | ||
| */ | ||
| private static final class LastCheckResult<V> { | ||
| /** | ||
| * Timestamp at which the check completed. | ||
| */ | ||
| private final long completedAt; | ||
|
|
||
| /** | ||
| * Result of running the check if it completed. null if it threw. | ||
| */ | ||
| @Nullable | ||
| private final V result; | ||
|
|
||
| /** | ||
| * Exception thrown by the check. null if it returned a result. | ||
| */ | ||
| private final Throwable exception; // null on success. | ||
|
|
||
| /** | ||
| * Initialize with a result. | ||
| * @param result | ||
| */ | ||
| private LastCheckResult(V result, long completedAt) { | ||
| this.result = result; | ||
| this.exception = null; | ||
| this.completedAt = completedAt; | ||
| } | ||
|
|
||
| /** | ||
| * Initialize with an exception. | ||
| * @param completedAt | ||
| * @param t | ||
| */ | ||
| private LastCheckResult(Throwable t, long completedAt) { | ||
| this.result = null; | ||
| this.exception = t; | ||
| this.completedAt = completedAt; | ||
| } | ||
| } | ||
| } |
26 changes: 26 additions & 0 deletions
26
...adoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/package-info.java
This file contains hidden or 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,26 @@ | ||
| /* | ||
| * 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. | ||
| */ | ||
|
|
||
| /** | ||
| * Datanode support for running disk checks. | ||
| */ | ||
| @InterfaceAudience.LimitedPrivate({"HDFS"}) | ||
| @InterfaceStability.Evolving | ||
| package org.apache.hadoop.hdfs.server.datanode.checker; | ||
| import org.apache.hadoop.classification.InterfaceAudience; | ||
| import org.apache.hadoop.classification.InterfaceStability; |
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.
Just trying to understand this a little better, From the signature and implementation this function looks more a awaitTermination in executorService. That is this function will wait for a while and cancel and task if the timeout occurs, in that case would you consider calling this await or awaitTermination. Java "join" seems to imply a wait without timeouts. Just making sure that the intended was indeed a shutdown/await pattern.
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 taking a look @anuengineer. The method covers both shutdown and awaitTermination semantics. I could call it shutdownAndAwaitTermination() to make it clearer.