|
| 1 | +// Copyright 2025 The Cockroach Authors. |
| 2 | +// |
| 3 | +// Use of this software is governed by the CockroachDB Software License |
| 4 | +// included in the /LICENSE file. |
| 5 | + |
| 6 | +package bulkmerge |
| 7 | + |
| 8 | +import ( |
| 9 | + "context" |
| 10 | + |
| 11 | + "github.com/cockroachdb/cockroach/pkg/base" |
| 12 | + "github.com/cockroachdb/cockroach/pkg/sql" |
| 13 | + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" |
| 14 | + "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" |
| 15 | + "github.com/cockroachdb/errors" |
| 16 | +) |
| 17 | + |
| 18 | +func newBulkMergePlan( |
| 19 | + ctx context.Context, execCtx sql.JobExecContext, |
| 20 | +) (*sql.PhysicalPlan, *sql.PlanningCtx, error) { |
| 21 | + // NOTE: This implementation is inspired by the physical plan created by |
| 22 | + // restore in `pkg/backup/restore_processor_planning.go` |
| 23 | + planCtx, sqlInstanceIDs, err := execCtx.DistSQLPlanner().SetupAllNodesPlanning( |
| 24 | + ctx, execCtx.ExtendedEvalContext(), execCtx.ExecCfg()) |
| 25 | + if err != nil { |
| 26 | + return nil, nil, err |
| 27 | + } |
| 28 | + |
| 29 | + plan := planCtx.NewPhysicalPlan() |
| 30 | + // Use the gateway node as the coordinator, which is where the job was initiated. |
| 31 | + coordinatorID := plan.GatewaySQLInstanceID |
| 32 | + |
| 33 | + router, err := physicalplan.MakeInstanceRouter(sqlInstanceIDs) |
| 34 | + if err != nil { |
| 35 | + return nil, nil, errors.Wrap(err, "unable to make instance router") |
| 36 | + } |
| 37 | + |
| 38 | + loopbackID := plan.AddProcessor(physicalplan.Processor{ |
| 39 | + SQLInstanceID: coordinatorID, |
| 40 | + Spec: execinfrapb.ProcessorSpec{ |
| 41 | + Core: execinfrapb.ProcessorCoreUnion{ |
| 42 | + MergeLoopback: &execinfrapb.MergeLoopbackSpec{}, |
| 43 | + }, |
| 44 | + Post: execinfrapb.PostProcessSpec{}, |
| 45 | + Output: []execinfrapb.OutputRouterSpec{{ |
| 46 | + Type: execinfrapb.OutputRouterSpec_BY_RANGE, |
| 47 | + RangeRouterSpec: router, |
| 48 | + }}, |
| 49 | + StageID: plan.NewStageOnNodes([]base.SQLInstanceID{coordinatorID}), |
| 50 | + ResultTypes: mergeLoopbackOutputTypes, |
| 51 | + }, |
| 52 | + }) |
| 53 | + |
| 54 | + mergeStage := plan.NewStageOnNodes(sqlInstanceIDs) |
| 55 | + for streamID, sqlInstanceID := range sqlInstanceIDs { |
| 56 | + pIdx := plan.AddProcessor(physicalplan.Processor{ |
| 57 | + SQLInstanceID: sqlInstanceID, |
| 58 | + Spec: execinfrapb.ProcessorSpec{ |
| 59 | + Input: []execinfrapb.InputSyncSpec{{ |
| 60 | + ColumnTypes: mergeLoopbackOutputTypes, |
| 61 | + }}, |
| 62 | + Core: execinfrapb.ProcessorCoreUnion{ |
| 63 | + BulkMerge: &execinfrapb.BulkMergeSpec{ |
| 64 | + // TODO(jeffswenson): fill in the rest of the spec |
| 65 | + }, |
| 66 | + }, |
| 67 | + Post: execinfrapb.PostProcessSpec{}, |
| 68 | + Output: []execinfrapb.OutputRouterSpec{{ |
| 69 | + Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, |
| 70 | + }}, |
| 71 | + StageID: mergeStage, |
| 72 | + ResultTypes: bulkMergeProcessorOutputTypes, |
| 73 | + }, |
| 74 | + }) |
| 75 | + plan.Streams = append(plan.Streams, physicalplan.Stream{ |
| 76 | + SourceProcessor: loopbackID, |
| 77 | + SourceRouterSlot: streamID, |
| 78 | + DestProcessor: pIdx, |
| 79 | + DestInput: 0, |
| 80 | + }) |
| 81 | + plan.ResultRouters = append(plan.ResultRouters, pIdx) |
| 82 | + } |
| 83 | + |
| 84 | + plan.AddSingleGroupStage(ctx, coordinatorID, execinfrapb.ProcessorCoreUnion{ |
| 85 | + MergeCoordinator: &execinfrapb.MergeCoordinatorSpec{ |
| 86 | + // TODO fill in the rest of the spec |
| 87 | + }, |
| 88 | + }, execinfrapb.PostProcessSpec{}, mergeCoordinatorOutputTypes, nil /* finalizeLastStageCb */) |
| 89 | + |
| 90 | + plan.PlanToStreamColMap = []int{0} // Needed for FinalizePlan to populate ResultTypes |
| 91 | + sql.FinalizePlan(ctx, planCtx, plan) |
| 92 | + |
| 93 | + return plan, planCtx, nil |
| 94 | +} |
0 commit comments