This is an automated email from the ASF dual-hosted git repository. zhangduo pushed a commit to branch branch-2 in repository https://gitbox.apache.org/repos/asf/hbase.git
commit 0b46ac5db329ce583d71b5060a293d557b70a2ef Author: zhangduo <[email protected]> AuthorDate: Sat Apr 6 22:17:46 2019 +0800 HBASE-22179 Fix RawAsyncHBaseAdmin.getCompactionState --- .../java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index fb202d6..55c6598 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -36,6 +36,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; @@ -2931,7 +2932,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin { future.completeExceptionally(err); return; } - List<CompactionState> regionStates = new ArrayList<>(); + ConcurrentLinkedQueue<CompactionState> regionStates = new ConcurrentLinkedQueue<>(); List<CompletableFuture<CompactionState>> futures = new ArrayList<>(); locations.stream().filter(loc -> loc.getServerName() != null) .filter(loc -> loc.getRegion() != null).filter(loc -> !loc.getRegion().isOffline()) @@ -2973,9 +2974,9 @@ class RawAsyncHBaseAdmin implements AsyncAdmin { case NONE: default: } - if (!future.isDone()) { - future.complete(state); - } + } + if (!future.isDone()) { + future.complete(state); } } });
