From db1767936897f156dcad3da388660d4cde5163e4 Mon Sep 17 00:00:00 2001 From: zhangduo Date: Tue, 10 Oct 2017 16:20:59 +0800 Subject: [PATCH] HBASE-18972 Use Builder pattern to remove nullable parameters for coprocessor methods in RawAsyncTable interface --- .../apache/hadoop/hbase/client/RawAsyncTable.java | 111 +++++++++++++-------- .../hadoop/hbase/client/RawAsyncTableImpl.java | 90 ++++++++++++++--- .../client/coprocessor/AsyncAggregationClient.java | 68 +++++++------ 3 files changed, 185 insertions(+), 84 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java index cd0226b..9ae6a17 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java @@ -200,49 +200,80 @@ public interface RawAsyncTable extends AsyncTableBase { } /** - * Execute the given coprocessor call on the regions which are covered by the range from - * {@code startKey} inclusive and {@code endKey} exclusive. See the comment of - * {@link #coprocessorService(Function, CoprocessorCallable, byte[], boolean, byte[], boolean, CoprocessorCallback)} - * for more details. - * @see #coprocessorService(Function, CoprocessorCallable, byte[], boolean, byte[], boolean, - * CoprocessorCallback) + * Helper class for sending coprocessorService request that executes a coprocessor call on regions + * which are covered by a range. + *

+ * If {@code from} is not specified the selection will start with the first table region. If + * {@code to} is not specified the selection will continue through the last table region. + * @param the type of the protobuf Service if you want to call. + * @param the type of the return value. */ - default void coprocessorService(Function stubMaker, - CoprocessorCallable callable, byte[] startKey, byte[] endKey, - CoprocessorCallback callback) { - coprocessorService(stubMaker, callable, startKey, true, endKey, false, callback); + interface CoprocessorServiceBuilder { + + /** + * The {@code stubMaker} is just a delegation to the {@code xxxService.newStub} call. Usually it + * is only a one line lambda expression, like: + * + *

+     * 
+     * channel -> xxxService.newStub(channel)
+     * 
+     * 
+ * + * @param stubMaker a delegation to the actual {@code newStub} call. + */ + CoprocessorServiceBuilder stubMaker(Function stubMaker); + + /** + * @param callable a delegation to the actual protobuf rpc call. See the comment of + * {@link CoprocessorCallable} for more details. + */ + CoprocessorServiceBuilder action(CoprocessorCallable callable); + + /** + * @param startKey start region selection with region containing this row, inclusive. + */ + default CoprocessorServiceBuilder from(byte[] startKey) { + return from(startKey, true); + } + + /** + * @param startKey start region selection with region containing this row + * @param inclusive whether to include the startKey + */ + CoprocessorServiceBuilder from(byte[] startKey, boolean inclusive); + + /** + * @param endKey select regions up to and including the region containing this row, exclusive. + */ + default CoprocessorServiceBuilder to(byte[] endKey) { + return to(endKey, false); + } + + /** + * @param endKey select regions up to and including the region containing this row + * @param inclusive whether to include the endKey + */ + CoprocessorServiceBuilder to(byte[] endKey, boolean inclusive); + + /** + * @param callback callback to get the response. See the comment of {@link CoprocessorCallback} + * for more details. + */ + CoprocessorServiceBuilder callback(CoprocessorCallback callback); + + /** + * Execute the coprocessorService request. You can get the response through the + * {@link CoprocessorCallback} specified by calling the {@link #callback(CoprocessorCallback)} + * method above. + */ + void execute(); } /** - * Execute the given coprocessor call on the regions which are covered by the range from - * {@code startKey} and {@code endKey}. The inclusive of boundaries are specified by - * {@code startKeyInclusive} and {@code endKeyInclusive}. The {@code stubMaker} is just a - * delegation to the {@code xxxService.newStub} call. Usually it is only a one line lambda - * expression, like: - * - *
-   * 
-   * channel -> xxxService.newStub(channel)
-   * 
-   * 
- * - * @param stubMaker a delegation to the actual {@code newStub} call. - * @param callable a delegation to the actual protobuf rpc call. See the comment of - * {@link CoprocessorCallable} for more details. - * @param startKey start region selection with region containing this row. If {@code null}, the - * selection will start with the first table region. - * @param startKeyInclusive whether to include the startKey - * @param endKey select regions up to and including the region containing this row. If - * {@code null}, selection will continue through the last table region. - * @param endKeyInclusive whether to include the endKey - * @param callback callback to get the response. See the comment of {@link CoprocessorCallback} - * for more details. - * @param the type of the asynchronous stub - * @param the type of the return value - * @see CoprocessorCallable - * @see CoprocessorCallback + * Execute a coprocessor call on the regions which are covered by a range. + *

+ * Use the returned {@link CoprocessorServiceBuilder} construct your request and then execute it. */ - void coprocessorService(Function stubMaker, - CoprocessorCallable callable, byte[] startKey, boolean startKeyInclusive, byte[] endKey, - boolean endKeyInclusive, CoprocessorCallback callback); + CoprocessorServiceBuilder coprocessorService(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java index 6107f7f..964c5a7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java @@ -18,8 +18,6 @@ package org.apache.hadoop.hbase.client; import static java.util.stream.Collectors.toList; -import static org.apache.hadoop.hbase.HConstants.EMPTY_END_ROW; -import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW; import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies; import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow; @@ -29,7 +27,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -38,6 +35,7 @@ import java.util.function.Function; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.SingleRequestCallerBuilder; @@ -560,19 +558,79 @@ class RawAsyncTableImpl implements RawAsyncTable { }); } - @Override - public void coprocessorService(Function stubMaker, - CoprocessorCallable callable, byte[] startKey, boolean startKeyInclusive, byte[] endKey, - boolean endKeyInclusive, CoprocessorCallback callback) { - byte[] nonNullStartKey = Optional.ofNullable(startKey).orElse(EMPTY_START_ROW); - byte[] nonNullEndKey = Optional.ofNullable(endKey).orElse(EMPTY_END_ROW); - List locs = new ArrayList<>(); - conn.getLocator() - .getRegionLocation(tableName, nonNullStartKey, - startKeyInclusive ? RegionLocateType.CURRENT : RegionLocateType.AFTER, operationTimeoutNs) - .whenComplete( - (loc, error) -> onLocateComplete(stubMaker, callable, callback, locs, nonNullEndKey, - endKeyInclusive, new AtomicBoolean(false), new AtomicInteger(0), loc, error)); + private final class CoprocessorServiceBuilderImpl + implements CoprocessorServiceBuilder { + + private Function stubMaker; + + private CoprocessorCallable callable; + + private byte[] startKey = HConstants.EMPTY_START_ROW; + + private boolean startKeyInclusive; + + private byte[] endKey = HConstants.EMPTY_END_ROW; + + private boolean endKeyInclusive; + + private CoprocessorCallback callback; + + @Override + public CoprocessorServiceBuilderImpl stubMaker(Function stubMaker) { + this.stubMaker = Preconditions.checkNotNull(stubMaker, "stubMaker is null"); + return this; + } + + @Override + public CoprocessorServiceBuilderImpl action(CoprocessorCallable callable) { + this.callable = Preconditions.checkNotNull(callable, "callable is null"); + return this; + } + + @Override + public CoprocessorServiceBuilderImpl from(byte[] startKey, boolean inclusive) { + this.startKey = Preconditions.checkNotNull(startKey, + "startKey is null. Consider using" + + " an empty byte array, or just do not call this method if you want to start selection" + + " from the first region"); + this.startKeyInclusive = inclusive; + return this; + } + + @Override + public CoprocessorServiceBuilderImpl to(byte[] endKey, boolean inclusive) { + this.endKey = Preconditions.checkNotNull(endKey, + "endKey is null. Consider using" + + " an empty byte array, or just do not call this method if you want to continue" + + " selection to the last region"); + this.endKeyInclusive = inclusive; + return this; + } + + @Override + public CoprocessorServiceBuilderImpl callback(CoprocessorCallback callback) { + this.callback = Preconditions.checkNotNull(callback, "callback is null"); + return this; + } + + @Override + public void execute() { + Preconditions.checkNotNull(stubMaker, + "stubMaker is null. You need to specify it by calling stubMaker method"); + Preconditions.checkNotNull(callable, + "callable is null. You need to specify it by calling action method"); + Preconditions.checkNotNull(callback, + "callback is null. You need to specify it by calling callback method"); + conn.getLocator().getRegionLocation(tableName, startKey, + startKeyInclusive ? RegionLocateType.CURRENT : RegionLocateType.AFTER, operationTimeoutNs) + .whenComplete( + (loc, error) -> onLocateComplete(stubMaker, callable, callback, new ArrayList<>(), + endKey, endKeyInclusive, new AtomicBoolean(false), new AtomicInteger(0), loc, error)); + } } + @Override + public CoprocessorServiceBuilder coprocessorService() { + return new CoprocessorServiceBuilderImpl<>(); + } } diff --git a/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncAggregationClient.java b/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncAggregationClient.java index 51c8248..2aa926a 100644 --- a/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncAggregationClient.java +++ b/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncAggregationClient.java @@ -29,6 +29,7 @@ import java.util.TreeMap; import java.util.concurrent.CompletableFuture; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.RawAsyncTable; import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallback; import org.apache.hadoop.hbase.client.RawScanResultConsumer; @@ -120,6 +121,10 @@ public class AsyncAggregationClient { return ci.getPromotedValueFromProto(t); } + private static byte[] nullToEmpty(byte[] b) { + return b != null ? b : HConstants.EMPTY_BYTE_ARRAY; + } + public static CompletableFuture max(RawAsyncTable table, ColumnInterpreter ci, Scan scan) { CompletableFuture future = new CompletableFuture<>(); @@ -149,10 +154,11 @@ public class AsyncAggregationClient { return max; } }; - table.coprocessorService(channel -> AggregateService.newStub(channel), - (stub, controller, rpcCallback) -> stub.getMax(controller, req, rpcCallback), - scan.getStartRow(), scan.includeStartRow(), scan.getStopRow(), scan.includeStopRow(), - callback); + table. coprocessorService() + .stubMaker(AggregateService::newStub) + .action((stub, controller, rpcCallback) -> stub.getMax(controller, req, rpcCallback)) + .from(nullToEmpty(scan.getStartRow()), scan.includeStartRow()) + .to(nullToEmpty(scan.getStopRow()), scan.includeStopRow()).callback(callback).execute(); return future; } @@ -185,10 +191,11 @@ public class AsyncAggregationClient { return min; } }; - table.coprocessorService(channel -> AggregateService.newStub(channel), - (stub, controller, rpcCallback) -> stub.getMin(controller, req, rpcCallback), - scan.getStartRow(), scan.includeStartRow(), scan.getStopRow(), scan.includeStopRow(), - callback); + table. coprocessorService() + .stubMaker(AggregateService::newStub) + .action((stub, controller, rpcCallback) -> stub.getMin(controller, req, rpcCallback)) + .from(nullToEmpty(scan.getStartRow()), scan.includeStartRow()) + .to(nullToEmpty(scan.getStopRow()), scan.includeStopRow()).callback(callback).execute(); return future; } @@ -217,10 +224,11 @@ public class AsyncAggregationClient { return count; } }; - table.coprocessorService(channel -> AggregateService.newStub(channel), - (stub, controller, rpcCallback) -> stub.getRowNum(controller, req, rpcCallback), - scan.getStartRow(), scan.includeStartRow(), scan.getStopRow(), scan.includeStopRow(), - callback); + table. coprocessorService() + .stubMaker(AggregateService::newStub) + .action((stub, controller, rpcCallback) -> stub.getRowNum(controller, req, rpcCallback)) + .from(nullToEmpty(scan.getStartRow()), scan.includeStartRow()) + .to(nullToEmpty(scan.getStopRow()), scan.includeStopRow()).callback(callback).execute(); return future; } @@ -251,10 +259,11 @@ public class AsyncAggregationClient { return sum; } }; - table.coprocessorService(channel -> AggregateService.newStub(channel), - (stub, controller, rpcCallback) -> stub.getSum(controller, req, rpcCallback), - scan.getStartRow(), scan.includeStartRow(), scan.getStopRow(), scan.includeStopRow(), - callback); + table. coprocessorService() + .stubMaker(AggregateService::newStub) + .action((stub, controller, rpcCallback) -> stub.getSum(controller, req, rpcCallback)) + .from(nullToEmpty(scan.getStartRow()), scan.includeStartRow()) + .to(nullToEmpty(scan.getStopRow()), scan.includeStopRow()).callback(callback).execute(); return future; } @@ -288,10 +297,11 @@ public class AsyncAggregationClient { return ci.divideForAvg(sum, count); } }; - table.coprocessorService(channel -> AggregateService.newStub(channel), - (stub, controller, rpcCallback) -> stub.getAvg(controller, req, rpcCallback), - scan.getStartRow(), scan.includeStartRow(), scan.getStopRow(), scan.includeStopRow(), - callback); + table. coprocessorService() + .stubMaker(AggregateService::newStub) + .action((stub, controller, rpcCallback) -> stub.getAvg(controller, req, rpcCallback)) + .from(nullToEmpty(scan.getStartRow()), scan.includeStartRow()) + .to(nullToEmpty(scan.getStopRow()), scan.includeStopRow()).callback(callback).execute(); return future; } @@ -330,10 +340,11 @@ public class AsyncAggregationClient { return Math.sqrt(avgSq - avg * avg); } }; - table.coprocessorService(channel -> AggregateService.newStub(channel), - (stub, controller, rpcCallback) -> stub.getStd(controller, req, rpcCallback), - scan.getStartRow(), scan.includeStartRow(), scan.getStopRow(), scan.includeStopRow(), - callback); + table. coprocessorService() + .stubMaker(AggregateService::newStub) + .action((stub, controller, rpcCallback) -> stub.getStd(controller, req, rpcCallback)) + .from(nullToEmpty(scan.getStartRow()), scan.includeStartRow()) + .to(nullToEmpty(scan.getStopRow()), scan.includeStopRow()).callback(callback).execute(); return future; } @@ -368,10 +379,11 @@ public class AsyncAggregationClient { return map; } }; - table.coprocessorService(channel -> AggregateService.newStub(channel), - (stub, controller, rpcCallback) -> stub.getMedian(controller, req, rpcCallback), - scan.getStartRow(), scan.includeStartRow(), scan.getStopRow(), scan.includeStopRow(), - callback); + table. coprocessorService() + .stubMaker(AggregateService::newStub) + .action((stub, controller, rpcCallback) -> stub.getMedian(controller, req, rpcCallback)) + .from(nullToEmpty(scan.getStartRow()), scan.includeStartRow()) + .to(nullToEmpty(scan.getStopRow()), scan.includeStopRow()).callback(callback).execute(); return future; } -- 2.7.4