浏览代码

HDDS-2102. HddsVolumeChecker should use java optional in place of Guava optional. Contributed by Mukul Kumar Singh. (#1416)

Mukul Kumar Singh 5 年之前
父节点
当前提交
d69b811ddd

+ 65 - 0
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/AsyncChecker.java

@@ -0,0 +1,65 @@
+/**
+ * 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.ozone.container.common.volume;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.datanode.checker.Checkable;
+
+import java.util.Optional;
+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 Optional of ListenableFuture} that can be used to
+   *         retrieve the result of the asynchronous check.
+   */
+  Optional<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 shutdownAndWait(long timeout, TimeUnit timeUnit)
+      throws InterruptedException;
+}

+ 1 - 2
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolumeChecker.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.ozone.container.common.volume;
 package org.apache.hadoop.ozone.container.common.volume;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
 import com.google.common.collect.Sets;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.Futures;
@@ -28,7 +27,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.datanode.checker.AsyncChecker;
 import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
 import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.Timer;
 import org.apache.hadoop.util.Timer;
@@ -43,6 +41,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.Set;
+import java.util.Optional;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ExecutorService;

+ 3 - 4
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/ThrottledAsyncChecker.java

@@ -18,7 +18,6 @@
 
 
 package org.apache.hadoop.ozone.container.common.volume;
 package org.apache.hadoop.ozone.container.common.volume;
 
 
-import com.google.common.base.Optional;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListenableFuture;
@@ -26,7 +25,6 @@ import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.MoreExecutors;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.datanode.checker.AsyncChecker;
 import org.apache.hadoop.hdfs.server.datanode.checker.Checkable;
 import org.apache.hadoop.hdfs.server.datanode.checker.Checkable;
 import org.apache.hadoop.util.Timer;
 import org.apache.hadoop.util.Timer;
 
 
@@ -37,6 +35,7 @@ import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 import javax.annotation.Nullable;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
+import java.util.Optional;
 import java.util.WeakHashMap;
 import java.util.WeakHashMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledExecutorService;
@@ -122,7 +121,7 @@ public class ThrottledAsyncChecker<K, V> implements AsyncChecker<K, V> {
   public Optional<ListenableFuture<V>> schedule(
   public Optional<ListenableFuture<V>> schedule(
       Checkable<K, V> target, K context) {
       Checkable<K, V> target, K context) {
     if (checksInProgress.containsKey(target)) {
     if (checksInProgress.containsKey(target)) {
-      return Optional.absent();
+      return Optional.empty();
     }
     }
 
 
     if (completedChecks.containsKey(target)) {
     if (completedChecks.containsKey(target)) {
@@ -133,7 +132,7 @@ public class ThrottledAsyncChecker<K, V> implements AsyncChecker<K, V> {
         LOG.debug("Skipped checking {}. Time since last check {}ms " +
         LOG.debug("Skipped checking {}. Time since last check {}ms " +
                 "is less than the min gap {}ms.",
                 "is less than the min gap {}ms.",
             target, msSinceLastCheck, minMsBetweenChecks);
             target, msSinceLastCheck, minMsBetweenChecks);
-        return Optional.absent();
+        return Optional.empty();
       }
       }
     }
     }
 
 

+ 1 - 2
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestHddsVolumeChecker.java

@@ -18,11 +18,9 @@
 
 
 package org.apache.hadoop.ozone.container.common.volume;
 package org.apache.hadoop.ozone.container.common.volume;
 
 
-import com.google.common.base.Optional;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListenableFuture;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.server.datanode.checker.AsyncChecker;
 import org.apache.hadoop.hdfs.server.datanode.checker.Checkable;
 import org.apache.hadoop.hdfs.server.datanode.checker.Checkable;
 import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
 import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -42,6 +40,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.List;
 import java.util.List;
 import java.util.Set;
 import java.util.Set;
+import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicLong;