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

This commit is contained in:
Mukul Kumar Singh 2019-09-10 02:47:28 +05:30 committed by Bharat Viswanadham
parent 469165e6f2
commit d69b811ddd
4 changed files with 70 additions and 8 deletions

View File

@ -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;
}

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.ozone.container.common.volume;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
@ -28,7 +27,6 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
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.util.DiskChecker.DiskErrorException;
import org.apache.hadoop.util.Timer;
@ -43,6 +41,7 @@
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import java.util.Optional;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;

View File

@ -18,7 +18,6 @@
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.Futures;
import com.google.common.util.concurrent.ListenableFuture;
@ -26,7 +25,6 @@
import com.google.common.util.concurrent.MoreExecutors;
import org.apache.hadoop.classification.InterfaceAudience;
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.util.Timer;
@ -37,6 +35,7 @@
import javax.annotation.Nullable;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.WeakHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledExecutorService;
@ -122,7 +121,7 @@ public ThrottledAsyncChecker(final Timer timer,
public Optional<ListenableFuture<V>> schedule(
Checkable<K, V> target, K context) {
if (checksInProgress.containsKey(target)) {
return Optional.absent();
return Optional.empty();
}
if (completedChecks.containsKey(target)) {
@ -133,7 +132,7 @@ public Optional<ListenableFuture<V>> schedule(
LOG.debug("Skipped checking {}. Time since last check {}ms " +
"is less than the min gap {}ms.",
target, msSinceLastCheck, minMsBetweenChecks);
return Optional.absent();
return Optional.empty();
}
}

View File

@ -18,11 +18,9 @@
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.ListenableFuture;
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.VolumeCheckResult;
import org.apache.hadoop.test.GenericTestUtils;
@ -42,6 +40,7 @@
import java.util.Collection;
import java.util.List;
import java.util.Set;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;