kvserver: instrument snapshot flow with tracing and timing#83003
kvserver: instrument snapshot flow with tracing and timing#83003craig[bot] merged 1 commit intocockroachdb:masterfrom
Conversation
irfansharif
left a comment
There was a problem hiding this comment.
Cool! This would be worth pairing with a cluster setting to log these traces if the snapshot duration exceeds some latency threshold, defaulting this setting to disabled. This would give you a way to collect traces on the fly for a running cluster. I'll leave the actual review to Alex.
pkg/kv/kvserver/store_snapshot.go
Outdated
| batchDuration := batchStopwatch.Elapsed().Seconds() | ||
| recvDuration := recvStopwatch.Elapsed().Seconds() | ||
|
|
||
| log.Infof( |
There was a problem hiding this comment.
Will this be spammy for every snapshot? Instead of log.Infof, stashing these in the traces alone seem good enough.
pkg/kv/kvserver/store_snapshot.go
Outdated
| defer msstw.Close() | ||
| msstwStopwatch.Stop() | ||
|
|
||
| log.Event(ctx, "Waiting for snapshot batches to begin.") |
There was a problem hiding this comment.
[drive by nit] We generally avoid capitalization at the start of these logs + the periods at the end since every particular message is only a fragment of a larger trace (so not quite a sentence).
b604430 to
98414de
Compare
98414de to
d563ace
Compare
ryanhzhao
left a comment
There was a problem hiding this comment.
Thank you for the comments Irfan! I wanted to clarify one thing. By "pairing with a cluster setting to log these traces", do you mean that all of these traces should only be recorded when the snapshot duration is longer than some latency? Or do you mean that we should always collect these traces, but the information would only get written to the log (e.g. log.Infof) when the latency exceeds a certain threshold?
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks and @irfansharif)
pkg/kv/kvserver/store_snapshot.go line 301 at r1 (raw file):
Previously, irfansharif (irfan sharif) wrote…
[drive by nit] We generally avoid capitalization at the start of these logs + the periods at the end since every particular message is only a fragment of a larger trace (so not quite a sentence).
Done.
pkg/kv/kvserver/store_snapshot.go line 377 at r1 (raw file):
Previously, irfansharif (irfan sharif) wrote…
Will this be spammy for every snapshot? Instead of log.Infof, stashing these in the traces alone seem good enough.
Done.
irfansharif
left a comment
There was a problem hiding this comment.
By "pairing with a cluster setting to log these traces", do you mean that all of these traces should only be recorded when the snapshot duration is longer than some latency?
I mean doing something like sql.trace.stmt.enable_threshold, which is "if set to 0, don't trace; if set to > 0, trace every subsequent operation, and write to log traces that exceed the specified latency target".
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks and @irfansharif)
AlexTalks
left a comment
There was a problem hiding this comment.
Looks great, but I think @irfansharif's suggestion also seems reasonable, especially if there is a significant overhead to tracing being enabled by default here (maybe someone else can clarify what our usual paradigm is here?).
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @irfansharif, and @KnightAsterial)
pkg/kv/kvserver/store_snapshot.go line 593 at r2 (raw file):
fn() } log.Event(ctx, "acquired spot in the snapshot semaphore")
This looks great!
pkg/kv/kvserver/store_snapshot.go line 892 at r2 (raw file):
ctx, sp := s.AnnotateCtxWithSpan(ctx, "receive snapshot data") inSnap, err := ss.Receive(ctx, stream, *header, recordBytesReceived) sp.Finish() // Ensure that the tracing span is closed, even if ss.Receive errors
nit: should this not be done with defer sp.Finish() immediately after s.AnnotateCtxWithSpan(..)?
ryanhzhao
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @irfansharif, and @KnightAsterial)
pkg/kv/kvserver/store_snapshot.go line 892 at r2 (raw file):
Previously, AlexTalks (Alex Sarkesian) wrote…
nit: should this not be done with
defer sp.Finish()immediately afters.AnnotateCtxWithSpan(..)?
My intention is for the "receive snapshot data" span to only encompass the call to ss.Receive. If I deferred the sp.Finish() function, it would only be run at the end of the parent function and encompass unwanted functions like processRaftSnapshotRequest(). I considered running Annotate and Defer Finish() within ss.Receive, but that would require passing the Store/AmbientContext inside the receive function and I did not want to change function signatures too much.
If you think one of these alternatives is better, or if better comments are needed, please let me know!
d563ace to
ae251e9
Compare
ae251e9 to
3b43f69
Compare
andreimatei
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @irfansharif, @KnightAsterial, and @miretskiy)
pkg/kv/kvserver/raft_transport.go line 749 at r3 (raw file):
} }() ctx, sp := t.AnnotateCtxWithSpan(ctx, "storage.RaftTransport: sending snapshot")
I don't think you want AnnotateCtx here. If indeed you want it, then I think you'd want to move it up before all the other uses of ctx. But I think this functions receives a ctx that's sufficiently annotated (RaftTransport does not seem to have any tags of it's own, so it's just the node id that we're talking about).
Also, I think tangling tracing with AmbientCtx through AnnotateCtxWithSpan is a bad idea - see also #74291 and linked discussions.
So I'd rather have RaftTransport take a Tracer explicitly, and use that directly.
Also, any particular reason to not push the new span into sendSnapshot? Is the "storage.RaftTransport" part of the name important? The only other use of sendSnapshot seems to be a recovery tool.
pkg/kv/kvserver/replica_command.go line 2680 at r3 (raw file):
note that enabling this may have a negative performance impact
pkg/kv/kvserver/replica_command.go line 2695 at r3 (raw file):
sendThreshold := traceSnapshotThreshold.Get(&r.ClusterSettings().SV) if sendThreshold > 0 { traceCtx, sp := r.AnnotateCtxWithSpan(ctx, "trace-snapshot-threshold")
Please use a more descriptive name, without "trace" and "threshold" in it.
pkg/kv/kvserver/store.go line 1397 at r3 (raw file):
// AnnotateCtxWithSpan is a convenience wrapper; see AmbientContext. func (s *Store) AnnotateCtxWithSpan(
No, please. Let's have less AnnotateCtxWithSpan, not more. Also I think most of the callers of this new method only want a span, they don't want the "annotate" part (i.e. the log tags); they should use the Tracer directly.
pkg/kv/kvserver/store_snapshot.go line 284 at r3 (raw file):
// and receiving the data from the stream. NB: this value encapsulates wait // time due to sender-side rate limiting totalTimeStopwatch := timeutil.NewStopWatch()
consider grouping this timing info into a LazyTag, so that the information is available (through the inflight span registry) while the snapshot is being received. Then you don't have to log it at the end.
pkg/kv/kvserver/store_snapshot.go line 286 at r3 (raw file):
totalTimeStopwatch := timeutil.NewStopWatch() batchStopwatch := timeutil.NewStopWatch() msstwStopwatch := timeutil.NewStopWatch()
what are the "m" and "w" in msstw? Consider renaming or getting rid of them.
Later edit: Multi Writer. I would still get rid of them.
pkg/kv/kvserver/store_snapshot.go line 292 at r3 (raw file):
// At the moment we'll write at most five SSTs. // TODO(jeffreyxiao): Re-evaluate as the default range size grows. msstwStopwatch.Start()
is this stanza blocking in any way / is it worth measuring? If not, consider getting rid of this use of msstwStopwatch since I think it's just noise.
pkg/kv/kvserver/store_snapshot.go line 337 at r3 (raw file):
} msstwStopwatch.Stop() batchStopwatch.Start()
is batchStopwatch worth it? I'm not sure how much hides behind batchReader, but I suspect it's unlikely for this measurement to be interesting.
pkg/kv/kvserver/store_snapshot.go line 379 at r3 (raw file):
log.Eventf( ctx, "Breakdown of time spent receiving snapshot. Total time taken: %.9f. "+
don't capitalize messages
pkg/kv/kvserver/store_snapshot.go line 382 at r3 (raw file):
"Time writing with msstw: %.9f (%.2f%%). Time reading from batch: %.9f (%.2f%%). Time waiting on & receiving from net: %.9f (%.2f%%).", totalTime, msstwDuration, msstwDuration/totalTime*100,
use humanizeutil.Duration
pkg/util/log/trace.go line 274 at r3 (raw file):
} // LogTraceAboveThreshold logs a span's recording if the duration is above a
let's not elevate this random function out of the (unexported) place it came from. I don't think it's worth it. Also, the r==nil protection stanza does not make sense for the new caller.
Also, the elapsed < threshold check is duplicated for the new caller. For the old callers, it would probably be a worth-while performance improvement to not generate the recording if the duration is below the threshold. It might be worth doing; you could do it by having this function take the span, or a function returning a recording, instead of the recording directly.
3b43f69 to
4ea6bfd
Compare
4ea6bfd to
9befa76
Compare
9befa76 to
eb2b8ab
Compare
ryanhzhao
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @andreimatei, @irfansharif, and @miretskiy)
pkg/kv/kvserver/raft_transport.go line 749 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
I don't think you want
AnnotateCtxhere. If indeed you want it, then I think you'd want to move it up before all the other uses ofctx. But I think this functions receives a ctx that's sufficiently annotated (RaftTransportdoes not seem to have any tags of it's own, so it's just the node id that we're talking about).
Also, I think tangling tracing withAmbientCtxthroughAnnotateCtxWithSpanis a bad idea - see also #74291 and linked discussions.
So I'd rather haveRaftTransporttake aTracerexplicitly, and use that directly.Also, any particular reason to not push the new span into
sendSnapshot? Is the"storage.RaftTransport"part of the name important? The only other use ofsendSnapshotseems to be a recovery tool.
Done.
pkg/kv/kvserver/replica_command.go line 2680 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
note that enabling this may have a negative performance impact
Could you clarify what you meant by this comment?
pkg/kv/kvserver/replica_command.go line 2695 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
Please use a more descriptive name, without "trace" and "threshold" in it.
Done.
pkg/kv/kvserver/store.go line 1397 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
No, please. Let's have less
AnnotateCtxWithSpan, not more. Also I think most of the callers of this new method only want a span, they don't want the "annotate" part (i.e. the log tags); they should use theTracerdirectly.
Done.
pkg/kv/kvserver/store_snapshot.go line 593 at r2 (raw file):
Previously, AlexTalks (Alex Sarkesian) wrote…
This looks great!
Done.
pkg/kv/kvserver/store_snapshot.go line 284 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
consider grouping this timing info into a
LazyTag, so that the information is available (through the inflight span registry) while the snapshot is being received. Then you don't have to log it at the end.
Done.
pkg/kv/kvserver/store_snapshot.go line 286 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
what are the "m" and "w" in
msstw? Consider renaming or getting rid of them.
Later edit: Multi Writer. I would still get rid of them.
Done.
pkg/kv/kvserver/store_snapshot.go line 292 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
is this stanza blocking in any way / is it worth measuring? If not, consider getting rid of this use of
msstwStopwatchsince I think it's just noise.
Done.
pkg/kv/kvserver/store_snapshot.go line 337 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
is
batchStopwatchworth it? I'm not sure how much hides behindbatchReader, but I suspect it's unlikely for this measurement to be interesting.
Done.
pkg/kv/kvserver/store_snapshot.go line 379 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
don't capitalize messages
Done.
pkg/kv/kvserver/store_snapshot.go line 382 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
use
humanizeutil.Duration
Done.
pkg/util/log/trace.go line 274 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
let's not elevate this random function out of the (unexported) place it came from. I don't think it's worth it. Also, the
r==nilprotection stanza does not make sense for the new caller.
Also, theelapsed < thresholdcheck is duplicated for the new caller. For the old callers, it would probably be a worth-while performance improvement to not generate the recording if the duration is below the threshold. It might be worth doing; you could do it by having this function take the span, or a function returning a recording, instead of the recording directly.
Done.
andreimatei
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @andreimatei, @irfansharif, @KnightAsterial, and @miretskiy)
pkg/kv/kvserver/replica_command.go line 2680 at r3 (raw file):
Previously, KnightAsterial (Ryan Zhao) wrote…
Could you clarify what you meant by this comment?
sorry, I left this hanging.
I think I wanted to ask if this note about the perf impact is based on something, or just fear mongering? Cause if it's not based on anything, let's get rid of it.
pkg/kv/kvserver/replica_command.go line 2683 at r6 (raw file):
"trace logged (set to 0 to disable); note that enabling this may have "+ "a negative performance impact;", 0, ).WithPublic()
I don't know exactly what WithPublic does, but is it is what we want?
If the setting appears in SHOW ALL CLUSTER SETTINGS even without it, I don't think the setting should be marked as public.
pkg/kv/kvserver/replica_command.go line 2697 at r6 (raw file):
sendThreshold := traceSnapshotThreshold.Get(&r.ClusterSettings().SV) if sendThreshold > 0 { traceCtx, sp := tracing.EnsureChildSpan(ctx, r.store.cfg.Tracer(),
why do you need the traceCtx variable? Assign ctx directly.
pkg/kv/kvserver/replica_command.go line 2700 at r6 (raw file):
"overarching span for snapshots activities") ctx = traceCtx sp.SetRecordingType(tracingpb.RecordingVerbose)
instead of this SetRecordingType call, pass a WithRecording(Verbose) to EnsureChildSpan.
pkg/kv/kvserver/replica_command.go line 2711 at r6 (raw file):
log.Infof(ctx, "%s took %s, exceeding threshold of %s:\n%s", "snapshot", sendDur, sendThreshold, dump) } else { sp.Finish()
nit: instead of the current if/else, with two flavors of Finish on either side, do
if sendDur > threshold {
log.Infof(..., sp.GetRecording())
}
sp.Finish()
The root span of recording is going to have an unfinished marker, but who cares.
pkg/kv/kvserver/store.go line 1156 at r6 (raw file):
// Tracer returns the tracer embedded within StoreConfig // TODO(ryan.zhao): This is temporary until #74291 is resolved
get rid of the TODO. :P
Also interns should generally not put their names on TODOs, unless they plan to address them imminently. :)
pkg/kv/kvserver/store_snapshot.go line 93 at r6 (raw file):
kvserverpb.SnapshotRequest_Header, snapshotRecordMetrics, bool,
You don't want an unnamed primitive type in an interface method; definitely not without a comment. If this stays, give it a name and name all the other arguments _.
But what's the deal with this argument? Is it really important that timing data is not produced when the cluster setting is not set?
pkg/kv/kvserver/store_snapshot.go line 265 at r6 (raw file):
// on how long individual parts of a snapshot take. Individual stopwatches can // be added to a snapshotTimingTag and timing will only be recorded if // shouldRecord is true
Please put periods after all method/struct comments; they're supposed to be full sentences.
pkg/kv/kvserver/store_snapshot.go line 363 at r6 (raw file):
// and receiving the data from the stream. NB: this value encapsulates wait // time due to sender-side rate limiting timingTag := newSnapshotTimingTag(shouldRecordTiming)
you want to do sp.SetLazyTag(timingTag), right? That was the point of grouping the timers.
eb2b8ab to
4f65e79
Compare
ryanhzhao
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @andreimatei, and @irfansharif)
pkg/kv/kvserver/replica_command.go line 2680 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
sorry, I left this hanging.
I think I wanted to ask if this note about the perf impact is based on something, or just fear mongering? Cause if it's not based on anything, let's get rid of it.
Removed!
I did not do any performance testing to verify if this setting would impact performance. I included the note because sql.trace.stmt.enable_threshold had a similar note, and this also enables additional tracing. However, I think that it would not add enough additional tracing to affect performance that much.
pkg/kv/kvserver/replica_command.go line 2683 at r6 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
I don't know exactly what
WithPublicdoes, but is it is what we want?
If the setting appears inSHOW ALL CLUSTER SETTINGSeven without it, I don't think the setting should be marked as public.
Done.
pkg/kv/kvserver/replica_command.go line 2697 at r6 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
why do you need the
traceCtxvariable? Assignctxdirectly.
Fixed. I created traceCtx before because the ctx, sp := caused the ctx variable to get shadowed, but I think that can be fixed by just avoiding the :=
pkg/kv/kvserver/replica_command.go line 2700 at r6 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
instead of this
SetRecordingTypecall, pass aWithRecording(Verbose)toEnsureChildSpan.
Done.
pkg/kv/kvserver/store.go line 1156 at r6 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
get rid of the TODO. :P
Also interns should generally not put their names on TODOs, unless they plan to address them imminently. :)
Makes sense. Thank you!
pkg/kv/kvserver/store_snapshot.go line 93 at r6 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
You don't want an unnamed primitive type in an interface method; definitely not without a comment. If this stays, give it a name and name all the other arguments
_.But what's the deal with this argument? Is it really important that timing data is not produced when the cluster setting is not set?
I added this argument because I noticed in pprof CPU profiles, calls to timeutil.Stopwatch and time.Now make up a non-insignificant percentage of the CPU in some of my experiments. For example, when I ran an experiment that created one 32 GiB range and transferred that between nodes, this was the pprof CPU profile. Thus, I thought we should avoid always leaving timing data. However, I am happy to hear other thoughts on this!
pkg/kv/kvserver/store_snapshot.go line 265 at r6 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
Please put periods after all method/struct comments; they're supposed to be full sentences.
Done.
pkg/kv/kvserver/store_snapshot.go line 363 at r6 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
you want to do
sp.SetLazyTag(timingTag), right? That was the point of grouping the timers.
Oh my gosh, thank you for the catch. Implemented!
andreimatei
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @andreimatei, @irfansharif, and @KnightAsterial)
pkg/kv/kvserver/replica_command.go line 2698 at r7 (raw file):
var sp *tracing.Span ctx, sp = tracing.EnsureChildSpan(ctx, r.store.cfg.Tracer(), "overarching span for snapshots activities", tracing.WithRecording(tracingpb.RecordingVerbose))
"overarching span for snapshots activities" is way too long and editorialized :)
Just delegate snapshot or so.
pkg/kv/kvserver/replica_command.go line 2706 at r7 (raw file):
// #50166). log.Infof(ctx, "%s took %s, exceeding threshold of %s:\n%s", "snapshot", sendDur, sendThreshold, sp.GetRecording(tracingpb.RecordingVerbose))
nit: sp.GetRecording(tracingpb.RecordingVerbose) -> sp.GetConfiguredRecording()
pkg/kv/kvserver/store.go line 1156 at r6 (raw file):
Previously, KnightAsterial (Ryan Zhao) wrote…
Makes sense. Thank you!
I meant get rid of the whole "this is temporary" line. It's not helping anybody.
pkg/kv/kvserver/store_snapshot.go line 93 at r6 (raw file):
Previously, KnightAsterial (Ryan Zhao) wrote…
I added this argument because I noticed in
pprofCPU profiles, calls totimeutil.Stopwatchandtime.Nowmake up a non-insignificant percentage of the CPU in some of my experiments. For example, when I ran an experiment that created one 32 GiB range and transferred that between nodes, this was thepprofCPU profile. Thus, I thought we should avoid always leaving timing data. However, I am happy to hear other thoughts on this!
We call timingTag.startIfRecording("sst") around every msstw.Put() call. That sounds really intense. Instead, can we have a single measurement around for batchReader.Next() { ? Then I hope that profile would look very different.
4f65e79 to
11aed96
Compare
ryanhzhao
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @andreimatei, and @irfansharif)
pkg/kv/kvserver/replica_command.go line 2698 at r7 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
"overarching span for snapshots activities" is way too long and editorialized :)
Justdelegate snapshotor so.
Done.
pkg/kv/kvserver/store.go line 1156 at r6 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
I meant get rid of the whole "this is temporary" line. It's not helping anybody.
Done.
pkg/kv/kvserver/store_snapshot.go line 93 at r6 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
We call
timingTag.startIfRecording("sst")around everymsstw.Put()call. That sounds really intense. Instead, can we have a single measurement aroundfor batchReader.Next() {? Then I hope that profile would look very different.
Done.
andreimatei
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 1 of 0 LGTMs obtained (waiting on @AlexTalks, @andreimatei, @irfansharif, and @KnightAsterial)
pkg/kv/kvserver/store_snapshot.go line 270 at r8 (raw file):
mu struct { syncutil.Mutex stopwatches map[string]*timeutil.StopWatch
If it was me, I wouldn't have this dynamic list of timers. Instead, I think it'd be simpler and more readable to have two different tags (one for send, one for receive), with static sets of timers.
If you do that, you can get rid of the mutex in the tag. In fact, even with the code as it is, I think you can get rid of the mutex and make the contract that addStopWatch cannot be called concurrently with anything else - in particular, it needs to be called before SetLazyTag(tag) is called. Individual timers are thread-safe, I believe.
4cd64e5 to
c324196
Compare
AlexTalks
left a comment
There was a problem hiding this comment.
Looks almost ready to merge, just fix the Yarn subproject commit (which I'm guessing was a mistake) and I will stamp finally!
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @AlexTalks, @andreimatei, @irfansharif, and @KnightAsterial)
pkg/ui/yarn-vendor line 1 at r10 (raw file):
Subproject commit 26434f605262cb3bd85d965f6e231aaab4f44741
I think this needs to be removed!
This commit adds additional tracing to both the sender and receiver sides of the snapshot flow. On the sender side, existing tracing was extended and new spans are added for sender-side snapshot reservation and the act of streaming the snapshot to a recipient. On the receiver side, new tracing was added to follow the reception of the snapshot bytes. Spans were added to show receiver-side snapshot reservation and the act of reading the snapshot of the gRPC stream. This tracing can be enabled using the new cluster setting kv.trace.snapshot.enable_threshold, and snapshots that take longer than the threshold have their traces written to the log. Additionally, timing was added to the snapshot Send() and Receive() functions. This change allows users to see what percentage of sending and receiving is spent on disk IO, network IO, and rate-limiting. These changes give better insight into where time is bring spent during rebalance and recovery operations, allowing for future works to better optimize these processes. Resolves: cockroachdb#82753 Release note: None
c324196 to
5745ba6
Compare
AlexTalks
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @AlexTalks, @andreimatei, @irfansharif, and @KnightAsterial)
|
bors r+ |
|
Build succeeded: |

This commit adds additional tracing to both the sender and receiver sides

of the snapshot flow. On the sender side, existing tracing was extended and
new spans are added for sender-side snapshot reservation and the act of
streaming the snapshot to a recipient. On the receiver side, new tracing was
added to follow the reception of the snapshot bytes. Spans were added to
show receiver-side snapshot reservation and the act of reading the snapshot
of the gRPC stream. This tracing can be enabled using the new cluster
setting
kv.trace.snapshot.enable_threshold, and snapshots that take longerthan the threshold have their traces written to the log.
Additionally, timing was added to the snapshot Send() and Receive()


functions. This change allows users to see what percentage of sending and
receiving is spent on disk IO, network IO, and rate-limiting.
These changes give better insight into where time is bring spent during
rebalance and recovery operations, allowing for future works to better
optimize these processes.
Resolves: #82753
Release note: None