Skip to content

Commit 02f2636

Browse files
authored
HBASE-27207 ConnectionUtils.allOf should be moved to FutureUtils (#4627)
Signed-off-by: Duo Zhang <[email protected]>
1 parent 01b45e2 commit 02f2636

File tree

3 files changed

+16
-6
lines changed

3 files changed

+16
-6
lines changed

hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,8 +18,8 @@
1818
package org.apache.hadoop.hbase.client;
1919

2020
import static java.util.stream.Collectors.toList;
21-
import static org.apache.hadoop.hbase.client.ConnectionUtils.allOf;
2221
import static org.apache.hadoop.hbase.client.ConnectionUtils.toCheckExistenceOnly;
22+
import static org.apache.hadoop.hbase.util.FutureUtils.allOf;
2323

2424
import java.util.List;
2525
import java.util.concurrent.CompletableFuture;

hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -306,11 +306,6 @@ static boolean noMoreResultsForReverseScan(Scan scan, RegionInfo info) {
306306
return Bytes.compareTo(info.getStartKey(), scan.getStopRow()) <= 0;
307307
}
308308

309-
static <T> CompletableFuture<List<T>> allOf(List<CompletableFuture<T>> futures) {
310-
return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]))
311-
.thenApply(v -> futures.stream().map(f -> f.getNow(null)).collect(toList()));
312-
}
313-
314309
public static ScanResultCache createScanResultCache(Scan scan) {
315310
if (scan.getAllowPartialResults()) {
316311
return new AllowPartialScanResultCache();

hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,11 @@
1717
*/
1818
package org.apache.hadoop.hbase.util;
1919

20+
import static java.util.stream.Collectors.toList;
21+
2022
import java.io.IOException;
2123
import java.io.InterruptedIOException;
24+
import java.util.List;
2225
import java.util.concurrent.CompletableFuture;
2326
import java.util.concurrent.CompletionException;
2427
import java.util.concurrent.ExecutionException;
@@ -197,4 +200,16 @@ public static <T> CompletableFuture<T> failedFuture(Throwable e) {
197200
future.completeExceptionally(e);
198201
return future;
199202
}
203+
204+
/**
205+
* Returns a new CompletableFuture that is completed when all of the given CompletableFutures
206+
* complete. If any of the given CompletableFutures complete exceptionally, then the returned
207+
* CompletableFuture also does so, with a CompletionException holding this exception as its cause.
208+
* Otherwise, the results of all given CompletableFutures could be obtained by the new returned
209+
* CompletableFuture.
210+
*/
211+
public static <T> CompletableFuture<List<T>> allOf(List<CompletableFuture<T>> futures) {
212+
return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]))
213+
.thenApply(v -> futures.stream().map(f -> f.getNow(null)).collect(toList()));
214+
}
200215
}

0 commit comments

Comments
 (0)