-
Notifications
You must be signed in to change notification settings - Fork 439
RATIS-2393 Add Span Context to RaftRpcRequestProto #1341
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,6 +24,7 @@ | |
| import org.apache.ratis.proto.RaftProtos.ReadRequestTypeProto; | ||
| import org.apache.ratis.proto.RaftProtos.ReplicationLevel; | ||
| import org.apache.ratis.proto.RaftProtos.SlidingWindowEntry; | ||
| import org.apache.ratis.proto.RaftProtos.SpanContextProto; | ||
| import org.apache.ratis.proto.RaftProtos.StaleReadRequestTypeProto; | ||
| import org.apache.ratis.proto.RaftProtos.WatchRequestTypeProto; | ||
| import org.apache.ratis.proto.RaftProtos.WriteRequestTypeProto; | ||
|
|
@@ -305,6 +306,7 @@ public static class Builder { | |
| private SlidingWindowEntry slidingWindowEntry; | ||
| private RoutingTable routingTable; | ||
| private long timeoutMs; | ||
| private SpanContextProto spanContext; | ||
|
|
||
| public RaftClientRequest build() { | ||
| return new RaftClientRequest(this); | ||
|
|
@@ -366,6 +368,11 @@ public Builder setTimeoutMs(long timeoutMs) { | |
| this.timeoutMs = timeoutMs; | ||
| return this; | ||
| } | ||
|
|
||
| public Builder setSpanContext(SpanContextProto spanContext) { | ||
| this.spanContext = spanContext; | ||
| return this; | ||
| } | ||
| } | ||
|
|
||
| public static Builder newBuilder() { | ||
|
|
@@ -397,6 +404,8 @@ public static RaftClientRequest toWriteRequest(RaftClientRequest r, Message mess | |
|
|
||
| private final boolean toLeader; | ||
|
|
||
| private SpanContextProto spanContext; | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please add |
||
|
|
||
| /** Construct a request for sending to the given server. */ | ||
| protected RaftClientRequest(ClientId clientId, RaftPeerId serverId, RaftGroupId groupId, long callId, Type type) { | ||
| this(newBuilder() | ||
|
|
@@ -429,6 +438,7 @@ private RaftClientRequest(Builder b) { | |
| this.slidingWindowEntry = b.slidingWindowEntry; | ||
| this.routingTable = b.routingTable; | ||
| this.timeoutMs = b.timeoutMs; | ||
| this.spanContext = b.spanContext; | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -472,6 +482,10 @@ public long getTimeoutMs() { | |
| return timeoutMs; | ||
| } | ||
|
|
||
| public SpanContextProto getSpanContext() { | ||
| return spanContext; | ||
| } | ||
|
|
||
| @Override | ||
| public String toString() { | ||
| return super.toString() + ", seq=" + ProtoUtils.toString(slidingWindowEntry) + ", " | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,34 @@ | ||
| /* | ||
| * 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.ratis.trace; | ||
|
|
||
| import io.opentelemetry.api.common.AttributeKey; | ||
|
|
||
| /** | ||
| * The constants in this class correspond with the guidance outlined by the OpenTelemetry <a href= | ||
| * "https://github.com/open-telemetry/semantic-conventions">Semantic | ||
| * Conventions</a>. | ||
| */ | ||
| public final class RatisAttributes { | ||
| public static final AttributeKey<String> ATTR_MEMBER_ID = AttributeKey.stringKey("raft.member.id"); | ||
| public static final AttributeKey<String> ATTR_CALLER_ID = AttributeKey.stringKey("raft.caller.id"); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The callId is unique only within a single client. We should use |
||
|
|
||
|
|
||
| private RatisAttributes() { | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,85 @@ | ||
| /* | ||
| * 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.ratis.trace; | ||
|
|
||
| import io.opentelemetry.api.GlobalOpenTelemetry; | ||
| import io.opentelemetry.api.trace.Span; | ||
| import io.opentelemetry.api.trace.SpanKind; | ||
| import io.opentelemetry.api.trace.Tracer; | ||
| import io.opentelemetry.context.Context; | ||
| import io.opentelemetry.context.propagation.TextMapPropagator; | ||
| import io.opentelemetry.context.propagation.TextMapGetter; | ||
| import org.apache.ratis.proto.RaftProtos; | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Import SpanContextProto instead. The code will be easier to read. import org.apache.ratis.proto.RaftProtos.SpanContextProto; |
||
| import org.apache.ratis.util.VersionInfo; | ||
|
|
||
| import java.util.HashMap; | ||
| import java.util.Map; | ||
| import java.util.Optional; | ||
|
|
||
| public final class TraceUtils { | ||
|
|
||
| private TraceUtils() { | ||
| } | ||
|
|
||
| public static Tracer getGlobalTracer() { | ||
| return GlobalOpenTelemetry.getTracer("org.apache.ratis", VersionInfo.getSoftwareInfoVersion()); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Add a static field, so we don't have to create the same Tracer multiple times. private static final Tracer TRACER = GlobalOpenTelemetry.getTracer("org.apache.ratis", VersionInfo.getSoftwareInfoVersion()); |
||
| } | ||
|
|
||
| /** | ||
| * Create a span which parent is from remote, i.e, passed through rpc. | ||
| * </p> | ||
| * We will set the kind of the returned span to {@link SpanKind#SERVER}, as this should be the top | ||
| * most span at server side. | ||
| */ | ||
| public static Span createRemoteSpan(String name, Context ctx) { | ||
| return getGlobalTracer().spanBuilder(name).setParent(ctx).setSpanKind(SpanKind.SERVER) | ||
| .startSpan(); | ||
| } | ||
|
|
||
| private static final TextMapPropagator PROPAGATOR = | ||
| GlobalOpenTelemetry.getPropagators().getTextMapPropagator(); | ||
|
|
||
| public static RaftProtos.SpanContextProto injectContextToProto(Context context) { | ||
| Map<String, String> carrier = new HashMap<>(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For String, would TreeMap be better? It will also sort them. |
||
| PROPAGATOR.inject(context, carrier, (map, key, value) -> map.put(key, value)); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Question: What exactly will be injected, ATTR_MEMBER_ID and ATTR_CALLER_ID? Anything else? |
||
| return RaftProtos.SpanContextProto.newBuilder().putAllContext(carrier).build(); | ||
| } | ||
|
|
||
| public static Context extractContextFromProto(RaftProtos.SpanContextProto proto) { | ||
| if (proto == null || proto.getContextMap().isEmpty()) { | ||
| return Context.current(); | ||
| } | ||
| final TextMapGetter<RaftProtos.SpanContextProto> getter = SpanContextGetter.INSTANCE; | ||
| return PROPAGATOR.extract(Context.current(), proto, getter); | ||
| } | ||
| } | ||
|
|
||
| class SpanContextGetter implements TextMapGetter<RaftProtos.SpanContextProto> { | ||
| static final SpanContextGetter INSTANCE = new SpanContextGetter(); | ||
|
|
||
| @Override | ||
| public Iterable<String> keys(RaftProtos.SpanContextProto carrier) { | ||
| return carrier.getContextMap().keySet(); | ||
| } | ||
|
|
||
| @Override | ||
| public String get(RaftProtos.SpanContextProto carrier, String key) { | ||
| return Optional.ofNullable(carrier).map(RaftProtos.SpanContextProto::getContextMap) | ||
| .map(map -> map.get(key)).orElse(null); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,6 +17,10 @@ | |
| */ | ||
| package org.apache.ratis.server.impl; | ||
|
|
||
| import io.opentelemetry.api.trace.Span; | ||
| import io.opentelemetry.api.trace.StatusCode; | ||
| import io.opentelemetry.context.Context; | ||
| import io.opentelemetry.context.Scope; | ||
| import org.apache.ratis.client.impl.ClientProtoUtils; | ||
| import org.apache.ratis.conf.RaftProperties; | ||
| import org.apache.ratis.metrics.Timekeeper; | ||
|
|
@@ -100,6 +104,8 @@ | |
| import org.apache.ratis.statemachine.impl.TransactionContextImpl; | ||
| import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; | ||
| import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; | ||
| import org.apache.ratis.trace.RatisAttributes; | ||
| import org.apache.ratis.trace.TraceUtils; | ||
| import org.apache.ratis.util.CodeInjectionForTesting; | ||
| import org.apache.ratis.util.CollectionUtils; | ||
| import org.apache.ratis.util.ConcurrentUtils; | ||
|
|
@@ -943,16 +949,44 @@ CompletableFuture<RaftClientReply> executeSubmitClientRequestAsync(RaftClientReq | |
| @Override | ||
| public CompletableFuture<RaftClientReply> submitClientRequestAsync( | ||
| RaftClientRequest request) throws IOException { | ||
| assertLifeCycleState(LifeCycle.States.RUNNING); | ||
| LOG.debug("{}: receive client request({})", getMemberId(), request); | ||
| final Timekeeper timer = raftServerMetrics.getClientRequestTimer(request.getType()); | ||
| final Optional<Timekeeper.Context> timerContext = Optional.ofNullable(timer).map(Timekeeper::time); | ||
| return replyFuture(request).whenComplete((clientReply, exception) -> { | ||
| timerContext.ifPresent(Timekeeper.Context::stop); | ||
| if (exception != null || clientReply.getException() != null) { | ||
| raftServerMetrics.incFailedRequestCount(request.getType()); | ||
| } | ||
| }); | ||
| final Context remoteContext = TraceUtils.extractContextFromProto(request.getSpanContext()); | ||
| final Span span = TraceUtils.createRemoteSpan("raft.server.submitClientRequestAsync", remoteContext); | ||
| span.setAttribute(RatisAttributes.ATTR_MEMBER_ID, getMemberId().toString()); | ||
| span.setAttribute(RatisAttributes.ATTR_CALLER_ID, String.valueOf(request.getCallId())); | ||
| try (Scope ignored = span.makeCurrent()) { | ||
| assertLifeCycleState(LifeCycle.States.RUNNING); | ||
| LOG.debug("{}: receive client request({})", getMemberId(), request); | ||
| final Timekeeper timer = raftServerMetrics.getClientRequestTimer(request.getType()); | ||
| final Optional<Timekeeper.Context> timerContext = Optional.ofNullable(timer).map(Timekeeper::time); | ||
| span.addEvent("Processing client request"); | ||
| final CompletableFuture<RaftClientReply> future = replyFuture(request); | ||
| return future.whenComplete((clientReply, exception) -> { | ||
| try (Scope completeScope = span.makeCurrent()) { | ||
| timerContext.ifPresent(Timekeeper.Context::stop); | ||
| if (exception != null || (clientReply != null && clientReply.getException() != null)) { | ||
| raftServerMetrics.incFailedRequestCount(request.getType()); | ||
| } | ||
| if (exception != null) { | ||
| span.recordException(exception); | ||
| span.setStatus(StatusCode.ERROR, exception.getMessage()); | ||
| } else if (clientReply != null && clientReply.getException() != null) { | ||
| span.recordException(clientReply.getException()); | ||
| span.setStatus(StatusCode.ERROR, clientReply.getException().getMessage()); | ||
| } | ||
| } finally { | ||
| span.addEvent("Completed client request"); | ||
| span.end(); | ||
| } | ||
| }); | ||
| } catch (Exception | Error e) { | ||
| // this catch block is for exceptions thrown before the future is returned. | ||
| // Any exception thrown after the future is returned should be handled in | ||
| // the whenComplete callback above. | ||
| span.recordException(e); | ||
| span.setStatus(StatusCode.ERROR, e.getMessage()); | ||
| span.end(); | ||
| throw e; | ||
| } | ||
|
Comment on lines
+952
to
+989
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let add traceAsyncMethod(..) to TraceUtils and move the opentelemetry code there. |
||
| } | ||
|
|
||
| private CompletableFuture<RaftClientReply> replyFuture(RaftClientRequest request) throws IOException { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
How about using the latest 1.59.0 and 1.40.0?