diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/trace/hamcrest/EventMatchers.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/trace/hamcrest/EventMatchers.java new file mode 100644 index 000000000000..e24245fb4c62 --- /dev/null +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/trace/hamcrest/EventMatchers.java @@ -0,0 +1,53 @@ +/* + * 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.trace.hamcrest; + +import static org.hamcrest.Matchers.equalTo; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.sdk.trace.data.EventData; +import org.hamcrest.FeatureMatcher; +import org.hamcrest.Matcher; + +/** + * Helper methods for matching against instances of {@link EventData}. + */ +public final class EventMatchers { + + private EventMatchers() { } + + public static Matcher hasAttributes(Matcher matcher) { + return new FeatureMatcher( + matcher, "EventData having attributes that ", "attributes") { + @Override protected Attributes featureValueOf(EventData actual) { + return actual.getAttributes(); + } + }; + } + + public static Matcher hasName(String name) { + return hasName(equalTo(name)); + } + + public static Matcher hasName(Matcher matcher) { + return new FeatureMatcher(matcher, "EventData with a name that ", "name") { + @Override protected String featureValueOf(EventData actual) { + return actual.getName(); + } + }; + } +} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/trace/hamcrest/SpanDataMatchers.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/trace/hamcrest/SpanDataMatchers.java index 9697d69f18ba..a9473dae5597 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/trace/hamcrest/SpanDataMatchers.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/trace/hamcrest/SpanDataMatchers.java @@ -22,6 +22,7 @@ import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.trace.SpanKind; import io.opentelemetry.api.trace.StatusCode; +import io.opentelemetry.sdk.trace.data.EventData; import io.opentelemetry.sdk.trace.data.SpanData; import io.opentelemetry.sdk.trace.data.StatusData; import java.time.Duration; @@ -68,6 +69,15 @@ public static Matcher hasEnded() { }; } + public static Matcher hasEvents(Matcher> matcher) { + return new FeatureMatcher>( + matcher, "SpanData having events that", "events") { + @Override protected Iterable featureValueOf(SpanData item) { + return item.getEvents(); + } + }; + } + public static Matcher hasKind(SpanKind kind) { return new FeatureMatcher( equalTo(kind), "SpanData with kind that", "SpanKind") { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java index 400b358497fb..1f85346908ff 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java @@ -22,7 +22,6 @@ import io.opentelemetry.context.Scope; import java.net.InetSocketAddress; import java.nio.channels.ClosedChannelException; -import java.util.Optional; import org.apache.hadoop.hbase.CallDroppedException; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HBaseInterfaceAudience; @@ -36,7 +35,6 @@ import org.apache.hadoop.util.StringUtils; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; - import org.apache.hbase.thirdparty.com.google.protobuf.Message; /** @@ -54,7 +52,8 @@ public class CallRunner { private RpcCall call; private RpcServerInterface rpcServer; private MonitoredRPCHandler status; - private volatile boolean sucessful; + private final Span span; + private volatile boolean successful; /** * On construction, adds the size of this call to the running count of outstanding call sizes. @@ -65,6 +64,7 @@ public class CallRunner { CallRunner(final RpcServerInterface rpcServer, final RpcCall call) { this.call = call; this.rpcServer = rpcServer; + this.span = Span.current(); // Add size of the call to queue size. if (call != null && rpcServer != null) { this.rpcServer.addCallSize(call.getSize()); @@ -89,32 +89,33 @@ private void cleanup() { } public void run() { - try { + try (Scope ignored = span.makeCurrent()) { if (call.disconnectSince() >= 0) { - if (RpcServer.LOG.isDebugEnabled()) { - RpcServer.LOG.debug(Thread.currentThread().getName() + ": skipped " + call); - } + RpcServer.LOG.debug("{}: skipped {}", Thread.currentThread().getName(), call); + span.addEvent("Client disconnect detected"); + span.setStatus(StatusCode.OK); return; } call.setStartTime(EnvironmentEdgeManager.currentTime()); if (call.getStartTime() > call.getDeadline()) { - RpcServer.LOG.warn("Dropping timed out call: " + call); + RpcServer.LOG.warn("Dropping timed out call: {}", call); this.rpcServer.getMetrics().callTimedOut(); + span.addEvent("Call deadline exceeded"); + span.setStatus(StatusCode.OK); return; } this.status.setStatus("Setting up call"); this.status.setConnection(call.getRemoteAddress().getHostAddress(), call.getRemotePort()); if (RpcServer.LOG.isTraceEnabled()) { - Optional remoteUser = call.getRequestUser(); - RpcServer.LOG.trace(call.toShortString() + " executing as " + - (remoteUser.isPresent() ? remoteUser.get().getName() : "NULL principal")); + RpcServer.LOG.trace("{} executing as {}", call.toShortString(), + call.getRequestUser().map(User::getName).orElse("NULL principal")); } Throwable errorThrowable = null; String error = null; Pair resultPair = null; RpcServer.CurCall.set(call); - Span span = new IpcServerSpanBuilder(call).build(); - try (Scope traceScope = span.makeCurrent()) { + final Span ipcServerSpan = new IpcServerSpanBuilder(call).build(); + try (Scope ignored1 = ipcServerSpan.makeCurrent()) { if (!this.rpcServer.isStarted()) { InetSocketAddress address = rpcServer.getListenerAddress(); throw new ServerNotRunningYetException("Server " + @@ -122,12 +123,12 @@ public void run() { } // make the call resultPair = this.rpcServer.call(call, this.status); - } catch (TimeoutIOException e){ - RpcServer.LOG.warn("Can not complete this request in time, drop it: " + call); - TraceUtil.setError(span, e); + } catch (TimeoutIOException e) { + RpcServer.LOG.warn("Can not complete this request in time, drop it: {}", call); + TraceUtil.setError(ipcServerSpan, e); return; } catch (Throwable e) { - TraceUtil.setError(span, e); + TraceUtil.setError(ipcServerSpan, e); if (e instanceof ServerNotRunningYetException) { // If ServerNotRunningYetException, don't spew stack trace. if (RpcServer.LOG.isTraceEnabled()) { @@ -135,7 +136,7 @@ public void run() { } } else { // Don't dump full exception.. just String version - RpcServer.LOG.debug(call.toShortString() + ", exception=" + e); + RpcServer.LOG.debug("{}, exception={}", call.toShortString(), e); } errorThrowable = e; error = StringUtils.stringifyException(e); @@ -146,40 +147,43 @@ public void run() { RpcServer.CurCall.set(null); if (resultPair != null) { this.rpcServer.addCallSize(call.getSize() * -1); - span.setStatus(StatusCode.OK); - sucessful = true; + ipcServerSpan.setStatus(StatusCode.OK); + successful = true; } - span.end(); + ipcServerSpan.end(); } this.status.markComplete("To send response"); - // return back the RPC request read BB we can do here. It is done by now. + // return the RPC request read BB we can do here. It is done by now. call.cleanup(); // Set the response Message param = resultPair != null ? resultPair.getFirst() : null; CellScanner cells = resultPair != null ? resultPair.getSecond() : null; call.setResponse(param, cells, errorThrowable, error); call.sendResponseIfReady(); + // don't touch `span` here because its status and `end()` are managed in `call#setResponse()` } catch (OutOfMemoryError e) { - if (this.rpcServer.getErrorHandler() != null) { - if (this.rpcServer.getErrorHandler().checkOOME(e)) { - RpcServer.LOG.info(Thread.currentThread().getName() + ": exiting on OutOfMemoryError"); - return; - } + TraceUtil.setError(span, e); + if (this.rpcServer.getErrorHandler() != null + && this.rpcServer.getErrorHandler().checkOOME(e)) { + RpcServer.LOG.info("{}: exiting on OutOfMemoryError", Thread.currentThread().getName()); + // exception intentionally swallowed } else { // rethrow if no handler throw e; } } catch (ClosedChannelException cce) { InetSocketAddress address = rpcServer.getListenerAddress(); - RpcServer.LOG.warn(Thread.currentThread().getName() + ": caught a ClosedChannelException, " + + RpcServer.LOG.warn("{}: caught a ClosedChannelException, " + "this means that the server " + (address != null ? address : "(channel closed)") + - " was processing a request but the client went away. The error message was: " + - cce.getMessage()); + " was processing a request but the client went away. The error message was: {}", + Thread.currentThread().getName(), cce.getMessage()); + TraceUtil.setError(span, cce); } catch (Exception e) { - RpcServer.LOG.warn(Thread.currentThread().getName() - + ": caught: " + StringUtils.stringifyException(e)); + RpcServer.LOG.warn("{}: caught: {}", Thread.currentThread().getName(), + StringUtils.stringifyException(e)); + TraceUtil.setError(span, e); } finally { - if (!sucessful) { + if (!successful) { this.rpcServer.addCallSize(call.getSize() * -1); } @@ -188,6 +192,7 @@ public void run() { } this.status.pause("Waiting for a call"); cleanup(); + span.end(); } } @@ -195,11 +200,11 @@ public void run() { * When we want to drop this call because of server is overloaded. */ public void drop() { - try { + try (Scope ignored = span.makeCurrent()) { if (call.disconnectSince() >= 0) { - if (RpcServer.LOG.isDebugEnabled()) { - RpcServer.LOG.debug(Thread.currentThread().getName() + ": skipped " + call); - } + RpcServer.LOG.debug("{}: skipped {}", Thread.currentThread().getName(), call); + span.addEvent("Client disconnect detected"); + span.setStatus(StatusCode.OK); return; } @@ -207,22 +212,26 @@ public void drop() { InetSocketAddress address = rpcServer.getListenerAddress(); call.setResponse(null, null, CALL_DROPPED_EXCEPTION, "Call dropped, server " + (address != null ? address : "(channel closed)") + " is overloaded, please retry."); + TraceUtil.setError(span, CALL_DROPPED_EXCEPTION); call.sendResponseIfReady(); this.rpcServer.getMetrics().exception(CALL_DROPPED_EXCEPTION); } catch (ClosedChannelException cce) { InetSocketAddress address = rpcServer.getListenerAddress(); - RpcServer.LOG.warn(Thread.currentThread().getName() + ": caught a ClosedChannelException, " + - "this means that the server " + (address != null ? address : "(channel closed)") + - " was processing a request but the client went away. The error message was: " + - cce.getMessage()); + RpcServer.LOG.warn("{}: caught a ClosedChannelException, " + + "this means that the server " + (address != null ? address : "(channel closed)") + + " was processing a request but the client went away. The error message was: {}", + Thread.currentThread().getName(), cce.getMessage()); + TraceUtil.setError(span, cce); } catch (Exception e) { - RpcServer.LOG.warn(Thread.currentThread().getName() - + ": caught: " + StringUtils.stringifyException(e)); + RpcServer.LOG.warn("{}: caught: {}", Thread.currentThread().getName(), + StringUtils.stringifyException(e)); + TraceUtil.setError(span, e); } finally { - if (!sucessful) { + if (!successful) { this.rpcServer.addCallSize(call.getSize() * -1); } cleanup(); + span.end(); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RPCTInfoGetter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RPCTInfoGetter.java new file mode 100644 index 000000000000..a5ed6fe0eae7 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RPCTInfoGetter.java @@ -0,0 +1,50 @@ +/* + * 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.ipc; + +import io.opentelemetry.context.Context; +import io.opentelemetry.context.propagation.TextMapGetter; +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos; + +/** + * Used to extract a tracing {@link Context} from an instance of {@link TracingProtos.RPCTInfo}. + */ +@InterfaceAudience.Private +final class RPCTInfoGetter implements TextMapGetter { + RPCTInfoGetter() { } + + @Override + public Iterable keys(TracingProtos.RPCTInfo carrier) { + return Optional.ofNullable(carrier) + .map(TracingProtos.RPCTInfo::getHeadersMap) + .map(Map::keySet) + .orElse(Collections.emptySet()); + } + + @Override + public String get(TracingProtos.RPCTInfo carrier, String key) { + return Optional.ofNullable(carrier) + .map(TracingProtos.RPCTInfo::getHeadersMap) + .map(map -> map.get(key)) + .orElse(null); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java index 0fd60721e833..f98bfc5cbcaf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java @@ -19,6 +19,7 @@ import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.StatusCode; +import io.opentelemetry.context.Scope; import java.io.IOException; import java.net.InetAddress; import java.nio.ByteBuffer; @@ -39,12 +40,10 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.util.StringUtils; import org.apache.yetus.audience.InterfaceAudience; - import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; import org.apache.hbase.thirdparty.com.google.protobuf.CodedOutputStream; import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor; import org.apache.hbase.thirdparty.com.google.protobuf.Message; - import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta; @@ -222,8 +221,7 @@ public String toShortString() { return "callId: " + this.id + " service: " + serviceName + " methodName: " + ((this.md != null) ? this.md.getName() : "n/a") + " size: " + StringUtils.TraditionalBinaryPrefix.long2String(this.size, "", 1) + - " connection: " + connection.toString() + - " deadline: " + deadline; + " connection: " + connection + " deadline: " + deadline; } @Override @@ -299,11 +297,12 @@ public synchronized void setResponse(Message m, final CellScanner cells, Throwab // Once a response message is created and set to this.response, this Call can be treated as // done. The Responder thread will do the n/w write of this message back to client. if (this.rpcCallback != null) { - try { + try (Scope ignored = span.makeCurrent()) { this.rpcCallback.run(); } catch (Exception e) { // Don't allow any exception here to kill this handler thread. RpcServer.LOG.warn("Exception while running the Rpc Callback.", e); + TraceUtil.setError(span, e); } } } @@ -571,8 +570,4 @@ public synchronized BufferChain getResponse() { return response; } } - - public Span getSpan() { - return span; - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java index 4ebc9fa5325a..89b1adc2cd89 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -94,7 +94,9 @@ justification="False positive according to http://sourceforge.net/p/findbugs/bugs/1032/") @InterfaceAudience.Private abstract class ServerRpcConnection implements Closeable { - /** */ + + private static final TextMapGetter getter = new RPCTInfoGetter(); + protected final RpcServer rpcServer; // If the connection header has been read or not. protected boolean connectionHeaderRead = false; @@ -616,22 +618,17 @@ protected void processRequest(ByteBuff buf) throws IOException, ProtobufUtil.mergeFrom(builder, cis, headerSize); RequestHeader header = (RequestHeader) builder.build(); offset += headerSize; - TextMapGetter getter = new TextMapGetter() { - - @Override - public Iterable keys(RPCTInfo carrier) { - return carrier.getHeadersMap().keySet(); - } - - @Override - public String get(RPCTInfo carrier, String key) { - return carrier.getHeadersMap().get(key); - } - }; Context traceCtx = GlobalOpenTelemetry.getPropagators().getTextMapPropagator() .extract(Context.current(), header.getTraceInfo(), getter); + + // n.b. Management of this Span instance is a little odd. Most exit paths from this try scope + // are early-exits due to error cases. There's only one success path, the asynchronous call to + // RpcScheduler#dispatch. The success path assumes ownership of the span, which is represented + // by null-ing out the reference in this scope. All other paths end the span. Thus, and in + // order to avoid accidentally orphaning the span, the call to Span#end happens in a finally + // block iff the span is non-null. Span span = TraceUtil.createRemoteSpan("RpcServer.process", traceCtx); - try (Scope scope = span.makeCurrent()) { + try (Scope ignored = span.makeCurrent()) { int id = header.getCallId(); if (RpcServer.LOG.isTraceEnabled()) { RpcServer.LOG.trace("RequestHeader " + TextFormat.shortDebugString(header) + @@ -648,6 +645,7 @@ public String get(RPCTInfo carrier, String key) { callTooBig.setResponse(null, null, RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION, "Call queue is full on " + this.rpcServer.server.getServerName() + ", is hbase.ipc.server.max.callqueue.size too small?"); + TraceUtil.setError(span, RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION); callTooBig.sendResponseIfReady(); return; } @@ -684,27 +682,30 @@ public String get(RPCTInfo carrier, String key) { cellScanner = this.rpcServer.cellBlockBuilder.createCellScannerReusingBuffers(this.codec, this.compressionCodec, dup); } - } catch (Throwable t) { + } catch (Throwable thrown) { InetSocketAddress address = this.rpcServer.getListenerAddress(); String msg = (address != null ? address : "(channel closed)") + " is unable to read call parameter from client " + getHostAddress(); - RpcServer.LOG.warn(msg, t); + RpcServer.LOG.warn(msg, thrown); - this.rpcServer.metrics.exception(t); + this.rpcServer.metrics.exception(thrown); - // probably the hbase hadoop version does not match the running hadoop - // version - if (t instanceof LinkageError) { - t = new DoNotRetryIOException(t); - } - // If the method is not present on the server, do not retry. - if (t instanceof UnsupportedOperationException) { - t = new DoNotRetryIOException(t); + final Throwable responseThrowable; + if (thrown instanceof LinkageError) { + // probably the hbase hadoop version does not match the running hadoop version + responseThrowable = new DoNotRetryIOException(thrown); + } else if (thrown instanceof UnsupportedOperationException) { + // If the method is not present on the server, do not retry. + responseThrowable = new DoNotRetryIOException(thrown); + } else { + responseThrowable = thrown; } ServerCall readParamsFailedCall = createCall(id, this.service, null, null, null, null, totalRequestSize, null, 0, this.callCleanup); - readParamsFailedCall.setResponse(null, null, t, msg + "; " + t.getMessage()); + readParamsFailedCall.setResponse(null, null, responseThrowable, msg + "; " + + responseThrowable.getMessage()); + TraceUtil.setError(span, responseThrowable); readParamsFailedCall.sendResponseIfReady(); return; } @@ -716,14 +717,22 @@ public String get(RPCTInfo carrier, String key) { ServerCall call = createCall(id, this.service, md, header, param, cellScanner, totalRequestSize, this.addr, timeout, this.callCleanup); - if (!this.rpcServer.scheduler.dispatch(new CallRunner(this.rpcServer, call))) { + if (this.rpcServer.scheduler.dispatch(new CallRunner(this.rpcServer, call))) { + // unset span do that it's not closed in the finally block + span = null; + } else { this.rpcServer.callQueueSizeInBytes.add(-1 * call.getSize()); this.rpcServer.metrics.exception(RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION); call.setResponse(null, null, RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION, "Call queue is full on " + this.rpcServer.server.getServerName() + ", too many items queued ?"); + TraceUtil.setError(span, RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION); call.sendResponseIfReady(); } + } finally { + if (span != null) { + span.end(); + } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/server/trace/IpcServerSpanBuilder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/server/trace/IpcServerSpanBuilder.java index 259268b19f62..342aa87feda2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/server/trace/IpcServerSpanBuilder.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/server/trace/IpcServerSpanBuilder.java @@ -25,14 +25,12 @@ import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.SpanBuilder; import io.opentelemetry.api.trace.SpanKind; -import io.opentelemetry.context.Context; import java.util.HashMap; import java.util.Map; import java.util.Optional; import java.util.function.Supplier; import org.apache.hadoop.hbase.client.trace.IpcClientSpanBuilder; import org.apache.hadoop.hbase.ipc.RpcCall; -import org.apache.hadoop.hbase.ipc.ServerCall; import org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.RpcSystem; import org.apache.hadoop.hbase.trace.TraceUtil; import org.apache.yetus.audience.InterfaceAudience; @@ -46,12 +44,10 @@ @InterfaceAudience.Private public class IpcServerSpanBuilder implements Supplier { - private final RpcCall rpcCall; private String name; private final Map, Object> attributes = new HashMap<>(); public IpcServerSpanBuilder(final RpcCall rpcCall) { - this.rpcCall = rpcCall; final String packageAndService = Optional.ofNullable(rpcCall.getService()) .map(BlockingService::getDescriptorForType) .map(IpcClientSpanBuilder::getRpcPackageAndService) @@ -86,7 +82,6 @@ public Span build() { .spanBuilder(name) .setSpanKind(SpanKind.SERVER); attributes.forEach((k, v) -> builder.setAttribute((AttributeKey) k, v)); - return builder.setParent(Context.current().with(((ServerCall) rpcCall).getSpan())) - .startSpan(); + return builder.startSpan(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestCallRunner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestCallRunner.java index 49b7caa29baf..d18ddbd49f5f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestCallRunner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestCallRunner.java @@ -17,15 +17,35 @@ */ package org.apache.hadoop.hbase.ipc; +import static org.apache.hadoop.hbase.client.trace.hamcrest.AttributesMatchers.containsEntry; +import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasEnded; +import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasEvents; +import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasName; +import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasStatusWithCode; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.hasItem; +import io.opentelemetry.api.trace.StatusCode; +import io.opentelemetry.sdk.testing.junit4.OpenTelemetryRule; import java.net.InetSocketAddress; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CallDroppedException; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.MatcherPredicate; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.client.trace.hamcrest.EventMatchers; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandlerImpl; import org.apache.hadoop.hbase.testclassification.RPCTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.trace.TraceUtil; +import org.junit.Before; import org.junit.ClassRule; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; import org.mockito.Mockito; @Category({RPCTests.class, SmallTests.class}) @@ -35,6 +55,20 @@ public class TestCallRunner { public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestCallRunner.class); + @Rule + public TestName testName = new TestName(); + + @Rule + public OpenTelemetryRule otelRule = OpenTelemetryRule.create(); + + private Configuration conf = null; + + @Before + public void before() { + final HBaseTestingUtil util = new HBaseTestingUtil(); + conf = util.getConfiguration(); + } + /** * Does nothing but exercise a {@link CallRunner} outside of {@link RpcServer} context. */ @@ -42,22 +76,37 @@ public class TestCallRunner { public void testSimpleCall() { RpcServerInterface mockRpcServer = Mockito.mock(RpcServerInterface.class); Mockito.when(mockRpcServer.isStarted()).thenReturn(true); - ServerCall mockCall = Mockito.mock(ServerCall.class); - CallRunner cr = new CallRunner(mockRpcServer, mockCall); - cr.setStatus(new MonitoredRPCHandlerImpl()); - cr.run(); + ServerCall mockCall = Mockito.mock(ServerCall.class); + + TraceUtil.trace(() -> { + CallRunner cr = new CallRunner(mockRpcServer, mockCall); + cr.setStatus(new MonitoredRPCHandlerImpl()); + cr.run(); + }, testName.getMethodName()); + + Waiter.waitFor(conf, TimeUnit.SECONDS.toMillis(5), new MatcherPredicate<>( + otelRule::getSpans, hasItem(allOf( + hasName(testName.getMethodName()), + hasEnded())))); + + assertThat(otelRule.getSpans(), hasItem(allOf( + hasName(testName.getMethodName()), + hasStatusWithCode(StatusCode.OK), + hasEnded()))); } @Test public void testCallCleanup() { RpcServerInterface mockRpcServer = Mockito.mock(RpcServerInterface.class); Mockito.when(mockRpcServer.isStarted()).thenReturn(true); - ServerCall mockCall = Mockito.mock(ServerCall.class); + ServerCall mockCall = Mockito.mock(ServerCall.class); Mockito.when(mockCall.disconnectSince()).thenReturn(1L); - CallRunner cr = new CallRunner(mockRpcServer, mockCall); - cr.setStatus(new MonitoredRPCHandlerImpl()); - cr.run(); + TraceUtil.trace(() -> { + CallRunner cr = new CallRunner(mockRpcServer, mockCall); + cr.setStatus(new MonitoredRPCHandlerImpl()); + cr.run(); + }, testName.getMethodName()); Mockito.verify(mockCall, Mockito.times(1)).cleanup(); } @@ -65,13 +114,26 @@ public void testCallCleanup() { public void testCallRunnerDropDisconnected() { RpcServerInterface mockRpcServer = Mockito.mock(RpcServerInterface.class); Mockito.when(mockRpcServer.isStarted()).thenReturn(true); - ServerCall mockCall = Mockito.mock(ServerCall.class); + ServerCall mockCall = Mockito.mock(ServerCall.class); Mockito.when(mockCall.disconnectSince()).thenReturn(1L); - CallRunner cr = new CallRunner(mockRpcServer, mockCall); - cr.setStatus(new MonitoredRPCHandlerImpl()); - cr.drop(); + TraceUtil.trace(() -> { + CallRunner cr = new CallRunner(mockRpcServer, mockCall); + cr.setStatus(new MonitoredRPCHandlerImpl()); + cr.drop(); + }, testName.getMethodName()); Mockito.verify(mockCall, Mockito.times(1)).cleanup(); + + Waiter.waitFor(conf, TimeUnit.SECONDS.toMillis(5), new MatcherPredicate<>( + otelRule::getSpans, hasItem(allOf( + hasName(testName.getMethodName()), + hasEnded())))); + + assertThat(otelRule.getSpans(), hasItem(allOf( + hasName(testName.getMethodName()), + hasStatusWithCode(StatusCode.OK), + hasEvents(hasItem(EventMatchers.hasName("Client disconnect detected"))), + hasEnded()))); } @Test @@ -80,14 +142,31 @@ public void testCallRunnerDropConnected() { MetricsHBaseServer mockMetrics = Mockito.mock(MetricsHBaseServer.class); Mockito.when(mockRpcServer.getMetrics()).thenReturn(mockMetrics); Mockito.when(mockRpcServer.isStarted()).thenReturn(true); - Mockito.when(mockRpcServer.getListenerAddress()).thenReturn(InetSocketAddress.createUnresolved("foo", 60020)); - ServerCall mockCall = Mockito.mock(ServerCall.class); + Mockito.when(mockRpcServer.getListenerAddress()) + .thenReturn(InetSocketAddress.createUnresolved("foo", 60020)); + ServerCall mockCall = Mockito.mock(ServerCall.class); Mockito.when(mockCall.disconnectSince()).thenReturn(-1L); - CallRunner cr = new CallRunner(mockRpcServer, mockCall); - cr.setStatus(new MonitoredRPCHandlerImpl()); - cr.drop(); + TraceUtil.trace(() -> { + CallRunner cr = new CallRunner(mockRpcServer, mockCall); + cr.setStatus(new MonitoredRPCHandlerImpl()); + cr.drop(); + }, testName.getMethodName()); Mockito.verify(mockCall, Mockito.times(1)).cleanup(); Mockito.verify(mockMetrics).exception(Mockito.any(CallDroppedException.class)); + + Waiter.waitFor(conf, TimeUnit.SECONDS.toMillis(5), new MatcherPredicate<>( + otelRule::getSpans, hasItem(allOf( + hasName(testName.getMethodName()), + hasEnded())))); + + assertThat(otelRule.getSpans(), hasItem(allOf( + hasName(testName.getMethodName()), + hasStatusWithCode(StatusCode.ERROR), + hasEvents(hasItem(allOf( + EventMatchers.hasName("exception"), + EventMatchers.hasAttributes( + containsEntry("exception.type", CallDroppedException.class.getName()))))), + hasEnded()))); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java index 3f05e861e3bb..ce283701b47f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -52,7 +52,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdge; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.hadoop.hbase.util.Threads; import org.junit.Before; import org.junit.ClassRule; @@ -782,7 +781,7 @@ public void sendResponseIfReady() throws IOException { } }; - CallRunner cr = new CallRunner(null, putCall) { + return new CallRunner(null, putCall) { @Override public void run() { if (sleepTime <= 0) { @@ -805,7 +804,5 @@ public RpcCall getRpcCall() { public void drop() { } }; - - return cr; } }