Skip to content

Commit

Permalink
HBASE-26759 Fix trace continuity through CallRunner (#4126)
Browse files Browse the repository at this point in the history
Signed-off-by: Andrew Purtell <[email protected]>
Signed-off-by: Duo Zhang <[email protected]>
  • Loading branch information
ndimiduk authored Mar 4, 2022
1 parent be59eb7 commit 2b9ab90
Show file tree
Hide file tree
Showing 9 changed files with 306 additions and 109 deletions.
Original file line number Diff line number Diff line change
@@ -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();
}
};
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -68,6 +69,15 @@ public static Matcher<SpanData> hasEnded() {
};
}

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") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;

/**
Expand All @@ -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.
Expand All @@ -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());
Expand All @@ -89,53 +89,54 @@ 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<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 " +
(address != null ? address : "(channel closed)") + " is not running yet");
}
// 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()) {
RpcServer.LOG.trace(call.toShortString(), e);
}
} 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);
Expand All @@ -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);
}

Expand All @@ -188,41 +192,46 @@ public void run() {
}
this.status.pause("Waiting for a call");
cleanup();
span.end();
}
}

/**
* 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;
}

// Set the response
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();
}
}
}
Original file line number Diff line number Diff line change
@@ -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);
}
}
Loading

0 comments on commit 2b9ab90

Please sign in to comment.