Skip to content
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

sql: add nodes for each EXPLAIN ANALYZE operator #60550

Merged
merged 2 commits into from
Feb 19, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions pkg/sql/colflow/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/sql/colflow",
visibility = ["//visibility:public"],
deps = [
"//pkg/base",
"//pkg/col/coldata",
"//pkg/col/coldataext",
"//pkg/rpc/nodedialer",
Expand Down
10 changes: 7 additions & 3 deletions pkg/sql/colflow/vectorized_flow.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"sync/atomic"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/col/coldataext"
"github.com/cockroachdb/cockroach/pkg/rpc/nodedialer"
Expand Down Expand Up @@ -831,9 +832,12 @@ func (s *vectorizedFlowCreator) setupInput(
s.addStreamEndpoint(inputStream.StreamID, inbox, s.waitGroup)
op := colexecbase.Operator(inbox)
if s.recordingStats {
op, err = s.wrapWithNetworkVectorizedStatsCollector(
inbox, flowCtx.StreamComponentID(inputStream.StreamID), latency,
// Note: we can't use flowCtx.StreamComponentID because the stream does
// not originate from this node (we are the target node).
compID := execinfrapb.StreamComponentID(
base.SQLInstanceID(inputStream.OriginNodeID), flowCtx.ID, inputStream.StreamID,
)
op, err = s.wrapWithNetworkVectorizedStatsCollector(inbox, compID, latency)
if err != nil {
return nil, nil, nil, err
}
Expand Down Expand Up @@ -958,7 +962,7 @@ func (s *vectorizedFlowCreator) setupOutput(
// flow-level span.
span.SetTag(execinfrapb.FlowIDTagKey, flowCtx.ID)
span.SetSpanStats(&execinfrapb.ComponentStats{
Component: execinfrapb.FlowComponentID(outputStream.OriginNodeID, flowCtx.ID),
Component: execinfrapb.FlowComponentID(base.SQLInstanceID(outputStream.OriginNodeID), flowCtx.ID),
FlowStats: execinfrapb.FlowStats{
MaxMemUsage: optional.MakeUint(uint64(flowCtx.EvalCtx.Mon.MaximumBytes())),
},
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/distsql/inbound_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,7 @@ func TestOutboxInboundStreamIntegration(t *testing.T) {
NodeDialer: nodedialer.New(rpcContext, staticAddressResolver(ln.Addr())),
Stopper: outboxStopper,
},
NodeID: base.TestingIDContainer,
}

streamID := execinfrapb.StreamID(1)
Expand Down
6 changes: 4 additions & 2 deletions pkg/sql/distsql_physical_planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"reflect"
"sort"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/keys"
Expand Down Expand Up @@ -2742,10 +2743,11 @@ func (dsp *DistSQLPlanner) createPhysPlanForPlanNode(

if planCtx.traceMetadata != nil {
processors := make(execComponents, len(plan.ResultRouters))
for i := range plan.ResultRouters {
for i, resultProcIdx := range plan.ResultRouters {
processors[i] = execinfrapb.ProcessorComponentID(
base.SQLInstanceID(plan.Processors[resultProcIdx].Node),
execinfrapb.FlowID{UUID: planCtx.infra.FlowID},
int32(plan.ResultRouters[i]),
int32(resultProcIdx),
)
}
planCtx.traceMetadata.associateNodeWithComponents(node, processors)
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/execinfra/flow_context.go
Original file line number Diff line number Diff line change
Expand Up @@ -108,10 +108,11 @@ func (ctx *FlowCtx) Codec() keys.SQLCodec {
// ProcessorComponentID returns a ComponentID for the given processor in this
// flow.
func (ctx *FlowCtx) ProcessorComponentID(procID int32) execinfrapb.ComponentID {
return execinfrapb.ProcessorComponentID(ctx.ID, procID)
return execinfrapb.ProcessorComponentID(ctx.NodeID.SQLInstanceID(), ctx.ID, procID)
}

// StreamComponentID returns a ComponentID for the given stream in this flow.
// The stream must originate from the node associated with this FlowCtx.
func (ctx *FlowCtx) StreamComponentID(streamID execinfrapb.StreamID) execinfrapb.ComponentID {
return execinfrapb.StreamComponentID(ctx.ID, streamID)
return execinfrapb.StreamComponentID(ctx.NodeID.SQLInstanceID(), ctx.ID, streamID)
}
32 changes: 19 additions & 13 deletions pkg/sql/execinfrapb/component_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@ import (
"strconv"
"strings"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/optional"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
Expand All @@ -26,29 +26,35 @@ import (
)

// ProcessorComponentID returns a ComponentID for the given processor in a flow.
func ProcessorComponentID(flowID FlowID, processorID int32) ComponentID {
func ProcessorComponentID(
instanceID base.SQLInstanceID, flowID FlowID, processorID int32,
) ComponentID {
return ComponentID{
FlowID: flowID,
Type: ComponentID_PROCESSOR,
ID: processorID,
FlowID: flowID,
Type: ComponentID_PROCESSOR,
ID: processorID,
SQLInstanceID: instanceID,
}
}

// StreamComponentID returns a ComponentID for the given stream in a flow.
func StreamComponentID(flowID FlowID, streamID StreamID) ComponentID {
func StreamComponentID(
originInstanceID base.SQLInstanceID, flowID FlowID, streamID StreamID,
) ComponentID {
return ComponentID{
FlowID: flowID,
Type: ComponentID_STREAM,
ID: int32(streamID),
FlowID: flowID,
Type: ComponentID_STREAM,
ID: int32(streamID),
SQLInstanceID: originInstanceID,
}
}

// FlowComponentID returns a ComponentID for the given flow.
func FlowComponentID(nodeID roachpb.NodeID, flowID FlowID) ComponentID {
func FlowComponentID(instanceID base.SQLInstanceID, flowID FlowID) ComponentID {
return ComponentID{
FlowID: flowID,
Type: ComponentID_FLOW,
NodeID: nodeID,
FlowID: flowID,
Type: ComponentID_FLOW,
SQLInstanceID: instanceID,
}
}

Expand Down
Loading