HBASE-22179 Fix RawAsyncHBaseAdmin.getCompactionState
This commit is contained in:
parent
0ca97d482a
commit
0247466210
|
@ -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;
|
||||
|
@ -2937,7 +2938,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())
|
||||
|
@ -2979,9 +2980,9 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
case NONE:
|
||||
default:
|
||||
}
|
||||
if (!future.isDone()) {
|
||||
future.complete(state);
|
||||
}
|
||||
}
|
||||
if (!future.isDone()) {
|
||||
future.complete(state);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
|
Loading…
Reference in New Issue