2626import static org .apache .hadoop .hbase .trace .TraceUtil .tracedFuture ;
2727import static org .apache .hadoop .hbase .trace .TraceUtil .tracedFutures ;
2828import static org .apache .hadoop .hbase .util .FutureUtils .addListener ;
29-
3029import io .opentelemetry .api .trace .Span ;
30+ import io .opentelemetry .api .trace .StatusCode ;
31+ import io .opentelemetry .context .Scope ;
3132import java .io .IOException ;
3233import java .util .ArrayList ;
3334import java .util .Arrays ;
5152import org .apache .hadoop .hbase .io .TimeRange ;
5253import org .apache .hadoop .hbase .ipc .HBaseRpcController ;
5354import org .apache .hadoop .hbase .trace .HBaseSemanticAttributes ;
55+ import org .apache .hadoop .hbase .trace .TraceUtil ;
5456import org .apache .hadoop .hbase .util .Bytes ;
5557import org .apache .hadoop .hbase .util .ReflectionUtils ;
5658import org .apache .yetus .audience .InterfaceAudience ;
5759import org .slf4j .Logger ;
5860import org .slf4j .LoggerFactory ;
59-
6061import org .apache .hbase .thirdparty .com .google .common .base .Preconditions ;
6162import org .apache .hbase .thirdparty .com .google .protobuf .RpcCallback ;
6263import org .apache .hbase .thirdparty .com .google .protobuf .RpcChannel ;
6364import org .apache .hbase .thirdparty .io .netty .util .Timer ;
64-
6565import org .apache .hadoop .hbase .shaded .protobuf .ProtobufUtil ;
6666import org .apache .hadoop .hbase .shaded .protobuf .RequestConverter ;
6767import 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
0 commit comments