diff --git a/pkg/ccl/backupccl/backup_processor_planning.go b/pkg/ccl/backupccl/backup_processor_planning.go index a2c0a357f51f..97fa9f6e16a8 100644 --- a/pkg/ccl/backupccl/backup_processor_planning.go +++ b/pkg/ccl/backupccl/backup_processor_planning.go @@ -11,6 +11,7 @@ package backupccl import ( "context" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" @@ -39,7 +40,7 @@ func distBackupPlanSpecs( encryption *jobspb.BackupEncryptionOptions, mvccFilter roachpb.MVCCFilter, startTime, endTime hlc.Timestamp, -) (map[roachpb.NodeID]*execinfrapb.BackupDataSpec, error) { +) (map[base.SQLInstanceID]*execinfrapb.BackupDataSpec, error) { var span *tracing.Span ctx, span = tracing.ChildSpan(ctx, "backup-plan-specs") _ = ctx // ctx is currently unused, but this new ctx should be used below in the future. @@ -88,7 +89,7 @@ func distBackupPlanSpecs( // First construct spans based on span partitions. Then add on // introducedSpans based on how those partition. - nodeToSpec := make(map[roachpb.NodeID]*execinfrapb.BackupDataSpec) + sqlInstanceIDToSpec := make(map[base.SQLInstanceID]*execinfrapb.BackupDataSpec) for _, partition := range spanPartitions { spec := &execinfrapb.BackupDataSpec{ Spans: partition.Spans, @@ -101,11 +102,11 @@ func distBackupPlanSpecs( BackupEndTime: endTime, UserProto: user.EncodeProto(), } - nodeToSpec[partition.Node] = spec + sqlInstanceIDToSpec[partition.SQLInstanceID] = spec } for _, partition := range introducedSpanPartitions { - if spec, ok := nodeToSpec[partition.Node]; ok { + if spec, ok := sqlInstanceIDToSpec[partition.SQLInstanceID]; ok { spec.IntroducedSpans = partition.Spans } else { // We may need to introduce a new spec in the case that there is a node @@ -122,21 +123,21 @@ func distBackupPlanSpecs( BackupEndTime: endTime, UserProto: user.EncodeProto(), } - nodeToSpec[partition.Node] = spec + sqlInstanceIDToSpec[partition.SQLInstanceID] = spec } } backupPlanningTraceEvent := BackupProcessorPlanningTraceEvent{ NodeToNumSpans: make(map[int32]int64), } - for node, spec := range nodeToSpec { + for node, spec := range sqlInstanceIDToSpec { numSpans := int64(len(spec.Spans) + len(spec.IntroducedSpans)) backupPlanningTraceEvent.NodeToNumSpans[int32(node)] = numSpans backupPlanningTraceEvent.TotalNumSpans += numSpans } span.RecordStructured(&backupPlanningTraceEvent) - return nodeToSpec, nil + return sqlInstanceIDToSpec, nil } // distBackup is used to plan the processors for a distributed backup. It @@ -148,7 +149,7 @@ func distBackup( planCtx *sql.PlanningCtx, dsp *sql.DistSQLPlanner, progCh chan *execinfrapb.RemoteProducerMetadata_BulkProcessorProgress, - backupSpecs map[roachpb.NodeID]*execinfrapb.BackupDataSpec, + backupSpecs map[base.SQLInstanceID]*execinfrapb.BackupDataSpec, ) error { ctx, span := tracing.ChildSpan(ctx, "backup-distsql") defer span.Finish() @@ -164,8 +165,8 @@ func distBackup( // Setup a one-stage plan with one proc per input spec. corePlacement := make([]physicalplan.ProcessorCorePlacement, len(backupSpecs)) i := 0 - for node, spec := range backupSpecs { - corePlacement[i].NodeID = node + for sqlInstanceID, spec := range backupSpecs { + corePlacement[i].SQLInstanceID = sqlInstanceID corePlacement[i].Core.BackupData = spec i++ } diff --git a/pkg/ccl/backupccl/restore_processor_planning.go b/pkg/ccl/backupccl/restore_processor_planning.go index 5bb20b53b05e..636dcca2ff74 100644 --- a/pkg/ccl/backupccl/restore_processor_planning.go +++ b/pkg/ccl/backupccl/restore_processor_planning.go @@ -13,6 +13,7 @@ import ( "context" "sort" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" @@ -76,12 +77,12 @@ func distRestore( fileEncryption = &roachpb.FileEncryptionOptions{Key: encryption.Key} } - planCtx, nodes, err := dsp.SetupAllNodesPlanning(ctx, evalCtx, execCtx.ExecCfg()) + planCtx, sqlInstanceIDs, err := dsp.SetupAllNodesPlanning(ctx, evalCtx, execCtx.ExecCfg()) if err != nil { return err } - splitAndScatterSpecs, err := makeSplitAndScatterSpecs(nodes, chunks, rekeys) + splitAndScatterSpecs, err := makeSplitAndScatterSpecs(sqlInstanceIDs, chunks, rekeys) if err != nil { return err } @@ -102,8 +103,8 @@ func distRestore( p := planCtx.NewPhysicalPlan() // Plan SplitAndScatter in a round-robin fashion. - splitAndScatterStageID := p.NewStageOnNodes(nodes) - splitAndScatterProcs := make(map[roachpb.NodeID]physicalplan.ProcessorIdx) + splitAndScatterStageID := p.NewStageOnNodes(sqlInstanceIDs) + splitAndScatterProcs := make(map[base.SQLInstanceID]physicalplan.ProcessorIdx) defaultStream := int32(0) rangeRouterSpec := execinfrapb.OutputRouterSpec_RangeRouterSpec{ @@ -116,8 +117,8 @@ func distRestore( }, }, } - for stream, nodeID := range nodes { - startBytes, endBytes, err := routingSpanForNode(nodeID) + for stream, sqlInstanceID := range sqlInstanceIDs { + startBytes, endBytes, err := routingSpanForSQLInstance(sqlInstanceID) if err != nil { return err } @@ -134,7 +135,7 @@ func distRestore( return bytes.Compare(rangeRouterSpec.Spans[i].Start, rangeRouterSpec.Spans[j].Start) == -1 }) - for _, n := range nodes { + for _, n := range sqlInstanceIDs { spec := splitAndScatterSpecs[n] if spec == nil { // We may have fewer chunks than we have nodes for very small imports. In @@ -144,7 +145,7 @@ func distRestore( continue } proc := physicalplan.Processor{ - Node: n, + SQLInstanceID: n, Spec: execinfrapb.ProcessorSpec{ Core: execinfrapb.ProcessorCoreUnion{SplitAndScatter: splitAndScatterSpecs[n]}, Post: execinfrapb.PostProcessSpec{}, @@ -163,11 +164,11 @@ func distRestore( } // Plan RestoreData. - restoreDataStageID := p.NewStageOnNodes(nodes) - restoreDataProcs := make(map[roachpb.NodeID]physicalplan.ProcessorIdx) - for _, n := range nodes { + restoreDataStageID := p.NewStageOnNodes(sqlInstanceIDs) + restoreDataProcs := make(map[base.SQLInstanceID]physicalplan.ProcessorIdx) + for _, sqlInstanceID := range sqlInstanceIDs { proc := physicalplan.Processor{ - Node: n, + SQLInstanceID: sqlInstanceID, Spec: execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{ {ColumnTypes: splitAndScatterOutputTypes}, @@ -180,17 +181,17 @@ func distRestore( }, } pIdx := p.AddProcessor(proc) - restoreDataProcs[n] = pIdx + restoreDataProcs[sqlInstanceID] = pIdx p.ResultRouters = append(p.ResultRouters, pIdx) } for _, srcProc := range splitAndScatterProcs { slot := 0 - for _, destNode := range nodes { + for _, destSQLInstanceID := range sqlInstanceIDs { // Streams were added to the range router in the same order that the // nodes appeared in `nodes`. Make sure that the `slot`s here are // ordered the same way. - destProc := restoreDataProcs[destNode] + destProc := restoreDataProcs[destSQLInstanceID] p.Streams = append(p.Streams, physicalplan.Stream{ SourceProcessor: srcProc, SourceRouterSlot: slot, @@ -236,17 +237,19 @@ func distRestore( // spec that should be planned on that node. Given the chunks of ranges to // import it round-robin distributes the chunks amongst the given nodes. func makeSplitAndScatterSpecs( - nodes []roachpb.NodeID, chunks [][]execinfrapb.RestoreSpanEntry, rekeys []execinfrapb.TableRekey, -) (map[roachpb.NodeID]*execinfrapb.SplitAndScatterSpec, error) { - specsByNodes := make(map[roachpb.NodeID]*execinfrapb.SplitAndScatterSpec) + sqlInstanceIDs []base.SQLInstanceID, + chunks [][]execinfrapb.RestoreSpanEntry, + rekeys []execinfrapb.TableRekey, +) (map[base.SQLInstanceID]*execinfrapb.SplitAndScatterSpec, error) { + specsBySQLInstanceID := make(map[base.SQLInstanceID]*execinfrapb.SplitAndScatterSpec) for i, chunk := range chunks { - node := nodes[i%len(nodes)] - if spec, ok := specsByNodes[node]; ok { + sqlInstanceID := sqlInstanceIDs[i%len(sqlInstanceIDs)] + if spec, ok := specsBySQLInstanceID[sqlInstanceID]; ok { spec.Chunks = append(spec.Chunks, execinfrapb.SplitAndScatterSpec_RestoreEntryChunk{ Entries: chunk, }) } else { - specsByNodes[node] = &execinfrapb.SplitAndScatterSpec{ + specsBySQLInstanceID[sqlInstanceID] = &execinfrapb.SplitAndScatterSpec{ Chunks: []execinfrapb.SplitAndScatterSpec_RestoreEntryChunk{{ Entries: chunk, }}, @@ -254,5 +257,5 @@ func makeSplitAndScatterSpecs( } } } - return specsByNodes, nil + return specsBySQLInstanceID, nil } diff --git a/pkg/ccl/backupccl/split_and_scatter_processor.go b/pkg/ccl/backupccl/split_and_scatter_processor.go index da06c91aafbd..2e11c11e5770 100644 --- a/pkg/ccl/backupccl/split_and_scatter_processor.go +++ b/pkg/ccl/backupccl/split_and_scatter_processor.go @@ -13,6 +13,7 @@ import ( "fmt" "time" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -291,7 +292,7 @@ func (ssp *splitAndScatterProcessor) Next() (rowenc.EncDatumRow, *execinfrapb.Pr // The routing datums informs the router which output stream should be used. routingDatum, ok := ssp.routingDatumCache[scatteredEntry.node] if !ok { - routingDatum, _ = routingDatumsForNode(scatteredEntry.node) + routingDatum, _ = routingDatumsForSQLInstance(base.SQLInstanceID(scatteredEntry.node)) ssp.routingDatumCache[scatteredEntry.node] = routingDatum } @@ -414,18 +415,20 @@ func runSplitAndScatter( return g.Wait() } -func routingDatumsForNode(nodeID roachpb.NodeID) (rowenc.EncDatum, rowenc.EncDatum) { - routingBytes := roachpb.Key(fmt.Sprintf("node%d", nodeID)) +func routingDatumsForSQLInstance( + sqlInstanceID base.SQLInstanceID, +) (rowenc.EncDatum, rowenc.EncDatum) { + routingBytes := roachpb.Key(fmt.Sprintf("node%d", sqlInstanceID)) startDatum := rowenc.DatumToEncDatum(types.Bytes, tree.NewDBytes(tree.DBytes(routingBytes))) endDatum := rowenc.DatumToEncDatum(types.Bytes, tree.NewDBytes(tree.DBytes(routingBytes.Next()))) return startDatum, endDatum } -// routingSpanForNode provides the mapping to be used during distsql planning +// routingSpanForSQLInstance provides the mapping to be used during distsql planning // when setting up the output router. -func routingSpanForNode(nodeID roachpb.NodeID) ([]byte, []byte, error) { +func routingSpanForSQLInstance(sqlInstanceID base.SQLInstanceID) ([]byte, []byte, error) { var alloc tree.DatumAlloc - startDatum, endDatum := routingDatumsForNode(nodeID) + startDatum, endDatum := routingDatumsForSQLInstance(sqlInstanceID) startBytes, endBytes := make([]byte, 0), make([]byte, 0) startBytes, err := startDatum.Encode(splitAndScatterOutputTypes[0], &alloc, descpb.DatumEncoding_ASCENDING_KEY, startBytes) diff --git a/pkg/ccl/backupccl/split_and_scatter_processor_test.go b/pkg/ccl/backupccl/split_and_scatter_processor_test.go index a2db7b105ff3..930f8a4ef3e6 100644 --- a/pkg/ccl/backupccl/split_and_scatter_processor_test.go +++ b/pkg/ccl/backupccl/split_and_scatter_processor_test.go @@ -198,7 +198,7 @@ func TestSplitAndScatterProcessor(t *testing.T) { } for stream := 0; stream < c.numStreams; stream++ { // In this test, nodes are 1 indexed. - startBytes, endBytes, err := routingSpanForNode(roachpb.NodeID(stream + 1)) + startBytes, endBytes, err := routingSpanForSQLInstance(base.SQLInstanceID(stream + 1)) require.NoError(t, err) span := execinfrapb.OutputRouterSpec_RangeRouterSpec_Span{ diff --git a/pkg/ccl/changefeedccl/changefeeddist/distflow.go b/pkg/ccl/changefeedccl/changefeeddist/distflow.go index 4efe9675b3f4..ead2c0791fec 100644 --- a/pkg/ccl/changefeedccl/changefeeddist/distflow.go +++ b/pkg/ccl/changefeedccl/changefeeddist/distflow.go @@ -52,7 +52,7 @@ func StartDistChangefeed( var spanPartitions []sql.SpanPartition if details.SinkURI == `` { // Sinkless feeds get one ChangeAggregator on the gateway. - spanPartitions = []sql.SpanPartition{{Node: dsp.GatewayID(), Spans: trackedSpans}} + spanPartitions = []sql.SpanPartition{{SQLInstanceID: dsp.GatewayID(), Spans: trackedSpans}} } else { // All other feeds get a ChangeAggregator local on the leaseholder. var err error @@ -86,7 +86,7 @@ func StartDistChangefeed( UserProto: execCtx.User().EncodeProto(), JobID: jobID, } - corePlacement[i].NodeID = sp.Node + corePlacement[i].SQLInstanceID = sp.SQLInstanceID corePlacement[i].Core.ChangeAggregator = spec } diff --git a/pkg/ccl/importccl/BUILD.bazel b/pkg/ccl/importccl/BUILD.bazel index 3d7e5e4b51b7..2c6f9d3063c9 100644 --- a/pkg/ccl/importccl/BUILD.bazel +++ b/pkg/ccl/importccl/BUILD.bazel @@ -23,6 +23,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/ccl/importccl", visibility = ["//visibility:public"], deps = [ + "//pkg/base", "//pkg/ccl/backupccl", "//pkg/ccl/storageccl", "//pkg/ccl/utilccl", diff --git a/pkg/ccl/importccl/import_processor_planning.go b/pkg/ccl/importccl/import_processor_planning.go index a5568afb93c9..bb60bbdac9c7 100644 --- a/pkg/ccl/importccl/import_processor_planning.go +++ b/pkg/ccl/importccl/import_processor_planning.go @@ -14,6 +14,7 @@ import ( "sync/atomic" "time" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" @@ -54,7 +55,7 @@ func distImport( dsp := execCtx.DistSQLPlanner() evalCtx := execCtx.ExtendedEvalContext() - planCtx, nodes, err := dsp.SetupAllNodesPlanning(ctx, evalCtx, execCtx.ExecCfg()) + planCtx, sqlInstanceIDs, err := dsp.SetupAllNodesPlanning(ctx, evalCtx, execCtx.ExecCfg()) if err != nil { return roachpb.BulkOpSummary{}, err } @@ -70,7 +71,7 @@ func distImport( accumulatedBulkSummary.BulkOpSummary = getLastImportSummary(job) accumulatedBulkSummary.Unlock() - inputSpecs := makeImportReaderSpecs(job, tables, typeDescs, from, format, nodes, walltime, + inputSpecs := makeImportReaderSpecs(job, tables, typeDescs, from, format, sqlInstanceIDs, walltime, execCtx.User()) p := planCtx.NewPhysicalPlan() @@ -78,7 +79,7 @@ func distImport( // Setup a one-stage plan with one proc per input spec. corePlacement := make([]physicalplan.ProcessorCorePlacement, len(inputSpecs)) for i := range inputSpecs { - corePlacement[i].NodeID = nodes[i] + corePlacement[i].SQLInstanceID = sqlInstanceIDs[i] corePlacement[i].Core.ReadImport = inputSpecs[i] } p.AddNoInputStage( @@ -237,19 +238,19 @@ func makeImportReaderSpecs( typeDescs []*descpb.TypeDescriptor, from []string, format roachpb.IOFileFormat, - nodes []roachpb.NodeID, + sqlInstanceIDs []base.SQLInstanceID, walltime int64, user security.SQLUsername, ) []*execinfrapb.ReadImportDataSpec { details := job.Details().(jobspb.ImportDetails) // For each input file, assign it to a node. - inputSpecs := make([]*execinfrapb.ReadImportDataSpec, 0, len(nodes)) + inputSpecs := make([]*execinfrapb.ReadImportDataSpec, 0, len(sqlInstanceIDs)) progress := job.Progress() importProgress := progress.GetImport() for i, input := range from { - // Round robin assign CSV files to nodes. Files 0 through len(nodes)-1 + // Round robin assign CSV files to sqlInstanceIDs. Files 0 through len(sqlInstanceIDs)-1 // creates the spec. Future files just add themselves to the Uris. - if i < len(nodes) { + if i < len(sqlInstanceIDs) { spec := &execinfrapb.ReadImportDataSpec{ Tables: tables, Types: typeDescs, @@ -266,7 +267,7 @@ func makeImportReaderSpecs( } inputSpecs = append(inputSpecs, spec) } - n := i % len(nodes) + n := i % len(sqlInstanceIDs) inputSpecs[n].Uri[int32(i)] = input if importProgress.ResumePos != nil { inputSpecs[n].ResumePos[int32(i)] = importProgress.ResumePos[int32(i)] diff --git a/pkg/ccl/streamingccl/streamingest/BUILD.bazel b/pkg/ccl/streamingccl/streamingest/BUILD.bazel index f6271f3e1eb0..2698dd77a06e 100644 --- a/pkg/ccl/streamingccl/streamingest/BUILD.bazel +++ b/pkg/ccl/streamingccl/streamingest/BUILD.bazel @@ -14,6 +14,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamingest", visibility = ["//visibility:public"], deps = [ + "//pkg/base", "//pkg/ccl/changefeedccl/cdctest", "//pkg/ccl/storageccl", "//pkg/ccl/streamingccl", diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go index 767f763ca711..03340bf7fb53 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go @@ -66,20 +66,20 @@ func ingest( evalCtx := execCtx.ExtendedEvalContext() dsp := execCtx.DistSQLPlanner() - planCtx, nodes, err := dsp.SetupAllNodesPlanning(ctx, evalCtx, execCtx.ExecCfg()) + planCtx, sqlInstanceIDs, err := dsp.SetupAllNodesPlanning(ctx, evalCtx, execCtx.ExecCfg()) if err != nil { return err } // Construct stream ingestion processor specs. streamIngestionSpecs, streamIngestionFrontierSpec, err := distStreamIngestionPlanSpecs( - streamAddress, topology, nodes, initialHighWater, jobID) + streamAddress, topology, sqlInstanceIDs, initialHighWater, jobID) if err != nil { return err } // Plan and run the DistSQL flow. - return distStreamIngest(ctx, execCtx, nodes, jobID, planCtx, dsp, streamIngestionSpecs, + return distStreamIngest(ctx, execCtx, sqlInstanceIDs, jobID, planCtx, dsp, streamIngestionSpecs, streamIngestionFrontierSpec) } diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_planning.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_planning.go index c26f0b3d93fa..6bb2ee816a69 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_planning.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_planning.go @@ -11,6 +11,7 @@ package streamingest import ( "context" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl" "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient" "github.com/cockroachdb/cockroach/pkg/jobs" @@ -30,20 +31,20 @@ import ( func distStreamIngestionPlanSpecs( streamAddress streamingccl.StreamAddress, topology streamclient.Topology, - nodes []roachpb.NodeID, + sqlInstanceIDs []base.SQLInstanceID, initialHighWater hlc.Timestamp, jobID jobspb.JobID, ) ([]*execinfrapb.StreamIngestionDataSpec, *execinfrapb.StreamIngestionFrontierSpec, error) { // For each stream partition in the topology, assign it to a node. - streamIngestionSpecs := make([]*execinfrapb.StreamIngestionDataSpec, 0, len(nodes)) + streamIngestionSpecs := make([]*execinfrapb.StreamIngestionDataSpec, 0, len(sqlInstanceIDs)) trackedSpans := make([]roachpb.Span, 0) for i, partition := range topology { // Round robin assign the stream partitions to nodes. Partitions 0 through // len(nodes) - 1 creates the spec. Future partitions just add themselves to // the partition addresses. - if i < len(nodes) { + if i < len(sqlInstanceIDs) { spec := &execinfrapb.StreamIngestionDataSpec{ JobID: int64(jobID), StartTime: initialHighWater, @@ -52,7 +53,7 @@ func distStreamIngestionPlanSpecs( } streamIngestionSpecs = append(streamIngestionSpecs, spec) } - n := i % len(nodes) + n := i % len(sqlInstanceIDs) streamIngestionSpecs[n].PartitionIds = append(streamIngestionSpecs[n].PartitionIds, partition.ID) streamIngestionSpecs[n].PartitionSpecs = append(streamIngestionSpecs[n].PartitionSpecs, @@ -80,7 +81,7 @@ func distStreamIngestionPlanSpecs( func distStreamIngest( ctx context.Context, execCtx sql.JobExecContext, - nodes []roachpb.NodeID, + sqlInstanceIDs []base.SQLInstanceID, jobID jobspb.JobID, planCtx *sql.PlanningCtx, dsp *sql.DistSQLPlanner, @@ -98,7 +99,7 @@ func distStreamIngest( // Setup a one-stage plan with one proc per input spec. corePlacement := make([]physicalplan.ProcessorCorePlacement, len(streamIngestionSpecs)) for i := range streamIngestionSpecs { - corePlacement[i].NodeID = nodes[i] + corePlacement[i].SQLInstanceID = sqlInstanceIDs[i] corePlacement[i].Core.StreamIngestionData = streamIngestionSpecs[i] } @@ -118,7 +119,7 @@ func distStreamIngest( // The ResultRouters from the previous stage will feed in to the // StreamIngestionFrontier processor. - p.AddSingleGroupStage(gatewayNodeID, + p.AddSingleGroupStage(base.SQLInstanceID(gatewayNodeID), execinfrapb.ProcessorCoreUnion{StreamIngestionFrontier: streamIngestionFrontierSpec}, execinfrapb.PostProcessSpec{}, streamIngestionResultTypes) diff --git a/pkg/ccl/streamingccl/streamproducer/replication_stream_planning.go b/pkg/ccl/streamingccl/streamproducer/replication_stream_planning.go index fd6ad47ccc69..325cfe2dbcc1 100644 --- a/pkg/ccl/streamingccl/streamproducer/replication_stream_planning.go +++ b/pkg/ccl/streamingccl/streamproducer/replication_stream_planning.go @@ -244,12 +244,12 @@ func getReplicationStreamSpec( Partitions: make([]streampb.ReplicationStreamSpec_Partition, 0, len(spanPartitions)), } for _, sp := range spanPartitions { - nodeInfo, err := dsp.GetNodeInfo(sp.Node) + nodeInfo, err := dsp.GetSQLInstanceInfo(sp.SQLInstanceID) if err != nil { return nil, err } res.Partitions = append(res.Partitions, streampb.ReplicationStreamSpec_Partition{ - NodeID: sp.Node, + NodeID: roachpb.NodeID(sp.SQLInstanceID), SQLAddress: nodeInfo.SQLAddress, Locality: nodeInfo.Locality, PartitionSpec: &streampb.StreamPartitionSpec{ diff --git a/pkg/gossip/keys.go b/pkg/gossip/keys.go index 2e1fcd893001..ac0d6fc82979 100644 --- a/pkg/gossip/keys.go +++ b/pkg/gossip/keys.go @@ -15,6 +15,7 @@ import ( "strconv" "strings" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/errors" ) @@ -172,14 +173,14 @@ func StoreIDFromKey(storeKey string) (roachpb.StoreID, error) { } // MakeDistSQLNodeVersionKey returns the gossip key for the given store. -func MakeDistSQLNodeVersionKey(nodeID roachpb.NodeID) string { - return MakeKey(KeyDistSQLNodeVersionKeyPrefix, nodeID.String()) +func MakeDistSQLNodeVersionKey(instanceID base.SQLInstanceID) string { + return MakeKey(KeyDistSQLNodeVersionKeyPrefix, instanceID.String()) } // MakeDistSQLDrainingKey returns the gossip key for the given node's distsql // draining state. -func MakeDistSQLDrainingKey(nodeID roachpb.NodeID) string { - return MakeKey(KeyDistSQLDrainingPrefix, nodeID.String()) +func MakeDistSQLDrainingKey(instanceID base.SQLInstanceID) string { + return MakeKey(KeyDistSQLDrainingPrefix, instanceID.String()) } // removePrefixFromKey removes the key prefix and separator and returns what's diff --git a/pkg/server/server.go b/pkg/server/server.go index bd231651fafe..b4ffde2b3344 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -113,7 +113,7 @@ import ( "google.golang.org/grpc/codes" "google.golang.org/grpc/metadata" grpcstatus "google.golang.org/grpc/status" - yaml "gopkg.in/yaml.v2" + "gopkg.in/yaml.v2" ) var ( @@ -1679,7 +1679,7 @@ func (s *Server) PreStart(ctx context.Context) error { s.cfg.NodeAttributes, s.cfg.Locality, s.cfg.LocalityAddresses, - s.sqlServer.execCfg.DistSQLPlanner.SetNodeInfo, + s.sqlServer.execCfg.DistSQLPlanner.SetSQLInstanceInfo, ); err != nil { return err } diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 65cd07b445b4..f7f736a6537e 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -605,7 +605,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { NodeID: cfg.nodeIDContainer, } - var isAvailable func(roachpb.NodeID) bool + var isAvailable func(sqlInstanceID base.SQLInstanceID) bool nodeLiveness, hasNodeLiveness := cfg.nodeLiveness.Optional(47900) if hasNodeLiveness { // TODO(erikgrinaker): We may want to use IsAvailableNotDraining instead, to @@ -613,11 +613,13 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { // that are being drained/decommissioned. However, these nodes can still be // leaseholders, and preventing processor scheduling on them can cause a // performance cliff for e.g. table reads that then hit the network. - isAvailable = nodeLiveness.IsAvailable + isAvailable = func(sqlInstanceID base.SQLInstanceID) bool { + return nodeLiveness.IsAvailable(roachpb.NodeID(sqlInstanceID)) + } } else { // We're on a SQL tenant, so this is the only node DistSQL will ever // schedule on - always returning true is fine. - isAvailable = func(roachpb.NodeID) bool { + isAvailable = func(sqlInstanceID base.SQLInstanceID) bool { return true } } @@ -671,7 +673,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { ctx, execinfra.Version, cfg.Settings, - roachpb.NodeID(cfg.nodeIDContainer.SQLInstanceID()), + cfg.nodeIDContainer.SQLInstanceID(), cfg.rpcContext, distSQLServer, cfg.distSender, @@ -1041,7 +1043,7 @@ func (s *SQLServer) initInstanceID(ctx context.Context) error { return err } s.sqlLivenessSessionID = sessionID - s.execCfg.DistSQLPlanner.SetNodeInfo(roachpb.NodeDescriptor{NodeID: roachpb.NodeID(instanceID)}) + s.execCfg.DistSQLPlanner.SetSQLInstanceInfo(roachpb.NodeDescriptor{NodeID: roachpb.NodeID(instanceID)}) return nil } diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index 8bba62ec42e8..731f83ee7217 100644 --- a/pkg/server/tenant.go +++ b/pkg/server/tenant.go @@ -177,7 +177,7 @@ func StartTenant( // TODO(asubiotto): remove this. Right now it is needed to initialize the // SpanResolver. - s.execCfg.DistSQLPlanner.SetNodeInfo(roachpb.NodeDescriptor{NodeID: 0}) + s.execCfg.DistSQLPlanner.SetSQLInstanceInfo(roachpb.NodeDescriptor{NodeID: 0}) // Register and start gRPC service on pod. This is separate from the // gRPC + Gateway services configured below. diff --git a/pkg/sql/colflow/colrpc/BUILD.bazel b/pkg/sql/colflow/colrpc/BUILD.bazel index fe333a017115..d8305be7712e 100644 --- a/pkg/sql/colflow/colrpc/BUILD.bazel +++ b/pkg/sql/colflow/colrpc/BUILD.bazel @@ -9,9 +9,9 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/colflow/colrpc", visibility = ["//visibility:public"], deps = [ + "//pkg/base", "//pkg/col/coldata", "//pkg/col/colserde", - "//pkg/roachpb:with-mocks", "//pkg/sql/colexec/colexecargs", "//pkg/sql/colexec/colexecutils", "//pkg/sql/colexecerror", @@ -46,10 +46,10 @@ go_test( ], embed = [":colrpc"], deps = [ + "//pkg/base", "//pkg/col/coldata", "//pkg/col/coldatatestutils", "//pkg/col/colserde", - "//pkg/roachpb:with-mocks", "//pkg/settings/cluster", "//pkg/sql/colexec/colexecargs", "//pkg/sql/colexec/colexectestutils", diff --git a/pkg/sql/colflow/colrpc/colrpc_test.go b/pkg/sql/colflow/colrpc/colrpc_test.go index a44d71eb3aea..74383e53c30b 100644 --- a/pkg/sql/colflow/colrpc/colrpc_test.go +++ b/pkg/sql/colflow/colrpc/colrpc_test.go @@ -20,9 +20,9 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" @@ -136,7 +136,7 @@ func TestOutboxInbox(t *testing.T) { defer stopper.Stop(ctx) clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - _, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, clock, stopper, execinfra.StaticNodeID) + _, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, clock, stopper, execinfra.StaticSQLInstanceID) require.NoError(t, err) // Generate a random cancellation scenario. @@ -487,7 +487,7 @@ func TestInboxHostCtxCancellation(t *testing.T) { defer stopper.Stop(ctx) clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - _, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, clock, stopper, execinfra.StaticNodeID) + _, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, clock, stopper, execinfra.StaticSQLInstanceID) require.NoError(t, err) rng, _ := randutil.NewTestRand() @@ -574,7 +574,7 @@ func TestOutboxInboxMetadataPropagation(t *testing.T) { defer stopper.Stop(ctx) _, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, - hlc.NewClock(hlc.UnixNano, time.Nanosecond), stopper, execinfra.StaticNodeID, + hlc.NewClock(hlc.UnixNano, time.Nanosecond), stopper, execinfra.StaticSQLInstanceID, ) require.NoError(t, err) @@ -757,7 +757,7 @@ func BenchmarkOutboxInbox(b *testing.B) { defer stopper.Stop(ctx) _, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, - hlc.NewClock(hlc.UnixNano, time.Nanosecond), stopper, execinfra.StaticNodeID, + hlc.NewClock(hlc.UnixNano, time.Nanosecond), stopper, execinfra.StaticSQLInstanceID, ) require.NoError(b, err) @@ -830,7 +830,7 @@ func TestOutboxStreamIDPropagation(t *testing.T) { defer stopper.Stop(ctx) _, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, - hlc.NewClock(hlc.UnixNano, time.Nanosecond), stopper, execinfra.StaticNodeID, + hlc.NewClock(hlc.UnixNano, time.Nanosecond), stopper, execinfra.StaticSQLInstanceID, ) require.NoError(t, err) dialer := &execinfrapb.MockDialer{Addr: addr} @@ -861,7 +861,7 @@ func TestOutboxStreamIDPropagation(t *testing.T) { outbox.Run( ctx, dialer, - roachpb.NodeID(0), + base.SQLInstanceID(0), execinfrapb.FlowID{UUID: uuid.MakeV4()}, outboxStreamID, nil, /* flowCtxCancel */ diff --git a/pkg/sql/colflow/colrpc/outbox.go b/pkg/sql/colflow/colrpc/outbox.go index c61fad313ce9..7a2a45d3b7e8 100644 --- a/pkg/sql/colflow/colrpc/outbox.go +++ b/pkg/sql/colflow/colrpc/outbox.go @@ -17,8 +17,8 @@ import ( "sync/atomic" "time" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/col/colserde" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" @@ -147,7 +147,7 @@ func (o *Outbox) close(ctx context.Context) { func (o *Outbox) Run( ctx context.Context, dialer execinfra.Dialer, - nodeID roachpb.NodeID, + sqlInstanceID base.SQLInstanceID, flowID execinfrapb.FlowID, streamID execinfrapb.StreamID, flowCtxCancel context.CancelFunc, @@ -169,11 +169,11 @@ func (o *Outbox) Run( o.runnerCtx = ctx ctx = logtags.AddTag(ctx, "streamID", streamID) - log.VEventf(ctx, 2, "Outbox Dialing %s", nodeID) + log.VEventf(ctx, 2, "Outbox Dialing %s", sqlInstanceID) var stream execinfrapb.DistSQL_FlowStreamClient if err := func() error { - conn, err := execinfra.GetConnForOutbox(ctx, dialer, nodeID, connectionTimeout) + conn, err := execinfra.GetConnForOutbox(ctx, dialer, sqlInstanceID, connectionTimeout) if err != nil { log.Warningf( ctx, diff --git a/pkg/sql/colflow/explain_vec.go b/pkg/sql/colflow/explain_vec.go index b8c461663576..803d936c0c2b 100644 --- a/pkg/sql/colflow/explain_vec.go +++ b/pkg/sql/colflow/explain_vec.go @@ -16,8 +16,8 @@ import ( "reflect" "sort" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" @@ -95,8 +95,8 @@ func (f fakeBatchReceiver) PushBatch( } type flowWithNode struct { - nodeID roachpb.NodeID - flow *execinfrapb.FlowSpec + sqlInstanceID base.SQLInstanceID + flow *execinfrapb.FlowSpec } // ExplainVec converts the flows (that are assumed to be vectorizable) into the @@ -113,10 +113,10 @@ type flowWithNode struct { func ExplainVec( ctx context.Context, flowCtx *execinfra.FlowCtx, - flows map[roachpb.NodeID]*execinfrapb.FlowSpec, + flows map[base.SQLInstanceID]*execinfrapb.FlowSpec, localProcessors []execinfra.LocalProcessor, opChains execinfra.OpChains, - gatewayNodeID roachpb.NodeID, + gatewaySQLInstanceID base.SQLInstanceID, verbose bool, distributed bool, ) (_ []string, cleanup func(), _ error) { @@ -139,15 +139,15 @@ func ExplainVec( // catching such errors. if err = colexecerror.CatchVectorizedRuntimeError(func() { if opChains != nil { - formatChains(root, gatewayNodeID, opChains, verbose) + formatChains(root, gatewaySQLInstanceID, opChains, verbose) } else { sortedFlows := make([]flowWithNode, 0, len(flows)) for nodeID, flow := range flows { - sortedFlows = append(sortedFlows, flowWithNode{nodeID: nodeID, flow: flow}) + sortedFlows = append(sortedFlows, flowWithNode{sqlInstanceID: nodeID, flow: flow}) } // Sort backward, since the first thing you add to a treeprinter will come // last. - sort.Slice(sortedFlows, func(i, j int) bool { return sortedFlows[i].nodeID < sortedFlows[j].nodeID }) + sort.Slice(sortedFlows, func(i, j int) bool { return sortedFlows[i].sqlInstanceID < sortedFlows[j].sqlInstanceID }) for _, flow := range sortedFlows { opChains, cleanup, err = convertToVecTree(ctx, flowCtx, flow.flow, localProcessors, !distributed) cleanups = append(cleanups, cleanup) @@ -155,7 +155,7 @@ func ExplainVec( conversionErr = err return } - formatChains(root, flow.nodeID, opChains, verbose) + formatChains(root, flow.sqlInstanceID, opChains, verbose) } } }); err != nil { @@ -168,9 +168,12 @@ func ExplainVec( } func formatChains( - root treeprinter.Node, nodeID roachpb.NodeID, opChains execinfra.OpChains, verbose bool, + root treeprinter.Node, + sqlInstanceID base.SQLInstanceID, + opChains execinfra.OpChains, + verbose bool, ) { - node := root.Childf("Node %d", nodeID) + node := root.Childf("Node %d", sqlInstanceID) for _, op := range opChains { formatOpChain(op, node, verbose) } diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index abdd4c060c37..f67d0bc82507 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -404,7 +404,7 @@ func (s *vectorizedFlowCreator) wrapWithNetworkVectorizedStatsCollector( func (s *vectorizedFlowCreator) makeGetStatsFnForOutbox( flowCtx *execinfra.FlowCtx, statsCollectors []colexecop.VectorizedStatsCollector, - originNodeID roachpb.NodeID, + originSQLInstanceID base.SQLInstanceID, ) func() []*execinfrapb.ComponentStats { if !s.recordingStats { return nil @@ -424,7 +424,7 @@ func (s *vectorizedFlowCreator) makeGetStatsFnForOutbox( // whole flow from parent monitors. These stats are added to a // flow-level span. result = append(result, &execinfrapb.ComponentStats{ - Component: execinfrapb.FlowComponentID(base.SQLInstanceID(originNodeID), flowCtx.ID), + Component: execinfrapb.FlowComponentID(originSQLInstanceID, flowCtx.ID), FlowStats: execinfrapb.FlowStats{ MaxMemUsage: optional.MakeUint(uint64(flowCtx.EvalCtx.Mon.MaximumBytes())), MaxDiskUsage: optional.MakeUint(uint64(flowCtx.DiskMonitor.MaximumBytes())), @@ -828,7 +828,7 @@ func (s *vectorizedFlowCreator) setupInput( // Retrieve the latency from the origin node (the one that has the // outbox). - latency, err := s.nodeDialer.Latency(inputStream.OriginNodeID) + latency, err := s.nodeDialer.Latency(roachpb.NodeID(inputStream.OriginNodeID)) if err != nil { // If an error occurred, latency's nil value of 0 is used. If latency is // 0, it is not included in the displayed stats for EXPLAIN ANALYZE @@ -864,7 +864,7 @@ func (s *vectorizedFlowCreator) setupInput( // 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, + inputStream.OriginNodeID, flowCtx.ID, inputStream.StreamID, ) s.wrapWithNetworkVectorizedStatsCollector(&opWithMetaInfo, inbox, compID, latency) } diff --git a/pkg/sql/colflow/vectorized_flow_shutdown_test.go b/pkg/sql/colflow/vectorized_flow_shutdown_test.go index 4bc4e2d8faed..a01f66ec9075 100644 --- a/pkg/sql/colflow/vectorized_flow_shutdown_test.go +++ b/pkg/sql/colflow/vectorized_flow_shutdown_test.go @@ -125,7 +125,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) _, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, - hlc.NewClock(hlc.UnixNano, time.Nanosecond), stopper, execinfra.StaticNodeID, + hlc.NewClock(hlc.UnixNano, time.Nanosecond), stopper, execinfra.StaticSQLInstanceID, ) require.NoError(t, err) dialer := &execinfrapb.MockDialer{Addr: addr} @@ -274,7 +274,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { outbox.Run( outboxCtx, dialer, - execinfra.StaticNodeID, + execinfra.StaticSQLInstanceID, flowID, execinfrapb.StreamID(id), flowCtxCancel, diff --git a/pkg/sql/conn_executor_internal_test.go b/pkg/sql/conn_executor_internal_test.go index 143336c507dd..4ca32c5db50d 100644 --- a/pkg/sql/conn_executor_internal_test.go +++ b/pkg/sql/conn_executor_internal_test.go @@ -292,7 +292,7 @@ func startConnExecutor( }, Codec: keys.SystemSQLCodec, DistSQLPlanner: NewDistSQLPlanner( - ctx, execinfra.Version, st, roachpb.NodeID(1), + ctx, execinfra.Version, st, 1, /* sqlInstanceID */ nil, /* rpcCtx */ distsql.NewServer( ctx, @@ -311,7 +311,7 @@ func startConnExecutor( nil, /* nodeDescs */ gw, stopper, - func(roachpb.NodeID) bool { return true }, // everybody is available + func(base.SQLInstanceID) bool { return true }, // everybody is available nil, /* nodeDialer */ ), QueryCache: querycache.New(0), diff --git a/pkg/sql/distsql/BUILD.bazel b/pkg/sql/distsql/BUILD.bazel index 6cabc87c53b3..6bc4bff4421c 100644 --- a/pkg/sql/distsql/BUILD.bazel +++ b/pkg/sql/distsql/BUILD.bazel @@ -6,6 +6,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/distsql", visibility = ["//visibility:public"], deps = [ + "//pkg/base", "//pkg/gossip", "//pkg/kv", "//pkg/roachpb:with-mocks", diff --git a/pkg/sql/distsql/inbound_test.go b/pkg/sql/distsql/inbound_test.go index fc5ed9cc3dfa..c667d574d5df 100644 --- a/pkg/sql/distsql/inbound_test.go +++ b/pkg/sql/distsql/inbound_test.go @@ -39,10 +39,10 @@ import ( "github.com/stretchr/testify/require" ) -// staticAddressResolver maps execinfra.StaticNodeID to the given address. +// staticAddressResolver maps execinfra.StaticSQLInstanceID to the given address. func staticAddressResolver(addr net.Addr) nodedialer.AddressResolver { return func(nodeID roachpb.NodeID) (net.Addr, error) { - if nodeID == execinfra.StaticNodeID { + if nodeID == roachpb.NodeID(execinfra.StaticSQLInstanceID) { return addr, nil } return nil, errors.Errorf("node %d not found", nodeID) @@ -98,7 +98,7 @@ func TestOutboxInboundStreamIntegration(t *testing.T) { } streamID := execinfrapb.StreamID(1) - outbox := flowinfra.NewOutbox(&flowCtx, execinfra.StaticNodeID, streamID, nil /* numOutboxes */, false /* isGatewayNode */) + outbox := flowinfra.NewOutbox(&flowCtx, execinfra.StaticSQLInstanceID, streamID, nil /* numOutboxes */, false /* isGatewayNode */) outbox.Init(types.OneIntCol) // WaitGroup for the outbox and inbound stream. If the WaitGroup is done, no diff --git a/pkg/sql/distsql/server.go b/pkg/sql/distsql/server.go index 144b22d61725..e9a266bf0333 100644 --- a/pkg/sql/distsql/server.go +++ b/pkg/sql/distsql/server.go @@ -16,6 +16,7 @@ import ( "sync" "time" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -108,7 +109,7 @@ func (ds *ServerImpl) Start() { if g, ok := ds.ServerConfig.Gossip.Optional(MultiTenancyIssueNo); ok { if nodeID, ok := ds.ServerConfig.NodeID.OptionalNodeID(); ok { if err := g.AddInfoProto( - gossip.MakeDistSQLNodeVersionKey(nodeID), + gossip.MakeDistSQLNodeVersionKey(base.SQLInstanceID(nodeID)), &execinfrapb.DistSQLVersionGossipInfo{ Version: execinfra.Version, MinAcceptedVersion: execinfra.MinAcceptedVersion, @@ -180,7 +181,7 @@ func (ds *ServerImpl) setDraining(drain bool) error { } if g, ok := ds.ServerConfig.Gossip.Optional(MultiTenancyIssueNo); ok { return g.AddInfoProto( - gossip.MakeDistSQLDrainingKey(nodeID), + gossip.MakeDistSQLDrainingKey(base.SQLInstanceID(nodeID)), &execinfrapb.DistSQLDrainingInfo{ Draining: drain, }, @@ -288,7 +289,7 @@ func (ds *ServerImpl) setupFlow( } // The flow will run in a LeafTxn because we do not want each distributed // Txn to heartbeat the transaction. - return kv.NewLeafTxn(ctx, ds.DB, req.Flow.Gateway, tis), nil + return kv.NewLeafTxn(ctx, ds.DB, roachpb.NodeID(req.Flow.Gateway), tis), nil } var evalCtx *tree.EvalContext @@ -366,7 +367,7 @@ func (ds *ServerImpl) setupFlow( // Create the FlowCtx for the flow. flowCtx := ds.newFlowContext( - ctx, req.Flow.FlowID, evalCtx, req.TraceKV, req.CollectStats, localState, req.Flow.Gateway == roachpb.NodeID(ds.NodeID.SQLInstanceID()), + ctx, req.Flow.FlowID, evalCtx, req.TraceKV, req.CollectStats, localState, req.Flow.Gateway == ds.NodeID.SQLInstanceID(), ) // req always contains the desired vectorize mode, regardless of whether we diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 56d9bd6804c7..ba371d6c0700 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -81,13 +81,11 @@ type DistSQLPlanner struct { planVersion execinfrapb.DistSQLVersion st *cluster.Settings - // The nodeID of the gateway node that initiated this query. - // TODO(asubiotto): This usage of NodeID instead of SQLInstanceID is - // temporary: https://github.com/cockroachdb/cockroach/issues/49596 - gatewayNodeID roachpb.NodeID - stopper *stop.Stopper - distSQLSrv *distsql.ServerImpl - spanResolver physicalplan.SpanResolver + // The SQLInstanceID of the gateway node that initiated this query. + gatewaySQLInstanceID base.SQLInstanceID + stopper *stop.Stopper + distSQLSrv *distsql.ServerImpl + spanResolver physicalplan.SpanResolver // metadataTestTolerance is the minimum level required to plan metadata test // processors. @@ -116,11 +114,11 @@ type DistSQLPlanner struct { // for the same stage of the fully local physical plans. parallelLocalScansSem *quotapool.IntPool - // distSender is used to construct the spanResolver upon SetNodeInfo. + // distSender is used to construct the spanResolver upon SetSQLInstanceInfo. distSender *kvcoord.DistSender - // nodeDescs is used to construct the spanResolver upon SetNodeInfo. + // nodeDescs is used to construct the spanResolver upon SetSQLInstanceInfo. nodeDescs kvcoord.NodeDescStore - // rpcCtx is used to construct the spanResolver upon SetNodeInfo. + // rpcCtx is used to construct the spanResolver upon SetSQLInstanceInfo. rpcCtx *rpc.Context } @@ -135,34 +133,32 @@ var logPlanDiagram = envutil.EnvOrDefaultBool("COCKROACH_DISTSQL_LOG_PLAN", fals // NewDistSQLPlanner initializes a DistSQLPlanner. // -// nodeID is the ID of the node on which this planner runs. It is used to favor -// itself and other close-by nodes when planning. An invalid nodeID can be -// passed to aid bootstrapping, but then SetNodeInfo() needs to be called +// sqlInstanceID is the ID of the node on which this planner runs. It is used to +// favor itself and other close-by nodes when planning. An invalid sqlInstanceID +// can be passed to aid bootstrapping, but then SetSQLInstanceInfo() needs to be called // before this planner is used. -// TODO(asubiotto): This usage of NodeID instead of SQLInstanceID is -// temporary: https://github.com/cockroachdb/cockroach/issues/49596 func NewDistSQLPlanner( ctx context.Context, planVersion execinfrapb.DistSQLVersion, st *cluster.Settings, - nodeID roachpb.NodeID, + sqlInstanceID base.SQLInstanceID, rpcCtx *rpc.Context, distSQLSrv *distsql.ServerImpl, distSender *kvcoord.DistSender, nodeDescs kvcoord.NodeDescStore, gw gossip.OptionalGossip, stopper *stop.Stopper, - isAvailable func(roachpb.NodeID) bool, + isAvailable func(base.SQLInstanceID) bool, nodeDialer *nodedialer.Dialer, ) *DistSQLPlanner { dsp := &DistSQLPlanner{ - planVersion: planVersion, - st: st, - gatewayNodeID: nodeID, - stopper: stopper, - distSQLSrv: distSQLSrv, - gossip: gw, - nodeDialer: nodeDialer, + planVersion: planVersion, + st: st, + gatewaySQLInstanceID: sqlInstanceID, + stopper: stopper, + distSQLSrv: distSQLSrv, + gossip: gw, + nodeDialer: nodeDialer, nodeHealth: distSQLNodeHealth{ gossip: gw, connHealth: nodeDialer.ConnHealthTryDial, @@ -193,15 +189,17 @@ func (dsp *DistSQLPlanner) shouldPlanTestMetadata() bool { return dsp.distSQLSrv.TestingKnobs.MetadataTestLevel >= dsp.metadataTestTolerance } -// GetNodeInfo gets a node descriptor by node ID. -func (dsp *DistSQLPlanner) GetNodeInfo(nodeID roachpb.NodeID) (*roachpb.NodeDescriptor, error) { - return dsp.nodeDescs.GetNodeDescriptor(nodeID) +// GetSQLInstanceInfo gets a node descriptor by node ID. +func (dsp *DistSQLPlanner) GetSQLInstanceInfo( + sqlInstanceID base.SQLInstanceID, +) (*roachpb.NodeDescriptor, error) { + return dsp.nodeDescs.GetNodeDescriptor(roachpb.NodeID(sqlInstanceID)) } -// SetNodeInfo sets the planner's node descriptor. -// The first call to SetNodeInfo leads to the construction of the SpanResolver. -func (dsp *DistSQLPlanner) SetNodeInfo(desc roachpb.NodeDescriptor) { - dsp.gatewayNodeID = desc.NodeID +// SetSQLInstanceInfo sets the planner's node descriptor. +// The first call to SetSQLInstanceInfo leads to the construction of the SpanResolver. +func (dsp *DistSQLPlanner) SetSQLInstanceInfo(desc roachpb.NodeDescriptor) { + dsp.gatewaySQLInstanceID = base.SQLInstanceID(desc.NodeID) if dsp.spanResolver == nil { sr := physicalplan.NewSpanResolver(dsp.st, dsp.distSender, dsp.nodeDescs, desc, dsp.rpcCtx, ReplicaOraclePolicy) @@ -210,8 +208,8 @@ func (dsp *DistSQLPlanner) SetNodeInfo(desc roachpb.NodeDescriptor) { } // GatewayID returns the ID of the gateway. -func (dsp *DistSQLPlanner) GatewayID() roachpb.NodeID { - return dsp.gatewayNodeID +func (dsp *DistSQLPlanner) GatewayID() base.SQLInstanceID { + return dsp.gatewaySQLInstanceID } // SetSpanResolver switches to a different SpanResolver. It is the caller's @@ -659,9 +657,9 @@ type PlanningCtx struct { ctx context.Context ExtendedEvalCtx *extendedEvalContext spanIter physicalplan.SpanResolverIterator - // NodesStatuses contains info for all NodeIDs that are referenced by any - // PhysicalPlan we generate with this context. - NodeStatuses map[roachpb.NodeID]NodeStatus + // NodesStatuses contains info for all SQLInstanceIDs that are referenced by + // any PhysicalPlan we generate with this context. + NodeStatuses map[base.SQLInstanceID]NodeStatus infra physicalplan.PhysicalInfrastructure @@ -681,7 +679,7 @@ type PlanningCtx struct { // If set, the flows for the physical plan will be passed to this function. // The flows are not safe for use past the lifetime of the saveFlows function. - saveFlows func(map[roachpb.NodeID]*execinfrapb.FlowSpec, execinfra.OpChains) error + saveFlows func(map[base.SQLInstanceID]*execinfrapb.FlowSpec, execinfra.OpChains) error // If set, we will record the mapping from planNode to tracing metadata to // later allow associating statistics with the planNode. @@ -735,8 +733,8 @@ func (p *PlanningCtx) IsLocal() bool { // plans and their diagrams. func (p *PlanningCtx) getDefaultSaveFlowsFunc( ctx context.Context, planner *planner, typ planComponentType, -) func(map[roachpb.NodeID]*execinfrapb.FlowSpec, execinfra.OpChains) error { - return func(flows map[roachpb.NodeID]*execinfrapb.FlowSpec, opChains execinfra.OpChains) error { +) func(map[base.SQLInstanceID]*execinfrapb.FlowSpec, execinfra.OpChains) error { + return func(flows map[base.SQLInstanceID]*execinfrapb.FlowSpec, opChains execinfra.OpChains) error { var diagram execinfrapb.FlowDiagram if planner.instrumentation.shouldSaveDiagrams() { diagramFlags := execinfrapb.DiagramFlags{ @@ -755,7 +753,7 @@ func (p *PlanningCtx) getDefaultSaveFlowsFunc( getExplain := func(verbose bool) []string { explain, cleanup, err := colflow.ExplainVec( ctx, flowCtx, flows, p.infra.LocalProcessors, opChains, - planner.extendedEvalCtx.DistSQLPlanner.gatewayNodeID, + planner.extendedEvalCtx.DistSQLPlanner.gatewaySQLInstanceID, verbose, planner.curPlan.flags.IsDistributed(), ) cleanup() @@ -790,7 +788,7 @@ func (p *PlanningCtx) getDefaultSaveFlowsFunc( // FlowDiagram using this PlanningCtx's information. func (p *PlanningCtx) flowSpecsToDiagram( ctx context.Context, - flows map[roachpb.NodeID]*execinfrapb.FlowSpec, + flows map[base.SQLInstanceID]*execinfrapb.FlowSpec, diagramFlags execinfrapb.DiagramFlags, ) (execinfrapb.FlowDiagram, error) { log.VEvent(ctx, 1, "creating plan diagram") @@ -880,20 +878,22 @@ func identityMapInPlace(slice []int) []int { return slice } -// SpanPartition is the intersection between a set of spans for a certain -// operation (e.g table scan) and the set of ranges owned by a given node. +// SpanPartition associates a subset of spans with a specific SQL instance, +// chosen to have the most efficient access to those spans. In the single-tenant +// case, the instance is the one running on the same node as the leaseholder for +// those spans. type SpanPartition struct { - Node roachpb.NodeID - Spans roachpb.Spans + SQLInstanceID base.SQLInstanceID + Spans roachpb.Spans } type distSQLNodeHealth struct { gossip gossip.OptionalGossip - isAvailable func(roachpb.NodeID) bool + isAvailable func(base.SQLInstanceID) bool connHealth func(roachpb.NodeID, rpc.ConnectionClass) error } -func (h *distSQLNodeHealth) check(ctx context.Context, nodeID roachpb.NodeID) error { +func (h *distSQLNodeHealth) check(ctx context.Context, sqlInstanceID base.SQLInstanceID) error { { // NB: as of #22658, ConnHealth does not work as expected; see the // comment within. We still keep this code for now because in @@ -902,23 +902,23 @@ func (h *distSQLNodeHealth) check(ctx context.Context, nodeID roachpb.NodeID) er // artifact of rpcContext's reconnection mechanism at the time of // writing). This is better than having it used in 100% of cases // (until the liveness check below kicks in). - err := h.connHealth(nodeID, rpc.DefaultClass) + err := h.connHealth(roachpb.NodeID(sqlInstanceID), rpc.DefaultClass) if err != nil && !errors.Is(err, rpc.ErrNotHeartbeated) { // This host is known to be unhealthy. Don't use it (use the gateway - // instead). Note: this can never happen for our nodeID (which + // instead). Note: this can never happen for our sqlInstanceID (which // always has its address in the nodeMap). - log.VEventf(ctx, 1, "marking n%d as unhealthy for this plan: %v", nodeID, err) + log.VEventf(ctx, 1, "marking n%d as unhealthy for this plan: %v", sqlInstanceID, err) return err } } - if !h.isAvailable(nodeID) { - return pgerror.Newf(pgcode.CannotConnectNow, "not using n%d since it is not available", nodeID) + if !h.isAvailable(sqlInstanceID) { + return pgerror.Newf(pgcode.CannotConnectNow, "not using n%d since it is not available", sqlInstanceID) } // Check that the node is not draining. if g, ok := h.gossip.Optional(distsql.MultiTenancyIssueNo); ok { drainingInfo := &execinfrapb.DistSQLDrainingInfo{} - if err := g.GetInfoProto(gossip.MakeDistSQLDrainingKey(nodeID), drainingInfo); err != nil { + if err := g.GetInfoProto(gossip.MakeDistSQLDrainingKey(sqlInstanceID), drainingInfo); err != nil { // Because draining info has no expiration, an error // implies that we have not yet received a node's // draining information. Since this information is @@ -930,7 +930,7 @@ func (h *distSQLNodeHealth) check(ctx context.Context, nodeID roachpb.NodeID) er } if drainingInfo.Draining { - err := errors.Newf("not using n%d because it is draining", nodeID) + err := errors.Newf("not using n%d because it is draining", sqlInstanceID) log.VEventf(ctx, 1, "%v", err) return err } @@ -959,11 +959,11 @@ func (dsp *DistSQLPlanner) PartitionSpans( if planCtx.isLocal { // If we're planning locally, map all spans to the local node. partitions = append(partitions, - SpanPartition{dsp.gatewayNodeID, spans}) + SpanPartition{dsp.gatewaySQLInstanceID, spans}) return partitions, nil } - // nodeMap maps a nodeID to an index inside the partitions array. - nodeMap := make(map[roachpb.NodeID]int) + // nodeMap maps a SQLInstanceID to an index inside the partitions array. + nodeMap := make(map[base.SQLInstanceID]int) it := planCtx.spanIter for i := range spans { @@ -989,7 +989,7 @@ func (dsp *DistSQLPlanner) PartitionSpans( return nil, err } - var lastNodeID roachpb.NodeID + var lastSQLInstanceID base.SQLInstanceID // lastKey maintains the EndKey of the last piece of `span`. lastKey := rSpan.Key if log.V(1) { @@ -1026,25 +1026,25 @@ func (dsp *DistSQLPlanner) PartitionSpans( endKey = rSpan.EndKey } - nodeID := replDesc.NodeID - partitionIdx, inNodeMap := nodeMap[nodeID] + sqlInstanceID := base.SQLInstanceID(replDesc.NodeID) + partitionIdx, inNodeMap := nodeMap[sqlInstanceID] if !inNodeMap { - // This is the first time we are seeing nodeID for these spans. Check - // its health. - status := dsp.CheckNodeHealthAndVersion(planCtx, nodeID) + // This is the first time we are seeing this sqlInstanceID for these + // spans. Check its health. + status := dsp.CheckInstanceHealthAndVersion(planCtx, sqlInstanceID) // If the node is unhealthy or its DistSQL version is incompatible, use // the gateway to process this span instead of the unhealthy host. // An empty address indicates an unhealthy host. if status != NodeOK { - log.Eventf(ctx, "not planning on node %d: %s", nodeID, status) - nodeID = dsp.gatewayNodeID - partitionIdx, inNodeMap = nodeMap[nodeID] + log.Eventf(ctx, "not planning on node %d: %s", sqlInstanceID, status) + sqlInstanceID = dsp.gatewaySQLInstanceID + partitionIdx, inNodeMap = nodeMap[sqlInstanceID] } if !inNodeMap { partitionIdx = len(partitions) - partitions = append(partitions, SpanPartition{Node: nodeID}) - nodeMap[nodeID] = partitionIdx + partitions = append(partitions, SpanPartition{SQLInstanceID: sqlInstanceID}) + nodeMap[sqlInstanceID] = partitionIdx } } partition := &partitions[partitionIdx] @@ -1058,7 +1058,7 @@ func (dsp *DistSQLPlanner) PartitionSpans( break } - if lastNodeID == nodeID { + if lastSQLInstanceID == sqlInstanceID { // Two consecutive ranges on the same node, merge the spans. partition.Spans[len(partition.Spans)-1].EndKey = endKey.AsRawKey() } else { @@ -1074,7 +1074,7 @@ func (dsp *DistSQLPlanner) PartitionSpans( } lastKey = endKey - lastNodeID = nodeID + lastSQLInstanceID = sqlInstanceID } } return partitions, nil @@ -1083,13 +1083,13 @@ func (dsp *DistSQLPlanner) PartitionSpans( // nodeVersionIsCompatible decides whether a particular node's DistSQL version // is compatible with dsp.planVersion. It uses gossip to find out the node's // version range. -func (dsp *DistSQLPlanner) nodeVersionIsCompatible(nodeID roachpb.NodeID) bool { +func (dsp *DistSQLPlanner) nodeVersionIsCompatible(sqlInstanceID base.SQLInstanceID) bool { g, ok := dsp.gossip.Optional(distsql.MultiTenancyIssueNo) if !ok { return true // no gossip - always compatible; only a single gateway running in Phase 2 } var v execinfrapb.DistSQLVersionGossipInfo - if err := g.GetInfoProto(gossip.MakeDistSQLNodeVersionKey(nodeID), &v); err != nil { + if err := g.GetInfoProto(gossip.MakeDistSQLNodeVersionKey(sqlInstanceID), &v); err != nil { return false } return distsql.FlowVerIsCompatible(dsp.planVersion, v.MinAcceptedVersion, v.Version) @@ -1199,13 +1199,13 @@ func (dsp *DistSQLPlanner) convertOrdering( return result } -// getNodeIDForScan retrieves the node ID where the single table reader should -// reside for a limited scan. Ideally this is the lease holder for the first -// range in the specified spans. But if that node is unhealthy or incompatible, -// we use the gateway node instead. -func (dsp *DistSQLPlanner) getNodeIDForScan( +// getInstanceIDForScan retrieves the SQL Instance ID where the single table reader +// should reside for a limited scan. Ideally this is the lease holder for the +// first range in the specified spans. But if that node is unhealthy or +// incompatible, we use the gateway node instead. +func (dsp *DistSQLPlanner) getInstanceIDForScan( planCtx *PlanningCtx, spans []roachpb.Span, reverse bool, -) (roachpb.NodeID, error) { +) (base.SQLInstanceID, error) { if len(spans) == 0 { panic("no spans") } @@ -1225,33 +1225,33 @@ func (dsp *DistSQLPlanner) getNodeIDForScan( return 0, err } - nodeID := replDesc.NodeID - status := dsp.CheckNodeHealthAndVersion(planCtx, nodeID) + sqlInstanceID := base.SQLInstanceID(replDesc.NodeID) + status := dsp.CheckInstanceHealthAndVersion(planCtx, sqlInstanceID) if status != NodeOK { - log.Eventf(planCtx.ctx, "not planning on node %d: %s", nodeID, status) - return dsp.gatewayNodeID, nil + log.Eventf(planCtx.ctx, "not planning on node %d: %s", sqlInstanceID, status) + return dsp.gatewaySQLInstanceID, nil } - return nodeID, nil + return sqlInstanceID, nil } -// CheckNodeHealthAndVersion returns a information about a node's health and +// CheckInstanceHealthAndVersion returns a information about a node's health and // compatibility. The info is also recorded in planCtx.Nodes. -func (dsp *DistSQLPlanner) CheckNodeHealthAndVersion( - planCtx *PlanningCtx, nodeID roachpb.NodeID, +func (dsp *DistSQLPlanner) CheckInstanceHealthAndVersion( + planCtx *PlanningCtx, sqlInstanceID base.SQLInstanceID, ) NodeStatus { - if status, ok := planCtx.NodeStatuses[nodeID]; ok { + if status, ok := planCtx.NodeStatuses[sqlInstanceID]; ok { return status } var status NodeStatus - if err := dsp.nodeHealth.check(planCtx.ctx, nodeID); err != nil { + if err := dsp.nodeHealth.check(planCtx.ctx, sqlInstanceID); err != nil { status = NodeUnhealthy - } else if !dsp.nodeVersionIsCompatible(nodeID) { + } else if !dsp.nodeVersionIsCompatible(sqlInstanceID) { status = NodeDistSQLVersionIncompatible } else { status = NodeOK } - planCtx.NodeStatuses[nodeID] = status + planCtx.NodeStatuses[sqlInstanceID] = status return status } @@ -1348,12 +1348,12 @@ func (dsp *DistSQLPlanner) maybeParallelizeLocalScans( if err != nil { // For some reason we couldn't partition the spans - fallback to // having a single TableReader. - spanPartitions = []SpanPartition{{dsp.gatewayNodeID, info.spans}} + spanPartitions = []SpanPartition{{dsp.gatewaySQLInstanceID, info.spans}} parallelizeLocal = false return spanPartitions, parallelizeLocal } for i := range spanPartitions { - spanPartitions[i].Node = dsp.gatewayNodeID + spanPartitions[i].SQLInstanceID = dsp.gatewaySQLInstanceID } if len(spanPartitions) > 1 { // We're touching ranges that have leaseholders on multiple nodes, @@ -1401,7 +1401,7 @@ func (dsp *DistSQLPlanner) maybeParallelizeLocalScans( // We weren't able to acquire the quota for any additional // goroutines, so we will fallback to having a single // TableReader. - spanPartitions = []SpanPartition{{dsp.gatewayNodeID, info.spans}} + spanPartitions = []SpanPartition{{dsp.gatewaySQLInstanceID, info.spans}} } } if len(spanPartitions) == 1 { @@ -1411,7 +1411,7 @@ func (dsp *DistSQLPlanner) maybeParallelizeLocalScans( parallelizeLocal = false } } else { - spanPartitions = []SpanPartition{{dsp.gatewayNodeID, info.spans}} + spanPartitions = []SpanPartition{{dsp.gatewaySQLInstanceID, info.spans}} } return spanPartitions, parallelizeLocal } @@ -1439,11 +1439,11 @@ func (dsp *DistSQLPlanner) planTableReaders( } else { // If the scan has a hard limit, use a single TableReader to avoid // reading more rows than necessary. - nodeID, err := dsp.getNodeIDForScan(planCtx, info.spans, info.reverse) + sqlInstanceID, err := dsp.getInstanceIDForScan(planCtx, info.spans, info.reverse) if err != nil { return err } - spanPartitions = []SpanPartition{{nodeID, info.spans}} + spanPartitions = []SpanPartition{{sqlInstanceID, info.spans}} } corePlacement := make([]physicalplan.ProcessorCorePlacement, len(spanPartitions)) @@ -1469,7 +1469,7 @@ func (dsp *DistSQLPlanner) planTableReaders( } p.TotalEstimatedScannedRows += info.estimatedRowCount - corePlacement[i].NodeID = sp.Node + corePlacement[i].SQLInstanceID = sp.SQLInstanceID corePlacement[i].EstimatedRowCount = info.estimatedRowCount corePlacement[i].Core.TableReader = tr } @@ -1486,7 +1486,7 @@ func (dsp *DistSQLPlanner) planTableReaders( if parallelizeLocal { // If we planned multiple table readers, we need to merge the streams // into one. - p.AddSingleGroupStage(dsp.gatewayNodeID, execinfrapb.ProcessorCoreUnion{Noop: &execinfrapb.NoopCoreSpec{}}, execinfrapb.PostProcessSpec{}, p.GetResultTypes()) + p.AddSingleGroupStage(dsp.gatewaySQLInstanceID, execinfrapb.ProcessorCoreUnion{Noop: &execinfrapb.NoopCoreSpec{}}, execinfrapb.PostProcessSpec{}, p.GetResultTypes()) } return nil @@ -1557,7 +1557,7 @@ func (dsp *DistSQLPlanner) addSorters( Limit: uint64(limit), } p.AddSingleGroupStage( - p.GatewayNodeID, + p.GatewaySQLInstanceID, execinfrapb.ProcessorCoreUnion{Noop: &execinfrapb.NoopCoreSpec{}}, post, p.GetResultTypes(), @@ -1711,9 +1711,9 @@ func (dsp *DistSQLPlanner) planAggregators( } // Check if the previous stage is all on one node. - prevStageNode := p.Processors[p.ResultRouters[0]].Node + prevStageNode := p.Processors[p.ResultRouters[0]].SQLInstanceID for i := 1; i < len(p.ResultRouters); i++ { - if n := p.Processors[p.ResultRouters[i]].Node; n != prevStageNode { + if n := p.Processors[p.ResultRouters[i]].SQLInstanceID; n != prevStageNode { prevStageNode = 0 break } @@ -2076,7 +2076,7 @@ func (dsp *DistSQLPlanner) planAggregators( // No GROUP BY, or we have a single stream. Use a single final aggregator. // If the previous stage was all on a single node, put the final // aggregator there. Otherwise, bring the results back on this node. - node := dsp.gatewayNodeID + node := dsp.gatewaySQLInstanceID if prevStageNode != 0 { node = prevStageNode } @@ -2108,7 +2108,7 @@ func (dsp *DistSQLPlanner) planAggregators( prevStageResultTypes := p.GetResultTypes() for _, resultProc := range p.ResultRouters { proc := physicalplan.Processor{ - Node: p.Processors[resultProc].Node, + SQLInstanceID: p.Processors[resultProc].SQLInstanceID, Spec: execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{{ // The other fields will be filled in by mergeResultStreams. @@ -2209,7 +2209,7 @@ func (dsp *DistSQLPlanner) createPlanForIndexJoin( } else { // We have a single stream, so use a single join reader on that node. plan.AddSingleGroupStage( - plan.Processors[plan.ResultRouters[0]].Node, + plan.Processors[plan.ResultRouters[0]].SQLInstanceID, execinfrapb.ProcessorCoreUnion{JoinReader: &joinReaderSpec}, post, types, @@ -2592,8 +2592,8 @@ func (dsp *DistSQLPlanner) planZigzagJoin( // processor across multiple nodes here. Until then, schedule on the current // node. corePlacement := []physicalplan.ProcessorCorePlacement{{ - NodeID: dsp.gatewayNodeID, - Core: execinfrapb.ProcessorCoreUnion{ZigzagJoiner: &zigzagJoinerSpec}, + SQLInstanceID: dsp.gatewaySQLInstanceID, + Core: execinfrapb.ProcessorCoreUnion{ZigzagJoiner: &zigzagJoinerSpec}, }} plan.AddNoInputStage(corePlacement, post, types, execinfrapb.Ordering{}) @@ -2634,8 +2634,8 @@ func (dsp *DistSQLPlanner) createPlanForInvertedFilter( // at the next stage. if len(plan.ResultRouters) == 1 { // Last stage is a single processor. - lastNodeID := plan.Processors[plan.ResultRouters[0]].Node - plan.AddSingleGroupStage(lastNodeID, + lastSQLInstanceID := plan.Processors[plan.ResultRouters[0]].SQLInstanceID + plan.AddSingleGroupStage(lastSQLInstanceID, execinfrapb.ProcessorCoreUnion{ InvertedFilterer: invertedFiltererSpec, }, @@ -2664,7 +2664,7 @@ func (dsp *DistSQLPlanner) createPlanForInvertedFilter( distinctColumns = append(distinctColumns, uint32(i)) } plan.AddSingleGroupStage( - dsp.gatewayNodeID, + dsp.gatewaySQLInstanceID, execinfrapb.ProcessorCoreUnion{ Distinct: dsp.createDistinctSpec( distinctColumns, @@ -2803,26 +2803,26 @@ func (dsp *DistSQLPlanner) planJoiners( leftRouters := info.leftPlan.ResultRouters rightRouters := info.rightPlan.ResultRouters - // Nodes where we will run the join processors. - var nodes []roachpb.NodeID + // Instances where we will run the join processors. + var sqlInstances []base.SQLInstanceID if numEq := len(info.leftEqCols); numEq != 0 { - nodes = findJoinProcessorNodes(leftRouters, rightRouters, p.Processors) + sqlInstances = findJoinProcessorNodes(leftRouters, rightRouters, p.Processors) } else { // Without column equality, we cannot distribute the join. Run a // single processor. - nodes = []roachpb.NodeID{dsp.gatewayNodeID} + sqlInstances = []base.SQLInstanceID{dsp.gatewaySQLInstanceID} // If either side has a single stream, put the processor on that node. We // prefer the left side because that is processed first by the hash joiner. if len(leftRouters) == 1 { - nodes[0] = p.Processors[leftRouters[0]].Node + sqlInstances[0] = p.Processors[leftRouters[0]].SQLInstanceID } else if len(rightRouters) == 1 { - nodes[0] = p.Processors[rightRouters[0]].Node + sqlInstances[0] = p.Processors[rightRouters[0]].SQLInstanceID } } p.AddJoinStage( - nodes, info.makeCoreSpec(), info.post, + sqlInstances, info.makeCoreSpec(), info.post, info.leftEqCols, info.rightEqCols, info.leftPlan.GetResultTypes(), info.rightPlan.GetResultTypes(), info.leftMergeOrd, info.rightMergeOrd, @@ -3005,7 +3005,7 @@ func (dsp *DistSQLPlanner) createPhysPlanForPlanNode( processors := make(execComponents, len(plan.ResultRouters)) for i, resultProcIdx := range plan.ResultRouters { processors[i] = execinfrapb.ProcessorComponentID( - base.SQLInstanceID(plan.Processors[resultProcIdx].Node), + plan.Processors[resultProcIdx].SQLInstanceID, execinfrapb.FlowID{UUID: planCtx.infra.FlowID}, int32(resultProcIdx), ) @@ -3119,7 +3119,7 @@ func (dsp *DistSQLPlanner) wrapPlan( } name := nodeName(n) proc := physicalplan.Processor{ - Node: dsp.gatewayNodeID, + SQLInstanceID: dsp.gatewaySQLInstanceID, Spec: execinfrapb.ProcessorSpec{ Input: input, Core: execinfrapb.ProcessorCoreUnion{LocalPlanNode: &execinfrapb.LocalPlanNodeSpec{ @@ -3186,7 +3186,7 @@ func (dsp *DistSQLPlanner) createValuesPlan( pIdx := p.AddProcessor(physicalplan.Processor{ // TODO: find a better node to place processor at - Node: dsp.gatewayNodeID, + SQLInstanceID: dsp.gatewaySQLInstanceID, Spec: execinfrapb.ProcessorSpec{ Core: execinfrapb.ProcessorCoreUnion{Values: spec}, Output: []execinfrapb.OutputRouterSpec{{Type: execinfrapb.OutputRouterSpec_PASS_THROUGH}}, @@ -3304,9 +3304,9 @@ func (dsp *DistSQLPlanner) addDistinctProcessors( return } - nodes := getNodesOfRouters(plan.ResultRouters, plan.Processors) + sqlInstanceIDs := getSQLInstanceIDsOfRouters(plan.ResultRouters, plan.Processors) plan.AddStageOnNodes( - nodes, distinctSpec, execinfrapb.PostProcessSpec{}, + sqlInstanceIDs, distinctSpec, execinfrapb.PostProcessSpec{}, distinctSpec.Distinct.DistinctColumns, plan.GetResultTypes(), plan.GetResultTypes(), plan.MergeOrdering, plan.ResultRouters, ) @@ -3330,7 +3330,7 @@ func (dsp *DistSQLPlanner) createPlanForOrdinality( // WITH ORDINALITY never gets distributed so that the gateway node can // always number each row in order. - plan.AddSingleGroupStage(dsp.gatewayNodeID, ordinalitySpec, execinfrapb.PostProcessSpec{}, outputTypes) + plan.AddSingleGroupStage(dsp.gatewaySQLInstanceID, ordinalitySpec, execinfrapb.PostProcessSpec{}, outputTypes) return plan, nil } @@ -3392,7 +3392,7 @@ func (dsp *DistSQLPlanner) addProjectSet( // filtered), we could try to detect these cases and use AddNoGroupingStage // instead. outputTypes := append(plan.GetResultTypes(), projectSetSpec.GeneratedColumns...) - plan.AddSingleGroupStage(dsp.gatewayNodeID, spec, execinfrapb.PostProcessSpec{}, outputTypes) + plan.AddSingleGroupStage(dsp.gatewaySQLInstanceID, spec, execinfrapb.PostProcessSpec{}, outputTypes) // Add generated columns to PlanToStreamColMap. for i := range projectSetSpec.GeneratedColumns { @@ -3406,7 +3406,7 @@ func (dsp *DistSQLPlanner) addProjectSet( func (dsp *DistSQLPlanner) isOnlyOnGateway(plan *PhysicalPlan) bool { if len(plan.ResultRouters) == 1 { processorIdx := plan.ResultRouters[0] - if plan.Processors[processorIdx].Node == dsp.gatewayNodeID { + if plan.Processors[processorIdx].SQLInstanceID == dsp.gatewaySQLInstanceID { return true } } @@ -3579,7 +3579,7 @@ func (dsp *DistSQLPlanner) createPlanForSetOp( mergeOrdering, ), } - p.AddSingleGroupStage(dsp.gatewayNodeID, distinctSpec, execinfrapb.PostProcessSpec{}, resultTypes) + p.AddSingleGroupStage(dsp.gatewaySQLInstanceID, distinctSpec, execinfrapb.PostProcessSpec{}, resultTypes) } else { // With UNION ALL, we can end up with multiple streams on the same node. // We don't want to have unnecessary routers and cross-node streams, so @@ -3730,26 +3730,26 @@ func (dsp *DistSQLPlanner) createPlanForWindow( } // Check if the previous stage is all on one node. - prevStageNode := plan.Processors[plan.ResultRouters[0]].Node + prevStageNode := plan.Processors[plan.ResultRouters[0]].SQLInstanceID for i := 1; i < len(plan.ResultRouters); i++ { - if n := plan.Processors[plan.ResultRouters[i]].Node; n != prevStageNode { + if n := plan.Processors[plan.ResultRouters[i]].SQLInstanceID; n != prevStageNode { prevStageNode = 0 break } } - // Get all nodes from the previous stage. - nodes := getNodesOfRouters(plan.ResultRouters, plan.Processors) - if len(partitionIdxs) == 0 || len(nodes) == 1 { + // Get all sqlInstanceIDs from the previous stage. + sqlInstanceIDs := getSQLInstanceIDsOfRouters(plan.ResultRouters, plan.Processors) + if len(partitionIdxs) == 0 || len(sqlInstanceIDs) == 1 { // No PARTITION BY or we have a single node. Use a single windower. // If the previous stage was all on a single node, put the windower // there. Otherwise, bring the results back on this node. - node := dsp.gatewayNodeID - if len(nodes) == 1 { - node = nodes[0] + sqlInstanceID := dsp.gatewaySQLInstanceID + if len(sqlInstanceIDs) == 1 { + sqlInstanceID = sqlInstanceIDs[0] } plan.AddSingleGroupStage( - node, + sqlInstanceID, execinfrapb.ProcessorCoreUnion{Windower: &windowerSpec}, execinfrapb.PostProcessSpec{}, newResultTypes, @@ -3776,10 +3776,10 @@ func (dsp *DistSQLPlanner) createPlanForWindow( // a node appears in nodes. prevStageRouters := plan.ResultRouters prevStageResultTypes := plan.GetResultTypes() - plan.ResultRouters = make([]physicalplan.ProcessorIdx, 0, len(nodes)) - for bucket, nodeID := range nodes { + plan.ResultRouters = make([]physicalplan.ProcessorIdx, 0, len(sqlInstanceIDs)) + for bucket, sqlInstanceID := range sqlInstanceIDs { proc := physicalplan.Processor{ - Node: nodeID, + SQLInstanceID: sqlInstanceID, Spec: execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{{ Type: execinfrapb.InputSyncSpec_PARALLEL_UNORDERED, @@ -3988,7 +3988,7 @@ func (dsp *DistSQLPlanner) NewPlanningCtx( planCtx := &PlanningCtx{ ctx: ctx, ExtendedEvalCtx: evalCtx, - infra: physicalplan.MakePhysicalInfrastructure(uuid.FastMakeV4(), dsp.gatewayNodeID), + infra: physicalplan.MakePhysicalInfrastructure(uuid.FastMakeV4(), dsp.gatewaySQLInstanceID), isLocal: !distribute, planner: planner, } @@ -4011,8 +4011,8 @@ func (dsp *DistSQLPlanner) NewPlanningCtx( planCtx.parallelizeScansIfLocal = true } planCtx.spanIter = dsp.spanResolver.NewSpanResolverIterator(txn) - planCtx.NodeStatuses = make(map[roachpb.NodeID]NodeStatus) - planCtx.NodeStatuses[dsp.gatewayNodeID] = NodeOK + planCtx.NodeStatuses = make(map[base.SQLInstanceID]NodeStatus) + planCtx.NodeStatuses[dsp.gatewaySQLInstanceID] = NodeOK return planCtx } @@ -4046,10 +4046,10 @@ func maybeMoveSingleFlowToGateway(planCtx *PlanningCtx, plan *PhysicalPlan, rowC } singleFlow := true moveFlowToGateway := false - nodeID := plan.Processors[0].Node + sqlInstanceID := plan.Processors[0].SQLInstanceID for _, p := range plan.Processors[1:] { - if p.Node != nodeID { - if p.Node != plan.GatewayNodeID || p.Spec.Core.Noop == nil { + if p.SQLInstanceID != sqlInstanceID { + if p.SQLInstanceID != plan.GatewaySQLInstanceID || p.Spec.Core.Noop == nil { // We want to ignore the noop processors planned on the // gateway because their job is to simply communicate the // results back to the client. If, however, there is another @@ -4070,7 +4070,7 @@ func maybeMoveSingleFlowToGateway(planCtx *PlanningCtx, plan *PhysicalPlan, rowC } if singleFlow && moveFlowToGateway { for i := range plan.Processors { - plan.Processors[i].Node = plan.GatewayNodeID + plan.Processors[i].SQLInstanceID = plan.GatewaySQLInstanceID } planCtx.isLocal = true planCtx.planner.curPlan.flags.Unset(planFlagFullyDistributed) @@ -4121,7 +4121,7 @@ func (dsp *DistSQLPlanner) finalizePlanWithRowCount( if len(metadataSenders) > 0 { plan.AddSingleGroupStage( - dsp.gatewayNodeID, + dsp.gatewaySQLInstanceID, execinfrapb.ProcessorCoreUnion{ MetadataTestReceiver: &execinfrapb.MetadataTestReceiverSpec{ SenderIDs: metadataSenders, diff --git a/pkg/sql/distsql_physical_planner_test.go b/pkg/sql/distsql_physical_planner_test.go index 5a3a6b9f434b..35b26fbee59f 100644 --- a/pkg/sql/distsql_physical_planner_test.go +++ b/pkg/sql/distsql_physical_planner_test.go @@ -855,16 +855,16 @@ func TestPartitionSpans(t *testing.T) { testStopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) var nodeDescs []*roachpb.NodeDescriptor for i := 1; i <= 10; i++ { - nodeID := roachpb.NodeID(i) + sqlInstanceID := base.SQLInstanceID(i) desc := &roachpb.NodeDescriptor{ - NodeID: nodeID, + NodeID: roachpb.NodeID(sqlInstanceID), Address: util.UnresolvedAddr{AddressField: fmt.Sprintf("addr%d", i)}, } if err := mockGossip.SetNodeDescriptor(desc); err != nil { t.Fatal(err) } if err := mockGossip.AddInfoProto( - gossip.MakeDistSQLNodeVersionKey(nodeID), + gossip.MakeDistSQLNodeVersionKey(sqlInstanceID), &execinfrapb.DistSQLVersionGossipInfo{ MinAcceptedVersion: execinfra.MinAcceptedVersion, Version: execinfra.Version, @@ -889,12 +889,12 @@ func TestPartitionSpans(t *testing.T) { gw := gossip.MakeOptionalGossip(mockGossip) dsp := DistSQLPlanner{ - planVersion: execinfra.Version, - st: cluster.MakeTestingClusterSettings(), - gatewayNodeID: tsp.nodes[tc.gatewayNode-1].NodeID, - stopper: stopper, - spanResolver: tsp, - gossip: gw, + planVersion: execinfra.Version, + st: cluster.MakeTestingClusterSettings(), + gatewaySQLInstanceID: base.SQLInstanceID(tsp.nodes[tc.gatewayNode-1].NodeID), + stopper: stopper, + spanResolver: tsp, + gossip: gw, nodeHealth: distSQLNodeHealth{ gossip: gw, connHealth: func(node roachpb.NodeID, _ rpc.ConnectionClass) error { @@ -905,7 +905,7 @@ func TestPartitionSpans(t *testing.T) { } return nil }, - isAvailable: func(nodeID roachpb.NodeID) bool { + isAvailable: func(base.SQLInstanceID) bool { return true }, }, @@ -926,14 +926,14 @@ func TestPartitionSpans(t *testing.T) { resMap := make(map[int][][2]string) for _, p := range partitions { - if _, ok := resMap[int(p.Node)]; ok { + if _, ok := resMap[int(p.SQLInstanceID)]; ok { t.Fatalf("node %d shows up in multiple partitions", p) } var spans [][2]string for _, s := range p.Spans { spans = append(spans, [2]string{string(s.Key), string(s.EndKey)}) } - resMap[int(p.Node)] = spans + resMap[int(p.SQLInstanceID)] = spans } if !reflect.DeepEqual(resMap, tc.partitions) { @@ -965,21 +965,21 @@ func TestPartitionSpansSkipsIncompatibleNodes(t *testing.T) { planVersion execinfrapb.DistSQLVersion // The versions accepted by each node. - nodeVersions map[roachpb.NodeID]execinfrapb.DistSQLVersionGossipInfo + nodeVersions map[base.SQLInstanceID]execinfrapb.DistSQLVersionGossipInfo // nodesNotAdvertisingDistSQLVersion is the set of nodes for which gossip is // not going to have information about the supported DistSQL version. This // is to simulate CRDB 1.0 nodes which don't advertise this information. - nodesNotAdvertisingDistSQLVersion map[roachpb.NodeID]struct{} + nodesNotAdvertisingDistSQLVersion map[base.SQLInstanceID]struct{} // expected result: a map of node to list of spans. - partitions map[roachpb.NodeID][][2]string + partitions map[base.SQLInstanceID][][2]string }{ { // In the first test, all nodes are compatible. name: "current_version", planVersion: 2, - nodeVersions: map[roachpb.NodeID]execinfrapb.DistSQLVersionGossipInfo{ + nodeVersions: map[base.SQLInstanceID]execinfrapb.DistSQLVersionGossipInfo{ 1: { MinAcceptedVersion: 1, Version: 2, @@ -989,7 +989,7 @@ func TestPartitionSpansSkipsIncompatibleNodes(t *testing.T) { Version: 2, }, }, - partitions: map[roachpb.NodeID][][2]string{ + partitions: map[base.SQLInstanceID][][2]string{ 1: {{"A", "B"}, {"C", "Z"}}, 2: {{"B", "C"}}, }, @@ -1000,7 +1000,7 @@ func TestPartitionSpansSkipsIncompatibleNodes(t *testing.T) { // Remember that the gateway is node 2. name: "next_version", planVersion: 3, - nodeVersions: map[roachpb.NodeID]execinfrapb.DistSQLVersionGossipInfo{ + nodeVersions: map[base.SQLInstanceID]execinfrapb.DistSQLVersionGossipInfo{ 1: { MinAcceptedVersion: 1, Version: 2, @@ -1010,7 +1010,7 @@ func TestPartitionSpansSkipsIncompatibleNodes(t *testing.T) { Version: 3, }, }, - partitions: map[roachpb.NodeID][][2]string{ + partitions: map[base.SQLInstanceID][][2]string{ 2: {{"A", "Z"}}, }, }, @@ -1019,16 +1019,16 @@ func TestPartitionSpansSkipsIncompatibleNodes(t *testing.T) { // a crdb 1.0 node). name: "crdb_1.0", planVersion: 3, - nodeVersions: map[roachpb.NodeID]execinfrapb.DistSQLVersionGossipInfo{ + nodeVersions: map[base.SQLInstanceID]execinfrapb.DistSQLVersionGossipInfo{ 2: { MinAcceptedVersion: 3, Version: 3, }, }, - nodesNotAdvertisingDistSQLVersion: map[roachpb.NodeID]struct{}{ + nodesNotAdvertisingDistSQLVersion: map[base.SQLInstanceID]struct{}{ 1: {}, }, - partitions: map[roachpb.NodeID][][2]string{ + partitions: map[base.SQLInstanceID][][2]string{ 2: {{"A", "Z"}}, }, }, @@ -1049,18 +1049,18 @@ func TestPartitionSpansSkipsIncompatibleNodes(t *testing.T) { testStopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) var nodeDescs []*roachpb.NodeDescriptor for i := 1; i <= 2; i++ { - nodeID := roachpb.NodeID(i) + sqlInstanceID := base.SQLInstanceID(i) desc := &roachpb.NodeDescriptor{ - NodeID: nodeID, + NodeID: roachpb.NodeID(sqlInstanceID), Address: util.UnresolvedAddr{AddressField: fmt.Sprintf("addr%d", i)}, } if err := mockGossip.SetNodeDescriptor(desc); err != nil { t.Fatal(err) } - if _, ok := tc.nodesNotAdvertisingDistSQLVersion[nodeID]; !ok { - verInfo := tc.nodeVersions[nodeID] + if _, ok := tc.nodesNotAdvertisingDistSQLVersion[sqlInstanceID]; !ok { + verInfo := tc.nodeVersions[sqlInstanceID] if err := mockGossip.AddInfoProto( - gossip.MakeDistSQLNodeVersionKey(nodeID), + gossip.MakeDistSQLNodeVersionKey(sqlInstanceID), &verInfo, 0, // ttl - no expiration ); err != nil { @@ -1077,19 +1077,19 @@ func TestPartitionSpansSkipsIncompatibleNodes(t *testing.T) { gw := gossip.MakeOptionalGossip(mockGossip) dsp := DistSQLPlanner{ - planVersion: tc.planVersion, - st: cluster.MakeTestingClusterSettings(), - gatewayNodeID: tsp.nodes[gatewayNode-1].NodeID, - stopper: stopper, - spanResolver: tsp, - gossip: gw, + planVersion: tc.planVersion, + st: cluster.MakeTestingClusterSettings(), + gatewaySQLInstanceID: base.SQLInstanceID(tsp.nodes[gatewayNode-1].NodeID), + stopper: stopper, + spanResolver: tsp, + gossip: gw, nodeHealth: distSQLNodeHealth{ gossip: gw, connHealth: func(roachpb.NodeID, rpc.ConnectionClass) error { // All the nodes are healthy. return nil }, - isAvailable: func(roachpb.NodeID) bool { + isAvailable: func(base.SQLInstanceID) bool { return true }, }, @@ -1103,16 +1103,16 @@ func TestPartitionSpansSkipsIncompatibleNodes(t *testing.T) { t.Fatal(err) } - resMap := make(map[roachpb.NodeID][][2]string) + resMap := make(map[base.SQLInstanceID][][2]string) for _, p := range partitions { - if _, ok := resMap[p.Node]; ok { + if _, ok := resMap[p.SQLInstanceID]; ok { t.Fatalf("node %d shows up in multiple partitions", p) } var spans [][2]string for _, s := range p.Spans { spans = append(spans, [2]string{string(s.Key), string(s.EndKey)}) } - resMap[p.Node] = spans + resMap[p.SQLInstanceID] = spans } if !reflect.DeepEqual(resMap, tc.partitions) { @@ -1142,9 +1142,9 @@ func TestPartitionSpansSkipsNodesNotInGossip(t *testing.T) { stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) var nodeDescs []*roachpb.NodeDescriptor for i := 1; i <= 2; i++ { - nodeID := roachpb.NodeID(i) + sqlInstanceID := base.SQLInstanceID(i) desc := &roachpb.NodeDescriptor{ - NodeID: nodeID, + NodeID: roachpb.NodeID(sqlInstanceID), Address: util.UnresolvedAddr{AddressField: fmt.Sprintf("addr%d", i)}, } if i == 2 { @@ -1157,7 +1157,7 @@ func TestPartitionSpansSkipsNodesNotInGossip(t *testing.T) { // the test comment - for such a node, the descriptor would be taken out of // the gossip data, but other datums it advertised are left in place. if err := mockGossip.AddInfoProto( - gossip.MakeDistSQLNodeVersionKey(nodeID), + gossip.MakeDistSQLNodeVersionKey(sqlInstanceID), &execinfrapb.DistSQLVersionGossipInfo{ MinAcceptedVersion: execinfra.MinAcceptedVersion, Version: execinfra.Version, @@ -1176,19 +1176,19 @@ func TestPartitionSpansSkipsNodesNotInGossip(t *testing.T) { gw := gossip.MakeOptionalGossip(mockGossip) dsp := DistSQLPlanner{ - planVersion: execinfra.Version, - st: cluster.MakeTestingClusterSettings(), - gatewayNodeID: tsp.nodes[gatewayNode-1].NodeID, - stopper: stopper, - spanResolver: tsp, - gossip: gw, + planVersion: execinfra.Version, + st: cluster.MakeTestingClusterSettings(), + gatewaySQLInstanceID: base.SQLInstanceID(tsp.nodes[gatewayNode-1].NodeID), + stopper: stopper, + spanResolver: tsp, + gossip: gw, nodeHealth: distSQLNodeHealth{ gossip: gw, connHealth: func(node roachpb.NodeID, _ rpc.ConnectionClass) error { _, err := mockGossip.GetNodeIDAddress(node) return err }, - isAvailable: func(roachpb.NodeID) bool { + isAvailable: func(base.SQLInstanceID) bool { return true }, }, @@ -1202,20 +1202,20 @@ func TestPartitionSpansSkipsNodesNotInGossip(t *testing.T) { t.Fatal(err) } - resMap := make(map[roachpb.NodeID][][2]string) + resMap := make(map[base.SQLInstanceID][][2]string) for _, p := range partitions { - if _, ok := resMap[p.Node]; ok { + if _, ok := resMap[p.SQLInstanceID]; ok { t.Fatalf("node %d shows up in multiple partitions", p) } var spans [][2]string for _, s := range p.Spans { spans = append(spans, [2]string{string(s.Key), string(s.EndKey)}) } - resMap[p.Node] = spans + resMap[p.SQLInstanceID] = spans } expectedPartitions := - map[roachpb.NodeID][][2]string{ + map[base.SQLInstanceID][][2]string{ 2: {{"A", "Z"}}, } if !reflect.DeepEqual(resMap, expectedPartitions) { @@ -1230,20 +1230,20 @@ func TestCheckNodeHealth(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(context.Background()) - const nodeID = roachpb.NodeID(5) + const sqlInstanceID = base.SQLInstanceID(5) - mockGossip := gossip.NewTest(nodeID, nil /* rpcContext */, nil, /* grpcServer */ + mockGossip := gossip.NewTest(roachpb.NodeID(sqlInstanceID), nil /* rpcContext */, nil, /* grpcServer */ stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) desc := &roachpb.NodeDescriptor{ - NodeID: nodeID, + NodeID: roachpb.NodeID(sqlInstanceID), Address: util.UnresolvedAddr{NetworkField: "tcp", AddressField: "testaddr"}, } if err := mockGossip.SetNodeDescriptor(desc); err != nil { t.Fatal(err) } if err := mockGossip.AddInfoProto( - gossip.MakeDistSQLNodeVersionKey(nodeID), + gossip.MakeDistSQLNodeVersionKey(sqlInstanceID), &execinfrapb.DistSQLVersionGossipInfo{ MinAcceptedVersion: execinfra.MinAcceptedVersion, Version: execinfra.Version, @@ -1253,10 +1253,10 @@ func TestCheckNodeHealth(t *testing.T) { t.Fatal(err) } - notAvailable := func(roachpb.NodeID) bool { + notAvailable := func(base.SQLInstanceID) bool { return false } - available := func(roachpb.NodeID) bool { + available := func(base.SQLInstanceID) bool { return true } @@ -1269,7 +1269,7 @@ func TestCheckNodeHealth(t *testing.T) { _ = connUnhealthy livenessTests := []struct { - isAvailable func(roachpb.NodeID) bool + isAvailable func(id base.SQLInstanceID) bool exp string }{ {available, ""}, @@ -1284,7 +1284,7 @@ func TestCheckNodeHealth(t *testing.T) { connHealth: connHealthy, isAvailable: test.isAvailable, } - if err := h.check(context.Background(), nodeID); !testutils.IsError(err, test.exp) { + if err := h.check(context.Background(), sqlInstanceID); !testutils.IsError(err, test.exp) { t.Fatalf("expected %v, got %v", test.exp, err) } }) @@ -1305,7 +1305,7 @@ func TestCheckNodeHealth(t *testing.T) { connHealth: test.connHealth, isAvailable: available, } - if err := h.check(context.Background(), nodeID); !testutils.IsError(err, test.exp) { + if err := h.check(context.Background(), sqlInstanceID); !testutils.IsError(err, test.exp) { t.Fatalf("expected %v, got %v", test.exp, err) } }) diff --git a/pkg/sql/distsql_plan_backfill.go b/pkg/sql/distsql_plan_backfill.go index f01a03dd8b8f..fbd656aed3f4 100644 --- a/pkg/sql/distsql_plan_backfill.go +++ b/pkg/sql/distsql_plan_backfill.go @@ -68,7 +68,7 @@ func (dsp *DistSQLPlanner) createBackfillerPhysicalPlan( ib.Spans = sp.Spans proc := physicalplan.Processor{ - Node: sp.Node, + SQLInstanceID: sp.SQLInstanceID, Spec: execinfrapb.ProcessorSpec{ Core: execinfrapb.ProcessorCoreUnion{Backfiller: ib}, Output: []execinfrapb.OutputRouterSpec{{Type: execinfrapb.OutputRouterSpec_PASS_THROUGH}}, diff --git a/pkg/sql/distsql_plan_bulk.go b/pkg/sql/distsql_plan_bulk.go index a7c16ca3c563..c566269fc0dd 100644 --- a/pkg/sql/distsql_plan_bulk.go +++ b/pkg/sql/distsql_plan_bulk.go @@ -14,7 +14,7 @@ import ( "context" "math/rand" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/server/serverpb" ) @@ -22,25 +22,25 @@ import ( // map for all nodes. It returns all nodes that can be used for planning. func (dsp *DistSQLPlanner) SetupAllNodesPlanning( ctx context.Context, evalCtx *extendedEvalContext, execCfg *ExecutorConfig, -) (*PlanningCtx, []roachpb.NodeID, error) { +) (*PlanningCtx, []base.SQLInstanceID, error) { distribute := evalCtx.Codec.ForSystemTenant() planCtx := dsp.NewPlanningCtx(ctx, evalCtx, nil /* planner */, nil /* txn */, distribute) ss, err := execCfg.NodesStatusServer.OptionalNodesStatusServer(47900) if err != nil { - return planCtx, []roachpb.NodeID{dsp.gatewayNodeID}, nil //nolint:returnerrcheck + return planCtx, []base.SQLInstanceID{dsp.gatewaySQLInstanceID}, nil //nolint:returnerrcheck } resp, err := ss.ListNodesInternal(ctx, &serverpb.NodesRequest{}) if err != nil { return nil, nil, err } // Because we're not going through the normal pathways, we have to set up the - // planCtx.NodeStatuses map ourselves. CheckNodeHealthAndVersion() will + // planCtx.NodeStatuses map ourselves. CheckInstanceHealthAndVersion() will // populate it. for _, node := range resp.Nodes { - _ /* NodeStatus */ = dsp.CheckNodeHealthAndVersion(planCtx, node.Desc.NodeID) + _ /* NodeStatus */ = dsp.CheckInstanceHealthAndVersion(planCtx, base.SQLInstanceID(node.Desc.NodeID)) } - nodes := make([]roachpb.NodeID, 0, len(planCtx.NodeStatuses)) + nodes := make([]base.SQLInstanceID, 0, len(planCtx.NodeStatuses)) for nodeID, status := range planCtx.NodeStatuses { if status == NodeOK { nodes = append(nodes, nodeID) diff --git a/pkg/sql/distsql_plan_join.go b/pkg/sql/distsql_plan_join.go index 5d6d89f14610..d18da150e32b 100644 --- a/pkg/sql/distsql_plan_join.go +++ b/pkg/sql/distsql_plan_join.go @@ -11,7 +11,7 @@ package sql import ( - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -202,25 +202,25 @@ func distsqlSetOpJoinType(setOpType tree.UnionType) descpb.JoinType { } } -// getNodesOfRouters returns all nodes that routers are put on. -func getNodesOfRouters( +// getSQLInstanceIDsOfRouters returns all nodes that routers are put on. +func getSQLInstanceIDsOfRouters( routers []physicalplan.ProcessorIdx, processors []physicalplan.Processor, -) (nodes []roachpb.NodeID) { - seen := make(map[roachpb.NodeID]struct{}) +) (sqlInstanceIDs []base.SQLInstanceID) { + seen := make(map[base.SQLInstanceID]struct{}) for _, pIdx := range routers { - n := processors[pIdx].Node + n := processors[pIdx].SQLInstanceID if _, ok := seen[n]; !ok { seen[n] = struct{}{} - nodes = append(nodes, n) + sqlInstanceIDs = append(sqlInstanceIDs, n) } } - return nodes + return sqlInstanceIDs } func findJoinProcessorNodes( leftRouters, rightRouters []physicalplan.ProcessorIdx, processors []physicalplan.Processor, -) (nodes []roachpb.NodeID) { +) (instances []base.SQLInstanceID) { // TODO(radu): for now we run a join processor on every node that produces // data for either source. In the future we should be smarter here. - return getNodesOfRouters(append(leftRouters, rightRouters...), processors) + return getSQLInstanceIDsOfRouters(append(leftRouters, rightRouters...), processors) } diff --git a/pkg/sql/distsql_plan_set_op_test.go b/pkg/sql/distsql_plan_set_op_test.go index ce41e2f3167b..3fbe171776a2 100644 --- a/pkg/sql/distsql_plan_set_op_test.go +++ b/pkg/sql/distsql_plan_set_op_test.go @@ -14,7 +14,7 @@ import ( "reflect" "testing" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -60,7 +60,7 @@ func TestMergeResultTypesForSetOp(t *testing.T) { } } } - infra := physicalplan.MakePhysicalInfrastructure(uuid.FastMakeV4(), roachpb.NodeID(1)) + infra := physicalplan.MakePhysicalInfrastructure(uuid.FastMakeV4(), base.SQLInstanceID(1)) var leftPlan, rightPlan PhysicalPlan leftPlan.PhysicalInfrastructure = &infra rightPlan.PhysicalInfrastructure = &infra diff --git a/pkg/sql/distsql_plan_stats.go b/pkg/sql/distsql_plan_stats.go index 5968644ee50e..1eca6e10a889 100644 --- a/pkg/sql/distsql_plan_stats.go +++ b/pkg/sql/distsql_plan_stats.go @@ -232,9 +232,9 @@ func (dsp *DistSQLPlanner) createStatsPlan( RowsExpected: rowsExpected, } // Plan the SampleAggregator on the gateway, unless we have a single Sampler. - node := dsp.gatewayNodeID + node := dsp.gatewaySQLInstanceID if len(p.ResultRouters) == 1 { - node = p.Processors[p.ResultRouters[0]].Node + node = p.Processors[p.ResultRouters[0]].SQLInstanceID } p.AddSingleGroupStage( node, diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index e61ca87a4762..e93af1ec3d79 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -18,6 +18,7 @@ import ( "sync/atomic" "time" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" @@ -63,24 +64,24 @@ const clientRejectedMsg string = "client rejected when attempting to run DistSQL // runnerRequest is the request that is sent (via a channel) to a worker. type runnerRequest struct { - ctx context.Context - nodeDialer *nodedialer.Dialer - flowReq *execinfrapb.SetupFlowRequest - nodeID roachpb.NodeID - resultChan chan<- runnerResult + ctx context.Context + nodeDialer *nodedialer.Dialer + flowReq *execinfrapb.SetupFlowRequest + sqlInstanceID base.SQLInstanceID + resultChan chan<- runnerResult } // runnerResult is returned by a worker (via a channel) for each received // request. type runnerResult struct { - nodeID roachpb.NodeID + nodeID base.SQLInstanceID err error } func (req runnerRequest) run() { - res := runnerResult{nodeID: req.nodeID} + res := runnerResult{nodeID: req.sqlInstanceID} - conn, err := req.nodeDialer.Dial(req.ctx, req.nodeID, rpc.DefaultClass) + conn, err := req.nodeDialer.Dial(req.ctx, roachpb.NodeID(req.sqlInstanceID), rpc.DefaultClass) if err != nil { res.err = err } else { @@ -147,7 +148,7 @@ func (dsp *DistSQLPlanner) initCancelingWorkers(initCtx context.Context) { continue } log.VEventf(parentCtx, 2, "worker %d is canceling at most %d flows on node %d", workerID, len(req.FlowIDs), nodeID) - conn, err := dsp.nodeDialer.Dial(parentCtx, nodeID, rpc.DefaultClass) + conn, err := dsp.nodeDialer.Dial(parentCtx, roachpb.NodeID(nodeID), rpc.DefaultClass) if err != nil { // We failed to dial the node, so we give up given that // our cancellation is best effort. It is possible that @@ -170,8 +171,8 @@ func (dsp *DistSQLPlanner) initCancelingWorkers(initCtx context.Context) { } type deadFlowsOnNode struct { - ids []execinfrapb.FlowID - nodeID roachpb.NodeID + ids []execinfrapb.FlowID + sqlInstanceID base.SQLInstanceID } // cancelFlowsCoordinator is responsible for batching up the requests to cancel @@ -193,34 +194,36 @@ type cancelFlowsCoordinator struct { // concurrent usage. func (c *cancelFlowsCoordinator) getFlowsToCancel() ( *execinfrapb.CancelDeadFlowsRequest, - roachpb.NodeID, + base.SQLInstanceID, ) { c.mu.Lock() defer c.mu.Unlock() if c.mu.deadFlowsByNode.Len() == 0 { - return nil, roachpb.NodeID(0) + return nil, base.SQLInstanceID(0) } deadFlows := c.mu.deadFlowsByNode.GetFirst().(*deadFlowsOnNode) c.mu.deadFlowsByNode.RemoveFirst() req := &execinfrapb.CancelDeadFlowsRequest{ FlowIDs: deadFlows.ids, } - return req, deadFlows.nodeID + return req, deadFlows.sqlInstanceID } // addFlowsToCancel adds all remote flows from flows map to be canceled via // CancelDeadFlows RPC. Safe for concurrent usage. -func (c *cancelFlowsCoordinator) addFlowsToCancel(flows map[roachpb.NodeID]*execinfrapb.FlowSpec) { +func (c *cancelFlowsCoordinator) addFlowsToCancel( + flows map[base.SQLInstanceID]*execinfrapb.FlowSpec, +) { c.mu.Lock() - for nodeID, f := range flows { - if nodeID != f.Gateway { + for sqlInstanceID, f := range flows { + if sqlInstanceID != f.Gateway { // c.mu.deadFlowsByNode.Len() is at most the number of nodes in the // cluster, so a linear search for the node ID should be // sufficiently fast. found := false for j := 0; j < c.mu.deadFlowsByNode.Len(); j++ { deadFlows := c.mu.deadFlowsByNode.Get(j).(*deadFlowsOnNode) - if nodeID == deadFlows.nodeID { + if sqlInstanceID == deadFlows.sqlInstanceID { deadFlows.ids = append(deadFlows.ids, f.FlowID) found = true break @@ -228,8 +231,8 @@ func (c *cancelFlowsCoordinator) addFlowsToCancel(flows map[roachpb.NodeID]*exec } if !found { c.mu.deadFlowsByNode.AddLast(&deadFlowsOnNode{ - ids: []execinfrapb.FlowID{f.FlowID}, - nodeID: nodeID, + ids: []execinfrapb.FlowID{f.FlowID}, + sqlInstanceID: sqlInstanceID, }) } } @@ -267,13 +270,13 @@ func (dsp *DistSQLPlanner) setupFlows( ctx context.Context, evalCtx *extendedEvalContext, leafInputState *roachpb.LeafTxnInputState, - flows map[roachpb.NodeID]*execinfrapb.FlowSpec, + flows map[base.SQLInstanceID]*execinfrapb.FlowSpec, recv *DistSQLReceiver, localState distsql.LocalState, collectStats bool, statementSQL string, ) (context.Context, flowinfra.Flow, execinfra.OpChains, error) { - thisNodeID := dsp.gatewayNodeID + thisNodeID := dsp.gatewaySQLInstanceID _, ok := flows[thisNodeID] if !ok { return nil, nil, nil, errors.AssertionFailedf("missing gateway flow") @@ -332,11 +335,11 @@ func (dsp *DistSQLPlanner) setupFlows( req := setupReq req.Flow = *flowSpec runReq := runnerRequest{ - ctx: ctx, - nodeDialer: dsp.nodeDialer, - flowReq: &req, - nodeID: nodeID, - resultChan: resultChan, + ctx: ctx, + nodeDialer: dsp.nodeDialer, + flowReq: &req, + sqlInstanceID: nodeID, + resultChan: resultChan, } // Send out a request to the workers; if no worker is available, run @@ -410,7 +413,7 @@ func (dsp *DistSQLPlanner) Run( physicalplan.ReleaseFlowSpec(flowSpec) } }() - if _, ok := flows[dsp.gatewayNodeID]; !ok { + if _, ok := flows[dsp.gatewaySQLInstanceID]; !ok { recv.SetError(errors.Errorf("expected to find gateway flow")) return func() {} } diff --git a/pkg/sql/distsql_running_test.go b/pkg/sql/distsql_running_test.go index 48ffa99be710..ee3a6b9316cb 100644 --- a/pkg/sql/distsql_running_test.go +++ b/pkg/sql/distsql_running_test.go @@ -466,7 +466,7 @@ func TestCancelFlowsCoordinator(t *testing.T) { globalRng, _ := randutil.NewTestRand() numNodes := globalRng.Intn(16) + 2 - gatewayNodeID := roachpb.NodeID(1) + gatewaySQLInstanceID := base.SQLInstanceID(1) assertInvariants := func() { c.mu.Lock() @@ -474,29 +474,29 @@ func TestCancelFlowsCoordinator(t *testing.T) { // Check that the coordinator hasn't created duplicate entries for some // nodes. require.GreaterOrEqual(t, numNodes-1, c.mu.deadFlowsByNode.Len()) - seen := make(map[roachpb.NodeID]struct{}) + seen := make(map[base.SQLInstanceID]struct{}) for i := 0; i < c.mu.deadFlowsByNode.Len(); i++ { deadFlows := c.mu.deadFlowsByNode.Get(i).(*deadFlowsOnNode) - require.NotEqual(t, gatewayNodeID, deadFlows.nodeID) - _, ok := seen[deadFlows.nodeID] + require.NotEqual(t, gatewaySQLInstanceID, deadFlows.sqlInstanceID) + _, ok := seen[deadFlows.sqlInstanceID] require.False(t, ok) - seen[deadFlows.nodeID] = struct{}{} + seen[deadFlows.sqlInstanceID] = struct{}{} } } // makeFlowsToCancel returns a fake flows map where each node in the cluster // has 67% probability of participating in the plan. - makeFlowsToCancel := func(rng *rand.Rand) map[roachpb.NodeID]*execinfrapb.FlowSpec { - res := make(map[roachpb.NodeID]*execinfrapb.FlowSpec) + makeFlowsToCancel := func(rng *rand.Rand) map[base.SQLInstanceID]*execinfrapb.FlowSpec { + res := make(map[base.SQLInstanceID]*execinfrapb.FlowSpec) flowID := execinfrapb.FlowID{UUID: uuid.FastMakeV4()} for id := 1; id <= numNodes; id++ { if rng.Float64() < 0.33 { // This node wasn't a part of the current plan. continue } - res[roachpb.NodeID(id)] = &execinfrapb.FlowSpec{ + res[base.SQLInstanceID(id)] = &execinfrapb.FlowSpec{ FlowID: flowID, - Gateway: gatewayNodeID, + Gateway: gatewaySQLInstanceID, } } return res diff --git a/pkg/sql/distsql_spec_exec_factory.go b/pkg/sql/distsql_spec_exec_factory.go index ad05aa1c4e04..9e8ff87bebb2 100644 --- a/pkg/sql/distsql_spec_exec_factory.go +++ b/pkg/sql/distsql_spec_exec_factory.go @@ -11,6 +11,7 @@ package sql import ( + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" @@ -35,10 +36,10 @@ type distSQLSpecExecFactory struct { planner *planner dsp *DistSQLPlanner // planCtx should not be used directly - getPlanCtx() should be used instead. - planCtx *PlanningCtx - singleTenant bool - planningMode distSQLPlanningMode - gatewayNodeID roachpb.NodeID + planCtx *PlanningCtx + singleTenant bool + planningMode distSQLPlanningMode + gatewaySQLInstanceID base.SQLInstanceID } var _ exec.Factory = &distSQLSpecExecFactory{} @@ -59,11 +60,11 @@ const ( func newDistSQLSpecExecFactory(p *planner, planningMode distSQLPlanningMode) exec.Factory { e := &distSQLSpecExecFactory{ - planner: p, - dsp: p.extendedEvalCtx.DistSQLPlanner, - singleTenant: p.execCfg.Codec.ForSystemTenant(), - planningMode: planningMode, - gatewayNodeID: p.extendedEvalCtx.DistSQLPlanner.gatewayNodeID, + planner: p, + dsp: p.extendedEvalCtx.DistSQLPlanner, + singleTenant: p.execCfg.Codec.ForSystemTenant(), + planningMode: planningMode, + gatewaySQLInstanceID: p.extendedEvalCtx.DistSQLPlanner.gatewaySQLInstanceID, } distribute := e.singleTenant && e.planningMode != distSQLLocalOnlyPlanning evalCtx := p.ExtendedEvalContext() diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 73e990240db0..aa4d642a70f3 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -1381,7 +1381,7 @@ type ExecutorTestingKnobs struct { // query (i.e. no subqueries). The physical plan is only safe for use for the // lifetime of this function. Note that returning a nil function is // unsupported and will lead to a panic. - TestingSaveFlows func(stmt string) func(map[roachpb.NodeID]*execinfrapb.FlowSpec, execinfra.OpChains) error + TestingSaveFlows func(stmt string) func(map[base.SQLInstanceID]*execinfrapb.FlowSpec, execinfra.OpChains) error // DeterministicExplain, if set, will result in overriding fields in EXPLAIN // and EXPLAIN ANALYZE that can vary between runs (like elapsed times). diff --git a/pkg/sql/execinfra/outboxbase.go b/pkg/sql/execinfra/outboxbase.go index d0ce09c6626f..0576c9a65ff4 100644 --- a/pkg/sql/execinfra/outboxbase.go +++ b/pkg/sql/execinfra/outboxbase.go @@ -40,11 +40,11 @@ type Dialer interface { // user, since the receiver at the other end will hang for // SettingFlowStreamTimeout waiting for a successful connection attempt. func GetConnForOutbox( - ctx context.Context, dialer Dialer, nodeID roachpb.NodeID, timeout time.Duration, + ctx context.Context, dialer Dialer, sqlInstanceID base.SQLInstanceID, timeout time.Duration, ) (conn *grpc.ClientConn, err error) { firstConnectionAttempt := timeutil.Now() for r := retry.StartWithCtx(ctx, base.DefaultRetryOptions()); r.Next(); { - conn, err = dialer.DialNoBreaker(ctx, nodeID, rpc.DefaultClass) + conn, err = dialer.DialNoBreaker(ctx, roachpb.NodeID(sqlInstanceID), rpc.DefaultClass) if err == nil || timeutil.Since(firstConnectionAttempt) > timeout { break } diff --git a/pkg/sql/execinfra/testutils.go b/pkg/sql/execinfra/testutils.go index 84e29940c29c..f2e2bec148bc 100644 --- a/pkg/sql/execinfra/testutils.go +++ b/pkg/sql/execinfra/testutils.go @@ -14,7 +14,7 @@ import ( "context" "math" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -24,8 +24,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/mon" ) -// StaticNodeID is the default Node ID to be used in tests. -const StaticNodeID = roachpb.NodeID(3) +// StaticSQLInstanceID is the default Node ID to be used in tests. +const StaticSQLInstanceID = base.SQLInstanceID(3) // RepeatableRowSource is a RowSource used in benchmarks to avoid having to // reinitialize a new RowSource every time during multiple passes of the input. diff --git a/pkg/sql/execinfrapb/BUILD.bazel b/pkg/sql/execinfrapb/BUILD.bazel index 1ef894e0d8d7..9bbb1d490a90 100644 --- a/pkg/sql/execinfrapb/BUILD.bazel +++ b/pkg/sql/execinfrapb/BUILD.bazel @@ -71,7 +71,7 @@ go_test( ], embed = [":execinfrapb"], deps = [ - "//pkg/roachpb:with-mocks", + "//pkg/base", "//pkg/security", "//pkg/settings/cluster", "//pkg/sql", diff --git a/pkg/sql/execinfrapb/api.proto b/pkg/sql/execinfrapb/api.proto index fcc8b6666d43..d7f2b7762351 100644 --- a/pkg/sql/execinfrapb/api.proto +++ b/pkg/sql/execinfrapb/api.proto @@ -41,14 +41,14 @@ message SetupFlowRequest { // Version of distsqlrun protocol; a server accepts a certain range of // versions, up to its own version. See server.go for more details. optional uint32 version = 5 [(gogoproto.nullable) = false, - (gogoproto.casttype) = "DistSQLVersion"]; + (gogoproto.casttype) = "DistSQLVersion"]; optional FlowSpec flow = 3 [(gogoproto.nullable) = false]; optional EvalContext eval_context = 6 [(gogoproto.nullable) = false]; optional bool trace_kv = 8 [(gogoproto.nullable) = false, - (gogoproto.customname) = "TraceKV"]; + (gogoproto.customname) = "TraceKV"]; // CollectStats specifies whether stats collection should be enabled for this // flow. Note that some form of tracing must be enabled for these stats to be @@ -59,18 +59,18 @@ message SetupFlowRequest { // StatementSQL is the SQL statement for which this flow is executing. It // is populated on a best effort basis. optional string statement_sql = 10 [(gogoproto.nullable) = false, - (gogoproto.customname) = "StatementSQL"]; + (gogoproto.customname) = "StatementSQL"]; } // FlowSpec describes a "flow" which is a subgraph of a distributed SQL // computation consisting of processors and streams. message FlowSpec { optional bytes flow_id = 1 [(gogoproto.nullable) = false, - (gogoproto.customname) = "FlowID", - (gogoproto.customtype) = "FlowID"]; - // The NodeID of the gateway that planned this Flow. Used for debugging. + (gogoproto.customname) = "FlowID", + (gogoproto.customtype) = "FlowID"]; + // The SQLInstanceID of the gateway that planned this Flow. Used for debugging. optional int32 gateway = 3 [(gogoproto.nullable) = false, - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/base.SQLInstanceID"]; repeated ProcessorSpec processors = 2 [(gogoproto.nullable) = false]; } @@ -127,9 +127,9 @@ message ConsumerHandshake { // The server's DistSQL version range. optional uint32 version = 3 [(gogoproto.nullable) = false, - (gogoproto.casttype) = "DistSQLVersion"]; + (gogoproto.casttype) = "DistSQLVersion"]; optional uint32 min_accepted_version = 4 [(gogoproto.nullable) = false, - (gogoproto.casttype) = "DistSQLVersion"]; + (gogoproto.casttype) = "DistSQLVersion"]; } // CancelDeadFlowsRequest is a request to cancel some flows that have been @@ -142,8 +142,8 @@ message CancelDeadFlowsRequest { // in the queue (i.e. it has already completed or is currently running), such // flow is ignored. repeated bytes flow_ids = 1 [(gogoproto.nullable) = false, - (gogoproto.customname) = "FlowIDs", - (gogoproto.customtype) = "FlowID"]; + (gogoproto.customname) = "FlowIDs", + (gogoproto.customtype) = "FlowID"]; } service DistSQL { diff --git a/pkg/sql/execinfrapb/data.proto b/pkg/sql/execinfrapb/data.proto index b058944cd352..da12cda7b420 100644 --- a/pkg/sql/execinfrapb/data.proto +++ b/pkg/sql/execinfrapb/data.proto @@ -38,7 +38,7 @@ import "google/protobuf/timestamp.proto"; message Error { option (gogoproto.goproto_stringer) = false; - reserved 1,2; + reserved 1, 2; // full_error contains a structured errors with possibly multiple // wrapping layers implementing the errors.Cause() interface. @@ -103,16 +103,16 @@ message StreamEndpointSpec { // // For SYNC_RESPONSE streams, the ID is unused. optional int32 stream_id = 2 [(gogoproto.nullable) = false, - (gogoproto.customname) = "StreamID", - (gogoproto.casttype) = "StreamID"]; - // Node ID of the target host, only used for outgoing REMOTE streams. + (gogoproto.customname) = "StreamID", + (gogoproto.casttype) = "StreamID"]; + // SQLInstanceID of the target host, only used for outgoing REMOTE streams. optional int32 target_node_id = 4 [(gogoproto.nullable) = false, - (gogoproto.customname) = "TargetNodeID", - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; - // Node ID of the origin node, only used for REMOTE streams. + (gogoproto.customname) = "TargetNodeID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/base.SQLInstanceID"]; + // SQLInstanceID of the origin node, only used for REMOTE streams. optional int32 origin_node_id = 5 [(gogoproto.nullable) = false, - (gogoproto.customname) = "OriginNodeID", - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; + (gogoproto.customname) = "OriginNodeID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/base.SQLInstanceID"]; reserved 3; } @@ -209,12 +209,12 @@ message DatumInfo { // ProducerHeader is a message that is sent once at the beginning of a stream. message ProducerHeader { optional bytes flow_id = 1 [(gogoproto.nullable) = false, - (gogoproto.customname) = "FlowID", - (gogoproto.customtype) = "FlowID"]; + (gogoproto.customname) = "FlowID", + (gogoproto.customtype) = "FlowID"]; optional int32 stream_id = 2 [(gogoproto.nullable) = false, - (gogoproto.customname) = "StreamID", - (gogoproto.casttype) = "StreamID"]; + (gogoproto.customname) = "StreamID", + (gogoproto.casttype) = "StreamID"]; } // ProducerData is a message that can be sent multiple times as part of a stream @@ -264,7 +264,7 @@ message RemoteProducerMetadata { message RowNum { // The ID of the processor that is producing rows. optional string sender_id = 1 [(gogoproto.nullable) = false, - (gogoproto.customname) = "SenderID"]; + (gogoproto.customname) = "SenderID"]; // A running count of the number of rows emitted from the sender so far. optional int32 row_num = 2 [(gogoproto.nullable) = false]; // When set, indicates that the row count contains the expected number of @@ -317,10 +317,10 @@ message RemoteProducerMetadata { // distsqlrun/server.go. message DistSQLVersionGossipInfo { optional uint32 version = 1 [(gogoproto.nullable) = false, - (gogoproto.casttype) = "DistSQLVersion"]; + (gogoproto.casttype) = "DistSQLVersion"]; optional uint32 min_accepted_version = 2 [(gogoproto.nullable) = false, - (gogoproto.casttype) = "DistSQLVersion"]; + (gogoproto.casttype) = "DistSQLVersion"]; } // DistSQLDrainingInfo represents the DistSQL draining state that gets gossiped diff --git a/pkg/sql/execinfrapb/flow_diagram.go b/pkg/sql/execinfrapb/flow_diagram.go index a9a87a07a484..f4bdaf561e11 100644 --- a/pkg/sql/execinfrapb/flow_diagram.go +++ b/pkg/sql/execinfrapb/flow_diagram.go @@ -21,7 +21,6 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" @@ -612,9 +611,9 @@ type diagramData struct { Processors []diagramProcessor `json:"processors"` Edges []diagramEdge `json:"edges"` - flags DiagramFlags - flowID FlowID - nodeIDs []roachpb.NodeID + flags DiagramFlags + flowID FlowID + sqlInstanceIDs []base.SQLInstanceID } var _ FlowDiagram = &diagramData{} @@ -633,15 +632,15 @@ func (d *diagramData) AddSpans(spans []tracingpb.RecordedSpan) { statsMap := ExtractStatsFromSpans(spans, d.flags.MakeDeterministic) for i := range d.Processors { p := &d.Processors[i] - nodeID := d.nodeIDs[p.NodeIdx] - component := ProcessorComponentID(base.SQLInstanceID(nodeID), d.flowID, p.processorID) + sqlInstanceID := d.sqlInstanceIDs[p.NodeIdx] + component := ProcessorComponentID(sqlInstanceID, d.flowID, p.processorID) if compStats := statsMap[component]; compStats != nil { p.Core.Details = append(p.Core.Details, compStats.StatsForQueryPlan()...) } } for i := range d.Edges { - originNodeID := d.nodeIDs[d.Processors[d.Edges[i].SourceProc].NodeIdx] - component := StreamComponentID(base.SQLInstanceID(originNodeID), d.flowID, d.Edges[i].streamID) + originSQLInstanceID := d.sqlInstanceIDs[d.Processors[d.Edges[i].SourceProc].NodeIdx] + component := StreamComponentID(originSQLInstanceID, d.flowID, d.Edges[i].streamID) if compStats := statsMap[component]; compStats != nil { d.Edges[i].Stats = compStats.StatsForQueryPlan() } @@ -649,18 +648,18 @@ func (d *diagramData) AddSpans(spans []tracingpb.RecordedSpan) { } // generateDiagramData generates the diagram data, given a list of flows (one -// per node). The nodeIDs list corresponds 1-1 to the flows list. +// per node). The sqlInstanceIDs list corresponds 1-1 to the flows list. func generateDiagramData( - sql string, flows []FlowSpec, nodeIDs []roachpb.NodeID, flags DiagramFlags, + sql string, flows []FlowSpec, sqlInstanceIDs []base.SQLInstanceID, flags DiagramFlags, ) (FlowDiagram, error) { d := &diagramData{ - SQL: sql, - nodeIDs: nodeIDs, - flags: flags, + SQL: sql, + sqlInstanceIDs: sqlInstanceIDs, + flags: flags, } - d.NodeNames = make([]string, len(nodeIDs)) + d.NodeNames = make([]string, len(sqlInstanceIDs)) for i := range d.NodeNames { - d.NodeNames[i] = nodeIDs[i].String() + d.NodeNames[i] = sqlInstanceIDs[i].String() } if len(flows) > 0 { @@ -789,31 +788,31 @@ func generateDiagramData( // one FlowSpec per node. The function assumes that StreamIDs are unique across // all flows. func GeneratePlanDiagram( - sql string, flows map[roachpb.NodeID]*FlowSpec, flags DiagramFlags, + sql string, flows map[base.SQLInstanceID]*FlowSpec, flags DiagramFlags, ) (FlowDiagram, error) { // We sort the flows by node because we want the diagram data to be // deterministic. - nodeIDs := make([]roachpb.NodeID, 0, len(flows)) + sqlInstanceIDs := make([]base.SQLInstanceID, 0, len(flows)) for n := range flows { - nodeIDs = append(nodeIDs, n) + sqlInstanceIDs = append(sqlInstanceIDs, n) } - sort.Slice(nodeIDs, func(i, j int) bool { - return nodeIDs[i] < nodeIDs[j] + sort.Slice(sqlInstanceIDs, func(i, j int) bool { + return sqlInstanceIDs[i] < sqlInstanceIDs[j] }) - flowSlice := make([]FlowSpec, len(nodeIDs)) - for i, n := range nodeIDs { + flowSlice := make([]FlowSpec, len(sqlInstanceIDs)) + for i, n := range sqlInstanceIDs { flowSlice[i] = *flows[n] } - return generateDiagramData(sql, flowSlice, nodeIDs, flags) + return generateDiagramData(sql, flowSlice, sqlInstanceIDs, flags) } // GeneratePlanDiagramURL generates the json data for a flow diagram and a // URL which encodes the diagram. There should be one FlowSpec per node. The // function assumes that StreamIDs are unique across all flows. func GeneratePlanDiagramURL( - sql string, flows map[roachpb.NodeID]*FlowSpec, flags DiagramFlags, + sql string, flows map[base.SQLInstanceID]*FlowSpec, flags DiagramFlags, ) (string, url.URL, error) { d, err := GeneratePlanDiagram(sql, flows, flags) if err != nil { diff --git a/pkg/sql/execinfrapb/flow_diagram_external_test.go b/pkg/sql/execinfrapb/flow_diagram_external_test.go index 2df9842e4414..168b94a36f4f 100644 --- a/pkg/sql/execinfrapb/flow_diagram_external_test.go +++ b/pkg/sql/execinfrapb/flow_diagram_external_test.go @@ -14,7 +14,7 @@ import ( "context" "testing" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -56,7 +56,7 @@ func TestPlanDiagramTableReaderWrapColumns(t *testing.T) { t.Fatal(err) } - flows := make(map[roachpb.NodeID]*execinfrapb.FlowSpec) + flows := make(map[base.SQLInstanceID]*execinfrapb.FlowSpec) tr := execinfrapb.TableReaderSpec{ Table: *desc.TableDesc(), diff --git a/pkg/sql/execinfrapb/flow_diagram_test.go b/pkg/sql/execinfrapb/flow_diagram_test.go index f124f22a2460..7a68dab8eb70 100644 --- a/pkg/sql/execinfrapb/flow_diagram_test.go +++ b/pkg/sql/execinfrapb/flow_diagram_test.go @@ -17,7 +17,7 @@ import ( "strings" "testing" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) @@ -45,7 +45,7 @@ func compareDiagrams(t *testing.T, result string, expected string) { func TestPlanDiagramIndexJoin(t *testing.T) { defer leaktest.AfterTest(t)() - flows := make(map[roachpb.NodeID]*FlowSpec) + flows := make(map[base.SQLInstanceID]*FlowSpec) desc := &descpb.TableDescriptor{ Name: "Table", @@ -164,7 +164,7 @@ func TestPlanDiagramIndexJoin(t *testing.T) { func TestPlanDiagramJoin(t *testing.T) { defer leaktest.AfterTest(t)() - flows := make(map[roachpb.NodeID]*FlowSpec) + flows := make(map[base.SQLInstanceID]*FlowSpec) descA := &descpb.TableDescriptor{Name: "TableA"} descB := &descpb.TableDescriptor{Name: "TableB"} diff --git a/pkg/sql/execinfrapb/testutils.go b/pkg/sql/execinfrapb/testutils.go index 5f03ac568d84..a535543f5c43 100644 --- a/pkg/sql/execinfrapb/testutils.go +++ b/pkg/sql/execinfrapb/testutils.go @@ -46,10 +46,10 @@ func newInsecureRPCContext(ctx context.Context, stopper *stop.Stopper) *rpc.Cont // StartMockDistSQLServer starts a MockDistSQLServer and returns the address on // which it's listening. func StartMockDistSQLServer( - ctx context.Context, clock *hlc.Clock, stopper *stop.Stopper, nodeID roachpb.NodeID, + ctx context.Context, clock *hlc.Clock, stopper *stop.Stopper, sqlInstanceID base.SQLInstanceID, ) (uuid.UUID, *MockDistSQLServer, net.Addr, error) { rpcContext := newInsecureRPCContext(ctx, stopper) - rpcContext.NodeID.Set(context.TODO(), nodeID) + rpcContext.NodeID.Set(context.TODO(), roachpb.NodeID(sqlInstanceID)) server := rpc.NewServer(rpcContext) mock := newMockDistSQLServer() RegisterDistSQLServer(server, mock) diff --git a/pkg/sql/execstats/BUILD.bazel b/pkg/sql/execstats/BUILD.bazel index 8cd311dabde0..28233648bb5c 100644 --- a/pkg/sql/execstats/BUILD.bazel +++ b/pkg/sql/execstats/BUILD.bazel @@ -7,7 +7,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", - "//pkg/roachpb:with-mocks", "//pkg/sql/execinfrapb", "//pkg/util", "//pkg/util/buildutil", @@ -27,7 +26,6 @@ go_test( embed = [":execstats"], deps = [ "//pkg/base", - "//pkg/roachpb:with-mocks", "//pkg/security", "//pkg/security/securitytest", "//pkg/server", diff --git a/pkg/sql/execstats/traceanalyzer.go b/pkg/sql/execstats/traceanalyzer.go index 20ac2260c3b4..da2b96a7bb9b 100644 --- a/pkg/sql/execstats/traceanalyzer.go +++ b/pkg/sql/execstats/traceanalyzer.go @@ -14,7 +14,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/buildutil" @@ -22,17 +21,10 @@ import ( "github.com/cockroachdb/errors" ) -type processorStats struct { - // TODO(radu): this field redundant with stats.Component.SQLInstanceID. - nodeID roachpb.NodeID - stats *execinfrapb.ComponentStats -} - type streamStats struct { - // TODO(radu): this field redundant with stats.Component.SQLInstanceID. - originNodeID roachpb.NodeID - destinationNodeID roachpb.NodeID - stats *execinfrapb.ComponentStats + originSQLInstanceID base.SQLInstanceID + destinationSQLInstanceID base.SQLInstanceID + stats *execinfrapb.ComponentStats } type flowStats struct { @@ -46,11 +38,11 @@ type FlowsMetadata struct { // flowID is the FlowID of the flows belonging to the physical plan. Note that // the same FlowID is used across multiple flows in the same query. flowID execinfrapb.FlowID - // processorStats maps a processor ID to stats associated with this - // processor extracted from a trace as well as some metadata. Note that it - // is possible for the processorStats to have nil stats, which indicates - // that no stats were found for the given processor in the trace. - processorStats map[execinfrapb.ProcessorID]*processorStats + // processorStats maps a processor ID to stats associated with this component + // extracted from a trace as well as some metadata. Note that it is possible + // for the processorStats to have nil stats, which indicates that no stats + // were found for the given processor in the trace. + processorStats map[execinfrapb.ProcessorID]*execinfrapb.ComponentStats // streamStats maps a stream ID to stats associated with this stream // extracted from a trace as well as some metadata. Note that it is possible // for the streamStats to have nil stats, which indicates that no stats were @@ -64,32 +56,35 @@ type FlowsMetadata struct { // NewFlowsMetadata creates a FlowsMetadata for the given physical plan // information. -func NewFlowsMetadata(flows map[roachpb.NodeID]*execinfrapb.FlowSpec) *FlowsMetadata { +func NewFlowsMetadata(flows map[base.SQLInstanceID]*execinfrapb.FlowSpec) *FlowsMetadata { a := &FlowsMetadata{ - processorStats: make(map[execinfrapb.ProcessorID]*processorStats), + processorStats: make(map[execinfrapb.ProcessorID]*execinfrapb.ComponentStats), streamStats: make(map[execinfrapb.StreamID]*streamStats), flowStats: make(map[base.SQLInstanceID]*flowStats), } // Annotate the maps with physical plan information. - for nodeID, flow := range flows { + for sqlInstanceID, flow := range flows { if a.flowID.IsUnset() { a.flowID = flow.FlowID } else if buildutil.CrdbTestBuild && !a.flowID.Equal(flow.FlowID) { panic( errors.AssertionFailedf( "expected the same FlowID to be used for all flows. UUID of first flow: %v, UUID of flow on node %s: %v", - a.flowID, nodeID, flow.FlowID), + a.flowID, sqlInstanceID, flow.FlowID), ) } - a.flowStats[base.SQLInstanceID(nodeID)] = &flowStats{} + a.flowStats[sqlInstanceID] = &flowStats{} for _, proc := range flow.Processors { - a.processorStats[execinfrapb.ProcessorID(proc.ProcessorID)] = &processorStats{nodeID: nodeID} + procID := execinfrapb.ProcessorID(proc.ProcessorID) + a.processorStats[procID] = &execinfrapb.ComponentStats{} + a.processorStats[procID].Component.SQLInstanceID = sqlInstanceID + for _, output := range proc.Output { for _, stream := range output.Streams { a.streamStats[stream.StreamID] = &streamStats{ - originNodeID: nodeID, - destinationNodeID: stream.TargetNodeID, + originSQLInstanceID: sqlInstanceID, + destinationSQLInstanceID: stream.TargetNodeID, } } } @@ -177,11 +172,7 @@ func (a *TraceAnalyzer) AddTrace(trace []tracingpb.RecordedSpan, makeDeterminist switch component.Type { case execinfrapb.ComponentID_PROCESSOR: id := component.ID - processorStats := a.processorStats[execinfrapb.ProcessorID(id)] - if processorStats == nil { - return errors.Errorf("trace has span for processor %d but the processor does not exist in the physical plan", id) - } - processorStats.stats = componentStats + a.processorStats[execinfrapb.ProcessorID(id)] = componentStats case execinfrapb.ComponentID_STREAM: id := component.ID @@ -227,14 +218,14 @@ func (a *TraceAnalyzer) ProcessStats() error { // Process processorStats. for _, stats := range a.processorStats { - if stats.stats == nil { + if stats == nil { continue } - instanceID := base.SQLInstanceID(stats.nodeID) - a.nodeLevelStats.KVBytesReadGroupedByNode[instanceID] += int64(stats.stats.KV.BytesRead.Value()) - a.nodeLevelStats.KVRowsReadGroupedByNode[instanceID] += int64(stats.stats.KV.TuplesRead.Value()) - a.nodeLevelStats.KVTimeGroupedByNode[instanceID] += stats.stats.KV.KVTime.Value() - a.nodeLevelStats.ContentionTimeGroupedByNode[instanceID] += stats.stats.KV.ContentionTime.Value() + instanceID := stats.Component.SQLInstanceID + a.nodeLevelStats.KVBytesReadGroupedByNode[instanceID] += int64(stats.KV.BytesRead.Value()) + a.nodeLevelStats.KVRowsReadGroupedByNode[instanceID] += int64(stats.KV.TuplesRead.Value()) + a.nodeLevelStats.KVTimeGroupedByNode[instanceID] += stats.KV.KVTime.Value() + a.nodeLevelStats.ContentionTimeGroupedByNode[instanceID] += stats.KV.ContentionTime.Value() } // Process streamStats. @@ -242,7 +233,7 @@ func (a *TraceAnalyzer) ProcessStats() error { if stats.stats == nil { continue } - originInstanceID := base.SQLInstanceID(stats.originNodeID) + originInstanceID := stats.originSQLInstanceID // Set networkBytesSentGroupedByNode. bytes, err := getNetworkBytesFromComponentStats(stats.stats) diff --git a/pkg/sql/execstats/traceanalyzer_test.go b/pkg/sql/execstats/traceanalyzer_test.go index d806aa39b61c..948f053922ae 100644 --- a/pkg/sql/execstats/traceanalyzer_test.go +++ b/pkg/sql/execstats/traceanalyzer_test.go @@ -18,7 +18,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" @@ -59,11 +58,11 @@ func TestTraceAnalyzer(t *testing.T) { UseDatabase: "test", Knobs: base.TestingKnobs{ SQLExecutor: &sql.ExecutorTestingKnobs{ - TestingSaveFlows: func(stmt string) func(map[roachpb.NodeID]*execinfrapb.FlowSpec, execinfra.OpChains) error { + TestingSaveFlows: func(stmt string) func(map[base.SQLInstanceID]*execinfrapb.FlowSpec, execinfra.OpChains) error { if stmt != testStmt { - return func(map[roachpb.NodeID]*execinfrapb.FlowSpec, execinfra.OpChains) error { return nil } + return func(map[base.SQLInstanceID]*execinfrapb.FlowSpec, execinfra.OpChains) error { return nil } } - return func(flows map[roachpb.NodeID]*execinfrapb.FlowSpec, _ execinfra.OpChains) error { + return func(flows map[base.SQLInstanceID]*execinfrapb.FlowSpec, _ execinfra.OpChains) error { flowsMetadata := execstats.NewFlowsMetadata(flows) analyzer := execstats.NewTraceAnalyzer(flowsMetadata) analyzerChan <- analyzer diff --git a/pkg/sql/execstats/utils_test.go b/pkg/sql/execstats/utils_test.go index c9488a054e5b..0c2b767852bd 100644 --- a/pkg/sql/execstats/utils_test.go +++ b/pkg/sql/execstats/utils_test.go @@ -12,7 +12,6 @@ package execstats import ( "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" ) @@ -27,18 +26,14 @@ func (a *TraceAnalyzer) AddComponentStats(stats *execinfrapb.ComponentStats) { func (m *FlowsMetadata) AddComponentStats(stats *execinfrapb.ComponentStats) { switch stats.Component.Type { case execinfrapb.ComponentID_PROCESSOR: - processorStat := &processorStats{ - nodeID: roachpb.NodeID(stats.Component.SQLInstanceID), - stats: stats, - } if m.processorStats == nil { - m.processorStats = make(map[execinfrapb.ProcessorID]*processorStats) + m.processorStats = make(map[execinfrapb.ProcessorID]*execinfrapb.ComponentStats) } - m.processorStats[execinfrapb.ProcessorID(stats.Component.ID)] = processorStat + m.processorStats[execinfrapb.ProcessorID(stats.Component.ID)] = stats case execinfrapb.ComponentID_STREAM: streamStat := &streamStats{ - originNodeID: roachpb.NodeID(stats.Component.SQLInstanceID), - stats: stats, + originSQLInstanceID: stats.Component.SQLInstanceID, + stats: stats, } if m.streamStats == nil { m.streamStats = make(map[execinfrapb.StreamID]*streamStats) diff --git a/pkg/sql/explain_vec.go b/pkg/sql/explain_vec.go index ad428c4d94fb..e3fe813e3af6 100644 --- a/pkg/sql/explain_vec.go +++ b/pkg/sql/explain_vec.go @@ -77,7 +77,7 @@ func (n *explainVecNode) startExec(params runParams) error { willDistribute := physPlan.Distribution.WillDistribute() n.run.lines, n.run.cleanup, err = colflow.ExplainVec( params.ctx, flowCtx, flows, physPlan.LocalProcessors, nil, /* opChains */ - distSQLPlanner.gatewayNodeID, verbose, willDistribute, + distSQLPlanner.gatewaySQLInstanceID, verbose, willDistribute, ) if err != nil { return err diff --git a/pkg/sql/flowinfra/BUILD.bazel b/pkg/sql/flowinfra/BUILD.bazel index 2e3b06ae988e..a139bdc2b66d 100644 --- a/pkg/sql/flowinfra/BUILD.bazel +++ b/pkg/sql/flowinfra/BUILD.bazel @@ -15,6 +15,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/flowinfra", visibility = ["//visibility:public"], deps = [ + "//pkg/base", "//pkg/kv", "//pkg/roachpb:with-mocks", "//pkg/server/telemetry", diff --git a/pkg/sql/flowinfra/cluster_test.go b/pkg/sql/flowinfra/cluster_test.go index 26573658f561..3e96c394349d 100644 --- a/pkg/sql/flowinfra/cluster_test.go +++ b/pkg/sql/flowinfra/cluster_test.go @@ -137,7 +137,7 @@ func TestClusterFlow(t *testing.T) { Output: []execinfrapb.OutputRouterSpec{{ Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, Streams: []execinfrapb.StreamEndpointSpec{ - {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 0, TargetNodeID: tc.Server(2).NodeID()}, + {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 0, TargetNodeID: base.SQLInstanceID(tc.Server(2).NodeID())}, }, }}, ResultTypes: types.TwoIntCols, @@ -156,7 +156,7 @@ func TestClusterFlow(t *testing.T) { Output: []execinfrapb.OutputRouterSpec{{ Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, Streams: []execinfrapb.StreamEndpointSpec{ - {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 1, TargetNodeID: tc.Server(2).NodeID()}, + {Type: execinfrapb.StreamEndpointSpec_REMOTE, StreamID: 1, TargetNodeID: base.SQLInstanceID(tc.Server(2).NodeID())}, }, }}, ResultTypes: types.TwoIntCols, @@ -715,7 +715,7 @@ func BenchmarkInfrastructure(b *testing.B) { Output: []execinfrapb.OutputRouterSpec{{ Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, Streams: []execinfrapb.StreamEndpointSpec{ - {Type: streamType(i), StreamID: execinfrapb.StreamID(i), TargetNodeID: tc.Server(0).NodeID()}, + {Type: streamType(i), StreamID: execinfrapb.StreamID(i), TargetNodeID: base.SQLInstanceID(tc.Server(0).NodeID())}, }, }}, ResultTypes: types.ThreeIntCols, diff --git a/pkg/sql/flowinfra/outbox.go b/pkg/sql/flowinfra/outbox.go index f0e61ab5518b..f445948e81bd 100644 --- a/pkg/sql/flowinfra/outbox.go +++ b/pkg/sql/flowinfra/outbox.go @@ -17,7 +17,7 @@ import ( "sync/atomic" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -47,9 +47,9 @@ type Outbox struct { // RowChannel implements the RowReceiver interface. execinfra.RowChannel - flowCtx *execinfra.FlowCtx - streamID execinfrapb.StreamID - nodeID roachpb.NodeID + flowCtx *execinfra.FlowCtx + streamID execinfrapb.StreamID + sqlInstanceID base.SQLInstanceID // The rows received from the RowChannel will be forwarded on this stream once // it is established. stream flowStream @@ -85,12 +85,12 @@ var _ Startable = &Outbox{} // NewOutbox creates a new Outbox. func NewOutbox( flowCtx *execinfra.FlowCtx, - nodeID roachpb.NodeID, + sqlInstanceID base.SQLInstanceID, streamID execinfrapb.StreamID, numOutboxes *int32, isGatewayNode bool, ) *Outbox { - m := &Outbox{flowCtx: flowCtx, nodeID: nodeID} + m := &Outbox{flowCtx: flowCtx, sqlInstanceID: sqlInstanceID} m.encoder.SetHeaderFields(flowCtx.ID, streamID) m.streamID = streamID m.numOutboxes = numOutboxes @@ -218,7 +218,7 @@ func (m *Outbox) mainLoop(ctx context.Context) error { if m.stream == nil { conn, err := execinfra.GetConnForOutbox( - ctx, m.flowCtx.Cfg.NodeDialer, m.nodeID, SettingFlowStreamTimeout.Get(&m.flowCtx.Cfg.Settings.SV), + ctx, m.flowCtx.Cfg.NodeDialer, m.sqlInstanceID, SettingFlowStreamTimeout.Get(&m.flowCtx.Cfg.Settings.SV), ) if err != nil { // Log any Dial errors. This does not have a verbosity check due to being diff --git a/pkg/sql/flowinfra/outbox_test.go b/pkg/sql/flowinfra/outbox_test.go index ec44eaedf9cd..9aade59f906b 100644 --- a/pkg/sql/flowinfra/outbox_test.go +++ b/pkg/sql/flowinfra/outbox_test.go @@ -40,10 +40,10 @@ import ( "github.com/cockroachdb/errors" ) -// staticAddressResolver maps StaticNodeID to the given address. +// staticAddressResolver maps StaticSQLInstanceID to the given address. func staticAddressResolver(addr net.Addr) nodedialer.AddressResolver { return func(nodeID roachpb.NodeID) (net.Addr, error) { - if nodeID == execinfra.StaticNodeID { + if nodeID == roachpb.NodeID(execinfra.StaticSQLInstanceID) { return addr, nil } return nil, errors.Errorf("node %d not found", nodeID) @@ -59,7 +59,7 @@ func TestOutbox(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - clusterID, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, clock, stopper, execinfra.StaticNodeID) + clusterID, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, clock, stopper, execinfra.StaticSQLInstanceID) if err != nil { t.Fatal(err) } @@ -79,7 +79,7 @@ func TestOutbox(t *testing.T) { NodeID: base.TestingIDContainer, } streamID := execinfrapb.StreamID(42) - outbox := flowinfra.NewOutbox(&flowCtx, execinfra.StaticNodeID, streamID, nil /* numOutboxes */, false /* isGatewayNode */) + outbox := flowinfra.NewOutbox(&flowCtx, execinfra.StaticSQLInstanceID, streamID, nil /* numOutboxes */, false /* isGatewayNode */) outbox.Init(types.OneIntCol) var outboxWG sync.WaitGroup var cancel func() @@ -221,7 +221,7 @@ func TestOutboxInitializesStreamBeforeReceivingAnyRows(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - clusterID, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, clock, stopper, execinfra.StaticNodeID) + clusterID, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, clock, stopper, execinfra.StaticSQLInstanceID) if err != nil { t.Fatal(err) } @@ -242,7 +242,7 @@ func TestOutboxInitializesStreamBeforeReceivingAnyRows(t *testing.T) { NodeID: base.TestingIDContainer, } streamID := execinfrapb.StreamID(42) - outbox := flowinfra.NewOutbox(&flowCtx, execinfra.StaticNodeID, streamID, nil /* numOutboxes */, false /* isGatewayNode */) + outbox := flowinfra.NewOutbox(&flowCtx, execinfra.StaticSQLInstanceID, streamID, nil /* numOutboxes */, false /* isGatewayNode */) var outboxWG sync.WaitGroup outbox.Init(types.OneIntCol) @@ -292,7 +292,7 @@ func TestOutboxClosesWhenConsumerCloses(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - clusterID, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, clock, stopper, execinfra.StaticNodeID) + clusterID, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, clock, stopper, execinfra.StaticSQLInstanceID) if err != nil { t.Fatal(err) } @@ -319,7 +319,7 @@ func TestOutboxClosesWhenConsumerCloses(t *testing.T) { var cancel func() ctx, cancel = context.WithCancel(ctx) defer cancel() - outbox = flowinfra.NewOutbox(&flowCtx, execinfra.StaticNodeID, streamID, nil /* numOutboxes */, false /* isGatewayNode */) + outbox = flowinfra.NewOutbox(&flowCtx, execinfra.StaticSQLInstanceID, streamID, nil /* numOutboxes */, false /* isGatewayNode */) outbox.Init(types.OneIntCol) outbox.Start(ctx, &wg, cancel) @@ -368,7 +368,7 @@ func TestOutboxCancelsFlowOnError(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - clusterID, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, clock, stopper, execinfra.StaticNodeID) + clusterID, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, clock, stopper, execinfra.StaticSQLInstanceID) if err != nil { t.Fatal(err) } @@ -402,7 +402,7 @@ func TestOutboxCancelsFlowOnError(t *testing.T) { ctxCanceled = true } - outbox = flowinfra.NewOutbox(&flowCtx, execinfra.StaticNodeID, streamID, nil /* numOutboxes */, false /* isGatewayNode */) + outbox = flowinfra.NewOutbox(&flowCtx, execinfra.StaticSQLInstanceID, streamID, nil /* numOutboxes */, false /* isGatewayNode */) outbox.Init(types.OneIntCol) outbox.Start(ctx, &wg, mockCancel) @@ -450,7 +450,7 @@ func TestOutboxUnblocksProducers(t *testing.T) { ctx, cancel := context.WithCancel(ctx) defer cancel() - outbox = flowinfra.NewOutbox(&flowCtx, execinfra.StaticNodeID, streamID, nil /* numOutboxes */, false /* isGatewayNode */) + outbox = flowinfra.NewOutbox(&flowCtx, execinfra.StaticSQLInstanceID, streamID, nil /* numOutboxes */, false /* isGatewayNode */) outbox.Init(types.OneIntCol) // Fill up the outbox. @@ -491,7 +491,7 @@ func BenchmarkOutbox(b *testing.B) { stopper := stop.NewStopper() defer stopper.Stop(bgCtx) clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - clusterID, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(bgCtx, clock, stopper, execinfra.StaticNodeID) + clusterID, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(bgCtx, clock, stopper, execinfra.StaticSQLInstanceID) if err != nil { b.Fatal(err) } @@ -517,7 +517,7 @@ func BenchmarkOutbox(b *testing.B) { }, NodeID: base.TestingIDContainer, } - outbox := flowinfra.NewOutbox(&flowCtx, execinfra.StaticNodeID, streamID, nil /* numOutboxes */, false /* isGatewayNode */) + outbox := flowinfra.NewOutbox(&flowCtx, execinfra.StaticSQLInstanceID, streamID, nil /* numOutboxes */, false /* isGatewayNode */) outbox.Init(types.MakeIntCols(numCols)) var outboxWG sync.WaitGroup var cancel func() diff --git a/pkg/sql/flowinfra/server_test.go b/pkg/sql/flowinfra/server_test.go index 7e05d8ddaf98..a09b1d718c58 100644 --- a/pkg/sql/flowinfra/server_test.go +++ b/pkg/sql/flowinfra/server_test.go @@ -146,7 +146,7 @@ func TestDistSQLServerGossipsVersion(t *testing.T) { var v execinfrapb.DistSQLVersionGossipInfo if err := s.GossipI().(*gossip.Gossip).GetInfoProto( - gossip.MakeDistSQLNodeVersionKey(s.NodeID()), &v, + gossip.MakeDistSQLNodeVersionKey(base.SQLInstanceID(s.NodeID())), &v, ); err != nil { t.Fatal(err) } diff --git a/pkg/sql/flowinfra/utils_test.go b/pkg/sql/flowinfra/utils_test.go index 9009c35c40c3..d9f818f9de9d 100644 --- a/pkg/sql/flowinfra/utils_test.go +++ b/pkg/sql/flowinfra/utils_test.go @@ -14,6 +14,7 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -39,13 +40,13 @@ func createDummyStream() ( stopper := stop.NewStopper() ctx := context.Background() clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) - clusterID, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, clock, stopper, execinfra.StaticNodeID) + clusterID, mockServer, addr, err := execinfrapb.StartMockDistSQLServer(ctx, clock, stopper, execinfra.StaticSQLInstanceID) if err != nil { return nil, nil, nil, err } rpcContext := rpc.NewInsecureTestingContextWithClusterID(ctx, clock, stopper, clusterID) - conn, err := rpcContext.GRPCDialNode(addr.String(), execinfra.StaticNodeID, + conn, err := rpcContext.GRPCDialNode(addr.String(), roachpb.NodeID(execinfra.StaticSQLInstanceID), rpc.DefaultClass).Connect(ctx) if err != nil { return nil, nil, nil, err diff --git a/pkg/sql/physicalplan/BUILD.bazel b/pkg/sql/physicalplan/BUILD.bazel index 4826d3b5eeb0..4a0842818be2 100644 --- a/pkg/sql/physicalplan/BUILD.bazel +++ b/pkg/sql/physicalplan/BUILD.bazel @@ -13,6 +13,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/physicalplan", visibility = ["//visibility:public"], deps = [ + "//pkg/base", "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvcoord:with-mocks", diff --git a/pkg/sql/physicalplan/physical_plan.go b/pkg/sql/physicalplan/physical_plan.go index d7af528b86c6..8c278d14fa51 100644 --- a/pkg/sql/physicalplan/physical_plan.go +++ b/pkg/sql/physicalplan/physical_plan.go @@ -17,7 +17,7 @@ package physicalplan import ( "math" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -31,7 +31,7 @@ import ( // Processor contains the information associated with a processor in a plan. type Processor struct { // Node where the processor must be instantiated. - Node roachpb.NodeID + SQLInstanceID base.SQLInstanceID // Spec for the processor; note that the StreamEndpointSpecs in the input // synchronizers and output routers are not set until the end of the planning @@ -67,8 +67,8 @@ type Stream struct { type PhysicalInfrastructure struct { // -- The following fields are immutable -- - FlowID uuid.UUID - GatewayNodeID roachpb.NodeID + FlowID uuid.UUID + GatewaySQLInstanceID base.SQLInstanceID // -- The following fields are mutable -- @@ -151,11 +151,11 @@ type PhysicalPlan struct { // MakePhysicalInfrastructure initializes a PhysicalInfrastructure that can then // be used with MakePhysicalPlan. func MakePhysicalInfrastructure( - flowID uuid.UUID, gatewayNodeID roachpb.NodeID, + flowID uuid.UUID, gatewaySQLInstanceID base.SQLInstanceID, ) PhysicalInfrastructure { return PhysicalInfrastructure{ - FlowID: flowID, - GatewayNodeID: gatewayNodeID, + FlowID: flowID, + GatewaySQLInstanceID: gatewaySQLInstanceID, } } @@ -198,12 +198,12 @@ func (p *PhysicalPlan) NewStage(containsRemoteProcessor bool, allowPartialDistri // NewStageOnNodes is the same as NewStage but takes in the information about // the nodes participating in the new stage and the gateway. -func (p *PhysicalPlan) NewStageOnNodes(nodes []roachpb.NodeID) int32 { +func (p *PhysicalPlan) NewStageOnNodes(sqlInstanceIDs []base.SQLInstanceID) int32 { // We have a remote processor either when we have multiple nodes // participating in the stage or the single processor is scheduled not on // the gateway. return p.NewStage( - len(nodes) > 1 || nodes[0] != p.GatewayNodeID, /* containsRemoteProcessor */ + len(sqlInstanceIDs) > 1 || sqlInstanceIDs[0] != p.GatewaySQLInstanceID, /* containsRemoteProcessor */ false, /* allowPartialDistribution */ ) } @@ -220,8 +220,8 @@ func (p *PhysicalPlan) SetMergeOrdering(o execinfrapb.Ordering) { // ProcessorCorePlacement indicates on which node a particular processor core // needs to be planned. type ProcessorCorePlacement struct { - NodeID roachpb.NodeID - Core execinfrapb.ProcessorCoreUnion + SQLInstanceID base.SQLInstanceID + Core execinfrapb.ProcessorCoreUnion // EstimatedRowCount, if set to non-zero, is the optimizer's guess of how // many rows will be emitted from this processor. EstimatedRowCount uint64 @@ -242,7 +242,7 @@ func (p *PhysicalPlan) AddNoInputStage( // plan multiple table readers on the gateway if the plan is local. containsRemoteProcessor := false for i := range corePlacements { - if corePlacements[i].NodeID != p.GatewayNodeID { + if corePlacements[i].SQLInstanceID != p.GatewaySQLInstanceID { containsRemoteProcessor = true break } @@ -251,7 +251,7 @@ func (p *PhysicalPlan) AddNoInputStage( p.ResultRouters = make([]ProcessorIdx, len(corePlacements)) for i := range p.ResultRouters { proc := Processor{ - Node: corePlacements[i].NodeID, + SQLInstanceID: corePlacements[i].SQLInstanceID, Spec: execinfrapb.ProcessorSpec{ Core: corePlacements[i].Core, Post: post, @@ -304,7 +304,7 @@ func (p *PhysicalPlan) AddNoGroupingStageWithCoreFunc( prevProc := &p.Processors[resultProc] proc := Processor{ - Node: prevProc.Node, + SQLInstanceID: prevProc.SQLInstanceID, Spec: execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{{ Type: execinfrapb.InputSyncSpec_PARALLEL_UNORDERED, @@ -374,13 +374,13 @@ func (p *PhysicalPlan) MergeResultStreams( // parallelized) which consists of a single processor on the specified node. The // previous stage (ResultRouters) are all connected to this processor. func (p *PhysicalPlan) AddSingleGroupStage( - nodeID roachpb.NodeID, + sqlInstanceID base.SQLInstanceID, core execinfrapb.ProcessorCoreUnion, post execinfrapb.PostProcessSpec, outputTypes []*types.T, ) { proc := Processor{ - Node: nodeID, + SQLInstanceID: sqlInstanceID, Spec: execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{{ // The other fields will be filled in by mergeResultStreams. @@ -391,10 +391,10 @@ func (p *PhysicalPlan) AddSingleGroupStage( Output: []execinfrapb.OutputRouterSpec{{ Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, }}, - // We're planning a single processor on the node nodeID, so we'll - // have a remote processor only when the node is different from the + // We're planning a single processor on the instance sqlInstanceID, so + // we'll have a remote processor only when the node is different from the // gateway. - StageID: p.NewStage(nodeID != p.GatewayNodeID, false /* allowPartialDistribution */), + StageID: p.NewStage(sqlInstanceID != p.GatewaySQLInstanceID, false /* allowPartialDistribution */), ResultTypes: outputTypes, }, } @@ -418,14 +418,14 @@ func (p *PhysicalPlan) EnsureSingleStreamOnGateway() { // If we don't already have a single result router on the gateway, add a // single grouping stage. if len(p.ResultRouters) != 1 || - p.Processors[p.ResultRouters[0]].Node != p.GatewayNodeID { + p.Processors[p.ResultRouters[0]].SQLInstanceID != p.GatewaySQLInstanceID { p.AddSingleGroupStage( - p.GatewayNodeID, + p.GatewaySQLInstanceID, execinfrapb.ProcessorCoreUnion{Noop: &execinfrapb.NoopCoreSpec{}}, execinfrapb.PostProcessSpec{}, p.GetResultTypes(), ) - if len(p.ResultRouters) != 1 || p.Processors[p.ResultRouters[0]].Node != p.GatewayNodeID { + if len(p.ResultRouters) != 1 || p.Processors[p.ResultRouters[0]].SQLInstanceID != p.GatewaySQLInstanceID { panic("ensuring a single stream on the gateway failed") } } @@ -818,7 +818,7 @@ func (p *PhysicalPlan) AddLimit(count int64, offset int64, exprCtx ExprContext) post.Limit = uint64(count) } p.AddSingleGroupStage( - p.GatewayNodeID, + p.GatewaySQLInstanceID, execinfrapb.ProcessorCoreUnion{Noop: &execinfrapb.NoopCoreSpec{}}, post, p.GetResultTypes(), @@ -842,14 +842,14 @@ func (p *PhysicalPlan) PopulateEndpoints() { p1 := &p.Processors[s.SourceProcessor] p2 := &p.Processors[s.DestProcessor] endpoint := execinfrapb.StreamEndpointSpec{StreamID: execinfrapb.StreamID(sIdx)} - if p1.Node == p2.Node { + if p1.SQLInstanceID == p2.SQLInstanceID { endpoint.Type = execinfrapb.StreamEndpointSpec_LOCAL } else { endpoint.Type = execinfrapb.StreamEndpointSpec_REMOTE } if endpoint.Type == execinfrapb.StreamEndpointSpec_REMOTE { - endpoint.OriginNodeID = p1.Node - endpoint.TargetNodeID = p2.Node + endpoint.OriginNodeID = p1.SQLInstanceID + endpoint.TargetNodeID = p2.SQLInstanceID } p2.Spec.Input[s.DestInput].Streams = append(p2.Spec.Input[s.DestInput].Streams, endpoint) @@ -870,18 +870,18 @@ func (p *PhysicalPlan) PopulateEndpoints() { // GenerateFlowSpecs takes a plan (with populated endpoints) and generates the // set of FlowSpecs (one per node involved in the plan). // -// gateway is the current node's NodeID. -func (p *PhysicalPlan) GenerateFlowSpecs() map[roachpb.NodeID]*execinfrapb.FlowSpec { +// gateway is the current node's SQLInstanceID. +func (p *PhysicalPlan) GenerateFlowSpecs() map[base.SQLInstanceID]*execinfrapb.FlowSpec { flowID := execinfrapb.FlowID{ UUID: p.FlowID, } - flows := make(map[roachpb.NodeID]*execinfrapb.FlowSpec, 1) + flows := make(map[base.SQLInstanceID]*execinfrapb.FlowSpec, 1) for _, proc := range p.Processors { - flowSpec, ok := flows[proc.Node] + flowSpec, ok := flows[proc.SQLInstanceID] if !ok { - flowSpec = NewFlowSpec(flowID, p.GatewayNodeID) - flows[proc.Node] = flowSpec + flowSpec = NewFlowSpec(flowID, p.GatewaySQLInstanceID) + flows[proc.SQLInstanceID] = flowSpec } flowSpec.Processors = append(flowSpec.Processors, proc.Spec) } @@ -913,7 +913,7 @@ func MergePlans( // AddJoinStage adds join processors at each of the specified nodes, and wires // the left and right-side outputs to these processors. func (p *PhysicalPlan) AddJoinStage( - nodes []roachpb.NodeID, + sqlInstanceIDs []base.SQLInstanceID, core execinfrapb.ProcessorCoreUnion, post execinfrapb.PostProcessSpec, leftEqCols, rightEqCols []uint32, @@ -923,15 +923,15 @@ func (p *PhysicalPlan) AddJoinStage( resultTypes []*types.T, ) { pIdxStart := ProcessorIdx(len(p.Processors)) - stageID := p.NewStageOnNodes(nodes) + stageID := p.NewStageOnNodes(sqlInstanceIDs) - for _, n := range nodes { + for _, sqlInstanceID := range sqlInstanceIDs { inputs := make([]execinfrapb.InputSyncSpec, 0, 2) inputs = append(inputs, execinfrapb.InputSyncSpec{ColumnTypes: leftTypes}) inputs = append(inputs, execinfrapb.InputSyncSpec{ColumnTypes: rightTypes}) proc := Processor{ - Node: n, + SQLInstanceID: sqlInstanceID, Spec: execinfrapb.ProcessorSpec{ Input: inputs, Core: core, @@ -944,7 +944,7 @@ func (p *PhysicalPlan) AddJoinStage( p.Processors = append(p.Processors, proc) } - if len(nodes) > 1 { + if len(sqlInstanceIDs) > 1 { // Parallel hash or merge join: we distribute rows (by hash of // equality columns) to len(nodes) join processors. @@ -967,7 +967,7 @@ func (p *PhysicalPlan) AddJoinStage( // Connect the left and right routers to the output joiners. Each joiner // corresponds to a hash bucket. - for bucket := 0; bucket < len(nodes); bucket++ { + for bucket := 0; bucket < len(sqlInstanceIDs); bucket++ { pIdx := pIdxStart + ProcessorIdx(bucket) // Connect left routers to the processor's first input. Currently the join @@ -984,7 +984,7 @@ func (p *PhysicalPlan) AddJoinStage( // logical stream on the specified nodes and connects them to the previous // stage via a hash router. func (p *PhysicalPlan) AddStageOnNodes( - nodes []roachpb.NodeID, + sqlInstanceIDs []base.SQLInstanceID, core execinfrapb.ProcessorCoreUnion, post execinfrapb.PostProcessSpec, hashCols []uint32, @@ -993,11 +993,11 @@ func (p *PhysicalPlan) AddStageOnNodes( routers []ProcessorIdx, ) { pIdxStart := len(p.Processors) - newStageID := p.NewStageOnNodes(nodes) + newStageID := p.NewStageOnNodes(sqlInstanceIDs) - for _, n := range nodes { + for _, sqlInstanceID := range sqlInstanceIDs { proc := Processor{ - Node: n, + SQLInstanceID: sqlInstanceID, Spec: execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{ {ColumnTypes: inputTypes}, @@ -1012,7 +1012,7 @@ func (p *PhysicalPlan) AddStageOnNodes( p.AddProcessor(proc) } - if len(nodes) > 1 { + if len(sqlInstanceIDs) > 1 { // Set up the routers. for _, resultProc := range routers { p.Processors[resultProc].Spec.Output[0] = execinfrapb.OutputRouterSpec{ @@ -1023,14 +1023,14 @@ func (p *PhysicalPlan) AddStageOnNodes( } // Connect the result streams to the processors. - for bucket := 0; bucket < len(nodes); bucket++ { + for bucket := 0; bucket < len(sqlInstanceIDs); bucket++ { pIdx := ProcessorIdx(pIdxStart + bucket) p.MergeResultStreams(routers, bucket, mergeOrd, pIdx, 0, false /* forceSerialization */) } // Set the new result routers. p.ResultRouters = p.ResultRouters[:0] - for i := 0; i < len(nodes); i++ { + for i := 0; i < len(sqlInstanceIDs); i++ { p.ResultRouters = append(p.ResultRouters, ProcessorIdx(pIdxStart+i)) } } @@ -1041,7 +1041,7 @@ func (p *PhysicalPlan) AddStageOnNodes( // TODO(yuzefovich): If there's a strong key on the left or right side, we // can elide the distinct stage on that side. func (p *PhysicalPlan) AddDistinctSetOpStage( - nodes []roachpb.NodeID, + sqlInstanceIDs []base.SQLInstanceID, joinCore execinfrapb.ProcessorCoreUnion, distinctCores []execinfrapb.ProcessorCoreUnion, post execinfrapb.PostProcessSpec, @@ -1057,32 +1057,32 @@ func (p *PhysicalPlan) AddDistinctSetOpStage( // produce correct results (e.g., (VALUES (1),(1),(2)) EXCEPT (VALUES (1)) // would return (1),(2) instead of (2) if there was no distinct processor // before the EXCEPT ALL join). - distinctProcs := make(map[roachpb.NodeID][]ProcessorIdx) + distinctProcs := make(map[base.SQLInstanceID][]ProcessorIdx) p.AddStageOnNodes( - nodes, distinctCores[0], execinfrapb.PostProcessSpec{}, eqCols, + sqlInstanceIDs, distinctCores[0], execinfrapb.PostProcessSpec{}, eqCols, leftTypes, leftTypes, leftMergeOrd, leftRouters, ) for _, leftDistinctProcIdx := range p.ResultRouters { - node := p.Processors[leftDistinctProcIdx].Node + node := p.Processors[leftDistinctProcIdx].SQLInstanceID distinctProcs[node] = append(distinctProcs[node], leftDistinctProcIdx) } p.AddStageOnNodes( - nodes, distinctCores[1], execinfrapb.PostProcessSpec{}, eqCols, + sqlInstanceIDs, distinctCores[1], execinfrapb.PostProcessSpec{}, eqCols, rightTypes, rightTypes, rightMergeOrd, rightRouters, ) for _, rightDistinctProcIdx := range p.ResultRouters { - node := p.Processors[rightDistinctProcIdx].Node + node := p.Processors[rightDistinctProcIdx].SQLInstanceID distinctProcs[node] = append(distinctProcs[node], rightDistinctProcIdx) } // Create a join stage, where the distinct processors on the same node are // connected to a join processor. - joinStageID := p.NewStageOnNodes(nodes) + joinStageID := p.NewStageOnNodes(sqlInstanceIDs) p.ResultRouters = p.ResultRouters[:0] - for _, n := range nodes { + for _, n := range sqlInstanceIDs { proc := Processor{ - Node: n, + SQLInstanceID: n, Spec: execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{ {ColumnTypes: leftTypes}, @@ -1126,11 +1126,11 @@ func (p *PhysicalPlan) EnsureSingleStreamPerNode( var foundDuplicates bool for _, pIdx := range p.ResultRouters { proc := &p.Processors[pIdx] - if nodes.Contains(int(proc.Node)) { + if nodes.Contains(int(proc.SQLInstanceID)) { foundDuplicates = true break } - nodes.Add(int(proc.Node)) + nodes.Add(int(proc.SQLInstanceID)) } if !foundDuplicates { return @@ -1139,11 +1139,11 @@ func (p *PhysicalPlan) EnsureSingleStreamPerNode( for i := 0; i < len(p.ResultRouters); i++ { pIdx := p.ResultRouters[i] - node := p.Processors[p.ResultRouters[i]].Node + node := p.Processors[p.ResultRouters[i]].SQLInstanceID streams = append(streams[:0], pIdx) // Find all streams on the same node. for j := i + 1; j < len(p.ResultRouters); { - if p.Processors[p.ResultRouters[j]].Node == node { + if p.Processors[p.ResultRouters[j]].SQLInstanceID == node { streams = append(streams, p.ResultRouters[j]) // Remove the stream. copy(p.ResultRouters[j:], p.ResultRouters[j+1:]) @@ -1159,7 +1159,7 @@ func (p *PhysicalPlan) EnsureSingleStreamPerNode( // Merge the streams into a no-op processor. proc := Processor{ - Node: node, + SQLInstanceID: node, Spec: execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{{ // The other fields will be filled in by MergeResultStreams. @@ -1182,7 +1182,7 @@ func (p *PhysicalPlan) EnsureSingleStreamPerNode( // remote node, such stage is considered distributed. func (p *PhysicalPlan) GetLastStageDistribution() PlanDistribution { for i := range p.ResultRouters { - if p.Processors[p.ResultRouters[i]].Node != p.GatewayNodeID { + if p.Processors[p.ResultRouters[i]].SQLInstanceID != p.GatewaySQLInstanceID { return FullyDistributedPlan } } diff --git a/pkg/sql/physicalplan/specs.go b/pkg/sql/physicalplan/specs.go index dbc6f66305d0..d7e2700162da 100644 --- a/pkg/sql/physicalplan/specs.go +++ b/pkg/sql/physicalplan/specs.go @@ -13,7 +13,7 @@ package physicalplan import ( "sync" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" ) @@ -25,7 +25,7 @@ var flowSpecPool = sync.Pool{ // NewFlowSpec returns a new FlowSpec, which may have non-zero capacity in its // slice fields. -func NewFlowSpec(flowID execinfrapb.FlowID, gateway roachpb.NodeID) *execinfrapb.FlowSpec { +func NewFlowSpec(flowID execinfrapb.FlowID, gateway base.SQLInstanceID) *execinfrapb.FlowSpec { spec := flowSpecPool.Get().(*execinfrapb.FlowSpec) spec.FlowID = flowID spec.Gateway = gateway