Skip to content

Commit

Permalink
HBASE-28770: Support partial results in AggregateImplementation and A…
Browse files Browse the repository at this point in the history
…syncAggregationClient (apache#6167)

Signed-off-by: Duo Zhang <zhangduo@apache.org>
Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
  • Loading branch information
charlesconnell authored and ndimiduk committed Sep 23, 2024
1 parent f0e22f0 commit 09b3d32
Show file tree
Hide file tree
Showing 12 changed files with 816 additions and 195 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import static org.apache.hadoop.hbase.util.FutureUtils.allOf;

import com.google.protobuf.RpcChannel;
import java.time.Duration;
import java.util.Collections;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -676,6 +677,47 @@ interface CoprocessorCallback<R> {
void onError(Throwable error);
}

/**
* Some coprocessors may support the idea of "partial results." If for some reason a coprocessor
* cannot return all results for a given region in a single response, the client side can be
* designed to recognize this and continuing requesting more results until they are completely
* accumulated in the client.
* <p>
* It is up to a particular coprocessor implementation and its corresponding clients to agree on
* what it means for results to be incomplete, how this state is communicated, and how multiple
* incomplete results are accumulated together.
* <p>
* Use this callback when you want to execute a coprocessor call on a range of regions, and that
* coprocessor may return incomplete results for a given region. See also the docs for
* {@link CoprocessorCallback}, which all apply here to its child interface too.
*/
@InterfaceAudience.Public
interface PartialResultCoprocessorCallback<S, R> extends CoprocessorCallback<R> {
/**
* Subclasses should implement this to tell AsyncTable whether the given response is "final" or
* whether the AsyncTable should send another request to the coprocessor to fetch more results
* from the given region. This method of fetching more results can be used many times until
* there are no more results to fetch from the region.
* @param response The response received from the coprocessor
* @param region The region the response came from
* @return A ServiceCaller object if the response was not final and therefore another request is
* required to continuing fetching results. null if no more requests need to be sent to
* the region.
*/
ServiceCaller<S, R> getNextCallable(R response, RegionInfo region);

/**
* Subclasses should implement this such that, when the above method returns non-null, this
* method returns the duration that AsyncTable should wait before sending the next request to
* the given region. You can use this to create a back-off behavior to reduce load on the
* RegionServer. If that's not desired, you can always return {@link Duration.ZERO}.
* @param response The response received from the coprocessor
* @param region The region the response came from
* @return The duration to wait.
*/
Duration getWaitInterval(R response, RegionInfo region);
}

/**
* Helper class for sending coprocessorService request that executes a coprocessor call on regions
* which are covered by a range.
Expand Down Expand Up @@ -744,4 +786,12 @@ default CoprocessorServiceBuilder<S, R> toRow(byte[] endKey) {
*/
<S, R> CoprocessorServiceBuilder<S, R> coprocessorService(Function<RpcChannel, S> stubMaker,
ServiceCaller<S, R> callable, CoprocessorCallback<R> callback);

/**
* Similar to above. Use when your coprocessor client+endpoint supports partial results. If the
* server does not offer partial results, it is still safe to use this, assuming you implement
* your {@link PartialResultCoprocessorCallback#getNextCallable(Object, RegionInfo)} correctly.
*/
<S, R> CoprocessorServiceBuilder<S, R> coprocessorService(Function<RpcChannel, S> stubMaker,
ServiceCaller<S, R> callable, PartialResultCoprocessorCallback<S, R> callback);
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import io.opentelemetry.context.Context;
import io.opentelemetry.context.Scope;
import java.io.IOException;
import java.time.Duration;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
Expand Down Expand Up @@ -296,49 +297,69 @@ public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> st
public <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(
Function<RpcChannel, S> stubMaker, ServiceCaller<S, R> callable,
CoprocessorCallback<R> callback) {
return coprocessorService(stubMaker, callable,
new NoopPartialResultCoprocessorCallback<>(callback));
}

@Override
public <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(
Function<RpcChannel, S> stubMaker, ServiceCaller<S, R> callable,
PartialResultCoprocessorCallback<S, R> callback) {
final Context context = Context.current();
CoprocessorCallback<R> wrappedCallback = new CoprocessorCallback<R>() {
PartialResultCoprocessorCallback<S, R> wrappedCallback =
new PartialResultCoprocessorCallback<S, R>() {

private final Phaser regionCompletesInProgress = new Phaser(1);

@Override
public void onRegionComplete(RegionInfo region, R resp) {
regionCompletesInProgress.register();
pool.execute(context.wrap(() -> {
try {
callback.onRegionComplete(region, resp);
} finally {
regionCompletesInProgress.arriveAndDeregister();
}
}));
}

private final Phaser regionCompletesInProgress = new Phaser(1);
@Override
public void onRegionError(RegionInfo region, Throwable error) {
regionCompletesInProgress.register();
pool.execute(context.wrap(() -> {
try {
callback.onRegionError(region, error);
} finally {
regionCompletesInProgress.arriveAndDeregister();
}
}));
}

@Override
public void onRegionComplete(RegionInfo region, R resp) {
regionCompletesInProgress.register();
pool.execute(context.wrap(() -> {
try {
callback.onRegionComplete(region, resp);
} finally {
regionCompletesInProgress.arriveAndDeregister();
}
}));
}
@Override
public void onComplete() {
pool.execute(context.wrap(() -> {
// Guarantee that onComplete() is called after all onRegionComplete()'s are called
regionCompletesInProgress.arriveAndAwaitAdvance();
callback.onComplete();
}));
}

@Override
public void onRegionError(RegionInfo region, Throwable error) {
regionCompletesInProgress.register();
pool.execute(context.wrap(() -> {
try {
callback.onRegionError(region, error);
} finally {
regionCompletesInProgress.arriveAndDeregister();
}
}));
}
@Override
public void onError(Throwable error) {
pool.execute(context.wrap(() -> callback.onError(error)));
}

@Override
public void onComplete() {
pool.execute(context.wrap(() -> {
// Guarantee that onComplete() is called after all onRegionComplete()'s are called
regionCompletesInProgress.arriveAndAwaitAdvance();
callback.onComplete();
}));
}
@Override
public ServiceCaller<S, R> getNextCallable(R response, RegionInfo region) {
return callback.getNextCallable(response, region);
}

@Override
public void onError(Throwable error) {
pool.execute(context.wrap(() -> callback.onError(error)));
}
};
@Override
public Duration getWaitInterval(R response, RegionInfo region) {
return callback.getWaitInterval(response, region);
}

};
CoprocessorServiceBuilder<S, R> builder =
rawTable.coprocessorService(stubMaker, callable, wrappedCallback);
return new CoprocessorServiceBuilder<S, R>() {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;

import java.time.Duration;
import org.apache.yetus.audience.InterfaceAudience;

@InterfaceAudience.Private
public class NoopPartialResultCoprocessorCallback<S, R>
implements AsyncTable.PartialResultCoprocessorCallback<S, R> {

private final AsyncTable.CoprocessorCallback<R> delegate;

public NoopPartialResultCoprocessorCallback(AsyncTable.CoprocessorCallback<R> delegate) {
this.delegate = delegate;
}

@Override
public void onRegionComplete(RegionInfo region, R resp) {
delegate.onRegionComplete(region, resp);
}

@Override
public void onRegionError(RegionInfo region, Throwable error) {
delegate.onRegionError(region, error);
}

@Override
public void onComplete() {
delegate.onComplete();
}

@Override
public void onError(Throwable error) {
delegate.onError(error);
}

@Override
public ServiceCaller<S, R> getNextCallable(R response, RegionInfo region) {
return null;
}

@Override
public Duration getWaitInterval(R response, RegionInfo region) {
return Duration.ZERO;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import io.opentelemetry.api.trace.StatusCode;
import io.opentelemetry.context.Scope;
import java.io.IOException;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
Expand Down Expand Up @@ -829,10 +830,55 @@ private boolean locateFinished(RegionInfo region, byte[] endKey, boolean endKeyI
}
}

private <S, R> void coprocessorServiceUntilComplete(Function<RpcChannel, S> stubMaker,
ServiceCaller<S, R> callable, PartialResultCoprocessorCallback<S, R> callback,
AtomicBoolean locateFinished, AtomicInteger unfinishedRequest, RegionInfo region, Span span) {
addListener(coprocessorService(stubMaker, callable, region, region.getStartKey()), (r, e) -> {
try (Scope ignored = span.makeCurrent()) {
if (e != null) {
callback.onRegionError(region, e);
} else {
callback.onRegionComplete(region, r);
}

ServiceCaller<S, R> updatedCallable;
if (e == null && r != null) {
updatedCallable = callback.getNextCallable(r, region);
} else {
updatedCallable = null;
}

// If updatedCallable is non-null, we will be sending another request, so no need to
// decrement unfinishedRequest (recall that && short-circuits).
// If updatedCallable is null, and unfinishedRequest decrements to 0, we're done with the
// requests for this coprocessor call.
if (
updatedCallable == null && unfinishedRequest.decrementAndGet() == 0
&& locateFinished.get()
) {
callback.onComplete();
} else if (updatedCallable != null) {
Duration waitInterval = callback.getWaitInterval(r, region);
LOG.trace("Coprocessor returned incomplete result. "
+ "Sleeping for {} before making follow-up request.", waitInterval);
if (waitInterval.isZero()) {
AsyncConnectionImpl.RETRY_TIMER.newTimeout(
(timeout) -> coprocessorServiceUntilComplete(stubMaker, updatedCallable, callback,
locateFinished, unfinishedRequest, region, span),
waitInterval.toMillis(), TimeUnit.MILLISECONDS);
} else {
coprocessorServiceUntilComplete(stubMaker, updatedCallable, callback, locateFinished,
unfinishedRequest, region, span);
}
}
}
});
}

private <S, R> void onLocateComplete(Function<RpcChannel, S> stubMaker,
ServiceCaller<S, R> callable, CoprocessorCallback<R> callback, List<HRegionLocation> locs,
byte[] endKey, boolean endKeyInclusive, AtomicBoolean locateFinished,
AtomicInteger unfinishedRequest, HRegionLocation loc, Throwable error) {
ServiceCaller<S, R> callable, PartialResultCoprocessorCallback<S, R> callback, byte[] endKey,
boolean endKeyInclusive, AtomicBoolean locateFinished, AtomicInteger unfinishedRequest,
HRegionLocation loc, Throwable error) {
final Span span = Span.current();
if (error != null) {
callback.onError(error);
Expand All @@ -848,23 +894,13 @@ private <S, R> void onLocateComplete(Function<RpcChannel, S> stubMaker,
addListener(conn.getLocator().getRegionLocation(tableName, region.getEndKey(),
RegionLocateType.CURRENT, operationTimeoutNs), (l, e) -> {
try (Scope ignored = span.makeCurrent()) {
onLocateComplete(stubMaker, callable, callback, locs, endKey, endKeyInclusive,
locateFinished, unfinishedRequest, l, e);
onLocateComplete(stubMaker, callable, callback, endKey, endKeyInclusive, locateFinished,
unfinishedRequest, l, e);
}
});
}
addListener(coprocessorService(stubMaker, callable, region, region.getStartKey()), (r, e) -> {
try (Scope ignored = span.makeCurrent()) {
if (e != null) {
callback.onRegionError(region, e);
} else {
callback.onRegionComplete(region, r);
}
if (unfinishedRequest.decrementAndGet() == 0 && locateFinished.get()) {
callback.onComplete();
}
}
});
coprocessorServiceUntilComplete(stubMaker, callable, callback, locateFinished,
unfinishedRequest, region, span);
}

private final class CoprocessorServiceBuilderImpl<S, R>
Expand All @@ -874,7 +910,7 @@ private final class CoprocessorServiceBuilderImpl<S, R>

private final ServiceCaller<S, R> callable;

private final CoprocessorCallback<R> callback;
private final PartialResultCoprocessorCallback<S, R> callback;

private byte[] startKey = HConstants.EMPTY_START_ROW;

Expand All @@ -885,7 +921,7 @@ private final class CoprocessorServiceBuilderImpl<S, R>
private boolean endKeyInclusive;

public CoprocessorServiceBuilderImpl(Function<RpcChannel, S> stubMaker,
ServiceCaller<S, R> callable, CoprocessorCallback<R> callback) {
ServiceCaller<S, R> callable, PartialResultCoprocessorCallback<S, R> callback) {
this.stubMaker = Preconditions.checkNotNull(stubMaker, "stubMaker is null");
this.callable = Preconditions.checkNotNull(callable, "callable is null");
this.callback = Preconditions.checkNotNull(callback, "callback is null");
Expand Down Expand Up @@ -922,8 +958,8 @@ public void execute() {
.getRegionLocation(tableName, startKey, regionLocateType, operationTimeoutNs);
addListener(future, (loc, error) -> {
try (Scope ignored1 = span.makeCurrent()) {
onLocateComplete(stubMaker, callable, callback, new ArrayList<>(), endKey,
endKeyInclusive, new AtomicBoolean(false), new AtomicInteger(0), loc, error);
onLocateComplete(stubMaker, callable, callback, endKey, endKeyInclusive,
new AtomicBoolean(false), new AtomicInteger(0), loc, error);
}
});
}
Expand All @@ -934,6 +970,14 @@ public void execute() {
public <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(
Function<RpcChannel, S> stubMaker, ServiceCaller<S, R> callable,
CoprocessorCallback<R> callback) {
return new CoprocessorServiceBuilderImpl<>(stubMaker, callable,
new NoopPartialResultCoprocessorCallback<>(callback));
}

@Override
public <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(
Function<RpcChannel, S> stubMaker, ServiceCaller<S, R> callable,
PartialResultCoprocessorCallback<S, R> callback) {
return new CoprocessorServiceBuilderImpl<>(stubMaker, callable, callback);
}
}
Loading

0 comments on commit 09b3d32

Please sign in to comment.