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:
parent
be59eb7e8d
commit
2b9ab90a03
|
@ -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();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -22,6 +22,7 @@ import static org.hamcrest.Matchers.is;
|
|||
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 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) {
|
||||
return new FeatureMatcher<SpanData, SpanKind>(
|
||||
equalTo(kind), "SpanData with kind that", "SpanKind") {
|
||||
|
|
|
@ -22,7 +22,6 @@ import io.opentelemetry.api.trace.StatusCode;
|
|||
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.hbase.util.Pair;
|
|||
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 @@ public class CallRunner {
|
|||
}
|
||||
|
||||
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<User> 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<Message, CellScanner> 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 class CallRunner {
|
|||
}
|
||||
// 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 class CallRunner {
|
|||
}
|
||||
} 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 class CallRunner {
|
|||
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 class CallRunner {
|
|||
}
|
||||
this.status.pause("Waiting for a call");
|
||||
cleanup();
|
||||
span.end();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -195,11 +200,11 @@ public class CallRunner {
|
|||
* 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 class CallRunner {
|
|||
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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.ipc;
|
|||
|
||||
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.Bytes;
|
|||
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 abstract class ServerCall<T extends ServerRpcConnection> implements RpcCa
|
|||
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 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
|
||||
// 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 abstract class ServerCall<T extends ServerRpcConnection> implements RpcCa
|
|||
return response;
|
||||
}
|
||||
}
|
||||
|
||||
public Span getSpan() {
|
||||
return span;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo;
|
|||
justification="False positive according to http://sourceforge.net/p/findbugs/bugs/1032/")
|
||||
@InterfaceAudience.Private
|
||||
abstract class ServerRpcConnection implements Closeable {
|
||||
/** */
|
||||
|
||||
private static final TextMapGetter<RPCTInfo> getter = new RPCTInfoGetter();
|
||||
|
||||
protected final RpcServer rpcServer;
|
||||
// If the connection header has been read or not.
|
||||
protected boolean connectionHeaderRead = false;
|
||||
|
@ -616,22 +618,17 @@ abstract class ServerRpcConnection implements Closeable {
|
|||
ProtobufUtil.mergeFrom(builder, cis, headerSize);
|
||||
RequestHeader header = (RequestHeader) builder.build();
|
||||
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()
|
||||
.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 @@ abstract class ServerRpcConnection implements Closeable {
|
|||
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 @@ abstract class ServerRpcConnection implements Closeable {
|
|||
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 @@ abstract class ServerRpcConnection implements Closeable {
|
|||
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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -25,14 +25,12 @@ import io.opentelemetry.api.common.AttributeKey;
|
|||
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 @@ import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
|
|||
@InterfaceAudience.Private
|
||||
public class IpcServerSpanBuilder implements Supplier<Span> {
|
||||
|
||||
private final RpcCall rpcCall;
|
||||
private String name;
|
||||
private final Map<AttributeKey<?>, 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 class IpcServerSpanBuilder implements Supplier<Span> {
|
|||
.spanBuilder(name)
|
||||
.setSpanKind(SpanKind.SERVER);
|
||||
attributes.forEach((k, v) -> builder.setAttribute((AttributeKey<? super Object>) k, v));
|
||||
return builder.setParent(Context.current().with(((ServerCall<?>) rpcCall).getSpan()))
|
||||
.startSpan();
|
||||
return builder.startSpan();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 class TestCallRunner {
|
|||
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 class TestCallRunner {
|
|||
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())));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.testclassification.RPCTests;
|
|||
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 class TestSimpleRpcScheduler {
|
|||
}
|
||||
};
|
||||
|
||||
CallRunner cr = new CallRunner(null, putCall) {
|
||||
return new CallRunner(null, putCall) {
|
||||
@Override
|
||||
public void run() {
|
||||
if (sleepTime <= 0) {
|
||||
|
@ -805,7 +804,5 @@ public class TestSimpleRpcScheduler {
|
|||
public void drop() {
|
||||
}
|
||||
};
|
||||
|
||||
return cr;
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue