HBASE-22179 Fix RawAsyncHBaseAdmin.getCompactionState
This commit is contained in:
parent
2f05376687
commit
08060b32b1
|
@ -34,6 +34,7 @@ import java.util.Map;
|
|||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.BiConsumer;
|
||||
|
@ -2856,7 +2857,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())
|
||||
|
@ -2898,9 +2899,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