sql: add query max memory usage to TraceAnalyzer#56300
sql: add query max memory usage to TraceAnalyzer#56300craig[bot] merged 2 commits intocockroachdb:masterfrom
Conversation
da76871 to
cfb1c45
Compare
RaduBerinde
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @cathymw)
pkg/sql/instrumentation.go, line 182 at r1 (raw file):
networkBytesSent := int64(0) queryMaxMem := int64(0) for _, flowInfo := range p.curPlan.distSQLFlowInfos {
Feels like this stuff should all be done inside traceanalyzer (or a new type in that package). There should be an interface that allows us to pass in multiple traces and flow maps and then get all the top-level stats in a struct.
pkg/sql/instrumentation.go, line 183 at r1 (raw file):
queryMaxMem := int64(0) for _, flowInfo := range p.curPlan.distSQLFlowInfos { analyzer := flowInfo.analyzer
It feels strange to store an analyzer in the struct. We should store the flow map and only initialize an analyzer when we analyze.
If we really want to not keep the flow map around, we can extract a separate FlowMetadata type from the TraceAnalyzer that is immutable and stores just the metadata (I realize TraceAnalyzer is already pretty much just that, it's more about the naming and how we describe these objects).
adc17f9 to
a52e6b4
Compare
asubiotto
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @cathymw and @RaduBerinde)
pkg/sql/instrumentation.go, line 183 at r1 (raw file):
Previously, RaduBerinde wrote…
It feels strange to store an analyzer in the struct. We should store the flow map and only initialize an analyzer when we analyze.
If we really want to not keep the flow map around, we can extract a separate
FlowMetadatatype from theTraceAnalyzerthat is immutable and stores just the metadata (I realize TraceAnalyzer is already pretty much just that, it's more about the naming and how we describe these objects).
Unfortunately we can't store the flow map because the lifetime doesn't extend to this piece of code (we release flow specs for allocation performance). I like the FlowMetadata idea.
pkg/sql/plan.go, line 316 at r2 (raw file):
// GetQueryStats is part of the QueryLevelStatsGetter interface. func (pt *planTop) GetQueryStats(
I think it'd be nicer to have a separate type in the execstats package or some helper function and keep this calculation separate from the planTop object.
pkg/sql/colflow/vectorized_flow.go, line 398 at r2 (raw file):
deterministicStats bool, vectorizedStatsCollectors []colexec.VectorizedStatsCollector, maxMemUsage int64,
I don't think this belongs here. It would be better to have a call after finishVectorizedStatsCollectors that adds this top-level flow information to the span. But I think we still need to do some work to send this message exactly once with the last outbox. I think what might be nice is to write a metadata source that is aware of the number of outboxes:
type FlowStatsReporter struct {
numOutboxes int32
}
func (r *FlowStatsReporter) AddOutbox() {
atomic.AddInt32(&r.numOutboxes, 1)
}
func (r *FlowStatsReporter) DrainMeta(ctx context.Context) []ProducerMetadata {
if atomic.AddInt32(&r.numOutboxes, -1) == 0 {
return []ProducerMetadata{FlowMaxMemory: r.flowCtx.Mon.GetMaxMemoryUsage()}
}
return nil
}
And then the outbox would check that this FlowMaxMemory field is set and add it to its span stats or something like this. The crucial part is that it has to be the last remote component because otherwise the Mon.GetMaxMemoryUsage might be incorrect. Keep in mind that we'll also need something similar in the row execution engine.
pkg/sql/rowflow/routers.go, line 378 at r2 (raw file):
ro.stats.Exec.MaxAllocatedMem.Set(uint64(ro.memoryMonitor.MaximumBytes())) ro.stats.Exec.MaxAllocatedDisk.Set(uint64(ro.diskMonitor.MaximumBytes())) ro.stats.FlowStats.MaxMemUsage.Set(uint64(rb.parent.MaximumBytes()))
Is there any reason why you went down the route of sending flowstats multiple times? In the traceanalyzer, I think the current approach will be to pick the latest value received, which means that you might send a value before the flow ends (e.g. imagine this router's input tree is done before the rest of the components running in this node) and for some reason receive it last in the trace analyzer. This will report an incorrect value for memory usage. I would prefer to find a way to send this stat exactly once at the end of the flow.
a52e6b4 to
e97e027
Compare
e97e027 to
cb55f4f
Compare
cathymw
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @asubiotto)
pkg/sql/instrumentation.go, line 182 at r1 (raw file):
Previously, RaduBerinde wrote…
Feels like this stuff should all be done inside
traceanalyzer(or a new type in that package). There should be an interface that allows us to pass in multiple traces and flow maps and then get all the top-level stats in a struct.
Done. But to prevent an import cycle, I couldn't pass in []flowInfo, so I get the flowMetadata from each of the flowInfos and pass in a slice of the flowMetadata. I don't know if that's too sketchy though
pkg/sql/instrumentation.go, line 183 at r1 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Unfortunately we can't store the flow map because the lifetime doesn't extend to this piece of code (we release flow specs for allocation performance). I like the
FlowMetadataidea.
Done.
pkg/sql/plan.go, line 316 at r2 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
I think it'd be nicer to have a separate type in the
execstatspackage or some helper function and keep this calculation separate from theplanTopobject.
Done.
pkg/sql/colflow/vectorized_flow.go, line 398 at r2 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
I don't think this belongs here. It would be better to have a call after
finishVectorizedStatsCollectorsthat adds this top-level flow information to the span. But I think we still need to do some work to send this message exactly once with the last outbox. I think what might be nice is to write a metadata source that is aware of the number of outboxes:type FlowStatsReporter struct { numOutboxes int32 } func (r *FlowStatsReporter) AddOutbox() { atomic.AddInt32(&r.numOutboxes, 1) } func (r *FlowStatsReporter) DrainMeta(ctx context.Context) []ProducerMetadata { if atomic.AddInt32(&r.numOutboxes, -1) == 0 { return []ProducerMetadata{FlowMaxMemory: r.flowCtx.Mon.GetMaxMemoryUsage()} } return nil }And then the outbox would check that this FlowMaxMemory field is set and add it to its span stats or something like this. The crucial part is that it has to be the last remote component because otherwise the
Mon.GetMaxMemoryUsagemight be incorrect. Keep in mind that we'll also need something similar in the row execution engine.
Done.
pkg/sql/rowflow/routers.go, line 378 at r2 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Is there any reason why you went down the route of sending flowstats multiple times? In the traceanalyzer, I think the current approach will be to pick the latest value received, which means that you might send a value before the flow ends (e.g. imagine this router's input tree is done before the rest of the components running in this node) and for some reason receive it last in the trace analyzer. This will report an incorrect value for memory usage. I would prefer to find a way to send this stat exactly once at the end of the flow.
Done.
cathymw
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @asubiotto)
pkg/sql/instrumentation.go, line 182 at r1 (raw file):
Previously, cathymw wrote…
Done. But to prevent an import cycle, I couldn't pass in
[]flowInfo, so I get theflowMetadatafrom each of theflowInfos and pass in a slice of the flowMetadata. I don't know if that's too sketchy though
^^ at sql/instrumentation.go:190
71788c8 to
e46bbcf
Compare
asubiotto
left a comment
There was a problem hiding this comment.
Reviewed 15 of 24 files at r3.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @asubiotto, @cathymw, @RaduBerinde, and @yuzefovich)
pkg/sql/instrumentation.go, line 194 at r3 (raw file):
flowMetadata = append(flowMetadata, flowInfo.flowMetadata) } queryLevelStats := execstats.GetQueryLevelStats(ctx, trace, cfg.TestingKnobs.DeterministicExplainAnalyze, ast, flowMetadata)
nit: don't make logging GetQueryLevelStats' responsibility. Change it to return an error and have callers decide what to do with it (in this case, log it). It also removes the need to pass in the ast just for logging purposes.
pkg/sql/plan.go, line 271 at r3 (raw file):
diagram execinfrapb.FlowDiagram // FlowMetadata stores metadata from flows that will be used by TraceAnalyzer. flowMetadata *execstats.FlowMetadata
nit: I think this refactor would've been nicer separated into an earlier commit.
pkg/sql/colflow/vectorized_flow.go, line 521 at r3 (raw file):
// When there is one outbox, the flow max memory usage is added to a flow // level span. currOutboxes int32
It's unfortunate that we now have numOutboxes and currOutboxes that essentially indicate the same thing but they're incremented and decremented separately. I understand the need for a separate counter, but maybe we can change it so that instead of incrementing numOutboxes and decrementing at plan time we would have a numOutboxes field incremented at plan time as it currently is and then separately increment two counters, on would be numOutboxesExited incremented where numOutboxes is currently decremented, and the other would be numOutboxesDrained which would be incremented where currOutboxes is currently decremented. We would then just check the atomic and perform whatever operation we need to once it reaches numOutboxes. I think this is cleaner than incrementing two counters that overlap semantically.
pkg/sql/colflow/vectorized_flow.go, line 987 at r3 (raw file):
if atomic.AddInt32(&s.currOutboxes, -1) == 0 { span.SetTag(execinfrapb.FlowIDTagKey, flowCtx.ID) span.SetSpanStats(&execinfrapb.ComponentStats{FlowStats: execinfrapb.FlowStats{MaxMemUsage: optional.MakeUint(uint64(flowCtx.EvalCtx.Mon.MaximumBytes()))}})
nit: I think this would be nicer on a couple of lines, it's a bit hard to read currently.
pkg/sql/colflow/vectorized_flow.go, line 990 at r3 (raw file):
} finishVectorizedStatsCollectors( ctx, flowCtx.ID, vscs,
nit: I don't think this needs to be on a new line
pkg/sql/execstats/traceanalyzer.go, line 56 at r3 (raw file):
// flowStats maps a flow ID, represented as a string, to flow level stats // extracted from a trace. flowStats map[string]*flowStats
Why is this a string and not a FlowID?
pkg/sql/execstats/traceanalyzer.go, line 100 at r3 (raw file):
// NewTraceAnalyzer creates a TraceAnalyzer with the corresponding physical // plan. Call AddTrace to calculate meaningful stats. func NewTraceAnalyzer(flowMetadata FlowMetadata) *TraceAnalyzer {
I believe this can be changed to MakeTraceAnalyzer which would return a TraceAnalyzer value (no pointer) if you embed FlowMetadata as *FlowMetadata then you could also remove change the pointer receiver in AddTrace to a value receiver.
pkg/sql/execstats/traceanalyzer.go, line 151 at r3 (raw file):
return errors.Errorf("trace has span for stream %d but the stream does not exist in the physical plan", id) } streamStats.stats = append(streamStats.stats, &stats)
This should no longer be necessary, right?
pkg/sql/execstats/traceanalyzer.go, line 181 at r3 (raw file):
if v.FlowStats.MaxMemUsage.HasValue() { // This stream might have multiple span stats, some of which may not have network bytes information. return 0, nil
How are we hitting this case? In colexec we set up stats with only a flow id tag so it can't be confused for stream stats and in rowexec we set the BytesSent value which would mean we exit above.
pkg/sql/execstats/traceanalyzer.go, line 255 at r3 (raw file):
// TODO(cathymw): maxMemUsage shouldn't be attached to span stats that are associated with streams, // since it's a flow level stat. However, due to the row exec engine infrastructure, it is too // complicated to attach this to a flow level span. If the row exec engine gets removed, getting
This comment should also be in the rowexec outbox when we set the stats.
pkg/sql/execstats/traceanalyzer.go, line 276 at r3 (raw file):
ast tree.Statement, flowMetadata []*FlowMetadata, ) *QueryLevelStats {
I don't think QueryLevelStats needs to be a pointer.
pkg/sql/flowinfra/outbox.go, line 84 at r3 (raw file):
flowID execinfrapb.FlowID, streamID execinfrapb.StreamID, numOutboxes int32,
I don't know how this is working. Passing in an int32 by value will make a copy of the value, so each outbox will independently decrement their own value. Only the first outbox created will end up setting stats because numOutboxes was 1 at its time of creation. You should probably pass in a pointer instead.
6cd0e89 to
c8de619
Compare
cathymw
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @asubiotto, @RaduBerinde, and @yuzefovich)
pkg/sql/instrumentation.go, line 194 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
nit: don't make logging
GetQueryLevelStats' responsibility. Change it to return an error and have callers decide what to do with it (in this case, log it). It also removes the need to pass in theastjust for logging purposes.
Done.
pkg/sql/plan.go, line 271 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
nit: I think this refactor would've been nicer separated into an earlier commit.
Done.
pkg/sql/colflow/vectorized_flow.go, line 521 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
It's unfortunate that we now have
numOutboxesandcurrOutboxesthat essentially indicate the same thing but they're incremented and decremented separately. I understand the need for a separate counter, but maybe we can change it so that instead of incrementingnumOutboxesand decrementing at plan time we would have anumOutboxesfield incremented at plan time as it currently is and then separately increment two counters, on would benumOutboxesExitedincremented wherenumOutboxesis currently decremented, and the other would benumOutboxesDrainedwhich would be incremented wherecurrOutboxesis currently decremented. We would then just check the atomic and perform whatever operation we need to once it reachesnumOutboxes. I think this is cleaner than incrementing two counters that overlap semantically.
Done.
pkg/sql/colflow/vectorized_flow.go, line 987 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
nit: I think this would be nicer on a couple of lines, it's a bit hard to read currently.
Done.
pkg/sql/colflow/vectorized_flow.go, line 990 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
nit: I don't think this needs to be on a new line
Done.
pkg/sql/execstats/traceanalyzer.go, line 56 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Why is this a string and not a
FlowID?
Done. Changed to FlowID.
pkg/sql/execstats/traceanalyzer.go, line 100 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
I believe this can be changed to
MakeTraceAnalyzerwhich would return aTraceAnalyzervalue (no pointer) if you embedFlowMetadataas*FlowMetadatathen you could also remove change the pointer receiver inAddTraceto a value receiver.
Done.
pkg/sql/execstats/traceanalyzer.go, line 151 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
This should no longer be necessary, right?
Done.
pkg/sql/execstats/traceanalyzer.go, line 181 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
How are we hitting this case? In colexec we set up stats with only a flow id tag so it can't be confused for stream stats and in rowexec we set the
BytesSentvalue which would mean we exit above.
Done. Oops, you're right, removed.
pkg/sql/execstats/traceanalyzer.go, line 255 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
This comment should also be in the rowexec outbox when we set the stats.
Done.
pkg/sql/execstats/traceanalyzer.go, line 276 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
I don't think
QueryLevelStatsneeds to be a pointer.
Done.
pkg/sql/flowinfra/outbox.go, line 84 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
I don't know how this is working. Passing in an
int32by value will make a copy of the value, so each outbox will independently decrement their own value. Only the first outbox created will end up setting stats becausenumOutboxeswas1at its time of creation. You should probably pass in a pointer instead.
Done.
asubiotto
left a comment
There was a problem hiding this comment.
Reviewed 1 of 24 files at r3, 13 of 13 files at r4, 11 of 11 files at r5.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @asubiotto, @cathymw, and @yuzefovich)
pkg/sql/instrumentation.go, line 194 at r3 (raw file):
Previously, cathymw wrote…
Done.
nit: errorMsgs is redundant with errors since the latter already contains a message. Oh, I saw you've amended this in the latter commit after I wrote this message, but that should probably be in the first commit.
pkg/sql/colflow/vectorized_flow.go, line 495 at r5 (raw file):
numOutboxesExited int32 // numOutboxesDrained is an atomic that keeps track of how many outboxes have // been drained. When numOutboxesDrained equals numOutboxes, the flow max memory
nit: it doesn't hurt to be more general to avoid this comment becoming stale in the future something like s/the flow max memory usage is added to a flow level span/flow-level metadata is sent.
pkg/sql/colflow/vectorized_flow.go, line 988 at r5 (raw file):
ctx, span := tracing.ChildSpanRemote(ctx, "") if atomic.AddInt32(&s.numOutboxesDrained, 1) == atomic.LoadInt32(&s.numOutboxes) { span.SetTag(execinfrapb.FlowIDTagKey, flowCtx.ID)
Add a comment about why the last outbox is doing this.
pkg/sql/execstats/traceanalyzer.go, line 52 at r4 (raw file):
// for the given stream in the trace. streamStats map[execinfrapb.StreamID]*streamStats // flowStats maps a flow ID, represented as a string, to flow level stats
nit: represented as a string is no longer true
pkg/sql/execstats/traceanalyzer.go, line 206 at r4 (raw file):
} // GetQueryLevelStats returns all the top-level stats in a QueryLevelStats struct.
nit: explain the multiple []error behavior since it's unusual. I think the important point is that GetQueryLevelStats is best-effort in that it tries to calculate as many stats as possible.
pkg/sql/execstats/traceanalyzer.go, line 225 at r4 (raw file):
if err != nil { errors = append(errors, err) errorMsgs = append(errorMsgs, "error calculating network bytes sent for stmt %s: %v")
Don't you need to continue here?
pkg/sql/execstats/traceanalyzer.go, line 268 at r5 (raw file):
analyzer := MakeTraceAnalyzer(metadata) if err := analyzer.AddTrace(trace, deterministicExplainAnalyze); err != nil { errors = append(errors, err)
It might be nice to still Wrap this error with some way to identify which stat produced an error (in the first commit).
pkg/sql/flowinfra/outbox.go, line 69 at r5 (raw file):
stats execinfrapb.ComponentStats // currOutboxes is an atomic that keeps track of how many outboxes are left.
nit: s/currOutboxes/numOutboxes
pkg/sql/flowinfra/outbox_test.go, line 77 at r5 (raw file):
streamID := execinfrapb.StreamID(42) numOutboxes := int32(0) outbox := NewOutbox(&flowCtx, execinfra.StaticNodeID, flowID, streamID, &numOutboxes)
nit: why not just pass in nil and only check m.numOutboxes in flowinfra/outbox.go if the pointer is non-nil.
pkg/sql/rowflow/row_based_flow.go, line 34 at r5 (raw file):
// currOutboxes is an atomic that counts how many outboxes have been created. // This atomic is then used by the last outbox to know when to report the flow's
nit: same comment regarding making this comment more general. Also mention somewhere in the rowflow code why it's necessary for the last outbox to do this.
pkg/sql/rowflow/row_based_flow.go, line 36 at r5 (raw file):
// This atomic is then used by the last outbox to know when to report the flow's // max memory usage. currOutboxes int32
nit: rename this to numOutboxes since that's what's used everywhere else.
c8de619 to
c05eabb
Compare
c05eabb to
5805254
Compare
cathymw
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @asubiotto and @yuzefovich)
pkg/sql/instrumentation.go, line 194 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
nit:
errorMsgsis redundant witherrorssince the latter already contains a message. Oh, I saw you've amended this in the latter commit after I wrote this message, but that should probably be in the first commit.
Done. Moved to first commit.
pkg/sql/colflow/vectorized_flow.go, line 495 at r5 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
nit: it doesn't hurt to be more general to avoid this comment becoming stale in the future something like
s/the flow max memory usage is added to a flow level span/flow-level metadata is sent.
Done.
pkg/sql/colflow/vectorized_flow.go, line 988 at r5 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Add a comment about why the last outbox is doing this.
Done.
pkg/sql/execstats/traceanalyzer.go, line 52 at r4 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
nit:
represented as a stringis no longer true
Done.
pkg/sql/execstats/traceanalyzer.go, line 206 at r4 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
nit: explain the multiple
[]errorbehavior since it's unusual. I think the important point is thatGetQueryLevelStatsis best-effort in that it tries to calculate as many stats as possible.
Done.
pkg/sql/execstats/traceanalyzer.go, line 225 at r4 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Don't you need to
continuehere?
Done.
pkg/sql/execstats/traceanalyzer.go, line 268 at r5 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
It might be nice to still
Wrapthis error with some way to identify which stat produced an error (in the first commit).
Done.
pkg/sql/flowinfra/outbox.go, line 69 at r5 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
nit:
s/currOutboxes/numOutboxes
Done.
pkg/sql/flowinfra/outbox_test.go, line 77 at r5 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
nit: why not just pass in
niland only checkm.numOutboxesinflowinfra/outbox.goif the pointer is non-nil.
Done.
pkg/sql/rowflow/row_based_flow.go, line 34 at r5 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
nit: same comment regarding making this comment more general. Also mention somewhere in the rowflow code why it's necessary for the last outbox to do this.
Done.
pkg/sql/rowflow/row_based_flow.go, line 36 at r5 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
nit: rename this to
numOutboxessince that's what's used everywhere else.
Done.
asubiotto
left a comment
There was a problem hiding this comment.
Good job! minus some remaining nits
Reviewed 10 of 10 files at r6, 10 of 10 files at r7.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @asubiotto, @cathymw, and @yuzefovich)
pkg/sql/execstats/traceanalyzer.go, line 268 at r5 (raw file):
Previously, cathymw wrote…
Done.
nit: I don't think the message makes sense as written, did you mean error analyzing?
pkg/sql/execstats/traceanalyzer.go, line 208 at r6 (raw file):
// GetQueryLevelStats tries to calculate as many stats as possible. If errors occur // while calculating stats, GetQueryLevelStats adds the error to a slice to be returned // for logging purposes, but continues calculating other stats.
small nit: GetQueryLevelStats doesn't know what the errors will be used for (some caller in the future might decide to return the last errror) so this should not reference any logging
2c6e1aa to
2b45421
Compare
cathymw
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @asubiotto and @yuzefovich)
pkg/sql/execstats/traceanalyzer.go, line 268 at r5 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
nit: I don't think the message makes sense as written, did you mean
error analyzing?
Done.
pkg/sql/execstats/traceanalyzer.go, line 208 at r6 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
small nit:
GetQueryLevelStatsdoesn't know what the errors will be used for (some caller in the future might decide to return the last errror) so this should not reference any logging
Done.
asubiotto
left a comment
There was a problem hiding this comment.
Reviewed 12 of 12 files at r8, 10 of 10 files at r9.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @yuzefovich)
This commit introduces a new type FlowMetadata to avoid flowInfo from storing a TraceAnalyzer. This commit also moves code from instrumentation.go to a helper function GetQueryLevelStats(), which takes in trace and flow metadata and returns top level stats. Release note: None.
This commit adds query max memory usage to TraceAnalyzer, which will later be surfaced to the statement stats page on the admin ui. This approach uses the parent memory monitor to accurately report the max memory that is being used at any point in the query. Release note: None.
2b45421 to
c25995c
Compare
|
TFTRs! |
|
Build succeeded: |
The first commit does a refactor of some TraceAnalyzer code.
The second commit adds max memory usage to TraceAnalyzer.
Please see individual commit messages for more details.
Closes: #54340
Release note: None.