HBASE-26759 Fix trace continuity through CallRunner (#4126)

Signed-off-by: Andrew Purtell <apurtell@apache.org>
Signed-off-by: Duo Zhang <zhangduo@apache.org>
This commit is contained in:
Nick Dimiduk 2022-03-05 00:23:25 +01:00 committed by GitHub
parent be59eb7e8d
commit 2b9ab90a03
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 306 additions and 109 deletions

View File

@ -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<EventData> hasAttributes(Matcher<Attributes> matcher) {
return new FeatureMatcher<EventData, Attributes>(
matcher, "EventData having attributes that ", "attributes") {
@Override protected Attributes featureValueOf(EventData actual) {
return actual.getAttributes();
}
};
}
public static Matcher<EventData> hasName(String name) {
return hasName(equalTo(name));
}
public static Matcher<EventData> hasName(Matcher<String> matcher) {
return new FeatureMatcher<EventData, String>(matcher, "EventData with a name that ", "name") {
@Override protected String featureValueOf(EventData actual) {
return actual.getName();
}
};
}
}

View File

@ -22,6 +22,7 @@ import static org.hamcrest.Matchers.is;
import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.common.Attributes;
import io.opentelemetry.api.trace.SpanKind; import io.opentelemetry.api.trace.SpanKind;
import io.opentelemetry.api.trace.StatusCode; 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.SpanData;
import io.opentelemetry.sdk.trace.data.StatusData; import io.opentelemetry.sdk.trace.data.StatusData;
import java.time.Duration; import java.time.Duration;
@ -68,6 +69,15 @@ public final class SpanDataMatchers {
}; };
} }
public static Matcher<SpanData> hasEvents(Matcher<Iterable<? super EventData>> matcher) {
return new FeatureMatcher<SpanData, Iterable<? super EventData>>(
matcher, "SpanData having events that", "events") {
@Override protected Iterable<? super EventData> featureValueOf(SpanData item) {
return item.getEvents();
}
};
}
public static Matcher<SpanData> hasKind(SpanKind kind) { public static Matcher<SpanData> hasKind(SpanKind kind) {
return new FeatureMatcher<SpanData, SpanKind>( return new FeatureMatcher<SpanData, SpanKind>(
equalTo(kind), "SpanData with kind that", "SpanKind") { equalTo(kind), "SpanData with kind that", "SpanKind") {

View File

@ -22,7 +22,6 @@ import io.opentelemetry.api.trace.StatusCode;
import io.opentelemetry.context.Scope; import io.opentelemetry.context.Scope;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.nio.channels.ClosedChannelException; import java.nio.channels.ClosedChannelException;
import java.util.Optional;
import org.apache.hadoop.hbase.CallDroppedException; import org.apache.hadoop.hbase.CallDroppedException;
import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HBaseInterfaceAudience;
@ -36,7 +35,6 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability; import org.apache.yetus.audience.InterfaceStability;
import org.apache.hbase.thirdparty.com.google.protobuf.Message; import org.apache.hbase.thirdparty.com.google.protobuf.Message;
/** /**
@ -54,7 +52,8 @@ public class CallRunner {
private RpcCall call; private RpcCall call;
private RpcServerInterface rpcServer; private RpcServerInterface rpcServer;
private MonitoredRPCHandler status; 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. * 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) { CallRunner(final RpcServerInterface rpcServer, final RpcCall call) {
this.call = call; this.call = call;
this.rpcServer = rpcServer; this.rpcServer = rpcServer;
this.span = Span.current();
// Add size of the call to queue size. // Add size of the call to queue size.
if (call != null && rpcServer != null) { if (call != null && rpcServer != null) {
this.rpcServer.addCallSize(call.getSize()); this.rpcServer.addCallSize(call.getSize());
@ -89,32 +89,33 @@ public class CallRunner {
} }
public void run() { public void run() {
try { try (Scope ignored = span.makeCurrent()) {
if (call.disconnectSince() >= 0) { if (call.disconnectSince() >= 0) {
if (RpcServer.LOG.isDebugEnabled()) { RpcServer.LOG.debug("{}: skipped {}", Thread.currentThread().getName(), call);
RpcServer.LOG.debug(Thread.currentThread().getName() + ": skipped " + call); span.addEvent("Client disconnect detected");
} span.setStatus(StatusCode.OK);
return; return;
} }
call.setStartTime(EnvironmentEdgeManager.currentTime()); call.setStartTime(EnvironmentEdgeManager.currentTime());
if (call.getStartTime() > call.getDeadline()) { 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(); this.rpcServer.getMetrics().callTimedOut();
span.addEvent("Call deadline exceeded");
span.setStatus(StatusCode.OK);
return; return;
} }
this.status.setStatus("Setting up call"); this.status.setStatus("Setting up call");
this.status.setConnection(call.getRemoteAddress().getHostAddress(), call.getRemotePort()); this.status.setConnection(call.getRemoteAddress().getHostAddress(), call.getRemotePort());
if (RpcServer.LOG.isTraceEnabled()) { if (RpcServer.LOG.isTraceEnabled()) {
Optional<User> remoteUser = call.getRequestUser(); RpcServer.LOG.trace("{} executing as {}", call.toShortString(),
RpcServer.LOG.trace(call.toShortString() + " executing as " + call.getRequestUser().map(User::getName).orElse("NULL principal"));
(remoteUser.isPresent() ? remoteUser.get().getName() : "NULL principal"));
} }
Throwable errorThrowable = null; Throwable errorThrowable = null;
String error = null; String error = null;
Pair<Message, CellScanner> resultPair = null; Pair<Message, CellScanner> resultPair = null;
RpcServer.CurCall.set(call); RpcServer.CurCall.set(call);
Span span = new IpcServerSpanBuilder(call).build(); final Span ipcServerSpan = new IpcServerSpanBuilder(call).build();
try (Scope traceScope = span.makeCurrent()) { try (Scope ignored1 = ipcServerSpan.makeCurrent()) {
if (!this.rpcServer.isStarted()) { if (!this.rpcServer.isStarted()) {
InetSocketAddress address = rpcServer.getListenerAddress(); InetSocketAddress address = rpcServer.getListenerAddress();
throw new ServerNotRunningYetException("Server " + throw new ServerNotRunningYetException("Server " +
@ -122,12 +123,12 @@ public class CallRunner {
} }
// make the call // make the call
resultPair = this.rpcServer.call(call, this.status); resultPair = this.rpcServer.call(call, this.status);
} catch (TimeoutIOException e){ } catch (TimeoutIOException e) {
RpcServer.LOG.warn("Can not complete this request in time, drop it: " + call); RpcServer.LOG.warn("Can not complete this request in time, drop it: {}", call);
TraceUtil.setError(span, e); TraceUtil.setError(ipcServerSpan, e);
return; return;
} catch (Throwable e) { } catch (Throwable e) {
TraceUtil.setError(span, e); TraceUtil.setError(ipcServerSpan, e);
if (e instanceof ServerNotRunningYetException) { if (e instanceof ServerNotRunningYetException) {
// If ServerNotRunningYetException, don't spew stack trace. // If ServerNotRunningYetException, don't spew stack trace.
if (RpcServer.LOG.isTraceEnabled()) { if (RpcServer.LOG.isTraceEnabled()) {
@ -135,7 +136,7 @@ public class CallRunner {
} }
} else { } else {
// Don't dump full exception.. just String version // Don't dump full exception.. just String version
RpcServer.LOG.debug(call.toShortString() + ", exception=" + e); RpcServer.LOG.debug("{}, exception={}", call.toShortString(), e);
} }
errorThrowable = e; errorThrowable = e;
error = StringUtils.stringifyException(e); error = StringUtils.stringifyException(e);
@ -146,40 +147,43 @@ public class CallRunner {
RpcServer.CurCall.set(null); RpcServer.CurCall.set(null);
if (resultPair != null) { if (resultPair != null) {
this.rpcServer.addCallSize(call.getSize() * -1); this.rpcServer.addCallSize(call.getSize() * -1);
span.setStatus(StatusCode.OK); ipcServerSpan.setStatus(StatusCode.OK);
sucessful = true; successful = true;
} }
span.end(); ipcServerSpan.end();
} }
this.status.markComplete("To send response"); 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(); call.cleanup();
// Set the response // Set the response
Message param = resultPair != null ? resultPair.getFirst() : null; Message param = resultPair != null ? resultPair.getFirst() : null;
CellScanner cells = resultPair != null ? resultPair.getSecond() : null; CellScanner cells = resultPair != null ? resultPair.getSecond() : null;
call.setResponse(param, cells, errorThrowable, error); call.setResponse(param, cells, errorThrowable, error);
call.sendResponseIfReady(); call.sendResponseIfReady();
// don't touch `span` here because its status and `end()` are managed in `call#setResponse()`
} catch (OutOfMemoryError e) { } catch (OutOfMemoryError e) {
if (this.rpcServer.getErrorHandler() != null) { TraceUtil.setError(span, e);
if (this.rpcServer.getErrorHandler().checkOOME(e)) { if (this.rpcServer.getErrorHandler() != null
RpcServer.LOG.info(Thread.currentThread().getName() + ": exiting on OutOfMemoryError"); && this.rpcServer.getErrorHandler().checkOOME(e)) {
return; RpcServer.LOG.info("{}: exiting on OutOfMemoryError", Thread.currentThread().getName());
} // exception intentionally swallowed
} else { } else {
// rethrow if no handler // rethrow if no handler
throw e; throw e;
} }
} catch (ClosedChannelException cce) { } catch (ClosedChannelException cce) {
InetSocketAddress address = rpcServer.getListenerAddress(); 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)") + "this means that the server " + (address != null ? address : "(channel closed)") +
" was processing a request but the client went away. The error message was: " + " was processing a request but the client went away. The error message was: {}",
cce.getMessage()); Thread.currentThread().getName(), cce.getMessage());
TraceUtil.setError(span, cce);
} catch (Exception e) { } catch (Exception e) {
RpcServer.LOG.warn(Thread.currentThread().getName() RpcServer.LOG.warn("{}: caught: {}", Thread.currentThread().getName(),
+ ": caught: " + StringUtils.stringifyException(e)); StringUtils.stringifyException(e));
TraceUtil.setError(span, e);
} finally { } finally {
if (!sucessful) { if (!successful) {
this.rpcServer.addCallSize(call.getSize() * -1); this.rpcServer.addCallSize(call.getSize() * -1);
} }
@ -188,6 +192,7 @@ public class CallRunner {
} }
this.status.pause("Waiting for a call"); this.status.pause("Waiting for a call");
cleanup(); cleanup();
span.end();
} }
} }
@ -195,11 +200,11 @@ public class CallRunner {
* When we want to drop this call because of server is overloaded. * When we want to drop this call because of server is overloaded.
*/ */
public void drop() { public void drop() {
try { try (Scope ignored = span.makeCurrent()) {
if (call.disconnectSince() >= 0) { if (call.disconnectSince() >= 0) {
if (RpcServer.LOG.isDebugEnabled()) { RpcServer.LOG.debug("{}: skipped {}", Thread.currentThread().getName(), call);
RpcServer.LOG.debug(Thread.currentThread().getName() + ": skipped " + call); span.addEvent("Client disconnect detected");
} span.setStatus(StatusCode.OK);
return; return;
} }
@ -207,22 +212,26 @@ public class CallRunner {
InetSocketAddress address = rpcServer.getListenerAddress(); InetSocketAddress address = rpcServer.getListenerAddress();
call.setResponse(null, null, CALL_DROPPED_EXCEPTION, "Call dropped, server " call.setResponse(null, null, CALL_DROPPED_EXCEPTION, "Call dropped, server "
+ (address != null ? address : "(channel closed)") + " is overloaded, please retry."); + (address != null ? address : "(channel closed)") + " is overloaded, please retry.");
TraceUtil.setError(span, CALL_DROPPED_EXCEPTION);
call.sendResponseIfReady(); call.sendResponseIfReady();
this.rpcServer.getMetrics().exception(CALL_DROPPED_EXCEPTION); this.rpcServer.getMetrics().exception(CALL_DROPPED_EXCEPTION);
} catch (ClosedChannelException cce) { } catch (ClosedChannelException cce) {
InetSocketAddress address = rpcServer.getListenerAddress(); 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)") + "this means that the server " + (address != null ? address : "(channel closed)") +
" was processing a request but the client went away. The error message was: " + " was processing a request but the client went away. The error message was: {}",
cce.getMessage()); Thread.currentThread().getName(), cce.getMessage());
TraceUtil.setError(span, cce);
} catch (Exception e) { } catch (Exception e) {
RpcServer.LOG.warn(Thread.currentThread().getName() RpcServer.LOG.warn("{}: caught: {}", Thread.currentThread().getName(),
+ ": caught: " + StringUtils.stringifyException(e)); StringUtils.stringifyException(e));
TraceUtil.setError(span, e);
} finally { } finally {
if (!sucessful) { if (!successful) {
this.rpcServer.addCallSize(call.getSize() * -1); this.rpcServer.addCallSize(call.getSize() * -1);
} }
cleanup(); cleanup();
span.end();
} }
} }
} }

View File

@ -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<TracingProtos.RPCTInfo> {
RPCTInfoGetter() { }
@Override
public Iterable<String> 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);
}
}

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.ipc;
import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.Span;
import io.opentelemetry.api.trace.StatusCode; import io.opentelemetry.api.trace.StatusCode;
import io.opentelemetry.context.Scope;
import java.io.IOException; import java.io.IOException;
import java.net.InetAddress; import java.net.InetAddress;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -39,12 +40,10 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; 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.CodedOutputStream;
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor; import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
import org.apache.hbase.thirdparty.com.google.protobuf.Message; import org.apache.hbase.thirdparty.com.google.protobuf.Message;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 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.HBaseProtos.VersionInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta;
@ -222,8 +221,7 @@ public abstract class ServerCall<T extends ServerRpcConnection> implements RpcCa
return "callId: " + this.id + " service: " + serviceName + return "callId: " + this.id + " service: " + serviceName +
" methodName: " + ((this.md != null) ? this.md.getName() : "n/a") + " methodName: " + ((this.md != null) ? this.md.getName() : "n/a") +
" size: " + StringUtils.TraditionalBinaryPrefix.long2String(this.size, "", 1) + " size: " + StringUtils.TraditionalBinaryPrefix.long2String(this.size, "", 1) +
" connection: " + connection.toString() + " connection: " + connection + " deadline: " + deadline;
" deadline: " + deadline;
} }
@Override @Override
@ -299,11 +297,12 @@ public abstract class ServerCall<T extends ServerRpcConnection> implements RpcCa
// Once a response message is created and set to this.response, this Call can be treated as // 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. // done. The Responder thread will do the n/w write of this message back to client.
if (this.rpcCallback != null) { if (this.rpcCallback != null) {
try { try (Scope ignored = span.makeCurrent()) {
this.rpcCallback.run(); this.rpcCallback.run();
} catch (Exception e) { } catch (Exception e) {
// Don't allow any exception here to kill this handler thread. // Don't allow any exception here to kill this handler thread.
RpcServer.LOG.warn("Exception while running the Rpc Callback.", e); RpcServer.LOG.warn("Exception while running the Rpc Callback.", e);
TraceUtil.setError(span, e);
} }
} }
} }
@ -571,8 +570,4 @@ public abstract class ServerCall<T extends ServerRpcConnection> implements RpcCa
return response; return response;
} }
} }
public Span getSpan() {
return span;
}
} }

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -94,7 +94,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo;
justification="False positive according to http://sourceforge.net/p/findbugs/bugs/1032/") justification="False positive according to http://sourceforge.net/p/findbugs/bugs/1032/")
@InterfaceAudience.Private @InterfaceAudience.Private
abstract class ServerRpcConnection implements Closeable { abstract class ServerRpcConnection implements Closeable {
/** */
private static final TextMapGetter<RPCTInfo> getter = new RPCTInfoGetter();
protected final RpcServer rpcServer; protected final RpcServer rpcServer;
// If the connection header has been read or not. // If the connection header has been read or not.
protected boolean connectionHeaderRead = false; protected boolean connectionHeaderRead = false;
@ -616,22 +618,17 @@ abstract class ServerRpcConnection implements Closeable {
ProtobufUtil.mergeFrom(builder, cis, headerSize); ProtobufUtil.mergeFrom(builder, cis, headerSize);
RequestHeader header = (RequestHeader) builder.build(); RequestHeader header = (RequestHeader) builder.build();
offset += headerSize; offset += headerSize;
TextMapGetter<RPCTInfo> getter = new TextMapGetter<RPCTInfo>() {
@Override
public Iterable<String> 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() Context traceCtx = GlobalOpenTelemetry.getPropagators().getTextMapPropagator()
.extract(Context.current(), header.getTraceInfo(), getter); .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); Span span = TraceUtil.createRemoteSpan("RpcServer.process", traceCtx);
try (Scope scope = span.makeCurrent()) { try (Scope ignored = span.makeCurrent()) {
int id = header.getCallId(); int id = header.getCallId();
if (RpcServer.LOG.isTraceEnabled()) { if (RpcServer.LOG.isTraceEnabled()) {
RpcServer.LOG.trace("RequestHeader " + TextFormat.shortDebugString(header) + RpcServer.LOG.trace("RequestHeader " + TextFormat.shortDebugString(header) +
@ -648,6 +645,7 @@ abstract class ServerRpcConnection implements Closeable {
callTooBig.setResponse(null, null, RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION, callTooBig.setResponse(null, null, RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION,
"Call queue is full on " + this.rpcServer.server.getServerName() + "Call queue is full on " + this.rpcServer.server.getServerName() +
", is hbase.ipc.server.max.callqueue.size too small?"); ", is hbase.ipc.server.max.callqueue.size too small?");
TraceUtil.setError(span, RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION);
callTooBig.sendResponseIfReady(); callTooBig.sendResponseIfReady();
return; return;
} }
@ -684,27 +682,30 @@ abstract class ServerRpcConnection implements Closeable {
cellScanner = this.rpcServer.cellBlockBuilder.createCellScannerReusingBuffers(this.codec, cellScanner = this.rpcServer.cellBlockBuilder.createCellScannerReusingBuffers(this.codec,
this.compressionCodec, dup); this.compressionCodec, dup);
} }
} catch (Throwable t) { } catch (Throwable thrown) {
InetSocketAddress address = this.rpcServer.getListenerAddress(); InetSocketAddress address = this.rpcServer.getListenerAddress();
String msg = (address != null ? address : "(channel closed)") + String msg = (address != null ? address : "(channel closed)") +
" is unable to read call parameter from client " + getHostAddress(); " 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 final Throwable responseThrowable;
// version if (thrown instanceof LinkageError) {
if (t instanceof LinkageError) { // probably the hbase hadoop version does not match the running hadoop version
t = new DoNotRetryIOException(t); responseThrowable = new DoNotRetryIOException(thrown);
} } else if (thrown instanceof UnsupportedOperationException) {
// If the method is not present on the server, do not retry. // If the method is not present on the server, do not retry.
if (t instanceof UnsupportedOperationException) { responseThrowable = new DoNotRetryIOException(thrown);
t = new DoNotRetryIOException(t); } else {
responseThrowable = thrown;
} }
ServerCall<?> readParamsFailedCall = createCall(id, this.service, null, null, null, null, ServerCall<?> readParamsFailedCall = createCall(id, this.service, null, null, null, null,
totalRequestSize, null, 0, this.callCleanup); 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(); readParamsFailedCall.sendResponseIfReady();
return; return;
} }
@ -716,14 +717,22 @@ abstract class ServerRpcConnection implements Closeable {
ServerCall<?> call = createCall(id, this.service, md, header, param, cellScanner, ServerCall<?> call = createCall(id, this.service, md, header, param, cellScanner,
totalRequestSize, this.addr, timeout, this.callCleanup); 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.callQueueSizeInBytes.add(-1 * call.getSize());
this.rpcServer.metrics.exception(RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION); this.rpcServer.metrics.exception(RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION);
call.setResponse(null, null, 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() + "Call queue is full on " + this.rpcServer.server.getServerName() +
", too many items queued ?"); ", too many items queued ?");
TraceUtil.setError(span, RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION);
call.sendResponseIfReady(); call.sendResponseIfReady();
} }
} finally {
if (span != null) {
span.end();
}
} }
} }

View File

@ -25,14 +25,12 @@ import io.opentelemetry.api.common.AttributeKey;
import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.Span;
import io.opentelemetry.api.trace.SpanBuilder; import io.opentelemetry.api.trace.SpanBuilder;
import io.opentelemetry.api.trace.SpanKind; import io.opentelemetry.api.trace.SpanKind;
import io.opentelemetry.context.Context;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.function.Supplier; import java.util.function.Supplier;
import org.apache.hadoop.hbase.client.trace.IpcClientSpanBuilder; import org.apache.hadoop.hbase.client.trace.IpcClientSpanBuilder;
import org.apache.hadoop.hbase.ipc.RpcCall; 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.HBaseSemanticAttributes.RpcSystem;
import org.apache.hadoop.hbase.trace.TraceUtil; import org.apache.hadoop.hbase.trace.TraceUtil;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
@ -46,12 +44,10 @@ import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
@InterfaceAudience.Private @InterfaceAudience.Private
public class IpcServerSpanBuilder implements Supplier<Span> { public class IpcServerSpanBuilder implements Supplier<Span> {
private final RpcCall rpcCall;
private String name; private String name;
private final Map<AttributeKey<?>, Object> attributes = new HashMap<>(); private final Map<AttributeKey<?>, Object> attributes = new HashMap<>();
public IpcServerSpanBuilder(final RpcCall rpcCall) { public IpcServerSpanBuilder(final RpcCall rpcCall) {
this.rpcCall = rpcCall;
final String packageAndService = Optional.ofNullable(rpcCall.getService()) final String packageAndService = Optional.ofNullable(rpcCall.getService())
.map(BlockingService::getDescriptorForType) .map(BlockingService::getDescriptorForType)
.map(IpcClientSpanBuilder::getRpcPackageAndService) .map(IpcClientSpanBuilder::getRpcPackageAndService)
@ -86,7 +82,6 @@ public class IpcServerSpanBuilder implements Supplier<Span> {
.spanBuilder(name) .spanBuilder(name)
.setSpanKind(SpanKind.SERVER); .setSpanKind(SpanKind.SERVER);
attributes.forEach((k, v) -> builder.setAttribute((AttributeKey<? super Object>) k, v)); attributes.forEach((k, v) -> builder.setAttribute((AttributeKey<? super Object>) k, v));
return builder.setParent(Context.current().with(((ServerCall<?>) rpcCall).getSpan())) return builder.startSpan();
.startSpan();
} }
} }

View File

@ -17,15 +17,35 @@
*/ */
package org.apache.hadoop.hbase.ipc; 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.net.InetSocketAddress;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CallDroppedException; import org.apache.hadoop.hbase.CallDroppedException;
import org.apache.hadoop.hbase.HBaseClassTestRule; 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.monitoring.MonitoredRPCHandlerImpl;
import org.apache.hadoop.hbase.testclassification.RPCTests; import org.apache.hadoop.hbase.testclassification.RPCTests;
import org.apache.hadoop.hbase.testclassification.SmallTests; 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.ClassRule;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.mockito.Mockito; import org.mockito.Mockito;
@Category({RPCTests.class, SmallTests.class}) @Category({RPCTests.class, SmallTests.class})
@ -35,6 +55,20 @@ public class TestCallRunner {
public static final HBaseClassTestRule CLASS_RULE = public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestCallRunner.class); 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. * Does nothing but exercise a {@link CallRunner} outside of {@link RpcServer} context.
*/ */
@ -42,22 +76,37 @@ public class TestCallRunner {
public void testSimpleCall() { public void testSimpleCall() {
RpcServerInterface mockRpcServer = Mockito.mock(RpcServerInterface.class); RpcServerInterface mockRpcServer = Mockito.mock(RpcServerInterface.class);
Mockito.when(mockRpcServer.isStarted()).thenReturn(true); Mockito.when(mockRpcServer.isStarted()).thenReturn(true);
ServerCall mockCall = Mockito.mock(ServerCall.class); ServerCall<?> mockCall = Mockito.mock(ServerCall.class);
TraceUtil.trace(() -> {
CallRunner cr = new CallRunner(mockRpcServer, mockCall); CallRunner cr = new CallRunner(mockRpcServer, mockCall);
cr.setStatus(new MonitoredRPCHandlerImpl()); cr.setStatus(new MonitoredRPCHandlerImpl());
cr.run(); 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 @Test
public void testCallCleanup() { public void testCallCleanup() {
RpcServerInterface mockRpcServer = Mockito.mock(RpcServerInterface.class); RpcServerInterface mockRpcServer = Mockito.mock(RpcServerInterface.class);
Mockito.when(mockRpcServer.isStarted()).thenReturn(true); Mockito.when(mockRpcServer.isStarted()).thenReturn(true);
ServerCall mockCall = Mockito.mock(ServerCall.class); ServerCall<?> mockCall = Mockito.mock(ServerCall.class);
Mockito.when(mockCall.disconnectSince()).thenReturn(1L); Mockito.when(mockCall.disconnectSince()).thenReturn(1L);
TraceUtil.trace(() -> {
CallRunner cr = new CallRunner(mockRpcServer, mockCall); CallRunner cr = new CallRunner(mockRpcServer, mockCall);
cr.setStatus(new MonitoredRPCHandlerImpl()); cr.setStatus(new MonitoredRPCHandlerImpl());
cr.run(); cr.run();
}, testName.getMethodName());
Mockito.verify(mockCall, Mockito.times(1)).cleanup(); Mockito.verify(mockCall, Mockito.times(1)).cleanup();
} }
@ -65,13 +114,26 @@ public class TestCallRunner {
public void testCallRunnerDropDisconnected() { public void testCallRunnerDropDisconnected() {
RpcServerInterface mockRpcServer = Mockito.mock(RpcServerInterface.class); RpcServerInterface mockRpcServer = Mockito.mock(RpcServerInterface.class);
Mockito.when(mockRpcServer.isStarted()).thenReturn(true); Mockito.when(mockRpcServer.isStarted()).thenReturn(true);
ServerCall mockCall = Mockito.mock(ServerCall.class); ServerCall<?> mockCall = Mockito.mock(ServerCall.class);
Mockito.when(mockCall.disconnectSince()).thenReturn(1L); Mockito.when(mockCall.disconnectSince()).thenReturn(1L);
TraceUtil.trace(() -> {
CallRunner cr = new CallRunner(mockRpcServer, mockCall); CallRunner cr = new CallRunner(mockRpcServer, mockCall);
cr.setStatus(new MonitoredRPCHandlerImpl()); cr.setStatus(new MonitoredRPCHandlerImpl());
cr.drop(); cr.drop();
}, testName.getMethodName());
Mockito.verify(mockCall, Mockito.times(1)).cleanup(); 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 @Test
@ -80,14 +142,31 @@ public class TestCallRunner {
MetricsHBaseServer mockMetrics = Mockito.mock(MetricsHBaseServer.class); MetricsHBaseServer mockMetrics = Mockito.mock(MetricsHBaseServer.class);
Mockito.when(mockRpcServer.getMetrics()).thenReturn(mockMetrics); Mockito.when(mockRpcServer.getMetrics()).thenReturn(mockMetrics);
Mockito.when(mockRpcServer.isStarted()).thenReturn(true); Mockito.when(mockRpcServer.isStarted()).thenReturn(true);
Mockito.when(mockRpcServer.getListenerAddress()).thenReturn(InetSocketAddress.createUnresolved("foo", 60020)); Mockito.when(mockRpcServer.getListenerAddress())
ServerCall mockCall = Mockito.mock(ServerCall.class); .thenReturn(InetSocketAddress.createUnresolved("foo", 60020));
ServerCall<?> mockCall = Mockito.mock(ServerCall.class);
Mockito.when(mockCall.disconnectSince()).thenReturn(-1L); Mockito.when(mockCall.disconnectSince()).thenReturn(-1L);
TraceUtil.trace(() -> {
CallRunner cr = new CallRunner(mockRpcServer, mockCall); CallRunner cr = new CallRunner(mockRpcServer, mockCall);
cr.setStatus(new MonitoredRPCHandlerImpl()); cr.setStatus(new MonitoredRPCHandlerImpl());
cr.drop(); cr.drop();
}, testName.getMethodName());
Mockito.verify(mockCall, Mockito.times(1)).cleanup(); Mockito.verify(mockCall, Mockito.times(1)).cleanup();
Mockito.verify(mockMetrics).exception(Mockito.any(CallDroppedException.class)); 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())));
} }
} }

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -52,7 +52,6 @@ import org.apache.hadoop.hbase.testclassification.RPCTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdge; import org.apache.hadoop.hbase.util.EnvironmentEdge;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
import org.junit.Before; import org.junit.Before;
import org.junit.ClassRule; import org.junit.ClassRule;
@ -782,7 +781,7 @@ public class TestSimpleRpcScheduler {
} }
}; };
CallRunner cr = new CallRunner(null, putCall) { return new CallRunner(null, putCall) {
@Override @Override
public void run() { public void run() {
if (sleepTime <= 0) { if (sleepTime <= 0) {
@ -805,7 +804,5 @@ public class TestSimpleRpcScheduler {
public void drop() { public void drop() {
} }
}; };
return cr;
} }
} }