2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org
.apache
.hadoop
.hbase
.trace
;
20 import io
.opentelemetry
.api
.GlobalOpenTelemetry
;
21 import io
.opentelemetry
.api
.trace
.Span
;
22 import io
.opentelemetry
.api
.trace
.SpanKind
;
23 import io
.opentelemetry
.api
.trace
.StatusCode
;
24 import io
.opentelemetry
.api
.trace
.Tracer
;
25 import io
.opentelemetry
.context
.Context
;
26 import io
.opentelemetry
.context
.Scope
;
27 import java
.util
.List
;
28 import java
.util
.concurrent
.Callable
;
29 import java
.util
.concurrent
.CompletableFuture
;
30 import java
.util
.function
.Supplier
;
31 import org
.apache
.hadoop
.hbase
.Version
;
32 import org
.apache
.hadoop
.hbase
.util
.FutureUtils
;
33 import org
.apache
.yetus
.audience
.InterfaceAudience
;
35 @InterfaceAudience.Private
36 public final class TraceUtil
{
41 public static Tracer
getGlobalTracer() {
42 return GlobalOpenTelemetry
.getTracer("org.apache.hbase", Version
.version
);
46 * Create a {@link SpanKind#INTERNAL} span.
48 public static Span
createSpan(String name
) {
49 return createSpan(name
, SpanKind
.INTERNAL
);
53 * Create a span with the given {@code kind}. Notice that, OpenTelemetry only expects one
54 * {@link SpanKind#CLIENT} span and one {@link SpanKind#SERVER} span for a traced request, so use
55 * this with caution when you want to create spans with kind other than {@link SpanKind#INTERNAL}.
57 private static Span
createSpan(String name
, SpanKind kind
) {
58 return getGlobalTracer().spanBuilder(name
).setSpanKind(kind
).startSpan();
62 * Create a span which parent is from remote, i.e, passed through rpc.
64 * We will set the kind of the returned span to {@link SpanKind#SERVER}, as this should be the top
65 * most span at server side.
67 public static Span
createRemoteSpan(String name
, Context ctx
) {
68 return getGlobalTracer().spanBuilder(name
).setParent(ctx
).setSpanKind(SpanKind
.SERVER
)
73 * Create a span with {@link SpanKind#CLIENT}.
75 public static Span
createClientSpan(String name
) {
76 return createSpan(name
, SpanKind
.CLIENT
);
80 * Trace an asynchronous operation for a table.
82 public static <T
> CompletableFuture
<T
> tracedFuture(
83 Supplier
<CompletableFuture
<T
>> action
,
84 Supplier
<Span
> spanSupplier
86 Span span
= spanSupplier
.get();
87 try (Scope ignored
= span
.makeCurrent()) {
88 CompletableFuture
<T
> future
= action
.get();
89 endSpan(future
, span
);
95 * Trace an asynchronous operation.
97 public static <T
> CompletableFuture
<T
> tracedFuture(Supplier
<CompletableFuture
<T
>> action
,
99 Span span
= createSpan(spanName
);
100 try (Scope ignored
= span
.makeCurrent()) {
101 CompletableFuture
<T
> future
= action
.get();
102 endSpan(future
, span
);
108 * Trace an asynchronous operation, and finish the create {@link Span} when all the given
109 * {@code futures} are completed.
111 public static <T
> List
<CompletableFuture
<T
>> tracedFutures(
112 Supplier
<List
<CompletableFuture
<T
>>> action
,
113 Supplier
<Span
> spanSupplier
115 Span span
= spanSupplier
.get();
116 try (Scope ignored
= span
.makeCurrent()) {
117 List
<CompletableFuture
<T
>> futures
= action
.get();
118 endSpan(CompletableFuture
.allOf(futures
.toArray(new CompletableFuture
[0])), span
);
123 public static void setError(Span span
, Throwable error
) {
124 span
.recordException(error
);
125 span
.setStatus(StatusCode
.ERROR
);
129 * Finish the {@code span} when the given {@code future} is completed.
131 private static void endSpan(CompletableFuture
<?
> future
, Span span
) {
132 FutureUtils
.addListener(future
, (resp
, error
) -> {
134 setError(span
, error
);
136 span
.setStatus(StatusCode
.OK
);
143 * A {@link Runnable} that may also throw.
144 * @param <T> the type of {@link Throwable} that can be produced.
147 public interface ThrowingRunnable
<T
extends Throwable
> {
151 public static <T
extends Throwable
> void trace(
152 final ThrowingRunnable
<T
> runnable
,
153 final String spanName
) throws T
{
154 trace(runnable
, () -> createSpan(spanName
));
157 public static <T
extends Throwable
> void trace(
158 final ThrowingRunnable
<T
> runnable
,
159 final Supplier
<Span
> spanSupplier
161 Span span
= spanSupplier
.get();
162 try (Scope ignored
= span
.makeCurrent()) {
164 span
.setStatus(StatusCode
.OK
);
165 } catch (Throwable e
) {
174 * A {@link Callable} that may also throw.
175 * @param <R> the result type of method call.
176 * @param <T> the type of {@link Throwable} that can be produced.
179 public interface ThrowingCallable
<R
, T
extends Throwable
> {
183 public static <R
, T
extends Throwable
> R
trace(
184 final ThrowingCallable
<R
, T
> callable
,
185 final String spanName
187 return trace(callable
, () -> createSpan(spanName
));
190 public static <R
, T
extends Throwable
> R
trace(
191 final ThrowingCallable
<R
, T
> callable
,
192 final Supplier
<Span
> spanSupplier
194 Span span
= spanSupplier
.get();
195 try (Scope ignored
= span
.makeCurrent()) {
196 final R ret
= callable
.call();
197 span
.setStatus(StatusCode
.OK
);
199 } catch (Throwable e
) {