Skip to content

Commit 36a5f86

Browse files
committed
HBASE-26531 Trace coprocessor exec endpoints
Trace table ExecService invocations as table operations. Ensure span relationships for both table and master invocations. Signed-off-by: Andrew Purtell <apurtell@apache.org>
1 parent a49d147 commit 36a5f86

10 files changed

Lines changed: 1018 additions & 73 deletions

File tree

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

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,4 @@
1-
/**
1+
/*
22
* Licensed to the Apache Software Foundation (ASF) under one
33
* or more contributor license agreements. See the NOTICE file
44
* distributed with this work for additional information
@@ -18,7 +18,7 @@
1818
package org.apache.hadoop.hbase.client;
1919

2020
import static java.util.stream.Collectors.toList;
21-
21+
import io.opentelemetry.context.Context;
2222
import java.io.IOException;
2323
import java.util.List;
2424
import java.util.concurrent.CompletableFuture;
@@ -32,7 +32,6 @@
3232
import org.apache.hadoop.hbase.io.TimeRange;
3333
import org.apache.hadoop.hbase.util.FutureUtils;
3434
import org.apache.yetus.audience.InterfaceAudience;
35-
3635
import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
3736

3837
/**
@@ -280,26 +279,27 @@ public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> st
280279
public <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(
281280
Function<RpcChannel, S> stubMaker, ServiceCaller<S, R> callable,
282281
CoprocessorCallback<R> callback) {
282+
final Context context = Context.current();
283283
CoprocessorCallback<R> wrappedCallback = new CoprocessorCallback<R>() {
284284

285285
@Override
286286
public void onRegionComplete(RegionInfo region, R resp) {
287-
pool.execute(() -> callback.onRegionComplete(region, resp));
287+
pool.execute(context.wrap(() -> callback.onRegionComplete(region, resp)));
288288
}
289289

290290
@Override
291291
public void onRegionError(RegionInfo region, Throwable error) {
292-
pool.execute(() -> callback.onRegionError(region, error));
292+
pool.execute(context.wrap(() -> callback.onRegionError(region, error)));
293293
}
294294

295295
@Override
296296
public void onComplete() {
297-
pool.execute(() -> callback.onComplete());
297+
pool.execute(context.wrap(callback::onComplete));
298298
}
299299

300300
@Override
301301
public void onError(Throwable error) {
302-
pool.execute(() -> callback.onError(error));
302+
pool.execute(context.wrap(() -> callback.onError(error)));
303303
}
304304
};
305305
CoprocessorServiceBuilder<S, R> builder =

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

Lines changed: 49 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -26,8 +26,9 @@
2626
import static org.apache.hadoop.hbase.trace.TraceUtil.tracedFuture;
2727
import static org.apache.hadoop.hbase.trace.TraceUtil.tracedFutures;
2828
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
29-
3029
import io.opentelemetry.api.trace.Span;
30+
import io.opentelemetry.api.trace.StatusCode;
31+
import io.opentelemetry.context.Scope;
3132
import java.io.IOException;
3233
import java.util.ArrayList;
3334
import java.util.Arrays;
@@ -51,17 +52,16 @@
5152
import org.apache.hadoop.hbase.io.TimeRange;
5253
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
5354
import org.apache.hadoop.hbase.trace.HBaseSemanticAttributes;
55+
import org.apache.hadoop.hbase.trace.TraceUtil;
5456
import org.apache.hadoop.hbase.util.Bytes;
5557
import org.apache.hadoop.hbase.util.ReflectionUtils;
5658
import org.apache.yetus.audience.InterfaceAudience;
5759
import org.slf4j.Logger;
5860
import org.slf4j.LoggerFactory;
59-
6061
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
6162
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
6263
import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
6364
import org.apache.hbase.thirdparty.io.netty.util.Timer;
64-
6565
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
6666
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
6767
import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
@@ -755,14 +755,22 @@ private <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> s
755755
ServiceCaller<S, R> callable, RegionInfo region, byte[] row) {
756756
RegionCoprocessorRpcChannelImpl channel = new RegionCoprocessorRpcChannelImpl(conn, tableName,
757757
region, row, rpcTimeoutNs, operationTimeoutNs);
758+
final Span span = Span.current();
758759
S stub = stubMaker.apply(channel);
759760
CompletableFuture<R> future = new CompletableFuture<>();
760761
ClientCoprocessorRpcController controller = new ClientCoprocessorRpcController();
761762
callable.call(stub, controller, resp -> {
762-
if (controller.failed()) {
763-
future.completeExceptionally(controller.getFailed());
764-
} else {
765-
future.complete(resp);
763+
try (Scope ignored = span.makeCurrent()) {
764+
if (controller.failed()) {
765+
final Throwable failure = controller.getFailed();
766+
future.completeExceptionally(failure);
767+
TraceUtil.setError(span, failure);
768+
} else {
769+
future.complete(resp);
770+
span.setStatus(StatusCode.OK);
771+
}
772+
} finally {
773+
span.end();
766774
}
767775
});
768776
return future;
@@ -795,8 +803,11 @@ private <S, R> void onLocateComplete(Function<RpcChannel, S> stubMaker,
795803
ServiceCaller<S, R> callable, CoprocessorCallback<R> callback, List<HRegionLocation> locs,
796804
byte[] endKey, boolean endKeyInclusive, AtomicBoolean locateFinished,
797805
AtomicInteger unfinishedRequest, HRegionLocation loc, Throwable error) {
806+
final Span span = Span.current();
798807
if (error != null) {
799808
callback.onError(error);
809+
TraceUtil.setError(span, error);
810+
span.end();
800811
return;
801812
}
802813
unfinishedRequest.incrementAndGet();
@@ -807,17 +818,23 @@ private <S, R> void onLocateComplete(Function<RpcChannel, S> stubMaker,
807818
addListener(
808819
conn.getLocator().getRegionLocation(tableName, region.getEndKey(), RegionLocateType.CURRENT,
809820
operationTimeoutNs),
810-
(l, e) -> onLocateComplete(stubMaker, callable, callback, locs, endKey, endKeyInclusive,
811-
locateFinished, unfinishedRequest, l, e));
821+
(l, e) -> {
822+
try (Scope ignored = span.makeCurrent()) {
823+
onLocateComplete(stubMaker, callable, callback, locs, endKey, endKeyInclusive,
824+
locateFinished, unfinishedRequest, l, e);
825+
}
826+
});
812827
}
813828
addListener(coprocessorService(stubMaker, callable, region, region.getStartKey()), (r, e) -> {
814-
if (e != null) {
815-
callback.onRegionError(region, e);
816-
} else {
817-
callback.onRegionComplete(region, r);
818-
}
819-
if (unfinishedRequest.decrementAndGet() == 0 && locateFinished.get()) {
820-
callback.onComplete();
829+
try (Scope ignored = span.makeCurrent()) {
830+
if (e != null) {
831+
callback.onRegionError(region, e);
832+
} else {
833+
callback.onRegionComplete(region, r);
834+
}
835+
if (unfinishedRequest.decrementAndGet() == 0 && locateFinished.get()) {
836+
callback.onComplete();
837+
}
821838
}
822839
});
823840
}
@@ -868,10 +885,22 @@ public CoprocessorServiceBuilderImpl<S, R> toRow(byte[] endKey, boolean inclusiv
868885

869886
@Override
870887
public void execute() {
871-
addListener(conn.getLocator().getRegionLocation(tableName, startKey,
872-
startKeyInclusive ? RegionLocateType.CURRENT : RegionLocateType.AFTER, operationTimeoutNs),
873-
(loc, error) -> onLocateComplete(stubMaker, callable, callback, new ArrayList<>(), endKey,
874-
endKeyInclusive, new AtomicBoolean(false), new AtomicInteger(0), loc, error));
888+
final Span span = newTableOperationSpanBuilder()
889+
.setOperation(HBaseSemanticAttributes.Operation.COPROC_EXEC)
890+
.build();
891+
try (Scope ignored = span.makeCurrent()) {
892+
final RegionLocateType regionLocateType = startKeyInclusive
893+
? RegionLocateType.CURRENT
894+
: RegionLocateType.AFTER;
895+
final CompletableFuture<HRegionLocation> future = conn.getLocator()
896+
.getRegionLocation(tableName, startKey, regionLocateType, operationTimeoutNs);
897+
addListener(future, (loc, error) -> {
898+
try (Scope ignored1 = span.makeCurrent()) {
899+
onLocateComplete(stubMaker, callable, callback, new ArrayList<>(), endKey,
900+
endKeyInclusive, new AtomicBoolean(false), new AtomicInteger(0), loc, error);
901+
}
902+
});
903+
}
875904
}
876905
}
877906

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

Lines changed: 27 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,4 @@
1-
/**
1+
/*
22
* Licensed to the Apache Software Foundation (ASF) under one
33
* or more contributor license agreements. See the NOTICE file
44
* distributed with this work for additional information
@@ -19,7 +19,8 @@
1919

2020
import static org.apache.hadoop.hbase.client.ConnectionUtils.setCoprocessorError;
2121
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
22-
22+
import io.opentelemetry.context.Context;
23+
import io.opentelemetry.context.Scope;
2324
import java.io.IOException;
2425
import java.util.concurrent.CompletableFuture;
2526
import java.util.concurrent.TimeUnit;
@@ -30,13 +31,11 @@
3031
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
3132
import org.apache.hadoop.hbase.util.Bytes;
3233
import org.apache.yetus.audience.InterfaceAudience;
33-
3434
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
3535
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
3636
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
3737
import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
3838
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
39-
4039
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
4140
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
4241
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
@@ -74,6 +73,7 @@ class RegionCoprocessorRpcChannelImpl implements RpcChannel {
7473
private CompletableFuture<Message> rpcCall(MethodDescriptor method, Message request,
7574
Message responsePrototype, HBaseRpcController controller, HRegionLocation loc,
7675
ClientService.Interface stub) {
76+
final Context context = Context.current();
7777
CompletableFuture<Message> future = new CompletableFuture<>();
7878
if (region != null && !Bytes.equals(loc.getRegion().getRegionName(), region.getRegionName())) {
7979
future.completeExceptionally(new DoNotRetryIOException("Region name is changed, expected " +
@@ -82,39 +82,43 @@ private CompletableFuture<Message> rpcCall(MethodDescriptor method, Message requ
8282
}
8383
CoprocessorServiceRequest csr = CoprocessorRpcUtils.getCoprocessorServiceRequest(method,
8484
request, row, loc.getRegion().getRegionName());
85-
stub.execService(controller, csr,
86-
new org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback<CoprocessorServiceResponse>() {
87-
88-
@Override
89-
public void run(CoprocessorServiceResponse resp) {
90-
if (controller.failed()) {
91-
future.completeExceptionally(controller.getFailed());
92-
} else {
93-
lastRegion = resp.getRegion().getValue().toByteArray();
94-
try {
95-
future.complete(CoprocessorRpcUtils.getResponse(resp, responsePrototype));
96-
} catch (IOException e) {
97-
future.completeExceptionally(e);
98-
}
85+
stub.execService(controller, csr, resp -> {
86+
try (Scope ignored = context.makeCurrent()) {
87+
if (controller.failed()) {
88+
future.completeExceptionally(controller.getFailed());
89+
} else {
90+
lastRegion = resp.getRegion().getValue().toByteArray();
91+
try {
92+
future.complete(CoprocessorRpcUtils.getResponse(resp, responsePrototype));
93+
} catch (IOException e) {
94+
future.completeExceptionally(e);
9995
}
10096
}
101-
});
97+
}
98+
});
10299
return future;
103100
}
104101

105102
@Override
106103
public void callMethod(MethodDescriptor method, RpcController controller, Message request,
107104
Message responsePrototype, RpcCallback<Message> done) {
105+
final Context context = Context.current();
108106
addListener(
109107
conn.callerFactory.<Message> single().table(tableName).row(row)
110108
.locateType(RegionLocateType.CURRENT).rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
111109
.operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS)
112-
.action((c, l, s) -> rpcCall(method, request, responsePrototype, c, l, s)).call(),
110+
.action((c, l, s) -> {
111+
try (Scope ignored = context.makeCurrent()) {
112+
return rpcCall(method, request, responsePrototype, c, l, s);
113+
}
114+
}).call(),
113115
(r, e) -> {
114-
if (e != null) {
115-
setCoprocessorError(controller, e);
116+
try (Scope ignored = context.makeCurrent()) {
117+
if (e != null) {
118+
setCoprocessorError(controller, e);
119+
}
120+
done.run(r);
116121
}
117-
done.run(r);
118122
});
119123
}
120124

0 commit comments

Comments
 (0)