diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index fd793ab3c7dc..8fe8d34e4ca1 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -170,8 +170,8 @@ ALL_TESTS = [ "//pkg/sql/colexec/colexecwindow:colexecwindow_test", "//pkg/sql/colexec/execgen:execgen_test", "//pkg/sql/colexec:colexec_test", - "//pkg/sql/colexecbase:colexecbase_test", "//pkg/sql/colexecerror:colexecerror_test", + "//pkg/sql/colexecop:colexecop_test", "//pkg/sql/colflow/colrpc:colrpc_test", "//pkg/sql/colflow:colflow_test", "//pkg/sql/colmem:colmem_test", diff --git a/pkg/col/coldatatestutils/BUILD.bazel b/pkg/col/coldatatestutils/BUILD.bazel index 7d0d7ed07ac9..6cf3b35b9203 100644 --- a/pkg/col/coldatatestutils/BUILD.bazel +++ b/pkg/col/coldatatestutils/BUILD.bazel @@ -10,8 +10,8 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/col/coldata", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/rowenc", diff --git a/pkg/col/coldatatestutils/random_testutils.go b/pkg/col/coldatatestutils/random_testutils.go index b808c3913e44..706f65e609da 100644 --- a/pkg/col/coldatatestutils/random_testutils.go +++ b/pkg/col/coldatatestutils/random_testutils.go @@ -16,8 +16,8 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -321,7 +321,7 @@ type RandomDataOp struct { nulls bool } -var _ colexecbase.Operator = &RandomDataOp{} +var _ colexecop.Operator = &RandomDataOp{} // NewRandomDataOp creates a new RandomDataOp. func NewRandomDataOp( diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index a85809de6335..d4cae41c7c4f 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -283,8 +283,8 @@ go_library( "//pkg/sql/catalog/tabledesc", "//pkg/sql/catalog/typedesc", "//pkg/sql/colexec", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colflow", "//pkg/sql/contention", "//pkg/sql/covering", diff --git a/pkg/sql/colcontainer/BUILD.bazel b/pkg/sql/colcontainer/BUILD.bazel index 721efae6b20e..3307d4c754ef 100644 --- a/pkg/sql/colcontainer/BUILD.bazel +++ b/pkg/sql/colcontainer/BUILD.bazel @@ -36,7 +36,7 @@ go_test( "//pkg/col/coldataext", "//pkg/col/coldatatestutils", "//pkg/settings/cluster", - "//pkg/sql/colexecbase", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/pgwire/pgcode", diff --git a/pkg/sql/colcontainer/diskqueue_test.go b/pkg/sql/colcontainer/diskqueue_test.go index f77987cfe483..640ebc15d736 100644 --- a/pkg/sql/colcontainer/diskqueue_test.go +++ b/pkg/sql/colcontainer/diskqueue_test.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -231,7 +231,7 @@ func BenchmarkDiskQueue(b *testing.B) { rng, _ := randutil.NewPseudoRand() typs := []*types.T{types.Int} batch := coldatatestutils.RandomBatch(testAllocator, rng, typs, coldata.BatchSize(), 0, 0) - op := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + op := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) ctx := context.Background() for i := 0; i < b.N; i++ { op.ResetBatchesToReturn(numBatches) diff --git a/pkg/sql/colcontainer/partitionedqueue_test.go b/pkg/sql/colcontainer/partitionedqueue_test.go index 50b89648ee08..b0d81217cfe3 100644 --- a/pkg/sql/colcontainer/partitionedqueue_test.go +++ b/pkg/sql/colcontainer/partitionedqueue_test.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/storage/fs" "github.com/cockroachdb/cockroach/pkg/testutils/colcontainerutils" @@ -95,7 +95,7 @@ func TestPartitionedDiskQueue(t *testing.T) { ctx = context.Background() typs = []*types.T{types.Int} batch = testAllocator.NewMemBatchWithMaxCapacity(typs) - sem = &colexecbase.TestingSemaphore{} + sem = &colexecop.TestingSemaphore{} ) batch.SetLength(coldata.BatchSize()) @@ -170,7 +170,7 @@ func TestPartitionedDiskQueueSimulatedExternal(t *testing.T) { // maxPartitions+1 are created. The +1 is the file descriptor of the // new partition being written to when closedForWrites from maxPartitions // and writing the merged result to a single new partition. - sem := colexecbase.NewTestingSemaphore(maxPartitions + 1) + sem := colexecop.NewTestingSemaphore(maxPartitions + 1) p := colcontainer.NewPartitionedDiskQueue(typs, queueCfg, sem, colcontainer.PartitionerStrategyCloseOnNewPartition, testDiskAcc) // Define sortRepartition to be able to call this helper function @@ -250,7 +250,7 @@ func TestPartitionedDiskQueueSimulatedExternal(t *testing.T) { // The limit for a hash join is maxPartitions + 2. maxPartitions is the // number of partitions partitioned to and 2 represents the file descriptors // for the left and right side in the case of a repartition. - sem := colexecbase.NewTestingSemaphore(maxPartitions + 2) + sem := colexecop.NewTestingSemaphore(maxPartitions + 2) p := colcontainer.NewPartitionedDiskQueue(typs, queueCfg, sem, colcontainer.PartitionerStrategyDefault, testDiskAcc) // joinRepartition will perform the partitioning that happens during a hash diff --git a/pkg/sql/colexec/BUILD.bazel b/pkg/sql/colexec/BUILD.bazel index 4b097c11a478..412a2220c678 100644 --- a/pkg/sql/colexec/BUILD.bazel +++ b/pkg/sql/colexec/BUILD.bazel @@ -1,5 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//pkg/sql/colexecbase:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") load(":COLEXEC.bzl", "gen_sort_partitioner_rule") go_library( @@ -56,8 +56,8 @@ go_library( "//pkg/sql/colexec/colexecmisc", "//pkg/sql/colexec/colexecutils", "//pkg/sql/colexec/execgen", # keep - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", @@ -138,8 +138,8 @@ go_test( "//pkg/sql/colexec/colexecmisc", "//pkg/sql/colexec/colexectestutils", "//pkg/sql/colexec/colexecutils", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", diff --git a/pkg/sql/colexec/aggregators_test.go b/pkg/sql/colexec/aggregators_test.go index 0485195c9e87..70e1bd196bb4 100644 --- a/pkg/sql/colexec/aggregators_test.go +++ b/pkg/sql/colexec/aggregators_test.go @@ -22,8 +22,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -72,7 +72,7 @@ type aggregatorTestCase struct { // aggType is a helper struct that allows tests to test both the ordered and // hash aggregators at the same time. type aggType struct { - new func(*colexecagg.NewAggregatorArgs) (colexecbase.ResettableOperator, error) + new func(*colexecagg.NewAggregatorArgs) (colexecop.ResettableOperator, error) name string } @@ -80,7 +80,7 @@ var aggTypes = []aggType{ { // This is a wrapper around NewHashAggregator so its signature is // compatible with NewOrderedAggregator. - new: func(args *colexecagg.NewAggregatorArgs) (colexecbase.ResettableOperator, error) { + new: func(args *colexecagg.NewAggregatorArgs) (colexecop.ResettableOperator, error) { return NewHashAggregator(args, nil /* newSpillingQueueArgs */) }, name: "hash", @@ -726,7 +726,7 @@ func TestAggregators(t *testing.T) { verifier = colexectestutils.UnorderedVerifier } colexectestutils.RunTestsWithTyps(t, testAllocator, []colexectestutils.Tuples{tc.input}, [][]*types.T{tc.typs}, tc.expected, verifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return agg.new(&colexecagg.NewAggregatorArgs{ Allocator: testAllocator, MemAccount: testMemAcc, @@ -1022,7 +1022,7 @@ func benchmarkAggregateFunction( // Exhaust aggregator until all batches have been read. for b := a.Next(ctx); b.Length() != 0; b = a.Next(ctx) { } - if err = a.(colexecbase.Closer).Close(ctx); err != nil { + if err = a.(colexecop.Closer).Close(ctx); err != nil { b.Fatal(err) } source.Reset(ctx) diff --git a/pkg/sql/colexec/aggregators_util.go b/pkg/sql/colexec/aggregators_util.go index 4b92574b2591..663e399652b5 100644 --- a/pkg/sql/colexec/aggregators_util.go +++ b/pkg/sql/colexec/aggregators_util.go @@ -18,8 +18,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -156,7 +156,7 @@ func (b *aggregatorHelperBase) restoreState() ([]coldata.Vec, int, []int) { // handling of a FILTER clause of a single aggregate function for the hash // aggregation. type filteringSingleFunctionHashHelper struct { - filter colexecbase.Operator + filter colexecop.Operator filterInput *singleBatchOperator } @@ -492,19 +492,19 @@ func (h *distinctOrderedAggregatorHelper) performAggregation( } } -// singleBatchOperator is a helper colexecbase.Operator that returns the +// singleBatchOperator is a helper colexecop.Operator that returns the // provided vectors as a batch on the first call to Next() and zero batch on // all consequent calls (until it is reset). It must be reset before it can be // used for the first time. type singleBatchOperator struct { - colexecbase.ZeroInputNode - colexecbase.NonExplainable + colexecop.ZeroInputNode + colexecop.NonExplainable nexted bool batch coldata.Batch } -var _ colexecbase.Operator = &singleBatchOperator{} +var _ colexecop.Operator = &singleBatchOperator{} func newSingleBatchOperator( allocator *colmem.Allocator, typs []*types.T, maxBatchSize int, diff --git a/pkg/sql/colexec/and_or_projection.eg.go b/pkg/sql/colexec/and_or_projection.eg.go index 666c66a483ba..41b3b2d38f1d 100644 --- a/pkg/sql/colexec/and_or_projection.eg.go +++ b/pkg/sql/colexec/and_or_projection.eg.go @@ -13,8 +13,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -23,11 +23,11 @@ import ( func NewAndProjOp( allocator *colmem.Allocator, - input, leftProjOpChain, rightProjOpChain colexecbase.Operator, - leftFeedOp, rightFeedOp *colexecbase.FeedOperator, + input, leftProjOpChain, rightProjOpChain colexecop.Operator, + leftFeedOp, rightFeedOp *colexecop.FeedOperator, leftInputType, rightInputType *types.T, leftIdx, rightIdx, outputIdx int, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { leftFamily := leftInputType.Family() leftIsBool := leftFamily == types.BoolFamily leftIsNull := leftFamily == types.UnknownFamily @@ -65,11 +65,11 @@ func NewAndProjOp( func NewOrProjOp( allocator *colmem.Allocator, - input, leftProjOpChain, rightProjOpChain colexecbase.Operator, - leftFeedOp, rightFeedOp *colexecbase.FeedOperator, + input, leftProjOpChain, rightProjOpChain colexecop.Operator, + leftFeedOp, rightFeedOp *colexecop.FeedOperator, leftInputType, rightInputType *types.T, leftIdx, rightIdx, outputIdx int, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { leftFamily := leftInputType.Family() leftIsBool := leftFamily == types.BoolFamily leftIsNull := leftFamily == types.UnknownFamily @@ -107,12 +107,12 @@ func NewOrProjOp( type andProjOp struct { allocator *colmem.Allocator - input colexecbase.Operator + input colexecop.Operator - leftProjOpChain colexecbase.Operator - rightProjOpChain colexecbase.Operator - leftFeedOp *colexecbase.FeedOperator - rightFeedOp *colexecbase.FeedOperator + leftProjOpChain colexecop.Operator + rightProjOpChain colexecop.Operator + leftFeedOp *colexecop.FeedOperator + rightFeedOp *colexecop.FeedOperator leftIdx int rightIdx int @@ -129,10 +129,10 @@ type andProjOp struct { // outputIdx. func newAndProjOp( allocator *colmem.Allocator, - input, leftProjOpChain, rightProjOpChain colexecbase.Operator, - leftFeedOp, rightFeedOp *colexecbase.FeedOperator, + input, leftProjOpChain, rightProjOpChain colexecop.Operator, + leftFeedOp, rightFeedOp *colexecop.FeedOperator, leftIdx, rightIdx, outputIdx int, -) colexecbase.Operator { +) colexecop.Operator { return &andProjOp{ allocator: allocator, input: input, @@ -545,12 +545,12 @@ func (o *andProjOp) Next(ctx context.Context) coldata.Batch { type andRightNullProjOp struct { allocator *colmem.Allocator - input colexecbase.Operator + input colexecop.Operator - leftProjOpChain colexecbase.Operator - rightProjOpChain colexecbase.Operator - leftFeedOp *colexecbase.FeedOperator - rightFeedOp *colexecbase.FeedOperator + leftProjOpChain colexecop.Operator + rightProjOpChain colexecop.Operator + leftFeedOp *colexecop.FeedOperator + rightFeedOp *colexecop.FeedOperator leftIdx int rightIdx int @@ -567,10 +567,10 @@ type andRightNullProjOp struct { // outputIdx. func newAndRightNullProjOp( allocator *colmem.Allocator, - input, leftProjOpChain, rightProjOpChain colexecbase.Operator, - leftFeedOp, rightFeedOp *colexecbase.FeedOperator, + input, leftProjOpChain, rightProjOpChain colexecop.Operator, + leftFeedOp, rightFeedOp *colexecop.FeedOperator, leftIdx, rightIdx, outputIdx int, -) colexecbase.Operator { +) colexecop.Operator { return &andRightNullProjOp{ allocator: allocator, input: input, @@ -950,12 +950,12 @@ func (o *andRightNullProjOp) Next(ctx context.Context) coldata.Batch { type andLeftNullProjOp struct { allocator *colmem.Allocator - input colexecbase.Operator + input colexecop.Operator - leftProjOpChain colexecbase.Operator - rightProjOpChain colexecbase.Operator - leftFeedOp *colexecbase.FeedOperator - rightFeedOp *colexecbase.FeedOperator + leftProjOpChain colexecop.Operator + rightProjOpChain colexecop.Operator + leftFeedOp *colexecop.FeedOperator + rightFeedOp *colexecop.FeedOperator leftIdx int rightIdx int @@ -972,10 +972,10 @@ type andLeftNullProjOp struct { // outputIdx. func newAndLeftNullProjOp( allocator *colmem.Allocator, - input, leftProjOpChain, rightProjOpChain colexecbase.Operator, - leftFeedOp, rightFeedOp *colexecbase.FeedOperator, + input, leftProjOpChain, rightProjOpChain colexecop.Operator, + leftFeedOp, rightFeedOp *colexecop.FeedOperator, leftIdx, rightIdx, outputIdx int, -) colexecbase.Operator { +) colexecop.Operator { return &andLeftNullProjOp{ allocator: allocator, input: input, @@ -1336,12 +1336,12 @@ func (o *andLeftNullProjOp) Next(ctx context.Context) coldata.Batch { type orProjOp struct { allocator *colmem.Allocator - input colexecbase.Operator + input colexecop.Operator - leftProjOpChain colexecbase.Operator - rightProjOpChain colexecbase.Operator - leftFeedOp *colexecbase.FeedOperator - rightFeedOp *colexecbase.FeedOperator + leftProjOpChain colexecop.Operator + rightProjOpChain colexecop.Operator + leftFeedOp *colexecop.FeedOperator + rightFeedOp *colexecop.FeedOperator leftIdx int rightIdx int @@ -1358,10 +1358,10 @@ type orProjOp struct { // outputIdx. func newOrProjOp( allocator *colmem.Allocator, - input, leftProjOpChain, rightProjOpChain colexecbase.Operator, - leftFeedOp, rightFeedOp *colexecbase.FeedOperator, + input, leftProjOpChain, rightProjOpChain colexecop.Operator, + leftFeedOp, rightFeedOp *colexecop.FeedOperator, leftIdx, rightIdx, outputIdx int, -) colexecbase.Operator { +) colexecop.Operator { return &orProjOp{ allocator: allocator, input: input, @@ -1775,12 +1775,12 @@ func (o *orProjOp) Next(ctx context.Context) coldata.Batch { type orRightNullProjOp struct { allocator *colmem.Allocator - input colexecbase.Operator + input colexecop.Operator - leftProjOpChain colexecbase.Operator - rightProjOpChain colexecbase.Operator - leftFeedOp *colexecbase.FeedOperator - rightFeedOp *colexecbase.FeedOperator + leftProjOpChain colexecop.Operator + rightProjOpChain colexecop.Operator + leftFeedOp *colexecop.FeedOperator + rightFeedOp *colexecop.FeedOperator leftIdx int rightIdx int @@ -1797,10 +1797,10 @@ type orRightNullProjOp struct { // outputIdx. func newOrRightNullProjOp( allocator *colmem.Allocator, - input, leftProjOpChain, rightProjOpChain colexecbase.Operator, - leftFeedOp, rightFeedOp *colexecbase.FeedOperator, + input, leftProjOpChain, rightProjOpChain colexecop.Operator, + leftFeedOp, rightFeedOp *colexecop.FeedOperator, leftIdx, rightIdx, outputIdx int, -) colexecbase.Operator { +) colexecop.Operator { return &orRightNullProjOp{ allocator: allocator, input: input, @@ -2181,12 +2181,12 @@ func (o *orRightNullProjOp) Next(ctx context.Context) coldata.Batch { type orLeftNullProjOp struct { allocator *colmem.Allocator - input colexecbase.Operator + input colexecop.Operator - leftProjOpChain colexecbase.Operator - rightProjOpChain colexecbase.Operator - leftFeedOp *colexecbase.FeedOperator - rightFeedOp *colexecbase.FeedOperator + leftProjOpChain colexecop.Operator + rightProjOpChain colexecop.Operator + leftFeedOp *colexecop.FeedOperator + rightFeedOp *colexecop.FeedOperator leftIdx int rightIdx int @@ -2203,10 +2203,10 @@ type orLeftNullProjOp struct { // outputIdx. func newOrLeftNullProjOp( allocator *colmem.Allocator, - input, leftProjOpChain, rightProjOpChain colexecbase.Operator, - leftFeedOp, rightFeedOp *colexecbase.FeedOperator, + input, leftProjOpChain, rightProjOpChain colexecop.Operator, + leftFeedOp, rightFeedOp *colexecop.FeedOperator, leftIdx, rightIdx, outputIdx int, -) colexecbase.Operator { +) colexecop.Operator { return &orLeftNullProjOp{ allocator: allocator, input: input, diff --git a/pkg/sql/colexec/and_or_projection_test.go b/pkg/sql/colexec/and_or_projection_test.go index 175599a82a2c..475b965b10b2 100644 --- a/pkg/sql/colexec/and_or_projection_test.go +++ b/pkg/sql/colexec/and_or_projection_test.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecmisc" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -202,7 +202,7 @@ func TestAndOrOps(t *testing.T) { [][]*types.T{{types.Bool, types.Bool}}, tc.expected, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { projOp, err := colexectestutils.CreateTestProjectingOperator( ctx, flowCtx, input[0], []*types.T{types.Bool, types.Bool}, fmt.Sprintf("@1 %s @2", test.operation), false /* canFallbackToRowexec */, testMemAcc, @@ -262,7 +262,7 @@ func benchmarkLogicalProjOp( } } typs := []*types.T{types.Bool, types.Bool} - input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + input := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) logicalProjOp, err := colexectestutils.CreateTestProjectingOperator( ctx, flowCtx, input, typs, fmt.Sprintf("@1 %s @2", operation), false /* canFallbackToRowexec */, testMemAcc, diff --git a/pkg/sql/colexec/and_or_projection_tmpl.go b/pkg/sql/colexec/and_or_projection_tmpl.go index fdcc9e0c6ac4..b8cf260b7e9f 100644 --- a/pkg/sql/colexec/and_or_projection_tmpl.go +++ b/pkg/sql/colexec/and_or_projection_tmpl.go @@ -23,8 +23,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -36,11 +36,11 @@ import ( func New_OPERATIONProjOp( allocator *colmem.Allocator, - input, leftProjOpChain, rightProjOpChain colexecbase.Operator, - leftFeedOp, rightFeedOp *colexecbase.FeedOperator, + input, leftProjOpChain, rightProjOpChain colexecop.Operator, + leftFeedOp, rightFeedOp *colexecop.FeedOperator, leftInputType, rightInputType *types.T, leftIdx, rightIdx, outputIdx int, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { leftFamily := leftInputType.Family() leftIsBool := leftFamily == types.BoolFamily leftIsNull := leftFamily == types.UnknownFamily @@ -82,12 +82,12 @@ func New_OPERATIONProjOp( type _OP_LOWERProjOp struct { allocator *colmem.Allocator - input colexecbase.Operator + input colexecop.Operator - leftProjOpChain colexecbase.Operator - rightProjOpChain colexecbase.Operator - leftFeedOp *colexecbase.FeedOperator - rightFeedOp *colexecbase.FeedOperator + leftProjOpChain colexecop.Operator + rightProjOpChain colexecop.Operator + leftFeedOp *colexecop.FeedOperator + rightFeedOp *colexecop.FeedOperator leftIdx int rightIdx int @@ -104,10 +104,10 @@ type _OP_LOWERProjOp struct { // outputIdx. func new_OP_TITLEProjOp( allocator *colmem.Allocator, - input, leftProjOpChain, rightProjOpChain colexecbase.Operator, - leftFeedOp, rightFeedOp *colexecbase.FeedOperator, + input, leftProjOpChain, rightProjOpChain colexecop.Operator, + leftFeedOp, rightFeedOp *colexecop.FeedOperator, leftIdx, rightIdx, outputIdx int, -) colexecbase.Operator { +) colexecop.Operator { return &_OP_LOWERProjOp{ allocator: allocator, input: input, diff --git a/pkg/sql/colexec/buffer.go b/pkg/sql/colexec/buffer.go index 1599c49b8575..e9a7aaa9ebde 100644 --- a/pkg/sql/colexec/buffer.go +++ b/pkg/sql/colexec/buffer.go @@ -14,27 +14,27 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" ) // bufferOp is an operator that buffers a single batch at a time from an input, // and makes it available to be read multiple times by downstream consumers. type bufferOp struct { - colexecbase.OneInputNode - initStatus colexecbase.OperatorInitStatus + colexecop.OneInputNode + initStatus colexecop.OperatorInitStatus // read is true if someone has read the current batch already. read bool batch coldata.Batch } -var _ colexecbase.Operator = &bufferOp{} +var _ colexecop.Operator = &bufferOp{} // NewBufferOp returns a new bufferOp, initialized to buffer batches from the // supplied input. -func NewBufferOp(input colexecbase.Operator) colexecbase.Operator { +func NewBufferOp(input colexecop.Operator) colexecop.Operator { return &bufferOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), } } @@ -43,9 +43,9 @@ func (b *bufferOp) Init() { // called multiple times. However, we do not want to call Init many times on // the input to bufferOp, so we do this check whether Init has already been // performed. - if b.initStatus == colexecbase.OperatorNotInitialized { + if b.initStatus == colexecop.OperatorNotInitialized { b.Input.Init() - b.initStatus = colexecbase.OperatorInitialized + b.initStatus = colexecop.OperatorInitialized } } diff --git a/pkg/sql/colexec/builtin_funcs.go b/pkg/sql/colexec/builtin_funcs.go index 2bf8aa9578cb..0dbcc1867742 100644 --- a/pkg/sql/colexec/builtin_funcs.go +++ b/pkg/sql/colexec/builtin_funcs.go @@ -16,15 +16,15 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" ) type defaultBuiltinFuncOperator struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator evalCtx *tree.EvalContext funcExpr *tree.FuncExpr @@ -38,7 +38,7 @@ type defaultBuiltinFuncOperator struct { row tree.Datums } -var _ colexecbase.Operator = &defaultBuiltinFuncOperator{} +var _ colexecop.Operator = &defaultBuiltinFuncOperator{} func (b *defaultBuiltinFuncOperator) Init() { b.Input.Init() @@ -115,8 +115,8 @@ func NewBuiltinFunctionOperator( columnTypes []*types.T, argumentCols []int, outputIdx int, - input colexecbase.Operator, -) (colexecbase.Operator, error) { + input colexecop.Operator, +) (colexecop.Operator, error) { switch funcExpr.ResolvedOverload().SpecializedVecBuiltin { case tree.SubstringStringIntInt: input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.String, outputIdx) @@ -127,7 +127,7 @@ func NewBuiltinFunctionOperator( outputType := funcExpr.ResolvedType() input = colexecutils.NewVectorTypeEnforcer(allocator, input, outputType, outputIdx) return &defaultBuiltinFuncOperator{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, evalCtx: evalCtx, funcExpr: funcExpr, diff --git a/pkg/sql/colexec/builtin_funcs_test.go b/pkg/sql/colexec/builtin_funcs_test.go index c5cb99f91341..7d60edb92ad9 100644 --- a/pkg/sql/colexec/builtin_funcs_test.go +++ b/pkg/sql/colexec/builtin_funcs_test.go @@ -21,7 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" @@ -77,7 +77,7 @@ func TestBasicBuiltinFunctions(t *testing.T) { for _, tc := range testCases { log.Infof(ctx, "%s", tc.desc) colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.inputTuples}, tc.outputTuples, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return colexectestutils.CreateTestProjectingOperator( ctx, flowCtx, input[0], tc.inputTypes, tc.expr, false /* canFallbackToRowexec */, testMemAcc, @@ -128,7 +128,7 @@ func benchmarkBuiltinFunctions(b *testing.B, useSelectionVector bool, hasNulls b } typs := []*types.T{types.Int} - source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) op, err := colexectestutils.CreateTestProjectingOperator( ctx, flowCtx, source, typs, "abs(@1)" /* projectingExpr */, false /* canFallbackToRowexec */, testMemAcc, @@ -173,8 +173,8 @@ func BenchmarkCompareSpecializedOperators(b *testing.B) { eCol[i] = 4 } batch.SetLength(coldata.BatchSize()) - var source colexecbase.Operator - source = colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + var source colexecop.Operator + source = colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) source = colexecutils.NewVectorTypeEnforcer(testAllocator, source, types.Bytes, outputIdx) // Set up the default operator. @@ -191,7 +191,7 @@ func BenchmarkCompareSpecializedOperators(b *testing.B) { b.Fatal(err) } defaultOp := &defaultBuiltinFuncOperator{ - OneInputNode: colexecbase.NewOneInputNode(source), + OneInputNode: colexecop.NewOneInputNode(source), allocator: testAllocator, evalCtx: evalCtx, funcExpr: typedExpr.(*tree.FuncExpr), diff --git a/pkg/sql/colexec/case.go b/pkg/sql/colexec/case.go index 1f579546c65b..270a0ed058f2 100644 --- a/pkg/sql/colexec/case.go +++ b/pkg/sql/colexec/case.go @@ -14,8 +14,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -26,8 +26,8 @@ type caseOp struct { allocator *colmem.Allocator buffer *bufferOp - caseOps []colexecbase.Operator - elseOp colexecbase.Operator + caseOps []colexecop.Operator + elseOp colexecop.Operator thenIdxs []int outputIdx int @@ -46,7 +46,7 @@ type caseOp struct { prevSel []int } -var _ colexecbase.Operator = &caseOp{} +var _ colexecop.Operator = &caseOp{} func (c *caseOp) ChildCount(verbose bool) int { return 1 + len(c.caseOps) + 1 @@ -77,13 +77,13 @@ func (c *caseOp) Child(nth int, verbose bool) execinfra.OpNode { // typ is the type of the CASE expression. func NewCaseOp( allocator *colmem.Allocator, - buffer colexecbase.Operator, - caseOps []colexecbase.Operator, - elseOp colexecbase.Operator, + buffer colexecop.Operator, + caseOps []colexecop.Operator, + elseOp colexecop.Operator, thenIdxs []int, outputIdx int, typ *types.T, -) colexecbase.Operator { +) colexecop.Operator { // We internally use two selection vectors, origSel and prevSel. allocator.AdjustMemoryUsage(int64(2 * colmem.SizeOfBatchSizeSelVector)) return &caseOp{ diff --git a/pkg/sql/colexec/case_test.go b/pkg/sql/colexec/case_test.go index bd434df4e239..551dd40ebc48 100644 --- a/pkg/sql/colexec/case_test.go +++ b/pkg/sql/colexec/case_test.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecmisc" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -67,7 +67,7 @@ func TestCaseOp(t *testing.T) { inputTypes: []*types.T{types.Int, types.Int}, }, } { - colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.expected, colexectestutils.OrderedVerifier, func(inputs []colexecbase.Operator) (colexecbase.Operator, error) { + colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.expected, colexectestutils.OrderedVerifier, func(inputs []colexecop.Operator) (colexecop.Operator, error) { caseOp, err := colexectestutils.CreateTestProjectingOperator( ctx, flowCtx, inputs[0], tc.inputTypes, tc.renderExpr, false /* canFallbackToRowexec */, testMemAcc, diff --git a/pkg/sql/colexec/colbuilder/BUILD.bazel b/pkg/sql/colexec/colbuilder/BUILD.bazel index ed27dd7c5434..df6f9b7da590 100644 --- a/pkg/sql/colexec/colbuilder/BUILD.bazel +++ b/pkg/sql/colexec/colbuilder/BUILD.bazel @@ -21,8 +21,8 @@ go_library( "//pkg/sql/colexec/colexecsel", "//pkg/sql/colexec/colexecutils", "//pkg/sql/colexec/colexecwindow", # keep - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colfetcher", "//pkg/sql/colmem", "//pkg/sql/execinfra", @@ -57,7 +57,7 @@ go_test( "//pkg/sql/catalog/catalogkv", "//pkg/sql/colexec", "//pkg/sql/colexec/colexecargs", - "//pkg/sql/colexecbase", + "//pkg/sql/colexecop", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", "//pkg/sql/rowenc", diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 1f4cc4aaff32..d05c49d922d6 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -31,8 +31,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecsel" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecwindow" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colfetcher" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" @@ -47,7 +47,7 @@ import ( "github.com/cockroachdb/errors" ) -func checkNumIn(inputs []colexecbase.Operator, numIn int) error { +func checkNumIn(inputs []colexecop.Operator, numIn int) error { if len(inputs) != numIn { return errors.Errorf("expected %d input(s), got %d", numIn, len(inputs)) } @@ -59,7 +59,7 @@ func checkNumIn(inputs []colexecbase.Operator, numIn int) error { func wrapRowSources( ctx context.Context, flowCtx *execinfra.FlowCtx, - inputs []colexecbase.Operator, + inputs []colexecop.Operator, inputTypes [][]*types.T, acc *mon.BoundAccount, processorID int32, @@ -360,7 +360,7 @@ func (r opResult) createDiskBackedSort( ctx context.Context, flowCtx *execinfra.FlowCtx, args *colexecargs.NewColOperatorArgs, - input colexecbase.Operator, + input colexecop.Operator, inputTypes []*types.T, ordering execinfrapb.Ordering, matchLen uint32, @@ -369,12 +369,12 @@ func (r opResult) createDiskBackedSort( post *execinfrapb.PostProcessSpec, memMonitorNamePrefix string, factory coldata.ColumnFactory, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { streamingMemAccount := args.StreamingMemAccount useStreamingMemAccountForBuffering := args.TestingKnobs.UseStreamingMemAccountForBuffering var ( sorterMemMonitorName string - inMemorySorter colexecbase.Operator + inMemorySorter colexecop.Operator err error ) if len(ordering.Columns) == int(matchLen) { @@ -447,9 +447,9 @@ func (r opResult) createDiskBackedSort( // we don't take advantage of the limits and of partial ordering). We // could improve this. return colexec.NewOneInputDiskSpiller( - input, inMemorySorter.(colexecbase.BufferingInMemoryOperator), + input, inMemorySorter.(colexecop.BufferingInMemoryOperator), sorterMemMonitorName, - func(input colexecbase.Operator) colexecbase.Operator { + func(input colexecop.Operator) colexecop.Operator { monitorNamePrefix := fmt.Sprintf("%sexternal-sorter", memMonitorNamePrefix) // We are using unlimited memory monitors here because external // sort itself is responsible for making sure that we stay within @@ -480,7 +480,7 @@ func (r opResult) createDiskBackedSort( args.FDSemaphore, diskAccount, ) - r.ToClose = append(r.ToClose, es.(colexecbase.Closer)) + r.ToClose = append(r.ToClose, es.(colexecop.Closer)) return es }, args.TestingKnobs.SpillingCallbackFn, @@ -498,11 +498,11 @@ func (r opResult) makeDiskBackedSorterConstructor( monitorNamePrefix string, factory coldata.ColumnFactory, ) colexec.DiskBackedSorterConstructor { - return func(input colexecbase.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column, maxNumberPartitions int) colexecbase.Operator { - if maxNumberPartitions < colexecbase.ExternalSorterMinPartitions { + return func(input colexecop.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column, maxNumberPartitions int) colexecop.Operator { + if maxNumberPartitions < colexecop.ExternalSorterMinPartitions { colexecerror.InternalError(errors.AssertionFailedf( "external sorter is attempted to be created with %d partitions, minimum %d required", - maxNumberPartitions, colexecbase.ExternalSorterMinPartitions, + maxNumberPartitions, colexecop.ExternalSorterMinPartitions, )) } sortArgs := *args @@ -540,7 +540,7 @@ func (r opResult) createAndWrapRowSource( ctx context.Context, flowCtx *execinfra.FlowCtx, args *colexecargs.NewColOperatorArgs, - inputs []colexecbase.Operator, + inputs []colexecop.Operator, inputTypes [][]*types.T, spec *execinfrapb.ProcessorSpec, factory coldata.ColumnFactory, @@ -698,7 +698,7 @@ func NewColOperator( if err := checkNumIn(inputs, 1); err != nil { return r, err } - result.Op = colexecbase.NewNoop(inputs[0]) + result.Op = colexecop.NewNoop(inputs[0]) result.ColumnTypes = make([]*types.T, len(spec.Input[0].ColumnTypes)) copy(result.ColumnTypes, spec.Input[0].ColumnTypes) @@ -848,7 +848,7 @@ func NewColOperator( spillingQueueCfg.SetDefaultBufferSizeBytesForCacheMode() newAggArgs.Allocator = colmem.NewAllocator(ctx, hashAggregatorMemAccount, factory) newAggArgs.MemAccount = hashAggregatorMemAccount - var inMemoryHashAggregator colexecbase.Operator + var inMemoryHashAggregator colexecop.Operator inMemoryHashAggregator, err = colexec.NewHashAggregator( newAggArgs, &colexecutils.NewSpillingQueueArgs{ @@ -873,9 +873,9 @@ func NewColOperator( // error even when used by the external hash aggregator). evalCtx.SingleDatumAggMemAccount = ehaMemAccount result.Op = colexec.NewOneInputDiskSpiller( - inputs[0], inMemoryHashAggregator.(colexecbase.BufferingInMemoryOperator), + inputs[0], inMemoryHashAggregator.(colexecop.BufferingInMemoryOperator), hashAggregatorMemMonitorName, - func(input colexecbase.Operator) colexecbase.Operator { + func(input colexecop.Operator) colexecop.Operator { newAggArgs := *newAggArgs // Note that the hash-based partitioner will make // sure that partitions to process using the @@ -901,7 +901,7 @@ func NewColOperator( newAggArgs.MemAccount = streamingMemAccount result.Op, err = colexec.NewOrderedAggregator(newAggArgs) } - result.ToClose = append(result.ToClose, result.Op.(colexecbase.Closer)) + result.ToClose = append(result.ToClose, result.Op.(colexecop.Closer)) case core.Distinct != nil: if err := checkNumIn(inputs, 1); err != nil { @@ -930,9 +930,9 @@ func NewColOperator( ) diskAccount := result.createDiskAccount(ctx, flowCtx, distinctMemMonitorName) result.Op = colexec.NewOneInputDiskSpiller( - inputs[0], inMemoryUnorderedDistinct.(colexecbase.BufferingInMemoryOperator), + inputs[0], inMemoryUnorderedDistinct.(colexecop.BufferingInMemoryOperator), distinctMemMonitorName, - func(input colexecbase.Operator) colexecbase.Operator { + func(input colexecop.Operator) colexecop.Operator { monitorNamePrefix := fmt.Sprintf("external-distinct-%d", spec.ProcessorID) unlimitedAllocator := colmem.NewAllocator( ctx, result.createBufferingUnlimitedMemAccount(ctx, flowCtx, monitorNamePrefix), factory, @@ -950,7 +950,7 @@ func NewColOperator( }, args.TestingKnobs.SpillingCallbackFn, ) - result.ToClose = append(result.ToClose, result.Op.(colexecbase.Closer)) + result.ToClose = append(result.ToClose, result.Op.(colexecop.Closer)) } case core.Ordinality != nil: @@ -988,7 +988,7 @@ func NewColOperator( leftTypes, rightTypes, crossJoinerDiskAcc, ) - result.ToClose = append(result.ToClose, result.Op.(colexecbase.Closer)) + result.ToClose = append(result.ToClose, result.Op.(colexecop.Closer)) } else { hashJoinerMemMonitorName := fmt.Sprintf("hash-joiner-%d", spec.ProcessorID) var hashJoinerMemAccount *mon.BoundAccount @@ -1026,9 +1026,9 @@ func NewColOperator( } else { diskAccount := result.createDiskAccount(ctx, flowCtx, hashJoinerMemMonitorName) result.Op = colexec.NewTwoInputDiskSpiller( - inputs[0], inputs[1], inMemoryHashJoiner.(colexecbase.BufferingInMemoryOperator), + inputs[0], inputs[1], inMemoryHashJoiner.(colexecop.BufferingInMemoryOperator), hashJoinerMemMonitorName, - func(inputOne, inputTwo colexecbase.Operator) colexecbase.Operator { + func(inputOne, inputTwo colexecop.Operator) colexecop.Operator { monitorNamePrefix := fmt.Sprintf("external-hash-joiner-%d", spec.ProcessorID) unlimitedAllocator := colmem.NewAllocator( ctx, result.createBufferingUnlimitedMemAccount(ctx, flowCtx, monitorNamePrefix), factory, @@ -1042,7 +1042,7 @@ func NewColOperator( result.makeDiskBackedSorterConstructor(ctx, flowCtx, args, monitorNamePrefix, factory), diskAccount, ) - result.ToClose = append(result.ToClose, ehj.(colexecbase.Closer)) + result.ToClose = append(result.ToClose, ehj.(colexecop.Closer)) return ehj }, args.TestingKnobs.SpillingCallbackFn, @@ -1103,7 +1103,7 @@ func NewColOperator( } result.Op = mj - result.ToClose = append(result.ToClose, mj.(colexecbase.Closer)) + result.ToClose = append(result.ToClose, mj.(colexecop.Closer)) result.ColumnTypes = core.MergeJoiner.Type.MakeOutputTypes(leftTypes, rightTypes) if onExpr != nil { @@ -1153,7 +1153,7 @@ func NewColOperator( input, err = colexecwindow.NewWindowSortingPartitioner( streamingAllocator, input, typs, core.Windower.PartitionBy, wf.Ordering.Columns, int(wf.OutputColIdx), - func(input colexecbase.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column) (colexecbase.Operator, error) { + func(input colexecop.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column) (colexecop.Operator, error) { return result.createDiskBackedSort( ctx, flowCtx, args, input, inputTypes, execinfrapb.Ordering{Columns: orderingCols}, 0, /* matchLen */ @@ -1216,7 +1216,7 @@ func NewColOperator( // NewRelativeRankOperator sometimes returns a constOp when // there are no ordering columns, so we check that the // returned operator is a Closer. - if c, ok := result.Op.(colexecbase.Closer); ok { + if c, ok := result.Op.(colexecop.Closer); ok { result.ToClose = append(result.ToClose, c) } default: @@ -1371,7 +1371,7 @@ func (r opResult) planAndMaybeWrapFilter( ResultTypes: args.Spec.ResultTypes, } return r.createAndWrapRowSource( - ctx, flowCtx, args, []colexecbase.Operator{r.Op}, [][]*types.T{r.ColumnTypes}, + ctx, flowCtx, args, []colexecop.Operator{r.Op}, [][]*types.T{r.ColumnTypes}, filtererSpec, factory, err, ) } @@ -1401,7 +1401,7 @@ func (r opResult) wrapPostProcessSpec( ResultTypes: resultTypes, } return r.createAndWrapRowSource( - ctx, flowCtx, args, []colexecbase.Operator{r.Op}, [][]*types.T{r.ColumnTypes}, + ctx, flowCtx, args, []colexecop.Operator{r.Op}, [][]*types.T{r.ColumnTypes}, noopSpec, factory, causeToWrap, ) } @@ -1533,7 +1533,7 @@ func (r opResult) createDiskAccount( } type postProcessResult struct { - Op colexecbase.Operator + Op colexecop.Operator ColumnTypes []*types.T } @@ -1548,13 +1548,13 @@ func planFilterExpr( ctx context.Context, flowCtx *execinfra.FlowCtx, evalCtx *tree.EvalContext, - input colexecbase.Operator, + input colexecop.Operator, columnTypes []*types.T, filter execinfrapb.Expression, acc *mon.BoundAccount, factory coldata.ColumnFactory, helper *colexecargs.ExprHelper, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { semaCtx := flowCtx.TypeResolverFactory.NewSemaContext(evalCtx.Txn) expr, err := helper.ProcessExpr(filter, semaCtx, evalCtx, columnTypes) if err != nil { @@ -1586,8 +1586,8 @@ func planFilterExpr( // addProjection adds a simple projection on top of op according to projection // and returns the updated operator and type schema. func addProjection( - op colexecbase.Operator, typs []*types.T, projection []uint32, -) (colexecbase.Operator, []*types.T) { + op colexecop.Operator, typs []*types.T, projection []uint32, +) (colexecop.Operator, []*types.T) { newTypes := make([]*types.T, len(projection)) for i, j := range projection { newTypes[i] = typs[j] @@ -1600,10 +1600,10 @@ func planSelectionOperators( evalCtx *tree.EvalContext, expr tree.TypedExpr, columnTypes []*types.T, - input colexecbase.Operator, + input colexecop.Operator, acc *mon.BoundAccount, factory coldata.ColumnFactory, -) (op colexecbase.Operator, resultIdx int, typs []*types.T, err error) { +) (op colexecop.Operator, resultIdx int, typs []*types.T, err error) { switch t := expr.(type) { case *tree.IndexedVar: op, err = colexecutils.BoolOrUnknownToSelOp(input, columnTypes, t.Idx) @@ -1613,7 +1613,7 @@ func planSelectionOperators( // vectors. First we select out the tuples that are true on the left // side, and then, only among the matched tuples, we select out the // tuples that are true on the right side. - var leftOp, rightOp colexecbase.Operator + var leftOp, rightOp colexecop.Operator leftOp, _, typs, err = planSelectionOperators( ctx, evalCtx, t.TypedLeft(), columnTypes, input, acc, factory, ) @@ -1756,12 +1756,12 @@ func planCastOperator( ctx context.Context, acc *mon.BoundAccount, columnTypes []*types.T, - input colexecbase.Operator, + input colexecop.Operator, inputIdx int, fromType *types.T, toType *types.T, factory coldata.ColumnFactory, -) (op colexecbase.Operator, resultIdx int, typs []*types.T, err error) { +) (op colexecop.Operator, resultIdx int, typs []*types.T, err error) { outputIdx := len(columnTypes) op, err = colexecmisc.GetCastOperator(colmem.NewAllocator(ctx, acc, factory), input, inputIdx, outputIdx, fromType, toType) typs = appendOneType(columnTypes, toType) @@ -1777,13 +1777,13 @@ func planProjectionOperators( evalCtx *tree.EvalContext, expr tree.TypedExpr, columnTypes []*types.T, - input colexecbase.Operator, + input colexecop.Operator, acc *mon.BoundAccount, factory coldata.ColumnFactory, -) (op colexecbase.Operator, resultIdx int, typs []*types.T, err error) { +) (op colexecop.Operator, resultIdx int, typs []*types.T, err error) { // projectDatum is a helper function that adds a new constant projection // operator for the given datum. typs are updated accordingly. - projectDatum := func(datum tree.Datum) (colexecbase.Operator, error) { + projectDatum := func(datum tree.Datum) (colexecop.Operator, error) { resultIdx = len(columnTypes) datumType := datum.ResolvedType() typs = appendOneType(columnTypes, datumType) @@ -1912,7 +1912,7 @@ func planProjectionOperators( allocator, input, nil /* typs */, caseOutputIdx, -1, /* subsetEndIdx */ ) buffer := colexec.NewBufferOp(schemaEnforcer) - caseOps := make([]colexecbase.Operator, len(t.Whens)) + caseOps := make([]colexecop.Operator, len(t.Whens)) typs = appendOneType(columnTypes, caseOutputType) thenIdxs := make([]int, len(t.Whens)+1) for i, when := range t.Whens { @@ -1962,7 +1962,7 @@ func planProjectionOperators( } } } - var elseOp colexecbase.Operator + var elseOp colexecop.Operator elseExpr := t.Else if elseExpr == nil { // If there's no ELSE arm, we write NULLs. @@ -2034,12 +2034,12 @@ func planProjectionExpr( outputType *types.T, left, right tree.TypedExpr, columnTypes []*types.T, - input colexecbase.Operator, + input colexecop.Operator, acc *mon.BoundAccount, factory coldata.ColumnFactory, binFn tree.TwoArgFn, cmpExpr *tree.ComparisonExpr, -) (op colexecbase.Operator, resultIdx int, typs []*types.T, err error) { +) (op colexecop.Operator, resultIdx int, typs []*types.T, err error) { if err := checkSupportedProjectionExpr(left, right); err != nil { return nil, resultIdx, typs, err } @@ -2164,20 +2164,20 @@ func planLogicalProjectionOp( evalCtx *tree.EvalContext, expr tree.TypedExpr, columnTypes []*types.T, - input colexecbase.Operator, + input colexecop.Operator, acc *mon.BoundAccount, factory coldata.ColumnFactory, -) (op colexecbase.Operator, resultIdx int, typs []*types.T, err error) { +) (op colexecop.Operator, resultIdx int, typs []*types.T, err error) { // Add a new boolean column that will store the result of the projection. resultIdx = len(columnTypes) typs = appendOneType(columnTypes, types.Bool) var ( typedLeft, typedRight tree.TypedExpr - leftProjOpChain, rightProjOpChain colexecbase.Operator + leftProjOpChain, rightProjOpChain colexecop.Operator leftIdx, rightIdx int ) - leftFeedOp := colexecbase.NewFeedOperator() - rightFeedOp := colexecbase.NewFeedOperator() + leftFeedOp := colexecop.NewFeedOperator() + rightFeedOp := colexecop.NewFeedOperator() switch t := expr.(type) { case *tree.AndExpr: typedLeft = t.TypedLeft() @@ -2231,11 +2231,11 @@ func planIsNullProjectionOp( outputType *types.T, expr tree.TypedExpr, columnTypes []*types.T, - input colexecbase.Operator, + input colexecop.Operator, acc *mon.BoundAccount, negate bool, factory coldata.ColumnFactory, -) (op colexecbase.Operator, resultIdx int, typs []*types.T, err error) { +) (op colexecop.Operator, resultIdx int, typs []*types.T, err error) { op, resultIdx, typs, err = planProjectionOperators( ctx, evalCtx, expr, columnTypes, input, acc, factory, ) diff --git a/pkg/sql/colexec/colbuilder/execplan_test.go b/pkg/sql/colexec/colbuilder/execplan_test.go index 5c9621c28291..1344499496f5 100644 --- a/pkg/sql/colexec/colbuilder/execplan_test.go +++ b/pkg/sql/colexec/colbuilder/execplan_test.go @@ -21,7 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/colexec" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -114,7 +114,7 @@ func TestNewColOperatorExpectedTypeSchema(t *testing.T) { Post: execinfrapb.PostProcessSpec{RenderExprs: []execinfrapb.Expression{{Expr: "@1 - 1"}}}, ResultTypes: []*types.T{types.Int}, }, - Inputs: []colexecbase.Operator{r.Op}, + Inputs: []colexecop.Operator{r.Op}, StreamingMemAccount: &streamingMemAcc, } r, err = NewColOperator(ctx, flowCtx, args) diff --git a/pkg/sql/colexec/colexecagg/BUILD.bazel b/pkg/sql/colexec/colexecagg/BUILD.bazel index 2797e9d5d8c8..4ac770910e21 100644 --- a/pkg/sql/colexec/colexecagg/BUILD.bazel +++ b/pkg/sql/colexec/colexecagg/BUILD.bazel @@ -1,5 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//pkg/sql/colexecbase:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") go_library( name = "colexecagg", @@ -17,8 +17,8 @@ go_library( "//pkg/col/typeconv", # keep "//pkg/sql/colconv", "//pkg/sql/colexec/execgen", # keep - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfrapb", "//pkg/sql/sem/tree", diff --git a/pkg/sql/colexec/colexecagg/aggregate_funcs.go b/pkg/sql/colexec/colexecagg/aggregate_funcs.go index 32485193ff2b..713839ff65df 100644 --- a/pkg/sql/colexec/colexecagg/aggregate_funcs.go +++ b/pkg/sql/colexec/colexecagg/aggregate_funcs.go @@ -16,8 +16,8 @@ import ( "github.com/cockroachdb/apd/v2" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -201,9 +201,9 @@ type AggregateFuncsAlloc struct { // NewAggregateFuncsAlloc returns a new AggregateFuncsAlloc. func NewAggregateFuncsAlloc( args *NewAggregatorArgs, allocSize int64, isHashAgg bool, -) (*AggregateFuncsAlloc, *colconv.VecToDatumConverter, colexecbase.Closers, error) { +) (*AggregateFuncsAlloc, *colconv.VecToDatumConverter, colexecop.Closers, error) { funcAllocs := make([]aggregateFuncAlloc, len(args.Spec.Aggregations)) - var toClose colexecbase.Closers + var toClose colexecop.Closers var vecIdxsToConvert []int for _, aggFn := range args.Spec.Aggregations { if !IsAggOptimized(aggFn.Func) { @@ -306,7 +306,7 @@ func NewAggregateFuncsAlloc( len(aggFn.ColIdx), args.ConstArguments[i], args.OutputTypes[i], allocSize, ) } - toClose = append(toClose, funcAllocs[i].(colexecbase.Closer)) + toClose = append(toClose, funcAllocs[i].(colexecop.Closer)) } if err != nil { diff --git a/pkg/sql/colexec/colexecagg/aggregators_util.go b/pkg/sql/colexec/colexecagg/aggregators_util.go index 161ffc9dde6b..c59b7cab74c9 100644 --- a/pkg/sql/colexec/colexecagg/aggregators_util.go +++ b/pkg/sql/colexec/colexecagg/aggregators_util.go @@ -11,7 +11,7 @@ package colexecagg import ( - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -26,7 +26,7 @@ type NewAggregatorArgs struct { // MemAccount should be the same as the one used by Allocator and will be // used by aggregatorHelper to handle DISTINCT clause. MemAccount *mon.BoundAccount - Input colexecbase.Operator + Input colexecop.Operator InputTypes []*types.T Spec *execinfrapb.AggregatorSpec EvalCtx *tree.EvalContext diff --git a/pkg/sql/colexec/colexecagg/default_agg_tmpl.go b/pkg/sql/colexec/colexecagg/default_agg_tmpl.go index 2cd1f6eafafa..c7d2c5d287ac 100644 --- a/pkg/sql/colexec/colexecagg/default_agg_tmpl.go +++ b/pkg/sql/colexec/colexecagg/default_agg_tmpl.go @@ -25,8 +25,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -182,7 +182,7 @@ type default_AGGKINDAggAlloc struct { } var _ aggregateFuncAlloc = &default_AGGKINDAggAlloc{} -var _ colexecbase.Closer = &default_AGGKINDAggAlloc{} +var _ colexecop.Closer = &default_AGGKINDAggAlloc{} const sizeOfDefault_AGGKINDAgg = int64(unsafe.Sizeof(default_AGGKINDAgg{})) const default_AGGKINDAggSliceOverhead = int64(unsafe.Sizeof([]default_AGGKINDAggAlloc{})) diff --git a/pkg/sql/colexec/colexecagg/hash_default_agg.eg.go b/pkg/sql/colexec/colexecagg/hash_default_agg.eg.go index 08d81c23f4cd..68ab0de704c6 100644 --- a/pkg/sql/colexec/colexecagg/hash_default_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/hash_default_agg.eg.go @@ -15,8 +15,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -44,9 +44,7 @@ func (a *defaultHashAgg) SetOutput(vec coldata.Vec) { a.hashAggregateFuncBase.SetOutput(vec) } -func (a *defaultHashAgg) Compute( - vecs []coldata.Vec, inputIdxs []uint32, inputLen int, sel []int, -) { +func (a *defaultHashAgg) Compute(vecs []coldata.Vec, inputIdxs []uint32, inputLen int, sel []int) { // Note that we only need to account for the memory of the output vector // and not for the intermediate results of aggregation since the aggregate // function itself does the latter. @@ -151,7 +149,7 @@ type defaultHashAggAlloc struct { } var _ aggregateFuncAlloc = &defaultHashAggAlloc{} -var _ colexecbase.Closer = &defaultHashAggAlloc{} +var _ colexecop.Closer = &defaultHashAggAlloc{} const sizeOfDefaultHashAgg = int64(unsafe.Sizeof(defaultHashAgg{})) const defaultHashAggSliceOverhead = int64(unsafe.Sizeof([]defaultHashAggAlloc{})) diff --git a/pkg/sql/colexec/colexecagg/ordered_default_agg.eg.go b/pkg/sql/colexec/colexecagg/ordered_default_agg.eg.go index b56f8c93bad6..43837a7cc4b5 100644 --- a/pkg/sql/colexec/colexecagg/ordered_default_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/ordered_default_agg.eg.go @@ -15,8 +15,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -216,7 +216,7 @@ type defaultOrderedAggAlloc struct { } var _ aggregateFuncAlloc = &defaultOrderedAggAlloc{} -var _ colexecbase.Closer = &defaultOrderedAggAlloc{} +var _ colexecop.Closer = &defaultOrderedAggAlloc{} const sizeOfDefaultOrderedAgg = int64(unsafe.Sizeof(defaultOrderedAgg{})) const defaultOrderedAggSliceOverhead = int64(unsafe.Sizeof([]defaultOrderedAggAlloc{})) diff --git a/pkg/sql/colexec/colexecargs/BUILD.bazel b/pkg/sql/colexec/colexecargs/BUILD.bazel index cdb7981473de..2ee9fc45ee50 100644 --- a/pkg/sql/colexec/colexecargs/BUILD.bazel +++ b/pkg/sql/colexec/colexecargs/BUILD.bazel @@ -11,7 +11,7 @@ go_library( deps = [ "//pkg/col/coldata", "//pkg/sql/colcontainer", - "//pkg/sql/colexecbase", + "//pkg/sql/colexecop", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", "//pkg/sql/parser", diff --git a/pkg/sql/colexec/colexecargs/dep_test.go b/pkg/sql/colexec/colexecargs/dep_test.go index 9fc4a66828f1..14ef0ea0c106 100644 --- a/pkg/sql/colexec/colexecargs/dep_test.go +++ b/pkg/sql/colexec/colexecargs/dep_test.go @@ -25,7 +25,7 @@ func TestNoLinkForbidden(t *testing.T) { // allowlist: "github.com/cockroachdb/cockroach/pkg/col/coldata", "github.com/cockroachdb/cockroach/pkg/sql/colcontainer", - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase", + "github.com/cockroachdb/cockroach/pkg/sql/colexecop", "github.com/cockroachdb/cockroach/pkg/sql/execinfra", "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb", "github.com/cockroachdb/cockroach/pkg/sql/types", diff --git a/pkg/sql/colexec/colexecargs/op_creation.go b/pkg/sql/colexec/colexecargs/op_creation.go index 2348e867d057..24cdacb6fadc 100644 --- a/pkg/sql/colexec/colexecargs/op_creation.go +++ b/pkg/sql/colexec/colexecargs/op_creation.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -34,7 +34,7 @@ var TestNewColOperator func(ctx context.Context, flowCtx *execinfra.FlowCtx, arg // arguments to NewColOperator call. type NewColOperatorArgs struct { Spec *execinfrapb.ProcessorSpec - Inputs []colexecbase.Operator + Inputs []colexecop.Operator StreamingMemAccount *mon.BoundAccount ProcessorConstructor execinfra.ProcessorConstructor LocalProcessors []execinfra.LocalProcessor @@ -78,12 +78,12 @@ type NewColOperatorArgs struct { // NewColOperatorResult is a helper struct that encompasses all of the return // values of NewColOperator call. type NewColOperatorResult struct { - Op colexecbase.Operator + Op colexecop.Operator KVReader execinfra.KVReader ColumnTypes []*types.T MetadataSources []execinfrapb.MetadataSource // ToClose is a slice of components that need to be Closed. - ToClose []colexecbase.Closer + ToClose []colexecop.Closer OpMonitors []*mon.BytesMonitor OpAccounts []*mon.BoundAccount Releasables []execinfra.Releasable diff --git a/pkg/sql/colexec/colexeccmp/BUILD.bazel b/pkg/sql/colexec/colexeccmp/BUILD.bazel index 9e29570768b4..f137b6d2ebee 100644 --- a/pkg/sql/colexec/colexeccmp/BUILD.bazel +++ b/pkg/sql/colexec/colexeccmp/BUILD.bazel @@ -1,5 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//pkg/sql/colexecbase:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") go_library( name = "colexeccmp", diff --git a/pkg/sql/colexec/colexechash/BUILD.bazel b/pkg/sql/colexec/colexechash/BUILD.bazel index 10875ec7a1ac..c0db4d3109b8 100644 --- a/pkg/sql/colexec/colexechash/BUILD.bazel +++ b/pkg/sql/colexec/colexechash/BUILD.bazel @@ -1,5 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//pkg/sql/colexecbase:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") go_library( name = "colexechash", @@ -17,8 +17,8 @@ go_library( "//pkg/col/typeconv", # keep "//pkg/sql/colexec/colexecutils", "//pkg/sql/colexec/execgen", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/rowenc", "//pkg/sql/sem/tree", # keep diff --git a/pkg/sql/colexec/colexechash/hashtable.go b/pkg/sql/colexec/colexechash/hashtable.go index ae6eb96e9eb2..aff5ebb27216 100644 --- a/pkg/sql/colexec/colexechash/hashtable.go +++ b/pkg/sql/colexec/colexechash/hashtable.go @@ -17,8 +17,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -190,7 +190,7 @@ type HashTable struct { probeMode HashTableProbeMode } -var _ colexecbase.Resetter = &HashTable{} +var _ colexecop.Resetter = &HashTable{} // NewHashTable returns a new HashTable. // @@ -347,7 +347,7 @@ func (ht *HashTable) buildFromBufferedTuples(ctx context.Context) { // FullBuild executes the entirety of the hash table build phase using the input // as the build source. The input is entirely consumed in the process. Note that // the hash table is assumed to operate in HashTableFullBuildMode. -func (ht *HashTable) FullBuild(ctx context.Context, input colexecbase.Operator) { +func (ht *HashTable) FullBuild(ctx context.Context, input colexecop.Operator) { if ht.BuildMode != HashTableFullBuildMode { colexecerror.InternalError(errors.AssertionFailedf( "HashTable.FullBuild is called in unexpected build mode %d", ht.BuildMode, diff --git a/pkg/sql/colexec/colexecjoin/BUILD.bazel b/pkg/sql/colexec/colexecjoin/BUILD.bazel index 5301f499ddb8..9e04bb6b2d73 100644 --- a/pkg/sql/colexec/colexecjoin/BUILD.bazel +++ b/pkg/sql/colexec/colexecjoin/BUILD.bazel @@ -1,5 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//pkg/sql/colexecbase:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") go_library( name = "colexecjoin", @@ -22,8 +22,8 @@ go_library( "//pkg/sql/colexec/colexechash", "//pkg/sql/colexec/colexecmisc", "//pkg/sql/colexec/colexecutils", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", @@ -52,8 +52,8 @@ go_test( "//pkg/settings/cluster", "//pkg/sql/catalog/descpb", "//pkg/sql/colexec/colexectestutils", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", diff --git a/pkg/sql/colexec/colexecjoin/crossjoiner.go b/pkg/sql/colexec/colexecjoin/crossjoiner.go index 8bb23cc2e8f5..5d7d07dc9a68 100644 --- a/pkg/sql/colexec/colexecjoin/crossjoiner.go +++ b/pkg/sql/colexec/colexecjoin/crossjoiner.go @@ -18,8 +18,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -34,12 +34,12 @@ func NewCrossJoiner( diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, joinType descpb.JoinType, - left colexecbase.Operator, - right colexecbase.Operator, + left colexecop.Operator, + right colexecop.Operator, leftTypes []*types.T, rightTypes []*types.T, diskAcc *mon.BoundAccount, -) colexecbase.Operator { +) colexecop.Operator { return &crossJoiner{ crossJoinerBase: newCrossJoinerBase( unlimitedAllocator, @@ -75,8 +75,8 @@ type crossJoiner struct { isLeftAllNulls, isRightAllNulls bool } -var _ colexecbase.ClosableOperator = &crossJoiner{} -var _ colexecbase.ResettableOperator = &crossJoiner{} +var _ colexecop.ClosableOperator = &crossJoiner{} +var _ colexecop.ResettableOperator = &crossJoiner{} func (c *crossJoiner) Init() { c.inputOne.Init() @@ -240,10 +240,10 @@ func setAllNulls(vecs []coldata.Vec, length int) { } func (c *crossJoiner) Reset(ctx context.Context) { - if r, ok := c.inputOne.(colexecbase.Resetter); ok { + if r, ok := c.inputOne.(colexecop.Resetter); ok { r.Reset(ctx) } - if r, ok := c.inputTwo.(colexecbase.Resetter); ok { + if r, ok := c.inputTwo.(colexecop.Resetter); ok { r.Reset(ctx) } c.crossJoinerBase.Reset(ctx) diff --git a/pkg/sql/colexec/colexecjoin/hashjoiner.go b/pkg/sql/colexec/colexecjoin/hashjoiner.go index bae45483f3db..37eb9895ef12 100644 --- a/pkg/sql/colexec/colexecjoin/hashjoiner.go +++ b/pkg/sql/colexec/colexecjoin/hashjoiner.go @@ -17,8 +17,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" @@ -236,8 +236,8 @@ type hashJoiner struct { } } -var _ colexecbase.BufferingInMemoryOperator = &hashJoiner{} -var _ colexecbase.Resetter = &hashJoiner{} +var _ colexecop.BufferingInMemoryOperator = &hashJoiner{} +var _ colexecop.Resetter = &hashJoiner{} // HashJoinerInitialNumBuckets is the number of the hash buckets initially // allocated by the hash table that is used by the in-memory hash joiner. @@ -664,9 +664,7 @@ func (hj *hashJoiner) congregate(nResults int, batch coldata.Batch) { }) } -func (hj *hashJoiner) ExportBuffered( - ctx context.Context, input colexecbase.Operator, -) coldata.Batch { +func (hj *hashJoiner) ExportBuffered(ctx context.Context, input colexecop.Operator) coldata.Batch { if hj.inputOne == input { // We do not buffer anything from the left source. Furthermore, the memory // limit can only hit during the building of the hash table step at which @@ -716,8 +714,8 @@ func (hj *hashJoiner) resetOutput(nResults int) { } func (hj *hashJoiner) Reset(ctx context.Context) { - for _, input := range []colexecbase.Operator{hj.inputOne, hj.inputTwo} { - if r, ok := input.(colexecbase.Resetter); ok { + for _, input := range []colexecop.Operator{hj.inputOne, hj.inputTwo} { + if r, ok := input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -804,10 +802,10 @@ func MakeHashJoinerSpec( func NewHashJoiner( buildSideAllocator, outputUnlimitedAllocator *colmem.Allocator, spec HashJoinerSpec, - leftSource, rightSource colexecbase.Operator, + leftSource, rightSource colexecop.Operator, initialNumBuckets uint64, memoryLimit int64, -) colexecbase.ResettableOperator { +) colexecop.ResettableOperator { return &hashJoiner{ twoInputNode: newTwoInputNode(leftSource, rightSource), buildSideAllocator: buildSideAllocator, diff --git a/pkg/sql/colexec/colexecjoin/joiner_utils.go b/pkg/sql/colexec/colexecjoin/joiner_utils.go index 0d1dcec53f1f..146b103fa2ea 100644 --- a/pkg/sql/colexec/colexecjoin/joiner_utils.go +++ b/pkg/sql/colexec/colexecjoin/joiner_utils.go @@ -11,20 +11,20 @@ package colexecjoin import ( - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/errors" ) // newTwoInputNode returns an execinfra.OpNode with two Operator inputs. -func newTwoInputNode(inputOne, inputTwo colexecbase.Operator) twoInputNode { +func newTwoInputNode(inputOne, inputTwo colexecop.Operator) twoInputNode { return twoInputNode{inputOne: inputOne, inputTwo: inputTwo} } type twoInputNode struct { - inputOne colexecbase.Operator - inputTwo colexecbase.Operator + inputOne colexecop.Operator + inputTwo colexecop.Operator } func (twoInputNode) ChildCount(verbose bool) int { diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner.go b/pkg/sql/colexec/colexecjoin/mergejoiner.go index d893f4d57395..cce8160b1d2f 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner.go @@ -20,8 +20,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecmisc" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -184,12 +184,12 @@ type mergeJoinInput struct { // The distincter is used in the finishGroup phase, and is used only to // determine where the current group ends, in the case that the group ended // with a batch. - distincterInput *colexecbase.FeedOperator - distincter colexecbase.Operator + distincterInput *colexecop.FeedOperator + distincter colexecop.Operator distinctOutput []bool // source specifies the input operator to the merge join. - source colexecbase.Operator + source colexecop.Operator } // The merge join operator uses a probe and build approach to generate the @@ -217,14 +217,14 @@ func NewMergeJoinOp( diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, joinType descpb.JoinType, - left colexecbase.Operator, - right colexecbase.Operator, + left colexecop.Operator, + right colexecop.Operator, leftTypes []*types.T, rightTypes []*types.T, leftOrdering []execinfrapb.Ordering_Column, rightOrdering []execinfrapb.Ordering_Column, diskAcc *mon.BoundAccount, -) (colexecbase.ResettableOperator, error) { +) (colexecop.ResettableOperator, error) { // Merge joiner only supports the case when the physical types in the // equality columns in both inputs are the same. We, however, also need to // support joining on numeric columns of different types or widths. If we @@ -298,7 +298,7 @@ func NewMergeJoinOp( if err != nil { return nil, err } - var mergeJoinerOp colexecbase.ResettableOperator + var mergeJoinerOp colexecop.ResettableOperator switch joinType { case descpb.InnerJoin: mergeJoinerOp = &mergeJoinInnerOp{base} @@ -357,7 +357,7 @@ func NewMergeJoinOp( } return colexecmisc.NewSimpleProjectOp( mergeJoinerOp, numActualLeftTypes+numActualRightTypes, projection, - ).(colexecbase.ResettableOperator), nil + ).(colexecop.ResettableOperator), nil } // Const declarations for the merge joiner cross product (MJCP) zero state. @@ -393,8 +393,8 @@ func newMergeJoinBase( diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, joinType descpb.JoinType, - left colexecbase.Operator, - right colexecbase.Operator, + left colexecop.Operator, + right colexecop.Operator, leftTypes []*types.T, rightTypes []*types.T, leftOrdering []execinfrapb.Ordering_Column, @@ -441,13 +441,13 @@ func newMergeJoinBase( diskAcc: diskAcc, } var err error - base.left.distincterInput = &colexecbase.FeedOperator{} + base.left.distincterInput = &colexecop.FeedOperator{} base.left.distincter, base.left.distinctOutput, err = colexecmisc.OrderedDistinctColsToOperators( base.left.distincterInput, lEqCols, leftTypes) if err != nil { return base, err } - base.right.distincterInput = &colexecbase.FeedOperator{} + base.right.distincterInput = &colexecop.FeedOperator{} base.right.distincter, base.right.distinctOutput, err = colexecmisc.OrderedDistinctColsToOperators( base.right.distincterInput, rEqCols, rightTypes) if err != nil { @@ -459,7 +459,7 @@ func newMergeJoinBase( // mergeJoinBase extracts the common logic between all merge join operators. type mergeJoinBase struct { twoInputNode - colexecbase.CloserHelper + colexecop.CloserHelper unlimitedAllocator *colmem.Allocator memoryLimit int64 @@ -487,14 +487,14 @@ type mergeJoinBase struct { diskAcc *mon.BoundAccount } -var _ colexecbase.Resetter = &mergeJoinBase{} -var _ colexecbase.Closer = &mergeJoinBase{} +var _ colexecop.Resetter = &mergeJoinBase{} +var _ colexecop.Closer = &mergeJoinBase{} func (o *mergeJoinBase) Reset(ctx context.Context) { - if r, ok := o.left.source.(colexecbase.Resetter); ok { + if r, ok := o.left.source.(colexecop.Resetter); ok { r.Reset(ctx) } - if r, ok := o.right.source.(colexecbase.Resetter); ok { + if r, ok := o.right.source.(colexecop.Resetter); ok { r.Reset(ctx) } o.outputReady = false @@ -698,7 +698,7 @@ func (o *mergeJoinBase) completeBufferedGroup( } isBufferedGroupComplete := false - input.distincter.(colexecbase.Resetter).Reset(ctx) + input.distincter.(colexecop.Resetter).Reset(ctx) // Ignore the first row of the distincter in the first pass since we already // know that we are in the same group and, thus, the row is not distinct, // regardless of what the distincter outputs. @@ -779,8 +779,8 @@ func (o *mergeJoinBase) Close(ctx context.Context) error { return nil } var lastErr error - for _, op := range []colexecbase.Operator{o.left.source, o.right.source} { - if c, ok := op.(colexecbase.Closer); ok { + for _, op := range []colexecop.Operator{o.left.source, o.right.source} { + if c, ok := op.(colexecop.Closer); ok { if err := c.Close(ctx); err != nil { lastErr = err } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_exceptall.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_exceptall.eg.go index 6baa8d9be474..84ef5781cb82 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_exceptall.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_exceptall.eg.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -42,7 +42,7 @@ type mergeJoinExceptAllOp struct { *mergeJoinBase } -var _ colexecbase.Operator = &mergeJoinExceptAllOp{} +var _ colexecop.Operator = &mergeJoinExceptAllOp{} func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue(ctx context.Context) { lSel := o.proberState.lBatch.Selection() diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_fullouter.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_fullouter.eg.go index de41a332bc34..0545a9196423 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_fullouter.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_fullouter.eg.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -42,7 +42,7 @@ type mergeJoinFullOuterOp struct { *mergeJoinBase } -var _ colexecbase.Operator = &mergeJoinFullOuterOp{} +var _ colexecop.Operator = &mergeJoinFullOuterOp{} func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue(ctx context.Context) { lSel := o.proberState.lBatch.Selection() diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_inner.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_inner.eg.go index 3330095a282d..dc5d033bb50c 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_inner.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_inner.eg.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -42,7 +42,7 @@ type mergeJoinInnerOp struct { *mergeJoinBase } -var _ colexecbase.Operator = &mergeJoinInnerOp{} +var _ colexecop.Operator = &mergeJoinInnerOp{} func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue(ctx context.Context) { lSel := o.proberState.lBatch.Selection() diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_intersectall.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_intersectall.eg.go index f31d4a5f3b03..c0b4a6dbdc0f 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_intersectall.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_intersectall.eg.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -42,7 +42,7 @@ type mergeJoinIntersectAllOp struct { *mergeJoinBase } -var _ colexecbase.Operator = &mergeJoinIntersectAllOp{} +var _ colexecop.Operator = &mergeJoinIntersectAllOp{} func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue(ctx context.Context) { lSel := o.proberState.lBatch.Selection() diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_leftanti.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_leftanti.eg.go index e59221d96001..7defb9ad1b89 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_leftanti.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_leftanti.eg.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -42,7 +42,7 @@ type mergeJoinLeftAntiOp struct { *mergeJoinBase } -var _ colexecbase.Operator = &mergeJoinLeftAntiOp{} +var _ colexecop.Operator = &mergeJoinLeftAntiOp{} func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue(ctx context.Context) { lSel := o.proberState.lBatch.Selection() diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_leftouter.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_leftouter.eg.go index cd3557f359d7..d8723d46633d 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_leftouter.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_leftouter.eg.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -42,7 +42,7 @@ type mergeJoinLeftOuterOp struct { *mergeJoinBase } -var _ colexecbase.Operator = &mergeJoinLeftOuterOp{} +var _ colexecop.Operator = &mergeJoinLeftOuterOp{} func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue(ctx context.Context) { lSel := o.proberState.lBatch.Selection() diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_leftsemi.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_leftsemi.eg.go index c218803f47cc..f5b28f2b6e1e 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_leftsemi.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_leftsemi.eg.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -42,7 +42,7 @@ type mergeJoinLeftSemiOp struct { *mergeJoinBase } -var _ colexecbase.Operator = &mergeJoinLeftSemiOp{} +var _ colexecop.Operator = &mergeJoinLeftSemiOp{} func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue(ctx context.Context) { lSel := o.proberState.lBatch.Selection() diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go index aad7db6435fa..73b39a37dc18 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -42,7 +42,7 @@ type mergeJoinRightAntiOp struct { *mergeJoinBase } -var _ colexecbase.Operator = &mergeJoinRightAntiOp{} +var _ colexecop.Operator = &mergeJoinRightAntiOp{} func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue(ctx context.Context) { lSel := o.proberState.lBatch.Selection() diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go index 05eb1b19344d..12cb60ad230d 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -42,7 +42,7 @@ type mergeJoinRightOuterOp struct { *mergeJoinBase } -var _ colexecbase.Operator = &mergeJoinRightOuterOp{} +var _ colexecop.Operator = &mergeJoinRightOuterOp{} func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue(ctx context.Context) { lSel := o.proberState.lBatch.Selection() diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go index 63a4b9da4cf4..0c5badd73746 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -42,7 +42,7 @@ type mergeJoinRightSemiOp struct { *mergeJoinBase } -var _ colexecbase.Operator = &mergeJoinRightSemiOp{} +var _ colexecop.Operator = &mergeJoinRightSemiOp{} func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue(ctx context.Context) { lSel := o.proberState.lBatch.Selection() diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_test.go b/pkg/sql/colexec/colexecjoin/mergejoiner_test.go index 0026fb2f4768..e75bca9c32d3 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_test.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_test.go @@ -21,7 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -95,8 +95,8 @@ func TestMergeJoinCrossProduct(t *testing.T) { leftHJSource := colexectestutils.NewChunkingBatchSource(testAllocator, typs, colsLeft, nTuples) rightHJSource := colexectestutils.NewChunkingBatchSource(testAllocator, typs, colsRight, nTuples) mj, err := NewMergeJoinOp( - testAllocator, colexecbase.DefaultMemoryLimit, queueCfg, - colexecbase.NewTestingSemaphore(mjFDLimit), descpb.InnerJoin, + testAllocator, colexecop.DefaultMemoryLimit, queueCfg, + colexecop.NewTestingSemaphore(mjFDLimit), descpb.InnerJoin, leftMJSource, rightMJSource, typs, typs, []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, @@ -115,7 +115,7 @@ func TestMergeJoinCrossProduct(t *testing.T) { Right: hashJoinerSourceSpec{ EqCols: []uint32{0}, SourceTypes: typs, }, - }, leftHJSource, rightHJSource, HashJoinerInitialNumBuckets, colexecbase.DefaultMemoryLimit, + }, leftHJSource, rightHJSource, HashJoinerInitialNumBuckets, colexecop.DefaultMemoryLimit, ) hj.Init() @@ -184,10 +184,10 @@ func BenchmarkMergeJoiner(b *testing.B) { benchMemAccount := testMemMonitor.MakeBoundAccount() defer benchMemAccount.Close(ctx) - getNewMergeJoiner := func(leftSource, rightSource colexecbase.Operator) colexecbase.Operator { + getNewMergeJoiner := func(leftSource, rightSource colexecop.Operator) colexecop.Operator { benchMemAccount.Clear(ctx) base, err := newMergeJoinBase( - colmem.NewAllocator(ctx, &benchMemAccount, testColumnFactory), colexecbase.DefaultMemoryLimit, queueCfg, colexecbase.NewTestingSemaphore(mjFDLimit), + colmem.NewAllocator(ctx, &benchMemAccount, testColumnFactory), colexecop.DefaultMemoryLimit, queueCfg, colexecop.NewTestingSemaphore(mjFDLimit), descpb.InnerJoin, leftSource, rightSource, sourceTypes, sourceTypes, []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_tmpl.go b/pkg/sql/colexec/colexecjoin/mergejoiner_tmpl.go index a66f4a0cf275..4248759341c0 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_tmpl.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_tmpl.go @@ -27,8 +27,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -98,7 +98,7 @@ type mergeJoin_JOIN_TYPE_STRINGOp struct { *mergeJoinBase } -var _ colexecbase.Operator = &mergeJoin_JOIN_TYPE_STRINGOp{} +var _ colexecop.Operator = &mergeJoin_JOIN_TYPE_STRINGOp{} // {{/* // This code snippet is the "meat" of the probing phase. diff --git a/pkg/sql/colexec/colexecmisc/BUILD.bazel b/pkg/sql/colexec/colexecmisc/BUILD.bazel index 4d2d2d873d68..0fb01c474cb5 100644 --- a/pkg/sql/colexec/colexecmisc/BUILD.bazel +++ b/pkg/sql/colexec/colexecmisc/BUILD.bazel @@ -1,5 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//pkg/sql/colexecbase:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") go_library( name = "colexecmisc", @@ -18,8 +18,8 @@ go_library( "//pkg/col/typeconv", # keep "//pkg/sql/colexec/colexecutils", "//pkg/sql/colexec/execgen", # keep - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", # keep "//pkg/sql/sem/tree", # keep "//pkg/sql/types", @@ -51,8 +51,8 @@ go_test( "//pkg/sql/colexec/colbuilder", "//pkg/sql/colexec/colexecargs", "//pkg/sql/colexec/colexectestutils", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", diff --git a/pkg/sql/colexec/colexecmisc/cast.eg.go b/pkg/sql/colexec/colexecmisc/cast.eg.go index 7edea7665579..3d3aebbd37cf 100644 --- a/pkg/sql/colexec/colexecmisc/cast.eg.go +++ b/pkg/sql/colexec/colexecmisc/cast.eg.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -33,16 +33,16 @@ var _ coldataext.Datum func GetCastOperator( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, colIdx int, resultIdx int, fromType *types.T, toType *types.T, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { input = colexecutils.NewVectorTypeEnforcer(allocator, input, toType, resultIdx) if fromType.Family() == types.UnknownFamily { return &castOpNullAny{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -60,7 +60,7 @@ func GetCastOperator( case -1: default: return &castBoolBoolOp{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -72,7 +72,7 @@ func GetCastOperator( case -1: default: return &castBoolFloat64Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -83,7 +83,7 @@ func GetCastOperator( switch rightType.Width() { case 16: return &castBoolInt16Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -91,7 +91,7 @@ func GetCastOperator( }, nil case 32: return &castBoolInt32Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -100,7 +100,7 @@ func GetCastOperator( case -1: default: return &castBoolInt64Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -119,7 +119,7 @@ func GetCastOperator( case -1: default: return &castDecimalBoolOp{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -131,7 +131,7 @@ func GetCastOperator( case -1: default: return &castDecimalDecimalOp{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -148,7 +148,7 @@ func GetCastOperator( switch rightType.Width() { case 16: return &castInt16Int16Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -156,7 +156,7 @@ func GetCastOperator( }, nil case 32: return &castInt16Int32Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -165,7 +165,7 @@ func GetCastOperator( case -1: default: return &castInt16Int64Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -177,7 +177,7 @@ func GetCastOperator( case -1: default: return &castInt16BoolOp{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -189,7 +189,7 @@ func GetCastOperator( case -1: default: return &castInt16DecimalOp{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -201,7 +201,7 @@ func GetCastOperator( case -1: default: return &castInt16Float64Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -215,7 +215,7 @@ func GetCastOperator( switch rightType.Width() { case 16: return &castInt32Int16Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -223,7 +223,7 @@ func GetCastOperator( }, nil case 32: return &castInt32Int32Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -232,7 +232,7 @@ func GetCastOperator( case -1: default: return &castInt32Int64Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -244,7 +244,7 @@ func GetCastOperator( case -1: default: return &castInt32BoolOp{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -256,7 +256,7 @@ func GetCastOperator( case -1: default: return &castInt32DecimalOp{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -268,7 +268,7 @@ func GetCastOperator( case -1: default: return &castInt32Float64Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -283,7 +283,7 @@ func GetCastOperator( switch rightType.Width() { case 16: return &castInt64Int16Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -291,7 +291,7 @@ func GetCastOperator( }, nil case 32: return &castInt64Int32Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -300,7 +300,7 @@ func GetCastOperator( case -1: default: return &castInt64Int64Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -312,7 +312,7 @@ func GetCastOperator( case -1: default: return &castInt64BoolOp{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -324,7 +324,7 @@ func GetCastOperator( case -1: default: return &castInt64DecimalOp{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -336,7 +336,7 @@ func GetCastOperator( case -1: default: return &castInt64Float64Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -355,7 +355,7 @@ func GetCastOperator( case -1: default: return &castFloat64Float64Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -367,7 +367,7 @@ func GetCastOperator( case -1: default: return &castFloat64BoolOp{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -379,7 +379,7 @@ func GetCastOperator( case -1: default: return &castFloat64DecimalOp{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -390,7 +390,7 @@ func GetCastOperator( switch rightType.Width() { case 16: return &castFloat64Int16Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -398,7 +398,7 @@ func GetCastOperator( }, nil case 32: return &castFloat64Int32Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -407,7 +407,7 @@ func GetCastOperator( case -1: default: return &castFloat64Int64Op{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -426,7 +426,7 @@ func GetCastOperator( case -1: default: return &castDatumBoolOp{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -438,7 +438,7 @@ func GetCastOperator( case -1: default: return &castDatumDatumOp{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -452,14 +452,14 @@ func GetCastOperator( } type castOpNullAny struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int outputIdx int } -var _ colexecbase.ClosableOperator = &castOpNullAny{} +var _ colexecop.ClosableOperator = &castOpNullAny{} func (c *castOpNullAny) Init() { c.Input.Init() @@ -506,7 +506,7 @@ func (c *castOpNullAny) Next(ctx context.Context) coldata.Batch { // probably require changing the way we handle cast overloads as well. type castBoolBoolOp struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -514,15 +514,15 @@ type castBoolBoolOp struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castBoolBoolOp{} -var _ colexecbase.ClosableOperator = &castBoolBoolOp{} +var _ colexecop.ResettableOperator = &castBoolBoolOp{} +var _ colexecop.ClosableOperator = &castBoolBoolOp{} func (c *castBoolBoolOp) Init() { c.Input.Init() } func (c *castBoolBoolOp) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -611,7 +611,7 @@ func (c *castBoolBoolOp) Next(ctx context.Context) coldata.Batch { } type castBoolFloat64Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -619,15 +619,15 @@ type castBoolFloat64Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castBoolFloat64Op{} -var _ colexecbase.ClosableOperator = &castBoolFloat64Op{} +var _ colexecop.ResettableOperator = &castBoolFloat64Op{} +var _ colexecop.ClosableOperator = &castBoolFloat64Op{} func (c *castBoolFloat64Op) Init() { c.Input.Init() } func (c *castBoolFloat64Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -736,7 +736,7 @@ func (c *castBoolFloat64Op) Next(ctx context.Context) coldata.Batch { } type castBoolInt16Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -744,15 +744,15 @@ type castBoolInt16Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castBoolInt16Op{} -var _ colexecbase.ClosableOperator = &castBoolInt16Op{} +var _ colexecop.ResettableOperator = &castBoolInt16Op{} +var _ colexecop.ClosableOperator = &castBoolInt16Op{} func (c *castBoolInt16Op) Init() { c.Input.Init() } func (c *castBoolInt16Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -861,7 +861,7 @@ func (c *castBoolInt16Op) Next(ctx context.Context) coldata.Batch { } type castBoolInt32Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -869,15 +869,15 @@ type castBoolInt32Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castBoolInt32Op{} -var _ colexecbase.ClosableOperator = &castBoolInt32Op{} +var _ colexecop.ResettableOperator = &castBoolInt32Op{} +var _ colexecop.ClosableOperator = &castBoolInt32Op{} func (c *castBoolInt32Op) Init() { c.Input.Init() } func (c *castBoolInt32Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -986,7 +986,7 @@ func (c *castBoolInt32Op) Next(ctx context.Context) coldata.Batch { } type castBoolInt64Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -994,15 +994,15 @@ type castBoolInt64Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castBoolInt64Op{} -var _ colexecbase.ClosableOperator = &castBoolInt64Op{} +var _ colexecop.ResettableOperator = &castBoolInt64Op{} +var _ colexecop.ClosableOperator = &castBoolInt64Op{} func (c *castBoolInt64Op) Init() { c.Input.Init() } func (c *castBoolInt64Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -1111,7 +1111,7 @@ func (c *castBoolInt64Op) Next(ctx context.Context) coldata.Batch { } type castDecimalBoolOp struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -1119,15 +1119,15 @@ type castDecimalBoolOp struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castDecimalBoolOp{} -var _ colexecbase.ClosableOperator = &castDecimalBoolOp{} +var _ colexecop.ResettableOperator = &castDecimalBoolOp{} +var _ colexecop.ClosableOperator = &castDecimalBoolOp{} func (c *castDecimalBoolOp) Init() { c.Input.Init() } func (c *castDecimalBoolOp) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -1216,7 +1216,7 @@ func (c *castDecimalBoolOp) Next(ctx context.Context) coldata.Batch { } type castDecimalDecimalOp struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -1224,15 +1224,15 @@ type castDecimalDecimalOp struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castDecimalDecimalOp{} -var _ colexecbase.ClosableOperator = &castDecimalDecimalOp{} +var _ colexecop.ResettableOperator = &castDecimalDecimalOp{} +var _ colexecop.ClosableOperator = &castDecimalDecimalOp{} func (c *castDecimalDecimalOp) Init() { c.Input.Init() } func (c *castDecimalDecimalOp) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -1341,7 +1341,7 @@ func (c *castDecimalDecimalOp) Next(ctx context.Context) coldata.Batch { } type castInt16Int16Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -1349,15 +1349,15 @@ type castInt16Int16Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt16Int16Op{} -var _ colexecbase.ClosableOperator = &castInt16Int16Op{} +var _ colexecop.ResettableOperator = &castInt16Int16Op{} +var _ colexecop.ClosableOperator = &castInt16Int16Op{} func (c *castInt16Int16Op) Init() { c.Input.Init() } func (c *castInt16Int16Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -1446,7 +1446,7 @@ func (c *castInt16Int16Op) Next(ctx context.Context) coldata.Batch { } type castInt16Int32Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -1454,15 +1454,15 @@ type castInt16Int32Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt16Int32Op{} -var _ colexecbase.ClosableOperator = &castInt16Int32Op{} +var _ colexecop.ResettableOperator = &castInt16Int32Op{} +var _ colexecop.ClosableOperator = &castInt16Int32Op{} func (c *castInt16Int32Op) Init() { c.Input.Init() } func (c *castInt16Int32Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -1559,7 +1559,7 @@ func (c *castInt16Int32Op) Next(ctx context.Context) coldata.Batch { } type castInt16Int64Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -1567,15 +1567,15 @@ type castInt16Int64Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt16Int64Op{} -var _ colexecbase.ClosableOperator = &castInt16Int64Op{} +var _ colexecop.ResettableOperator = &castInt16Int64Op{} +var _ colexecop.ClosableOperator = &castInt16Int64Op{} func (c *castInt16Int64Op) Init() { c.Input.Init() } func (c *castInt16Int64Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -1672,7 +1672,7 @@ func (c *castInt16Int64Op) Next(ctx context.Context) coldata.Batch { } type castInt16BoolOp struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -1680,15 +1680,15 @@ type castInt16BoolOp struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt16BoolOp{} -var _ colexecbase.ClosableOperator = &castInt16BoolOp{} +var _ colexecop.ResettableOperator = &castInt16BoolOp{} +var _ colexecop.ClosableOperator = &castInt16BoolOp{} func (c *castInt16BoolOp) Init() { c.Input.Init() } func (c *castInt16BoolOp) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -1785,7 +1785,7 @@ func (c *castInt16BoolOp) Next(ctx context.Context) coldata.Batch { } type castInt16DecimalOp struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -1793,15 +1793,15 @@ type castInt16DecimalOp struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt16DecimalOp{} -var _ colexecbase.ClosableOperator = &castInt16DecimalOp{} +var _ colexecop.ResettableOperator = &castInt16DecimalOp{} +var _ colexecop.ClosableOperator = &castInt16DecimalOp{} func (c *castInt16DecimalOp) Init() { c.Input.Init() } func (c *castInt16DecimalOp) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -1914,7 +1914,7 @@ func (c *castInt16DecimalOp) Next(ctx context.Context) coldata.Batch { } type castInt16Float64Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -1922,15 +1922,15 @@ type castInt16Float64Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt16Float64Op{} -var _ colexecbase.ClosableOperator = &castInt16Float64Op{} +var _ colexecop.ResettableOperator = &castInt16Float64Op{} +var _ colexecop.ClosableOperator = &castInt16Float64Op{} func (c *castInt16Float64Op) Init() { c.Input.Init() } func (c *castInt16Float64Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -2027,7 +2027,7 @@ func (c *castInt16Float64Op) Next(ctx context.Context) coldata.Batch { } type castInt32Int16Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -2035,15 +2035,15 @@ type castInt32Int16Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt32Int16Op{} -var _ colexecbase.ClosableOperator = &castInt32Int16Op{} +var _ colexecop.ResettableOperator = &castInt32Int16Op{} +var _ colexecop.ClosableOperator = &castInt32Int16Op{} func (c *castInt32Int16Op) Init() { c.Input.Init() } func (c *castInt32Int16Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -2140,7 +2140,7 @@ func (c *castInt32Int16Op) Next(ctx context.Context) coldata.Batch { } type castInt32Int32Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -2148,15 +2148,15 @@ type castInt32Int32Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt32Int32Op{} -var _ colexecbase.ClosableOperator = &castInt32Int32Op{} +var _ colexecop.ResettableOperator = &castInt32Int32Op{} +var _ colexecop.ClosableOperator = &castInt32Int32Op{} func (c *castInt32Int32Op) Init() { c.Input.Init() } func (c *castInt32Int32Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -2245,7 +2245,7 @@ func (c *castInt32Int32Op) Next(ctx context.Context) coldata.Batch { } type castInt32Int64Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -2253,15 +2253,15 @@ type castInt32Int64Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt32Int64Op{} -var _ colexecbase.ClosableOperator = &castInt32Int64Op{} +var _ colexecop.ResettableOperator = &castInt32Int64Op{} +var _ colexecop.ClosableOperator = &castInt32Int64Op{} func (c *castInt32Int64Op) Init() { c.Input.Init() } func (c *castInt32Int64Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -2358,7 +2358,7 @@ func (c *castInt32Int64Op) Next(ctx context.Context) coldata.Batch { } type castInt32BoolOp struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -2366,15 +2366,15 @@ type castInt32BoolOp struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt32BoolOp{} -var _ colexecbase.ClosableOperator = &castInt32BoolOp{} +var _ colexecop.ResettableOperator = &castInt32BoolOp{} +var _ colexecop.ClosableOperator = &castInt32BoolOp{} func (c *castInt32BoolOp) Init() { c.Input.Init() } func (c *castInt32BoolOp) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -2471,7 +2471,7 @@ func (c *castInt32BoolOp) Next(ctx context.Context) coldata.Batch { } type castInt32DecimalOp struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -2479,15 +2479,15 @@ type castInt32DecimalOp struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt32DecimalOp{} -var _ colexecbase.ClosableOperator = &castInt32DecimalOp{} +var _ colexecop.ResettableOperator = &castInt32DecimalOp{} +var _ colexecop.ClosableOperator = &castInt32DecimalOp{} func (c *castInt32DecimalOp) Init() { c.Input.Init() } func (c *castInt32DecimalOp) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -2600,7 +2600,7 @@ func (c *castInt32DecimalOp) Next(ctx context.Context) coldata.Batch { } type castInt32Float64Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -2608,15 +2608,15 @@ type castInt32Float64Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt32Float64Op{} -var _ colexecbase.ClosableOperator = &castInt32Float64Op{} +var _ colexecop.ResettableOperator = &castInt32Float64Op{} +var _ colexecop.ClosableOperator = &castInt32Float64Op{} func (c *castInt32Float64Op) Init() { c.Input.Init() } func (c *castInt32Float64Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -2713,7 +2713,7 @@ func (c *castInt32Float64Op) Next(ctx context.Context) coldata.Batch { } type castInt64Int16Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -2721,15 +2721,15 @@ type castInt64Int16Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt64Int16Op{} -var _ colexecbase.ClosableOperator = &castInt64Int16Op{} +var _ colexecop.ResettableOperator = &castInt64Int16Op{} +var _ colexecop.ClosableOperator = &castInt64Int16Op{} func (c *castInt64Int16Op) Init() { c.Input.Init() } func (c *castInt64Int16Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -2826,7 +2826,7 @@ func (c *castInt64Int16Op) Next(ctx context.Context) coldata.Batch { } type castInt64Int32Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -2834,15 +2834,15 @@ type castInt64Int32Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt64Int32Op{} -var _ colexecbase.ClosableOperator = &castInt64Int32Op{} +var _ colexecop.ResettableOperator = &castInt64Int32Op{} +var _ colexecop.ClosableOperator = &castInt64Int32Op{} func (c *castInt64Int32Op) Init() { c.Input.Init() } func (c *castInt64Int32Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -2939,7 +2939,7 @@ func (c *castInt64Int32Op) Next(ctx context.Context) coldata.Batch { } type castInt64Int64Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -2947,15 +2947,15 @@ type castInt64Int64Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt64Int64Op{} -var _ colexecbase.ClosableOperator = &castInt64Int64Op{} +var _ colexecop.ResettableOperator = &castInt64Int64Op{} +var _ colexecop.ClosableOperator = &castInt64Int64Op{} func (c *castInt64Int64Op) Init() { c.Input.Init() } func (c *castInt64Int64Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -3044,7 +3044,7 @@ func (c *castInt64Int64Op) Next(ctx context.Context) coldata.Batch { } type castInt64BoolOp struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -3052,15 +3052,15 @@ type castInt64BoolOp struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt64BoolOp{} -var _ colexecbase.ClosableOperator = &castInt64BoolOp{} +var _ colexecop.ResettableOperator = &castInt64BoolOp{} +var _ colexecop.ClosableOperator = &castInt64BoolOp{} func (c *castInt64BoolOp) Init() { c.Input.Init() } func (c *castInt64BoolOp) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -3157,7 +3157,7 @@ func (c *castInt64BoolOp) Next(ctx context.Context) coldata.Batch { } type castInt64DecimalOp struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -3165,15 +3165,15 @@ type castInt64DecimalOp struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt64DecimalOp{} -var _ colexecbase.ClosableOperator = &castInt64DecimalOp{} +var _ colexecop.ResettableOperator = &castInt64DecimalOp{} +var _ colexecop.ClosableOperator = &castInt64DecimalOp{} func (c *castInt64DecimalOp) Init() { c.Input.Init() } func (c *castInt64DecimalOp) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -3286,7 +3286,7 @@ func (c *castInt64DecimalOp) Next(ctx context.Context) coldata.Batch { } type castInt64Float64Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -3294,15 +3294,15 @@ type castInt64Float64Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castInt64Float64Op{} -var _ colexecbase.ClosableOperator = &castInt64Float64Op{} +var _ colexecop.ResettableOperator = &castInt64Float64Op{} +var _ colexecop.ClosableOperator = &castInt64Float64Op{} func (c *castInt64Float64Op) Init() { c.Input.Init() } func (c *castInt64Float64Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -3399,7 +3399,7 @@ func (c *castInt64Float64Op) Next(ctx context.Context) coldata.Batch { } type castFloat64Float64Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -3407,15 +3407,15 @@ type castFloat64Float64Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castFloat64Float64Op{} -var _ colexecbase.ClosableOperator = &castFloat64Float64Op{} +var _ colexecop.ResettableOperator = &castFloat64Float64Op{} +var _ colexecop.ClosableOperator = &castFloat64Float64Op{} func (c *castFloat64Float64Op) Init() { c.Input.Init() } func (c *castFloat64Float64Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -3504,7 +3504,7 @@ func (c *castFloat64Float64Op) Next(ctx context.Context) coldata.Batch { } type castFloat64BoolOp struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -3512,15 +3512,15 @@ type castFloat64BoolOp struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castFloat64BoolOp{} -var _ colexecbase.ClosableOperator = &castFloat64BoolOp{} +var _ colexecop.ResettableOperator = &castFloat64BoolOp{} +var _ colexecop.ClosableOperator = &castFloat64BoolOp{} func (c *castFloat64BoolOp) Init() { c.Input.Init() } func (c *castFloat64BoolOp) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -3617,7 +3617,7 @@ func (c *castFloat64BoolOp) Next(ctx context.Context) coldata.Batch { } type castFloat64DecimalOp struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -3625,15 +3625,15 @@ type castFloat64DecimalOp struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castFloat64DecimalOp{} -var _ colexecbase.ClosableOperator = &castFloat64DecimalOp{} +var _ colexecop.ResettableOperator = &castFloat64DecimalOp{} +var _ colexecop.ClosableOperator = &castFloat64DecimalOp{} func (c *castFloat64DecimalOp) Init() { c.Input.Init() } func (c *castFloat64DecimalOp) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -3754,7 +3754,7 @@ func (c *castFloat64DecimalOp) Next(ctx context.Context) coldata.Batch { } type castFloat64Int16Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -3762,15 +3762,15 @@ type castFloat64Int16Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castFloat64Int16Op{} -var _ colexecbase.ClosableOperator = &castFloat64Int16Op{} +var _ colexecop.ResettableOperator = &castFloat64Int16Op{} +var _ colexecop.ClosableOperator = &castFloat64Int16Op{} func (c *castFloat64Int16Op) Init() { c.Input.Init() } func (c *castFloat64Int16Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -3879,7 +3879,7 @@ func (c *castFloat64Int16Op) Next(ctx context.Context) coldata.Batch { } type castFloat64Int32Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -3887,15 +3887,15 @@ type castFloat64Int32Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castFloat64Int32Op{} -var _ colexecbase.ClosableOperator = &castFloat64Int32Op{} +var _ colexecop.ResettableOperator = &castFloat64Int32Op{} +var _ colexecop.ClosableOperator = &castFloat64Int32Op{} func (c *castFloat64Int32Op) Init() { c.Input.Init() } func (c *castFloat64Int32Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -4004,7 +4004,7 @@ func (c *castFloat64Int32Op) Next(ctx context.Context) coldata.Batch { } type castFloat64Int64Op struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -4012,15 +4012,15 @@ type castFloat64Int64Op struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castFloat64Int64Op{} -var _ colexecbase.ClosableOperator = &castFloat64Int64Op{} +var _ colexecop.ResettableOperator = &castFloat64Int64Op{} +var _ colexecop.ClosableOperator = &castFloat64Int64Op{} func (c *castFloat64Int64Op) Init() { c.Input.Init() } func (c *castFloat64Int64Op) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -4129,7 +4129,7 @@ func (c *castFloat64Int64Op) Next(ctx context.Context) coldata.Batch { } type castDatumBoolOp struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -4137,15 +4137,15 @@ type castDatumBoolOp struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castDatumBoolOp{} -var _ colexecbase.ClosableOperator = &castDatumBoolOp{} +var _ colexecop.ResettableOperator = &castDatumBoolOp{} +var _ colexecop.ClosableOperator = &castDatumBoolOp{} func (c *castDatumBoolOp) Init() { c.Input.Init() } func (c *castDatumBoolOp) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -4266,7 +4266,7 @@ func (c *castDatumBoolOp) Next(ctx context.Context) coldata.Batch { } type castDatumDatumOp struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -4274,15 +4274,15 @@ type castDatumDatumOp struct { toType *types.T } -var _ colexecbase.ResettableOperator = &castDatumDatumOp{} -var _ colexecbase.ClosableOperator = &castDatumDatumOp{} +var _ colexecop.ResettableOperator = &castDatumDatumOp{} +var _ colexecop.ClosableOperator = &castDatumDatumOp{} func (c *castDatumDatumOp) Init() { c.Input.Init() } func (c *castDatumDatumOp) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } diff --git a/pkg/sql/colexec/colexecmisc/cast_test.go b/pkg/sql/colexec/colexecmisc/cast_test.go index 5b8979549e4d..cf3adcf1f6c5 100644 --- a/pkg/sql/colexec/colexecmisc/cast_test.go +++ b/pkg/sql/colexec/colexecmisc/cast_test.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -151,7 +151,7 @@ func TestRandomizedCast(t *testing.T) { output = append(output, colexectestutils.Tuple{c.fromPhysType(fromDatum), c.toPhysType(toDatum)}) } colexectestutils.RunTestsWithTyps(t, testAllocator, []colexectestutils.Tuples{input}, [][]*types.T{{c.fromTyp}}, output, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return createTestCastOperator(ctx, flowCtx, input[0], c.fromTyp, c.toTyp) }) } @@ -193,7 +193,7 @@ func BenchmarkCastOp(b *testing.B) { testAllocator, rng, typs, coldata.BatchSize(), nullProbability, selectivity, ) - source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) op, err := createTestCastOperator(ctx, flowCtx, source, typePair[0], typePair[1]) require.NoError(b, err) b.SetBytes(int64(8 * coldata.BatchSize())) @@ -211,10 +211,10 @@ func BenchmarkCastOp(b *testing.B) { func createTestCastOperator( ctx context.Context, flowCtx *execinfra.FlowCtx, - input colexecbase.Operator, + input colexecop.Operator, fromTyp *types.T, toTyp *types.T, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { // We currently don't support casting to decimal type (other than when // casting from decimal with the same precision), so we will allow falling // back to row-by-row engine. diff --git a/pkg/sql/colexec/colexecmisc/cast_tmpl.go b/pkg/sql/colexec/colexecmisc/cast_tmpl.go index 27e64c6579a0..76f07bd11027 100644 --- a/pkg/sql/colexec/colexecmisc/cast_tmpl.go +++ b/pkg/sql/colexec/colexecmisc/cast_tmpl.go @@ -29,8 +29,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -74,16 +74,16 @@ func _R_SET(to, from interface{}) { func GetCastOperator( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, colIdx int, resultIdx int, fromType *types.T, toType *types.T, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { input = colexecutils.NewVectorTypeEnforcer(allocator, input, toType, resultIdx) if fromType.Family() == types.UnknownFamily { return &castOpNullAny{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -103,7 +103,7 @@ func GetCastOperator( // {{range .RightWidths}} case _RIGHT_TYPE_WIDTH: return &cast_NAMEOp{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -121,14 +121,14 @@ func GetCastOperator( } type castOpNullAny struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int outputIdx int } -var _ colexecbase.ClosableOperator = &castOpNullAny{} +var _ colexecop.ClosableOperator = &castOpNullAny{} func (c *castOpNullAny) Init() { c.Input.Init() @@ -180,7 +180,7 @@ func (c *castOpNullAny) Next(ctx context.Context) coldata.Batch { // {{range .RightWidths}} type cast_NAMEOp struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper allocator *colmem.Allocator colIdx int @@ -188,15 +188,15 @@ type cast_NAMEOp struct { toType *types.T } -var _ colexecbase.ResettableOperator = &cast_NAMEOp{} -var _ colexecbase.ClosableOperator = &cast_NAMEOp{} +var _ colexecop.ResettableOperator = &cast_NAMEOp{} +var _ colexecop.ClosableOperator = &cast_NAMEOp{} func (c *cast_NAMEOp) Init() { c.Input.Init() } func (c *cast_NAMEOp) Reset(ctx context.Context) { - if r, ok := c.Input.(colexecbase.Resetter); ok { + if r, ok := c.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } diff --git a/pkg/sql/colexec/colexecmisc/const.eg.go b/pkg/sql/colexec/colexecmisc/const.eg.go index e1a0571dd0b9..f2dc2ed9b6c5 100644 --- a/pkg/sql/colexec/colexecmisc/const.eg.go +++ b/pkg/sql/colexec/colexecmisc/const.eg.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" @@ -35,11 +35,11 @@ var ( // type t at index outputIdx. func NewConstOp( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, t *types.T, constVal interface{}, outputIdx int, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { input = colexecutils.NewVectorTypeEnforcer(allocator, input, t, outputIdx) switch typeconv.TypeFamilyToCanonicalTypeFamily(t.Family()) { case types.BoolFamily: @@ -47,7 +47,7 @@ func NewConstOp( case -1: default: return &constBoolOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, outputIdx: outputIdx, constVal: constVal.(bool), @@ -58,7 +58,7 @@ func NewConstOp( case -1: default: return &constBytesOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, outputIdx: outputIdx, constVal: constVal.([]byte), @@ -69,7 +69,7 @@ func NewConstOp( case -1: default: return &constDecimalOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, outputIdx: outputIdx, constVal: constVal.(apd.Decimal), @@ -79,14 +79,14 @@ func NewConstOp( switch t.Width() { case 16: return &constInt16Op{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, outputIdx: outputIdx, constVal: constVal.(int16), }, nil case 32: return &constInt32Op{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, outputIdx: outputIdx, constVal: constVal.(int32), @@ -94,7 +94,7 @@ func NewConstOp( case -1: default: return &constInt64Op{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, outputIdx: outputIdx, constVal: constVal.(int64), @@ -105,7 +105,7 @@ func NewConstOp( case -1: default: return &constFloat64Op{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, outputIdx: outputIdx, constVal: constVal.(float64), @@ -116,7 +116,7 @@ func NewConstOp( case -1: default: return &constTimestampOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, outputIdx: outputIdx, constVal: constVal.(time.Time), @@ -127,7 +127,7 @@ func NewConstOp( case -1: default: return &constIntervalOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, outputIdx: outputIdx, constVal: constVal.(duration.Duration), @@ -138,7 +138,7 @@ func NewConstOp( case -1: default: return &constDatumOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, outputIdx: outputIdx, constVal: constVal.(interface{}), @@ -149,7 +149,7 @@ func NewConstOp( } type constBoolOp struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator outputIdx int @@ -197,7 +197,7 @@ func (c constBoolOp) Next(ctx context.Context) coldata.Batch { } type constBytesOp struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator outputIdx int @@ -244,7 +244,7 @@ func (c constBytesOp) Next(ctx context.Context) coldata.Batch { } type constDecimalOp struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator outputIdx int @@ -292,7 +292,7 @@ func (c constDecimalOp) Next(ctx context.Context) coldata.Batch { } type constInt16Op struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator outputIdx int @@ -340,7 +340,7 @@ func (c constInt16Op) Next(ctx context.Context) coldata.Batch { } type constInt32Op struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator outputIdx int @@ -388,7 +388,7 @@ func (c constInt32Op) Next(ctx context.Context) coldata.Batch { } type constInt64Op struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator outputIdx int @@ -436,7 +436,7 @@ func (c constInt64Op) Next(ctx context.Context) coldata.Batch { } type constFloat64Op struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator outputIdx int @@ -484,7 +484,7 @@ func (c constFloat64Op) Next(ctx context.Context) coldata.Batch { } type constTimestampOp struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator outputIdx int @@ -532,7 +532,7 @@ func (c constTimestampOp) Next(ctx context.Context) coldata.Batch { } type constIntervalOp struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator outputIdx int @@ -580,7 +580,7 @@ func (c constIntervalOp) Next(ctx context.Context) coldata.Batch { } type constDatumOp struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator outputIdx int @@ -629,21 +629,21 @@ func (c constDatumOp) Next(ctx context.Context) coldata.Batch { // NewConstNullOp creates a new operator that produces a constant (untyped) NULL // value at index outputIdx. func NewConstNullOp( - allocator *colmem.Allocator, input colexecbase.Operator, outputIdx int, -) colexecbase.Operator { + allocator *colmem.Allocator, input colexecop.Operator, outputIdx int, +) colexecop.Operator { input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.Unknown, outputIdx) return &constNullOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), outputIdx: outputIdx, } } type constNullOp struct { - colexecbase.OneInputNode + colexecop.OneInputNode outputIdx int } -var _ colexecbase.Operator = &constNullOp{} +var _ colexecop.Operator = &constNullOp{} func (c constNullOp) Init() { c.Input.Init() diff --git a/pkg/sql/colexec/colexecmisc/const_test.go b/pkg/sql/colexec/colexecmisc/const_test.go index cd72e37c8aee..fba2071b4938 100644 --- a/pkg/sql/colexec/colexecmisc/const_test.go +++ b/pkg/sql/colexec/colexecmisc/const_test.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -52,7 +52,7 @@ func TestConst(t *testing.T) { } for _, tc := range tcs { colexectestutils.RunTestsWithTyps(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, [][]*types.T{{types.Int}}, tc.expected, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return colexectestutils.CreateTestProjectingOperator( ctx, flowCtx, input[0], []*types.T{types.Int}, "9" /* projectingExpr */, false /* canFallbackToRowexec */, testMemAcc, @@ -89,7 +89,7 @@ func TestConstNull(t *testing.T) { } for _, tc := range tcs { colexectestutils.RunTestsWithTyps(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, [][]*types.T{{types.Int}}, tc.expected, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return colexectestutils.CreateTestProjectingOperator( ctx, flowCtx, input[0], []*types.T{types.Int}, "NULL::INT" /* projectingExpr */, false /* canFallbackToRowexec */, testMemAcc, diff --git a/pkg/sql/colexec/colexecmisc/const_tmpl.go b/pkg/sql/colexec/colexecmisc/const_tmpl.go index 0041d7fa1a02..aea533350f44 100644 --- a/pkg/sql/colexec/colexecmisc/const_tmpl.go +++ b/pkg/sql/colexec/colexecmisc/const_tmpl.go @@ -27,7 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" @@ -60,11 +60,11 @@ const _TYPE_WIDTH = 0 // type t at index outputIdx. func NewConstOp( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, t *types.T, constVal interface{}, outputIdx int, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { input = colexecutils.NewVectorTypeEnforcer(allocator, input, t, outputIdx) switch typeconv.TypeFamilyToCanonicalTypeFamily(t.Family()) { // {{range .}} @@ -73,7 +73,7 @@ func NewConstOp( // {{range .WidthOverloads}} case _TYPE_WIDTH: return &const_TYPEOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, outputIdx: outputIdx, constVal: constVal.(_GOTYPE), @@ -89,7 +89,7 @@ func NewConstOp( // {{range .WidthOverloads}} type const_TYPEOp struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator outputIdx int @@ -144,21 +144,21 @@ func (c const_TYPEOp) Next(ctx context.Context) coldata.Batch { // NewConstNullOp creates a new operator that produces a constant (untyped) NULL // value at index outputIdx. func NewConstNullOp( - allocator *colmem.Allocator, input colexecbase.Operator, outputIdx int, -) colexecbase.Operator { + allocator *colmem.Allocator, input colexecop.Operator, outputIdx int, +) colexecop.Operator { input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.Unknown, outputIdx) return &constNullOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), outputIdx: outputIdx, } } type constNullOp struct { - colexecbase.OneInputNode + colexecop.OneInputNode outputIdx int } -var _ colexecbase.Operator = &constNullOp{} +var _ colexecop.Operator = &constNullOp{} func (c constNullOp) Init() { c.Input.Init() diff --git a/pkg/sql/colexec/colexecmisc/distinct.eg.go b/pkg/sql/colexec/colexecmisc/distinct.eg.go index 5a2ffa24937e..dceb1c5ffde7 100644 --- a/pkg/sql/colexec/colexecmisc/distinct.eg.go +++ b/pkg/sql/colexec/colexecmisc/distinct.eg.go @@ -20,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" @@ -28,15 +28,15 @@ import ( ) func newSingleDistinct( - input colexecbase.Operator, distinctColIdx int, outputCol []bool, t *types.T, -) (colexecbase.Operator, error) { + input colexecop.Operator, distinctColIdx int, outputCol []bool, t *types.T, +) (colexecop.Operator, error) { switch typeconv.TypeFamilyToCanonicalTypeFamily(t.Family()) { case types.BoolFamily: switch t.Width() { case -1: default: return &distinctBoolOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), distinctColIdx: distinctColIdx, outputCol: outputCol, }, nil @@ -46,7 +46,7 @@ func newSingleDistinct( case -1: default: return &distinctBytesOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), distinctColIdx: distinctColIdx, outputCol: outputCol, }, nil @@ -56,7 +56,7 @@ func newSingleDistinct( case -1: default: return &distinctDecimalOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), distinctColIdx: distinctColIdx, outputCol: outputCol, }, nil @@ -65,20 +65,20 @@ func newSingleDistinct( switch t.Width() { case 16: return &distinctInt16Op{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), distinctColIdx: distinctColIdx, outputCol: outputCol, }, nil case 32: return &distinctInt32Op{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), distinctColIdx: distinctColIdx, outputCol: outputCol, }, nil case -1: default: return &distinctInt64Op{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), distinctColIdx: distinctColIdx, outputCol: outputCol, }, nil @@ -88,7 +88,7 @@ func newSingleDistinct( case -1: default: return &distinctFloat64Op{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), distinctColIdx: distinctColIdx, outputCol: outputCol, }, nil @@ -98,7 +98,7 @@ func newSingleDistinct( case -1: default: return &distinctTimestampOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), distinctColIdx: distinctColIdx, outputCol: outputCol, }, nil @@ -108,7 +108,7 @@ func newSingleDistinct( case -1: default: return &distinctIntervalOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), distinctColIdx: distinctColIdx, outputCol: outputCol, }, nil @@ -118,7 +118,7 @@ func newSingleDistinct( case -1: default: return &distinctDatumOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), distinctColIdx: distinctColIdx, outputCol: outputCol, }, nil @@ -139,7 +139,7 @@ type distinctBoolOp struct { // still works across batch boundaries. lastVal bool - colexecbase.OneInputNode + colexecop.OneInputNode // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -151,7 +151,7 @@ type distinctBoolOp struct { lastValNull bool } -var _ colexecbase.ResettableOperator = &distinctBoolOp{} +var _ colexecop.ResettableOperator = &distinctBoolOp{} func (p *distinctBoolOp) Init() { p.Input.Init() @@ -160,7 +160,7 @@ func (p *distinctBoolOp) Init() { func (p *distinctBoolOp) Reset(ctx context.Context) { p.foundFirstRow = false p.lastValNull = false - if resetter, ok := p.Input.(colexecbase.Resetter); ok { + if resetter, ok := p.Input.(colexecop.Resetter); ok { resetter.Reset(ctx) } } @@ -403,7 +403,7 @@ type distinctBytesOp struct { // still works across batch boundaries. lastVal []byte - colexecbase.OneInputNode + colexecop.OneInputNode // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -415,7 +415,7 @@ type distinctBytesOp struct { lastValNull bool } -var _ colexecbase.ResettableOperator = &distinctBytesOp{} +var _ colexecop.ResettableOperator = &distinctBytesOp{} func (p *distinctBytesOp) Init() { p.Input.Init() @@ -424,7 +424,7 @@ func (p *distinctBytesOp) Init() { func (p *distinctBytesOp) Reset(ctx context.Context) { p.foundFirstRow = false p.lastValNull = false - if resetter, ok := p.Input.(colexecbase.Resetter); ok { + if resetter, ok := p.Input.(colexecop.Resetter); ok { resetter.Reset(ctx) } } @@ -635,7 +635,7 @@ type distinctDecimalOp struct { // still works across batch boundaries. lastVal apd.Decimal - colexecbase.OneInputNode + colexecop.OneInputNode // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -647,7 +647,7 @@ type distinctDecimalOp struct { lastValNull bool } -var _ colexecbase.ResettableOperator = &distinctDecimalOp{} +var _ colexecop.ResettableOperator = &distinctDecimalOp{} func (p *distinctDecimalOp) Init() { p.Input.Init() @@ -656,7 +656,7 @@ func (p *distinctDecimalOp) Init() { func (p *distinctDecimalOp) Reset(ctx context.Context) { p.foundFirstRow = false p.lastValNull = false - if resetter, ok := p.Input.(colexecbase.Resetter); ok { + if resetter, ok := p.Input.(colexecop.Resetter); ok { resetter.Reset(ctx) } } @@ -867,7 +867,7 @@ type distinctInt16Op struct { // still works across batch boundaries. lastVal int16 - colexecbase.OneInputNode + colexecop.OneInputNode // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -879,7 +879,7 @@ type distinctInt16Op struct { lastValNull bool } -var _ colexecbase.ResettableOperator = &distinctInt16Op{} +var _ colexecop.ResettableOperator = &distinctInt16Op{} func (p *distinctInt16Op) Init() { p.Input.Init() @@ -888,7 +888,7 @@ func (p *distinctInt16Op) Init() { func (p *distinctInt16Op) Reset(ctx context.Context) { p.foundFirstRow = false p.lastValNull = false - if resetter, ok := p.Input.(colexecbase.Resetter); ok { + if resetter, ok := p.Input.(colexecop.Resetter); ok { resetter.Reset(ctx) } } @@ -1143,7 +1143,7 @@ type distinctInt32Op struct { // still works across batch boundaries. lastVal int32 - colexecbase.OneInputNode + colexecop.OneInputNode // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -1155,7 +1155,7 @@ type distinctInt32Op struct { lastValNull bool } -var _ colexecbase.ResettableOperator = &distinctInt32Op{} +var _ colexecop.ResettableOperator = &distinctInt32Op{} func (p *distinctInt32Op) Init() { p.Input.Init() @@ -1164,7 +1164,7 @@ func (p *distinctInt32Op) Init() { func (p *distinctInt32Op) Reset(ctx context.Context) { p.foundFirstRow = false p.lastValNull = false - if resetter, ok := p.Input.(colexecbase.Resetter); ok { + if resetter, ok := p.Input.(colexecop.Resetter); ok { resetter.Reset(ctx) } } @@ -1419,7 +1419,7 @@ type distinctInt64Op struct { // still works across batch boundaries. lastVal int64 - colexecbase.OneInputNode + colexecop.OneInputNode // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -1431,7 +1431,7 @@ type distinctInt64Op struct { lastValNull bool } -var _ colexecbase.ResettableOperator = &distinctInt64Op{} +var _ colexecop.ResettableOperator = &distinctInt64Op{} func (p *distinctInt64Op) Init() { p.Input.Init() @@ -1440,7 +1440,7 @@ func (p *distinctInt64Op) Init() { func (p *distinctInt64Op) Reset(ctx context.Context) { p.foundFirstRow = false p.lastValNull = false - if resetter, ok := p.Input.(colexecbase.Resetter); ok { + if resetter, ok := p.Input.(colexecop.Resetter); ok { resetter.Reset(ctx) } } @@ -1695,7 +1695,7 @@ type distinctFloat64Op struct { // still works across batch boundaries. lastVal float64 - colexecbase.OneInputNode + colexecop.OneInputNode // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -1707,7 +1707,7 @@ type distinctFloat64Op struct { lastValNull bool } -var _ colexecbase.ResettableOperator = &distinctFloat64Op{} +var _ colexecop.ResettableOperator = &distinctFloat64Op{} func (p *distinctFloat64Op) Init() { p.Input.Init() @@ -1716,7 +1716,7 @@ func (p *distinctFloat64Op) Init() { func (p *distinctFloat64Op) Reset(ctx context.Context) { p.foundFirstRow = false p.lastValNull = false - if resetter, ok := p.Input.(colexecbase.Resetter); ok { + if resetter, ok := p.Input.(colexecop.Resetter); ok { resetter.Reset(ctx) } } @@ -2003,7 +2003,7 @@ type distinctTimestampOp struct { // still works across batch boundaries. lastVal time.Time - colexecbase.OneInputNode + colexecop.OneInputNode // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -2015,7 +2015,7 @@ type distinctTimestampOp struct { lastValNull bool } -var _ colexecbase.ResettableOperator = &distinctTimestampOp{} +var _ colexecop.ResettableOperator = &distinctTimestampOp{} func (p *distinctTimestampOp) Init() { p.Input.Init() @@ -2024,7 +2024,7 @@ func (p *distinctTimestampOp) Init() { func (p *distinctTimestampOp) Reset(ctx context.Context) { p.foundFirstRow = false p.lastValNull = false - if resetter, ok := p.Input.(colexecbase.Resetter); ok { + if resetter, ok := p.Input.(colexecop.Resetter); ok { resetter.Reset(ctx) } } @@ -2263,7 +2263,7 @@ type distinctIntervalOp struct { // still works across batch boundaries. lastVal duration.Duration - colexecbase.OneInputNode + colexecop.OneInputNode // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -2275,7 +2275,7 @@ type distinctIntervalOp struct { lastValNull bool } -var _ colexecbase.ResettableOperator = &distinctIntervalOp{} +var _ colexecop.ResettableOperator = &distinctIntervalOp{} func (p *distinctIntervalOp) Init() { p.Input.Init() @@ -2284,7 +2284,7 @@ func (p *distinctIntervalOp) Init() { func (p *distinctIntervalOp) Reset(ctx context.Context) { p.foundFirstRow = false p.lastValNull = false - if resetter, ok := p.Input.(colexecbase.Resetter); ok { + if resetter, ok := p.Input.(colexecop.Resetter); ok { resetter.Reset(ctx) } } @@ -2495,7 +2495,7 @@ type distinctDatumOp struct { // still works across batch boundaries. lastVal interface{} - colexecbase.OneInputNode + colexecop.OneInputNode // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -2507,7 +2507,7 @@ type distinctDatumOp struct { lastValNull bool } -var _ colexecbase.ResettableOperator = &distinctDatumOp{} +var _ colexecop.ResettableOperator = &distinctDatumOp{} func (p *distinctDatumOp) Init() { p.Input.Init() @@ -2516,7 +2516,7 @@ func (p *distinctDatumOp) Init() { func (p *distinctDatumOp) Reset(ctx context.Context) { p.foundFirstRow = false p.lastValNull = false - if resetter, ok := p.Input.(colexecbase.Resetter); ok { + if resetter, ok := p.Input.(colexecop.Resetter); ok { resetter.Reset(ctx) } } diff --git a/pkg/sql/colexec/colexecmisc/distinct.go b/pkg/sql/colexec/colexecmisc/distinct.go index 8f89f2ceab7d..50fc47ae7ff3 100644 --- a/pkg/sql/colexec/colexecmisc/distinct.go +++ b/pkg/sql/colexec/colexecmisc/distinct.go @@ -13,8 +13,8 @@ package colexecmisc import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" ) @@ -23,17 +23,17 @@ import ( // a slice of columns, creates a chain of distinct operators and returns the // last distinct operator in that chain as well as its output column. func OrderedDistinctColsToOperators( - input colexecbase.Operator, distinctCols []uint32, typs []*types.T, -) (colexecbase.ResettableOperator, []bool, error) { + input colexecop.Operator, distinctCols []uint32, typs []*types.T, +) (colexecop.ResettableOperator, []bool, error) { distinctCol := make([]bool, coldata.BatchSize()) // zero the boolean column on every iteration. input = fnOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), fn: func() { copy(distinctCol, colexecutils.ZeroBoolColumn) }, } var ( err error - r colexecbase.ResettableOperator + r colexecop.ResettableOperator ok bool ) for i := range distinctCols { @@ -42,7 +42,7 @@ func OrderedDistinctColsToOperators( return nil, nil, err } } - if r, ok = input.(colexecbase.ResettableOperator); !ok { + if r, ok = input.(colexecop.ResettableOperator); !ok { colexecerror.InternalError(errors.AssertionFailedf("unexpectedly an ordered distinct is not a Resetter")) } distinctChain := &distinctChainOps{ @@ -52,22 +52,22 @@ func OrderedDistinctColsToOperators( } type distinctChainOps struct { - colexecbase.ResettableOperator + colexecop.ResettableOperator } -var _ colexecbase.ResettableOperator = &distinctChainOps{} +var _ colexecop.ResettableOperator = &distinctChainOps{} // NewOrderedDistinct creates a new ordered distinct operator on the given // input columns with the given types. func NewOrderedDistinct( - input colexecbase.Operator, distinctCols []uint32, typs []*types.T, -) (colexecbase.ResettableOperator, error) { + input colexecop.Operator, distinctCols []uint32, typs []*types.T, +) (colexecop.ResettableOperator, error) { op, outputCol, err := OrderedDistinctColsToOperators(input, distinctCols, typs) if err != nil { return nil, err } return &colexecutils.BoolVecToSelOp{ - OneInputNode: colexecbase.NewOneInputNode(op), + OneInputNode: colexecop.NewOneInputNode(op), OutputCol: outputCol, }, nil } diff --git a/pkg/sql/colexec/colexecmisc/distinct_tmpl.go b/pkg/sql/colexec/colexecmisc/distinct_tmpl.go index bb880af1168c..83136b06dfd4 100644 --- a/pkg/sql/colexec/colexecmisc/distinct_tmpl.go +++ b/pkg/sql/colexec/colexecmisc/distinct_tmpl.go @@ -27,8 +27,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" @@ -71,8 +71,8 @@ const _TYPE_WIDTH = 0 // {{define "distinctOpConstructor"}} func newSingleDistinct( - input colexecbase.Operator, distinctColIdx int, outputCol []bool, t *types.T, -) (colexecbase.Operator, error) { + input colexecop.Operator, distinctColIdx int, outputCol []bool, t *types.T, +) (colexecop.Operator, error) { switch typeconv.TypeFamilyToCanonicalTypeFamily(t.Family()) { // {{range .}} case _CANONICAL_TYPE_FAMILY: @@ -80,7 +80,7 @@ func newSingleDistinct( // {{range .WidthOverloads}} case _TYPE_WIDTH: return &distinct_TYPEOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), distinctColIdx: distinctColIdx, outputCol: outputCol, }, nil @@ -144,7 +144,7 @@ type distinct_TYPEOp struct { // still works across batch boundaries. lastVal _GOTYPE - colexecbase.OneInputNode + colexecop.OneInputNode // distinctColIdx is the index of the column to distinct upon. distinctColIdx int @@ -156,7 +156,7 @@ type distinct_TYPEOp struct { lastValNull bool } -var _ colexecbase.ResettableOperator = &distinct_TYPEOp{} +var _ colexecop.ResettableOperator = &distinct_TYPEOp{} func (p *distinct_TYPEOp) Init() { p.Input.Init() @@ -165,7 +165,7 @@ func (p *distinct_TYPEOp) Init() { func (p *distinct_TYPEOp) Reset(ctx context.Context) { p.foundFirstRow = false p.lastValNull = false - if resetter, ok := p.Input.(colexecbase.Resetter); ok { + if resetter, ok := p.Input.(colexecop.Resetter); ok { resetter.Reset(ctx) } } diff --git a/pkg/sql/colexec/colexecmisc/fn_op.go b/pkg/sql/colexec/colexecmisc/fn_op.go index 82e16e208d38..2d31217bb7b2 100644 --- a/pkg/sql/colexec/colexecmisc/fn_op.go +++ b/pkg/sql/colexec/colexecmisc/fn_op.go @@ -14,19 +14,19 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" ) // fnOp is an operator that executes an arbitrary function for its side-effects, // once per input batch, passing the input batch unmodified along. type fnOp struct { - colexecbase.OneInputNode - colexecbase.NonExplainable + colexecop.OneInputNode + colexecop.NonExplainable fn func() } -var _ colexecbase.ResettableOperator = fnOp{} +var _ colexecop.ResettableOperator = fnOp{} func (f fnOp) Init() { f.Input.Init() @@ -39,7 +39,7 @@ func (f fnOp) Next(ctx context.Context) coldata.Batch { } func (f fnOp) Reset(ctx context.Context) { - if resettableOp, ok := f.Input.(colexecbase.Resetter); ok { + if resettableOp, ok := f.Input.(colexecop.Resetter); ok { resettableOp.Reset(ctx) } } diff --git a/pkg/sql/colexec/colexecmisc/ordinality.go b/pkg/sql/colexec/colexecmisc/ordinality.go index 8ffb7cea192f..938d94bde5d1 100644 --- a/pkg/sql/colexec/colexecmisc/ordinality.go +++ b/pkg/sql/colexec/colexecmisc/ordinality.go @@ -15,7 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -23,7 +23,7 @@ import ( // ordinalityOp is an operator that implements WITH ORDINALITY, which adds // an additional column to the result with an ordinal number. type ordinalityOp struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator // outputIdx is the index of the column in which ordinalityOp will write the @@ -33,15 +33,15 @@ type ordinalityOp struct { counter int64 } -var _ colexecbase.Operator = &ordinalityOp{} +var _ colexecop.Operator = &ordinalityOp{} // NewOrdinalityOp returns a new WITH ORDINALITY operator. func NewOrdinalityOp( - allocator *colmem.Allocator, input colexecbase.Operator, outputIdx int, -) colexecbase.Operator { + allocator *colmem.Allocator, input colexecop.Operator, outputIdx int, +) colexecop.Operator { input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.Int, outputIdx) c := &ordinalityOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, outputIdx: outputIdx, counter: 1, diff --git a/pkg/sql/colexec/colexecmisc/ordinality_test.go b/pkg/sql/colexec/colexecmisc/ordinality_test.go index 61c093dd84e6..2c4381f3956d 100644 --- a/pkg/sql/colexec/colexecmisc/ordinality_test.go +++ b/pkg/sql/colexec/colexecmisc/ordinality_test.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -70,7 +70,7 @@ func TestOrdinality(t *testing.T) { for _, tc := range tcs { colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.expected, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return createTestOrdinalityOperator(ctx, flowCtx, input[0], tc.inputTypes) }) } @@ -91,7 +91,7 @@ func BenchmarkOrdinality(b *testing.B) { typs := []*types.T{types.Int, types.Int, types.Int} batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) - source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) ordinality, err := createTestOrdinalityOperator(ctx, flowCtx, source, []*types.T{types.Int, types.Int, types.Int}) require.NoError(b, err) ordinality.Init() @@ -103,11 +103,8 @@ func BenchmarkOrdinality(b *testing.B) { } func createTestOrdinalityOperator( - ctx context.Context, - flowCtx *execinfra.FlowCtx, - input colexecbase.Operator, - inputTypes []*types.T, -) (colexecbase.Operator, error) { + ctx context.Context, flowCtx *execinfra.FlowCtx, input colexecop.Operator, inputTypes []*types.T, +) (colexecop.Operator, error) { spec := &execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{{ColumnTypes: inputTypes}}, Core: execinfrapb.ProcessorCoreUnion{ @@ -117,7 +114,7 @@ func createTestOrdinalityOperator( } args := &colexecargs.NewColOperatorArgs{ Spec: spec, - Inputs: []colexecbase.Operator{input}, + Inputs: []colexecop.Operator{input}, StreamingMemAccount: testMemAcc, } result, err := colexecargs.TestNewColOperator(ctx, flowCtx, args) diff --git a/pkg/sql/colexec/colexecmisc/simple_project.go b/pkg/sql/colexec/colexecmisc/simple_project.go index 274b3bc30861..8962c4fdad46 100644 --- a/pkg/sql/colexec/colexecmisc/simple_project.go +++ b/pkg/sql/colexec/colexecmisc/simple_project.go @@ -14,15 +14,15 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/util/log" ) // simpleProjectOp is an operator that implements "simple projection" - removal of // columns that aren't needed by later operators. type simpleProjectOp struct { - colexecbase.OneInputCloserHelper - colexecbase.NonExplainable + colexecop.OneInputCloserHelper + colexecop.NonExplainable projection []uint32 batches map[coldata.Batch]*projectingBatch @@ -32,8 +32,8 @@ type simpleProjectOp struct { numBatchesLoggingThreshold int } -var _ colexecbase.ClosableOperator = &simpleProjectOp{} -var _ colexecbase.ResettableOperator = &simpleProjectOp{} +var _ colexecop.ClosableOperator = &simpleProjectOp{} +var _ colexecop.ResettableOperator = &simpleProjectOp{} // projectingBatch is a Batch that applies a simple projection to another, // underlying batch, discarding all columns but the ones in its projection @@ -92,8 +92,8 @@ func (b *projectingBatch) ReplaceCol(col coldata.Vec, idx int) { // when input already outputs batches that satisfy the projection, a // simpleProjectOp is not planned and input is returned. func NewSimpleProjectOp( - input colexecbase.Operator, numInputCols int, projection []uint32, -) colexecbase.Operator { + input colexecop.Operator, numInputCols int, projection []uint32, +) colexecop.Operator { if numInputCols == len(projection) { projectionIsRedundant := true for i := range projection { @@ -106,7 +106,7 @@ func NewSimpleProjectOp( } } s := &simpleProjectOp{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), projection: make([]uint32, len(projection)), batches: make(map[coldata.Batch]*projectingBatch), numBatchesLoggingThreshold: 128, @@ -141,7 +141,7 @@ func (d *simpleProjectOp) Next(ctx context.Context) coldata.Batch { } func (d *simpleProjectOp) Reset(ctx context.Context) { - if r, ok := d.Input.(colexecbase.Resetter); ok { + if r, ok := d.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } diff --git a/pkg/sql/colexec/colexecmisc/simple_project_test.go b/pkg/sql/colexec/colexecmisc/simple_project_test.go index a373abceb534..5732ea312747 100644 --- a/pkg/sql/colexec/colexecmisc/simple_project_test.go +++ b/pkg/sql/colexec/colexecmisc/simple_project_test.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecmisc" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -67,7 +67,7 @@ func TestSimpleProjectOp(t *testing.T) { }, } for _, tc := range tcs { - colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.expected, colexectestutils.OrderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { + colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.expected, colexectestutils.OrderedVerifier, func(input []colexecop.Operator) (colexecop.Operator, error) { return colexecmisc.NewSimpleProjectOp(input[0], len(tc.tuples[0]), tc.colsToKeep), nil }) } @@ -75,7 +75,7 @@ func TestSimpleProjectOp(t *testing.T) { // Empty projection. The all nulls injection test case will also return // nothing. colexectestutils.RunTestsWithoutAllNullsInjection(t, testAllocator, []colexectestutils.Tuples{{{1, 2, 3}, {1, 2, 3}}}, nil, colexectestutils.Tuples{{}, {}}, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return colexecmisc.NewSimpleProjectOp(input[0], 3 /* numInputCols */, nil), nil }) @@ -114,8 +114,8 @@ func TestSimpleProjectOpWithUnorderedSynchronizer(t *testing.T) { {"bb", constVal}, } colexectestutils.RunTestsWithoutAllNullsInjection(t, testAllocator, inputTuples, [][]*types.T{inputTypes, inputTypes}, expected, colexectestutils.UnorderedVerifier, - func(inputs []colexecbase.Operator) (colexecbase.Operator, error) { - var input colexecbase.Operator + func(inputs []colexecop.Operator) (colexecop.Operator, error) { + var input colexecop.Operator parallelUnorderedSynchronizerInputs := make([]colexec.SynchronizerInput, len(inputs)) for i := range parallelUnorderedSynchronizerInputs { parallelUnorderedSynchronizerInputs[i].Op = inputs[i] diff --git a/pkg/sql/colexec/colexecproj/BUILD.bazel b/pkg/sql/colexec/colexecproj/BUILD.bazel index 687d0529858b..f78a0842a494 100644 --- a/pkg/sql/colexec/colexecproj/BUILD.bazel +++ b/pkg/sql/colexec/colexecproj/BUILD.bazel @@ -1,5 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//pkg/sql/colexecbase:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") go_library( name = "colexecproj", @@ -18,8 +18,8 @@ go_library( "//pkg/sql/colexec/colexeccmp", "//pkg/sql/colexec/colexecutils", "//pkg/sql/colexec/execgen", # keep - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", # keep + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/sem/tree", "//pkg/sql/sqltelemetry", # keep @@ -50,8 +50,8 @@ go_test( "//pkg/sql/colexec/colexecargs", "//pkg/sql/colexec/colexectestutils", "//pkg/sql/colexec/execgen", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/rowenc", diff --git a/pkg/sql/colexec/colexecproj/default_cmp_op_test.go b/pkg/sql/colexec/colexecproj/default_cmp_op_test.go index edfa24a15c9c..3863179a3d83 100644 --- a/pkg/sql/colexec/colexecproj/default_cmp_op_test.go +++ b/pkg/sql/colexec/colexecproj/default_cmp_op_test.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -108,7 +108,7 @@ func TestDefaultCmpProjOps(t *testing.T) { for _, c := range testCases { t.Run(c.cmpExpr, func(t *testing.T) { colexectestutils.RunTestsWithTyps(t, testAllocator, []colexectestutils.Tuples{c.inputTuples}, [][]*types.T{c.inputTypes}, c.outputTuples, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return colexectestutils.CreateTestProjectingOperator( ctx, flowCtx, input[0], c.inputTypes, c.cmpExpr, false /* canFallbackToRowexec */, testMemAcc, @@ -133,7 +133,7 @@ func BenchmarkDefaultCmpProjOp(b *testing.B) { for _, hasNulls := range []bool{false, true} { inputTypes := []*types.T{types.String, types.String} name := fmt.Sprintf("IS DISTINCT FROM/useSel=%t/hasNulls=%t", useSel, hasNulls) - benchmarkProjOp(b, name, func(source *colexecbase.RepeatableBatchSource) (colexecbase.Operator, error) { + benchmarkProjOp(b, name, func(source *colexecop.RepeatableBatchSource) (colexecop.Operator, error) { return colexectestutils.CreateTestProjectingOperator( ctx, flowCtx, source, inputTypes, "@1 IS DISTINCT FROM @2", false /* canFallbackToRowexec */, testMemAcc, diff --git a/pkg/sql/colexec/colexecproj/default_cmp_proj_ops.eg.go b/pkg/sql/colexec/colexecproj/default_cmp_proj_ops.eg.go index 73cd6a4970d4..515591b94ff1 100644 --- a/pkg/sql/colexec/colexecproj/default_cmp_proj_ops.eg.go +++ b/pkg/sql/colexec/colexecproj/default_cmp_proj_ops.eg.go @@ -15,8 +15,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" ) @@ -28,7 +28,7 @@ type defaultCmpProjOp struct { datumToVecConverter func(tree.Datum) interface{} } -var _ colexecbase.Operator = &defaultCmpProjOp{} +var _ colexecop.Operator = &defaultCmpProjOp{} func (d *defaultCmpProjOp) Init() { d.Input.Init() @@ -89,7 +89,7 @@ type defaultCmpRConstProjOp struct { datumToVecConverter func(tree.Datum) interface{} } -var _ colexecbase.Operator = &defaultCmpRConstProjOp{} +var _ colexecop.Operator = &defaultCmpRConstProjOp{} func (d *defaultCmpRConstProjOp) Init() { d.Input.Init() diff --git a/pkg/sql/colexec/colexecproj/default_cmp_proj_ops_tmpl.go b/pkg/sql/colexec/colexecproj/default_cmp_proj_ops_tmpl.go index 20698a034e70..5e6b824b6613 100644 --- a/pkg/sql/colexec/colexecproj/default_cmp_proj_ops_tmpl.go +++ b/pkg/sql/colexec/colexecproj/default_cmp_proj_ops_tmpl.go @@ -25,8 +25,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" ) @@ -45,7 +45,7 @@ type defaultCmp_KINDProjOp struct { datumToVecConverter func(tree.Datum) interface{} } -var _ colexecbase.Operator = &defaultCmp_KINDProjOp{} +var _ colexecop.Operator = &defaultCmp_KINDProjOp{} func (d *defaultCmp_KINDProjOp) Init() { d.Input.Init() diff --git a/pkg/sql/colexec/colexecproj/like_ops.go b/pkg/sql/colexec/colexecproj/like_ops.go index 7cbd89f9a8e0..3844e3e44a61 100644 --- a/pkg/sql/colexec/colexecproj/like_ops.go +++ b/pkg/sql/colexec/colexecproj/like_ops.go @@ -13,7 +13,7 @@ package colexecproj import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -26,12 +26,12 @@ import ( func GetLikeProjectionOperator( allocator *colmem.Allocator, ctx *tree.EvalContext, - input colexecbase.Operator, + input colexecop.Operator, colIdx int, resultIdx int, pattern string, negate bool, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { likeOpType, pattern, err := colexeccmp.GetLikeOperatorType(pattern, negate) if err != nil { return nil, err @@ -39,7 +39,7 @@ func GetLikeProjectionOperator( pat := []byte(pattern) input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.Bool, resultIdx) base := projConstOpBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, diff --git a/pkg/sql/colexec/colexecproj/proj_const_left_ops.eg.go b/pkg/sql/colexec/colexecproj/proj_const_left_ops.eg.go index b42d0e4d91ff..d52d4378c2f4 100644 --- a/pkg/sql/colexec/colexecproj/proj_const_left_ops.eg.go +++ b/pkg/sql/colexec/colexecproj/proj_const_left_ops.eg.go @@ -22,8 +22,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" @@ -28689,17 +28689,17 @@ func GetProjectionLConstOperator( constType *types.T, outputType *types.T, op tree.Operator, - input colexecbase.Operator, + input colexecop.Operator, colIdx int, constArg tree.Datum, outputIdx int, evalCtx *tree.EvalContext, binFn tree.TwoArgFn, cmpExpr *tree.ComparisonExpr, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { input = colexecutils.NewVectorTypeEnforcer(allocator, input, outputType, outputIdx) projConstOpBase := projConstOpBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: outputIdx, diff --git a/pkg/sql/colexec/colexecproj/proj_const_ops_tmpl.go b/pkg/sql/colexec/colexecproj/proj_const_ops_tmpl.go index 3f081e5a77db..a77e96505f71 100644 --- a/pkg/sql/colexec/colexecproj/proj_const_ops_tmpl.go +++ b/pkg/sql/colexec/colexecproj/proj_const_ops_tmpl.go @@ -31,8 +31,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" @@ -256,17 +256,17 @@ func GetProjection_CONST_SIDEConstOperator( constType *types.T, outputType *types.T, op tree.Operator, - input colexecbase.Operator, + input colexecop.Operator, colIdx int, constArg tree.Datum, outputIdx int, evalCtx *tree.EvalContext, binFn tree.TwoArgFn, cmpExpr *tree.ComparisonExpr, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { input = colexecutils.NewVectorTypeEnforcer(allocator, input, outputType, outputIdx) projConstOpBase := projConstOpBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: outputIdx, diff --git a/pkg/sql/colexec/colexecproj/proj_const_right_ops.eg.go b/pkg/sql/colexec/colexecproj/proj_const_right_ops.eg.go index 96805dc05eb3..7b0f6025702b 100644 --- a/pkg/sql/colexec/colexecproj/proj_const_right_ops.eg.go +++ b/pkg/sql/colexec/colexecproj/proj_const_right_ops.eg.go @@ -24,8 +24,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" @@ -57869,17 +57869,17 @@ func GetProjectionRConstOperator( constType *types.T, outputType *types.T, op tree.Operator, - input colexecbase.Operator, + input colexecop.Operator, colIdx int, constArg tree.Datum, outputIdx int, evalCtx *tree.EvalContext, binFn tree.TwoArgFn, cmpExpr *tree.ComparisonExpr, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { input = colexecutils.NewVectorTypeEnforcer(allocator, input, outputType, outputIdx) projConstOpBase := projConstOpBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: outputIdx, diff --git a/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go b/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go index b3408968d346..40b05a1fc8fd 100644 --- a/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go +++ b/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go @@ -22,8 +22,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" @@ -47,7 +47,7 @@ var ( // so, it'll be redeclared because we execute that template twice. To go // around the problem we specify it here. type projConstOpBase struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator colIdx int outputIdx int @@ -56,7 +56,7 @@ type projConstOpBase struct { // projOpBase contains all of the fields for non-constant projections. type projOpBase struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator col1Idx int col2Idx int @@ -60981,17 +60981,17 @@ func GetProjectionOperator( inputTypes []*types.T, outputType *types.T, op tree.Operator, - input colexecbase.Operator, + input colexecop.Operator, col1Idx int, col2Idx int, outputIdx int, evalCtx *tree.EvalContext, binFn tree.TwoArgFn, cmpExpr *tree.ComparisonExpr, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { input = colexecutils.NewVectorTypeEnforcer(allocator, input, outputType, outputIdx) projOpBase := projOpBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, col1Idx: col1Idx, col2Idx: col2Idx, diff --git a/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go b/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go index 014fbe63395f..49daf481a83b 100644 --- a/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go +++ b/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go @@ -30,8 +30,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" @@ -78,7 +78,7 @@ func _ASSIGN(_, _, _, _, _, _ interface{}) { // so, it'll be redeclared because we execute that template twice. To go // around the problem we specify it here. type projConstOpBase struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator colIdx int outputIdx int @@ -87,7 +87,7 @@ type projConstOpBase struct { // projOpBase contains all of the fields for non-constant projections. type projOpBase struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator col1Idx int col2Idx int @@ -251,17 +251,17 @@ func GetProjectionOperator( inputTypes []*types.T, outputType *types.T, op tree.Operator, - input colexecbase.Operator, + input colexecop.Operator, col1Idx int, col2Idx int, outputIdx int, evalCtx *tree.EvalContext, binFn tree.TwoArgFn, cmpExpr *tree.ComparisonExpr, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { input = colexecutils.NewVectorTypeEnforcer(allocator, input, outputType, outputIdx) projOpBase := projOpBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, col1Idx: col1Idx, col2Idx: col2Idx, diff --git a/pkg/sql/colexec/colexecproj/projection_ops_test.go b/pkg/sql/colexec/colexecproj/projection_ops_test.go index 49e2184d4eb3..21aa785e3f40 100644 --- a/pkg/sql/colexec/colexecproj/projection_ops_test.go +++ b/pkg/sql/colexec/colexecproj/projection_ops_test.go @@ -22,7 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -49,7 +49,7 @@ func TestProjPlusInt64Int64ConstOp(t *testing.T) { }, } colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{{{1}, {2}, {nil}}}, colexectestutils.Tuples{{1, 2}, {2, 3}, {nil, nil}}, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return colexectestutils.CreateTestProjectingOperator( ctx, flowCtx, input[0], []*types.T{types.Int}, "@1 + 1" /* projectingExpr */, false /* canFallbackToRowexec */, testMemAcc, @@ -71,7 +71,7 @@ func TestProjPlusInt64Int64Op(t *testing.T) { }, } colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{{{1, 2}, {3, 4}, {5, nil}}}, colexectestutils.Tuples{{1, 2, 3}, {3, 4, 7}, {5, nil, nil}}, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return colexectestutils.CreateTestProjectingOperator( ctx, flowCtx, input[0], []*types.T{types.Int, types.Int}, "@1 + @2" /* projectingExpr */, false /* canFallbackToRowexec */, testMemAcc, @@ -93,7 +93,7 @@ func TestProjDivFloat64Float64Op(t *testing.T) { }, } colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{{{1.0, 2.0}, {3.0, 4.0}, {5.0, nil}}}, colexectestutils.Tuples{{1.0, 2.0, 0.5}, {3.0, 4.0, 0.75}, {5.0, nil, nil}}, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return colexectestutils.CreateTestProjectingOperator( ctx, flowCtx, input[0], []*types.T{types.Float, types.Float}, "@1 / @2" /* projectingExpr */, false /* canFallbackToRowexec */, testMemAcc, @@ -105,7 +105,7 @@ func TestGetProjectionConstOperator(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) binOp := tree.Mult - var input colexecbase.Operator + var input colexecop.Operator colIdx := 3 inputTypes := make([]*types.T, colIdx+1) inputTypes[colIdx] = types.Float @@ -121,7 +121,7 @@ func TestGetProjectionConstOperator(t *testing.T) { } expected := &projMultFloat64Float64ConstOp{ projConstOpBase: projConstOpBase{ - OneInputNode: colexecbase.NewOneInputNode(op.(*projMultFloat64Float64ConstOp).Input), + OneInputNode: colexecop.NewOneInputNode(op.(*projMultFloat64Float64ConstOp).Input), allocator: testAllocator, colIdx: colIdx, outputIdx: outputIdx, @@ -137,7 +137,7 @@ func TestGetProjectionConstMixedTypeOperator(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) cmpOp := tree.GE - var input colexecbase.Operator + var input colexecop.Operator colIdx := 3 inputTypes := make([]*types.T, colIdx+1) inputTypes[colIdx] = types.Int @@ -153,7 +153,7 @@ func TestGetProjectionConstMixedTypeOperator(t *testing.T) { } expected := &projGEInt64Int16ConstOp{ projConstOpBase: projConstOpBase{ - OneInputNode: colexecbase.NewOneInputNode(op.(*projGEInt64Int16ConstOp).Input), + OneInputNode: colexecop.NewOneInputNode(op.(*projGEInt64Int16ConstOp).Input), allocator: testAllocator, colIdx: colIdx, outputIdx: outputIdx, @@ -265,7 +265,7 @@ func TestGetProjectionOperator(t *testing.T) { defer log.Scope(t).Close(t) typ := types.Int2 binOp := tree.Mult - var input colexecbase.Operator + var input colexecop.Operator col1Idx := 5 col2Idx := 7 inputTypes := make([]*types.T, col2Idx+1) @@ -281,7 +281,7 @@ func TestGetProjectionOperator(t *testing.T) { } expected := &projMultInt16Int16Op{ projOpBase: projOpBase{ - OneInputNode: colexecbase.NewOneInputNode(op.(*projMultInt16Int16Op).Input), + OneInputNode: colexecop.NewOneInputNode(op.(*projMultInt16Int16Op).Input), allocator: testAllocator, col1Idx: col1Idx, col2Idx: col2Idx, @@ -296,7 +296,7 @@ func TestGetProjectionOperator(t *testing.T) { func benchmarkProjOp( b *testing.B, name string, - makeProjOp func(source *colexecbase.RepeatableBatchSource) (colexecbase.Operator, error), + makeProjOp func(source *colexecop.RepeatableBatchSource) (colexecop.Operator, error), inputTypes []*types.T, useSelectionVector bool, hasNulls bool, @@ -329,7 +329,7 @@ func benchmarkProjOp( sel[i] = i } } - source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, inputTypes) + source := colexecop.NewRepeatableBatchSource(testAllocator, batch, inputTypes) op, err := makeProjOp(source) require.NoError(b, err) op.Init() @@ -379,7 +379,7 @@ func BenchmarkProjOp(b *testing.B) { inputTypes = inputTypes[:1] } name := fmt.Sprintf("proj%sInt64Int64%s/useSel=%t/hasNulls=%t", opName, kind, useSel, hasNulls) - benchmarkProjOp(b, name, func(source *colexecbase.RepeatableBatchSource) (colexecbase.Operator, error) { + benchmarkProjOp(b, name, func(source *colexecop.RepeatableBatchSource) (colexecop.Operator, error) { expr := fmt.Sprintf("@1 %s @2", opInfixForm) if rightConst { expr = fmt.Sprintf("@1 %s 2", opInfixForm) diff --git a/pkg/sql/colexec/colexecsel/BUILD.bazel b/pkg/sql/colexec/colexecsel/BUILD.bazel index 744d3c16be45..bb49f15e9e3c 100644 --- a/pkg/sql/colexec/colexecsel/BUILD.bazel +++ b/pkg/sql/colexec/colexecsel/BUILD.bazel @@ -1,5 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//pkg/sql/colexecbase:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") go_library( name = "colexecsel", @@ -16,8 +16,8 @@ go_library( "//pkg/sql/colconv", # keep "//pkg/sql/colexec/colexeccmp", "//pkg/sql/colexec/execgen", # keep - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", # keep + "//pkg/sql/colexecop", "//pkg/sql/sem/tree", "//pkg/sql/types", # keep "//pkg/util/duration", # keep @@ -40,8 +40,8 @@ go_test( "//pkg/col/coldataext", "//pkg/settings/cluster", "//pkg/sql/colexec/colexectestutils", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/sem/tree", diff --git a/pkg/sql/colexec/colexecsel/default_cmp_sel_ops.eg.go b/pkg/sql/colexec/colexecsel/default_cmp_sel_ops.eg.go index 9b37497cde18..ab6132635dfd 100644 --- a/pkg/sql/colexec/colexecsel/default_cmp_sel_ops.eg.go +++ b/pkg/sql/colexec/colexecsel/default_cmp_sel_ops.eg.go @@ -15,8 +15,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" ) @@ -27,7 +27,7 @@ type defaultCmpSelOp struct { toDatumConverter *colconv.VecToDatumConverter } -var _ colexecbase.Operator = &defaultCmpSelOp{} +var _ colexecop.Operator = &defaultCmpSelOp{} func (d *defaultCmpSelOp) Init() { d.Input.Init() @@ -83,7 +83,7 @@ type defaultCmpConstSelOp struct { toDatumConverter *colconv.VecToDatumConverter } -var _ colexecbase.Operator = &defaultCmpConstSelOp{} +var _ colexecop.Operator = &defaultCmpConstSelOp{} func (d *defaultCmpConstSelOp) Init() { d.Input.Init() diff --git a/pkg/sql/colexec/colexecsel/default_cmp_sel_ops_tmpl.go b/pkg/sql/colexec/colexecsel/default_cmp_sel_ops_tmpl.go index 54c9c523e657..2f8f34cc87c4 100644 --- a/pkg/sql/colexec/colexecsel/default_cmp_sel_ops_tmpl.go +++ b/pkg/sql/colexec/colexecsel/default_cmp_sel_ops_tmpl.go @@ -25,8 +25,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" ) @@ -44,7 +44,7 @@ type defaultCmp_KINDSelOp struct { toDatumConverter *colconv.VecToDatumConverter } -var _ colexecbase.Operator = &defaultCmp_KINDSelOp{} +var _ colexecop.Operator = &defaultCmp_KINDSelOp{} func (d *defaultCmp_KINDSelOp) Init() { d.Input.Init() diff --git a/pkg/sql/colexec/colexecsel/like_ops.go b/pkg/sql/colexec/colexecsel/like_ops.go index 08fab8f0d70f..655a467aea0a 100644 --- a/pkg/sql/colexec/colexecsel/like_ops.go +++ b/pkg/sql/colexec/colexecsel/like_ops.go @@ -12,7 +12,7 @@ package colexecsel import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/errors" ) @@ -21,15 +21,15 @@ import ( // pattern, or NOT LIKE if the negate argument is true. The implementation // varies depending on the complexity of the pattern. func GetLikeOperator( - ctx *tree.EvalContext, input colexecbase.Operator, colIdx int, pattern string, negate bool, -) (colexecbase.Operator, error) { + ctx *tree.EvalContext, input colexecop.Operator, colIdx int, pattern string, negate bool, +) (colexecop.Operator, error) { likeOpType, pattern, err := colexeccmp.GetLikeOperatorType(pattern, negate) if err != nil { return nil, err } pat := []byte(pattern) base := selConstOpBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, } switch likeOpType { diff --git a/pkg/sql/colexec/colexecsel/like_ops_test.go b/pkg/sql/colexec/colexecsel/like_ops_test.go index 32878b865afd..92fcc3b33752 100644 --- a/pkg/sql/colexec/colexecsel/like_ops_test.go +++ b/pkg/sql/colexec/colexecsel/like_ops_test.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -94,7 +94,7 @@ func TestLikeOperators(t *testing.T) { } { colexectestutils.RunTests( t, testAllocator, []colexectestutils.Tuples{tc.tups}, tc.expected, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { ctx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) return GetLikeOperator(&ctx, input[0], 0, tc.pattern, tc.negate) }) @@ -125,11 +125,11 @@ func BenchmarkLikeOps(b *testing.B) { } batch.SetLength(coldata.BatchSize()) - source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) source.Init() base := selConstOpBase{ - OneInputNode: colexecbase.NewOneInputNode(source), + OneInputNode: colexecop.NewOneInputNode(source), colIdx: 0, } prefixOp := &selPrefixBytesBytesConstOp{ @@ -152,7 +152,7 @@ func BenchmarkLikeOps(b *testing.B) { testCases := []struct { name string - op colexecbase.Operator + op colexecop.Operator }{ {name: "selPrefixBytesBytesConstOp", op: prefixOp}, {name: "selSuffixBytesBytesConstOp", op: suffixOp}, diff --git a/pkg/sql/colexec/colexecsel/selection_ops.eg.go b/pkg/sql/colexec/colexecsel/selection_ops.eg.go index ac1782155368..b92cd3462671 100644 --- a/pkg/sql/colexec/colexecsel/selection_ops.eg.go +++ b/pkg/sql/colexec/colexecsel/selection_ops.eg.go @@ -22,8 +22,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" @@ -40,14 +40,14 @@ var ( // selConstOpBase contains all of the fields for binary selections with a // constant, except for the constant itself. type selConstOpBase struct { - colexecbase.OneInputNode + colexecop.OneInputNode colIdx int overloadHelper execgen.OverloadHelper } // selOpBase contains all of the fields for non-constant binary selections. type selOpBase struct { - colexecbase.OneInputNode + colexecop.OneInputNode col1Idx int col2Idx int overloadHelper execgen.OverloadHelper @@ -58713,17 +58713,17 @@ func (p *selGEDatumDatumOp) Init() { // for the given left and right column types and comparison. func GetSelectionConstOperator( cmpOp tree.ComparisonOperator, - input colexecbase.Operator, + input colexecop.Operator, inputTypes []*types.T, colIdx int, constArg tree.Datum, evalCtx *tree.EvalContext, cmpExpr *tree.ComparisonExpr, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { leftType, constType := inputTypes[colIdx], constArg.ResolvedType() c := colconv.GetDatumToPhysicalFn(constType)(constArg) selConstOpBase := selConstOpBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, } if leftType.Family() != types.TupleFamily && constType.Family() != types.TupleFamily { @@ -59975,16 +59975,16 @@ func GetSelectionConstOperator( // for the given left and right column types and comparison. func GetSelectionOperator( cmpOp tree.ComparisonOperator, - input colexecbase.Operator, + input colexecop.Operator, inputTypes []*types.T, col1Idx int, col2Idx int, evalCtx *tree.EvalContext, cmpExpr *tree.ComparisonExpr, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { leftType, rightType := inputTypes[col1Idx], inputTypes[col2Idx] selOpBase := selOpBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), col1Idx: col1Idx, col2Idx: col2Idx, } diff --git a/pkg/sql/colexec/colexecsel/selection_ops_test.go b/pkg/sql/colexec/colexecsel/selection_ops_test.go index 5a149485364e..c9589fe400a4 100644 --- a/pkg/sql/colexec/colexecsel/selection_ops_test.go +++ b/pkg/sql/colexec/colexecsel/selection_ops_test.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -36,10 +36,10 @@ func TestSelLTInt64Int64ConstOp(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) tups := colexectestutils.Tuples{{0}, {1}, {2}, {nil}} - colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tups}, colexectestutils.Tuples{{0}, {1}}, colexectestutils.OrderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { + colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tups}, colexectestutils.Tuples{{0}, {1}}, colexectestutils.OrderedVerifier, func(input []colexecop.Operator) (colexecop.Operator, error) { return &selLTInt64Int64ConstOp{ selConstOpBase: selConstOpBase{ - OneInputNode: colexecbase.NewOneInputNode(input[0]), + OneInputNode: colexecop.NewOneInputNode(input[0]), colIdx: 0, }, constArg: 2, @@ -59,10 +59,10 @@ func TestSelLTInt64Int64(t *testing.T) { {-1, nil}, {nil, nil}, } - colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tups}, colexectestutils.Tuples{{0, 1}}, colexectestutils.OrderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { + colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tups}, colexectestutils.Tuples{{0, 1}}, colexectestutils.OrderedVerifier, func(input []colexecop.Operator) (colexecop.Operator, error) { return &selLTInt64Int64Op{ selOpBase: selOpBase{ - OneInputNode: colexecbase.NewOneInputNode(input[0]), + OneInputNode: colexecop.NewOneInputNode(input[0]), col1Idx: 0, col2Idx: 1, }, @@ -74,7 +74,7 @@ func TestGetSelectionConstOperator(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) cmpOp := tree.LT - var input colexecbase.Operator + var input colexecop.Operator colIdx := 3 inputTypes := make([]*types.T, colIdx+1) inputTypes[colIdx] = types.Date @@ -88,7 +88,7 @@ func TestGetSelectionConstOperator(t *testing.T) { } expected := &selLTInt64Int64ConstOp{ selConstOpBase: selConstOpBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, }, constArg: constVal, @@ -102,7 +102,7 @@ func TestGetSelectionConstMixedTypeOperator(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) cmpOp := tree.LT - var input colexecbase.Operator + var input colexecop.Operator colIdx := 3 inputTypes := make([]*types.T, colIdx+1) inputTypes[colIdx] = types.Int2 @@ -116,7 +116,7 @@ func TestGetSelectionConstMixedTypeOperator(t *testing.T) { } expected := &selLTInt16Int64ConstOp{ selConstOpBase: selConstOpBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, }, constArg: constVal, @@ -131,7 +131,7 @@ func TestGetSelectionOperator(t *testing.T) { defer log.Scope(t).Close(t) ct := types.Int2 cmpOp := tree.GE - var input colexecbase.Operator + var input colexecop.Operator col1Idx := 5 col2Idx := 7 inputTypes := make([]*types.T, col2Idx+1) @@ -145,7 +145,7 @@ func TestGetSelectionOperator(t *testing.T) { } expected := &selGEInt16Int16Op{ selOpBase: selOpBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), col1Idx: col1Idx, col2Idx: col2Idx, }, @@ -183,12 +183,12 @@ func benchmarkSelLTInt64Int64ConstOp(b *testing.B, useSelectionVector bool, hasN sel[i] = i } } - source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) source.Init() plusOp := &selLTInt64Int64ConstOp{ selConstOpBase: selConstOpBase{ - OneInputNode: colexecbase.NewOneInputNode(source), + OneInputNode: colexecop.NewOneInputNode(source), colIdx: 0, }, constArg: 0, @@ -244,12 +244,12 @@ func benchmarkSelLTInt64Int64Op(b *testing.B, useSelectionVector bool, hasNulls sel[i] = i } } - source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) source.Init() plusOp := &selLTInt64Int64Op{ selOpBase: selOpBase{ - OneInputNode: colexecbase.NewOneInputNode(source), + OneInputNode: colexecop.NewOneInputNode(source), col1Idx: 0, col2Idx: 1, }, diff --git a/pkg/sql/colexec/colexecsel/selection_ops_tmpl.go b/pkg/sql/colexec/colexecsel/selection_ops_tmpl.go index 9d4fccf03a39..06e3e3433f78 100644 --- a/pkg/sql/colexec/colexecsel/selection_ops_tmpl.go +++ b/pkg/sql/colexec/colexecsel/selection_ops_tmpl.go @@ -29,8 +29,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" @@ -173,14 +173,14 @@ func _SEL_LOOP(_HAS_NULLS bool) { // */}} // selConstOpBase contains all of the fields for binary selections with a // constant, except for the constant itself. type selConstOpBase struct { - colexecbase.OneInputNode + colexecop.OneInputNode colIdx int overloadHelper execgen.OverloadHelper } // selOpBase contains all of the fields for non-constant binary selections. type selOpBase struct { - colexecbase.OneInputNode + colexecop.OneInputNode col1Idx int col2Idx int overloadHelper execgen.OverloadHelper @@ -293,17 +293,17 @@ func (p *_OP_NAME) Init() { // for the given left and right column types and comparison. func GetSelectionConstOperator( cmpOp tree.ComparisonOperator, - input colexecbase.Operator, + input colexecop.Operator, inputTypes []*types.T, colIdx int, constArg tree.Datum, evalCtx *tree.EvalContext, cmpExpr *tree.ComparisonExpr, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { leftType, constType := inputTypes[colIdx], constArg.ResolvedType() c := colconv.GetDatumToPhysicalFn(constType)(constArg) selConstOpBase := selConstOpBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, } if leftType.Family() != types.TupleFamily && constType.Family() != types.TupleFamily { @@ -349,16 +349,16 @@ func GetSelectionConstOperator( // for the given left and right column types and comparison. func GetSelectionOperator( cmpOp tree.ComparisonOperator, - input colexecbase.Operator, + input colexecop.Operator, inputTypes []*types.T, col1Idx int, col2Idx int, evalCtx *tree.EvalContext, cmpExpr *tree.ComparisonExpr, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { leftType, rightType := inputTypes[col1Idx], inputTypes[col2Idx] selOpBase := selOpBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), col1Idx: col1Idx, col2Idx: col2Idx, } diff --git a/pkg/sql/colexec/colexectestutils/BUILD.bazel b/pkg/sql/colexec/colexectestutils/BUILD.bazel index 21ac1bd6ee02..f7add0bfc513 100644 --- a/pkg/sql/colexec/colexectestutils/BUILD.bazel +++ b/pkg/sql/colexec/colexectestutils/BUILD.bazel @@ -14,8 +14,8 @@ go_library( "//pkg/col/typeconv", "//pkg/settings/cluster", "//pkg/sql/colexec/colexecargs", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", @@ -51,7 +51,7 @@ go_test( "//pkg/col/coldataext", "//pkg/col/coldatatestutils", "//pkg/settings/cluster", - "//pkg/sql/colexecbase", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/sem/tree", diff --git a/pkg/sql/colexec/colexectestutils/proj_utils.go b/pkg/sql/colexec/colexectestutils/proj_utils.go index 46e03bd3576e..bc708e42a76e 100644 --- a/pkg/sql/colexec/colexectestutils/proj_utils.go +++ b/pkg/sql/colexec/colexectestutils/proj_utils.go @@ -15,7 +15,7 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -64,12 +64,12 @@ func (p *MockTypeContext) IndexedVarNodeFormatter(idx int) tree.NodeFormatter { func CreateTestProjectingOperator( ctx context.Context, flowCtx *execinfra.FlowCtx, - input colexecbase.Operator, + input colexecop.Operator, inputTypes []*types.T, projectingExpr string, canFallbackToRowexec bool, testMemAcc *mon.BoundAccount, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { expr, err := parser.ParseExpr(projectingExpr) if err != nil { return nil, err @@ -98,7 +98,7 @@ func CreateTestProjectingOperator( } args := &colexecargs.NewColOperatorArgs{ Spec: spec, - Inputs: []colexecbase.Operator{input}, + Inputs: []colexecop.Operator{input}, StreamingMemAccount: testMemAcc, } if canFallbackToRowexec { diff --git a/pkg/sql/colexec/colexectestutils/utils.go b/pkg/sql/colexec/colexectestutils/utils.go index 0bf0adb7de46..f9904c62931e 100644 --- a/pkg/sql/colexec/colexectestutils/utils.go +++ b/pkg/sql/colexec/colexectestutils/utils.go @@ -26,8 +26,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -244,7 +244,7 @@ func maybeHasNulls(b coldata.Batch) bool { // TestRunner is the signature of RunTestsWithTyps that can be used to // substitute it with RunTestsWithoutAllNullsInjection when applicable. -type TestRunner func(*testing.T, *colmem.Allocator, []Tuples, [][]*types.T, Tuples, VerifierType, func([]colexecbase.Operator) (colexecbase.Operator, error)) +type TestRunner func(*testing.T, *colmem.Allocator, []Tuples, [][]*types.T, Tuples, VerifierType, func([]colexecop.Operator) (colexecop.Operator, error)) // RunTests is a helper that automatically runs your tests with varied batch // sizes and with and without a random selection vector. @@ -258,7 +258,7 @@ func RunTests( tups []Tuples, expected Tuples, verifier VerifierType, - constructor func(inputs []colexecbase.Operator) (colexecbase.Operator, error), + constructor func(inputs []colexecop.Operator) (colexecop.Operator, error), ) { RunTestsWithTyps(t, allocator, tups, nil /* typs */, expected, verifier, constructor) } @@ -275,7 +275,7 @@ func RunTestsWithTyps( typs [][]*types.T, expected Tuples, verifier VerifierType, - constructor func(inputs []colexecbase.Operator) (colexecbase.Operator, error), + constructor func(inputs []colexecop.Operator) (colexecop.Operator, error), ) { RunTestsWithoutAllNullsInjection(t, allocator, tups, typs, expected, verifier, constructor) @@ -299,8 +299,8 @@ func RunTestsWithTyps( } } } - opConstructor := func(injectAllNulls bool) colexecbase.Operator { - inputSources := make([]colexecbase.Operator, len(tups)) + opConstructor := func(injectAllNulls bool) colexecop.Operator { + inputSources := make([]colexecop.Operator, len(tups)) var inputTypes []*types.T for i, tup := range tups { if typs != nil { @@ -357,12 +357,12 @@ func RunTestsWithTyps( } // closeIfCloser is a testing utility function that checks whether op is a -// colexecbase.Closer and closes it if so. +// colexecop.Closer and closes it if so. // // RunTests harness needs to do that once it is done with op. In non-test // setting, the closing happens at the end of the query execution. -func closeIfCloser(ctx context.Context, t *testing.T, op colexecbase.Operator) { - if c, ok := op.(colexecbase.Closer); ok { +func closeIfCloser(ctx context.Context, t *testing.T, op colexecop.Operator) { + if c, ok := op.(colexecop.Closer); ok { if err := c.Close(ctx); err != nil { t.Fatal(err) } @@ -372,7 +372,7 @@ func closeIfCloser(ctx context.Context, t *testing.T, op colexecbase.Operator) { // isOperatorChainResettable traverses the whole operator tree rooted at op and // returns true if all nodes are resetters. func isOperatorChainResettable(op execinfra.OpNode) bool { - if _, resettable := op.(colexecbase.ResettableOperator); !resettable { + if _, resettable := op.(colexecop.ResettableOperator); !resettable { return false } for i := 0; i < op.ChildCount(true /* verbose */); i++ { @@ -395,7 +395,7 @@ func RunTestsWithoutAllNullsInjection( typs [][]*types.T, expected Tuples, verifier VerifierType, - constructor func(inputs []colexecbase.Operator) (colexecbase.Operator, error), + constructor func(inputs []colexecop.Operator) (colexecop.Operator, error), ) { ctx := context.Background() verifyFn := (*OpTestOutput).VerifyAnyOrder @@ -407,7 +407,7 @@ func RunTestsWithoutAllNullsInjection( // vector or nulls info can be different and that is totally valid). skipVerifySelAndNullsResets = false } - RunTestsWithFn(t, allocator, tups, typs, func(t *testing.T, inputs []colexecbase.Operator) { + RunTestsWithFn(t, allocator, tups, typs, func(t *testing.T, inputs []colexecop.Operator) { op, err := constructor(inputs) if err != nil { t.Fatal(err) @@ -443,7 +443,7 @@ func RunTestsWithoutAllNullsInjection( inputTypes []*types.T ) for round := 0; round < 2; round++ { - inputSources := make([]colexecbase.Operator, len(tups)) + inputSources := make([]colexecop.Operator, len(tups)) for i, tup := range tups { if typs != nil { inputTypes = typs[i] @@ -502,7 +502,7 @@ func RunTestsWithoutAllNullsInjection( log.Info(ctx, "randomNullsInjection") // This test randomly injects nulls in the input tuples and ensures that // the operator doesn't panic. - inputSources := make([]colexecbase.Operator, len(tups)) + inputSources := make([]colexecop.Operator, len(tups)) var inputTypes []*types.T for i, tup := range tups { if typs != nil { @@ -542,7 +542,7 @@ func RunTestsWithFn( allocator *colmem.Allocator, tups []Tuples, typs [][]*types.T, - test func(t *testing.T, inputs []colexecbase.Operator), + test func(t *testing.T, inputs []colexecop.Operator), ) { // Run tests over batchSizes of 1, (sometimes) a batch size that is small but // greater than 1, and a full coldata.BatchSize(). @@ -557,7 +557,7 @@ func RunTestsWithFn( for _, batchSize := range batchSizes { for _, useSel := range []bool{false, true} { log.Infof(context.Background(), "batchSize=%d/sel=%t", batchSize, useSel) - inputSources := make([]colexecbase.Operator, len(tups)) + inputSources := make([]colexecop.Operator, len(tups)) var inputTypes []*types.T if useSel { for i, tup := range tups { @@ -590,11 +590,11 @@ func RunTestsWithFixedSel( tups []Tuples, typs []*types.T, sel []int, - test func(t *testing.T, inputs []colexecbase.Operator), + test func(t *testing.T, inputs []colexecop.Operator), ) { for _, batchSize := range []int{1, 2, 3, 16, 1024} { log.Infof(context.Background(), "batchSize=%d/fixedSel", batchSize) - inputSources := make([]colexecbase.Operator, len(tups)) + inputSources := make([]colexecop.Operator, len(tups)) for i, tup := range tups { inputSources[i] = NewOpFixedSelTestInput(allocator, sel, batchSize, tup, typs) } @@ -701,7 +701,7 @@ func extrapolateTypesFromTuples(tups Tuples) []*types.T { // t.Fatal(err) // } type opTestInput struct { - colexecbase.ZeroInputNode + colexecop.ZeroInputNode allocator *colmem.Allocator @@ -727,14 +727,14 @@ type opTestInput struct { injectRandomNulls bool } -var _ colexecbase.ResettableOperator = &opTestInput{} +var _ colexecop.ResettableOperator = &opTestInput{} // NewOpTestInput returns a new opTestInput with the given input tuples and the // given type schema. If typs is nil, the input tuples are translated into // types automatically, using simple rules (e.g. integers always become Int64). func NewOpTestInput( allocator *colmem.Allocator, batchSize int, tuples Tuples, typs []*types.T, -) colexecbase.Operator { +) colexecop.Operator { ret := &opTestInput{ allocator: allocator, batchSize: batchSize, @@ -919,7 +919,7 @@ func (s *opTestInput) Reset(context.Context) { } type opFixedSelTestInput struct { - colexecbase.ZeroInputNode + colexecop.ZeroInputNode allocator *colmem.Allocator @@ -936,14 +936,14 @@ type opFixedSelTestInput struct { idx int } -var _ colexecbase.ResettableOperator = &opFixedSelTestInput{} +var _ colexecop.ResettableOperator = &opFixedSelTestInput{} // NewOpFixedSelTestInput returns a new opFixedSelTestInput with the given // input tuples and selection vector. The input tuples are translated into // types automatically, using simple rules (e.g. integers always become Int64). func NewOpFixedSelTestInput( allocator *colmem.Allocator, sel []int, batchSize int, tuples Tuples, typs []*types.T, -) colexecbase.Operator { +) colexecop.Operator { ret := &opFixedSelTestInput{ allocator: allocator, batchSize: batchSize, @@ -1044,7 +1044,7 @@ func (s *opFixedSelTestInput) Reset(context.Context) { // OpTestOutput is a test verification struct that ensures its input batches // match some expected output tuples. type OpTestOutput struct { - colexecbase.OneInputNode + colexecop.OneInputNode expected Tuples evalCtx *tree.EvalContext @@ -1054,11 +1054,11 @@ type OpTestOutput struct { // NewOpTestOutput returns a new OpTestOutput, initialized with the given input // to verify that the output is exactly equal to the expected tuples. -func NewOpTestOutput(input colexecbase.Operator, expected Tuples) *OpTestOutput { +func NewOpTestOutput(input colexecop.Operator, expected Tuples) *OpTestOutput { input.Init() return &OpTestOutput{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), expected: expected, evalCtx: tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()), } @@ -1112,7 +1112,7 @@ func (r *OpTestOutput) next(ctx context.Context) Tuple { // Reset implements the Resetter interface. func (r *OpTestOutput) Reset(ctx context.Context) { - if r, ok := r.Input.(colexecbase.Resetter); ok { + if r, ok := r.Input.(colexecop.Resetter); ok { r.Reset(ctx) } r.curIdx = 0 @@ -1279,14 +1279,14 @@ func assertTuplesOrderedEqual(expected Tuples, actual Tuples, evalCtx *tree.Eval // FiniteBatchSource is an Operator that returns the same batch a specified // number of times. type FiniteBatchSource struct { - colexecbase.ZeroInputNode + colexecop.ZeroInputNode - repeatableBatch *colexecbase.RepeatableBatchSource + repeatableBatch *colexecop.RepeatableBatchSource usableCount int } -var _ colexecbase.Operator = &FiniteBatchSource{} +var _ colexecop.Operator = &FiniteBatchSource{} // NewFiniteBatchSource returns a new Operator initialized to return its input // batch a specified number of times. @@ -1294,7 +1294,7 @@ func NewFiniteBatchSource( allocator *colmem.Allocator, batch coldata.Batch, typs []*types.T, usableCount int, ) *FiniteBatchSource { return &FiniteBatchSource{ - repeatableBatch: colexecbase.NewRepeatableBatchSource(allocator, batch, typs), + repeatableBatch: colexecop.NewRepeatableBatchSource(allocator, batch, typs), usableCount: usableCount, } } @@ -1324,22 +1324,22 @@ func (f *FiniteBatchSource) Reset(usableCount int) { // (except for the first) the batch is returned to emulate source that is // already ordered on matchLen columns. type finiteChunksSource struct { - colexecbase.ZeroInputNode - repeatableBatch *colexecbase.RepeatableBatchSource + colexecop.ZeroInputNode + repeatableBatch *colexecop.RepeatableBatchSource usableCount int matchLen int adjustment []int64 } -var _ colexecbase.Operator = &finiteChunksSource{} +var _ colexecop.Operator = &finiteChunksSource{} // NewFiniteChunksSource returns a new finiteChunksSource. func NewFiniteChunksSource( allocator *colmem.Allocator, batch coldata.Batch, typs []*types.T, usableCount int, matchLen int, -) colexecbase.Operator { +) colexecop.Operator { return &finiteChunksSource{ - repeatableBatch: colexecbase.NewRepeatableBatchSource(allocator, batch, typs), + repeatableBatch: colexecop.NewRepeatableBatchSource(allocator, batch, typs), usableCount: usableCount, matchLen: matchLen, } @@ -1386,7 +1386,7 @@ func (f *finiteChunksSource) Next(ctx context.Context) coldata.Batch { // chunkingBatchSource is a batch source that takes unlimited-size columns and // chunks them into BatchSize()-sized chunks when Nexted. type chunkingBatchSource struct { - colexecbase.ZeroInputNode + colexecop.ZeroInputNode allocator *colmem.Allocator typs []*types.T cols []coldata.Vec @@ -1396,13 +1396,13 @@ type chunkingBatchSource struct { batch coldata.Batch } -var _ colexecbase.ResettableOperator = &chunkingBatchSource{} +var _ colexecop.ResettableOperator = &chunkingBatchSource{} // NewChunkingBatchSource returns a new chunkingBatchSource with the given // column types, columns, and length. func NewChunkingBatchSource( allocator *colmem.Allocator, typs []*types.T, cols []coldata.Vec, len int, -) colexecbase.ResettableOperator { +) colexecop.ResettableOperator { return &chunkingBatchSource{ allocator: allocator, typs: typs, diff --git a/pkg/sql/colexec/colexectestutils/utils_test.go b/pkg/sql/colexec/colexectestutils/utils_test.go index c215b2b0dac2..a76ce4c5916a 100644 --- a/pkg/sql/colexec/colexectestutils/utils_test.go +++ b/pkg/sql/colexec/colexectestutils/utils_test.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -34,7 +34,7 @@ func TestOpTestInputOutput(t *testing.T) { {1, 5, 0}, }, } - RunTestsWithFn(t, testAllocator, inputs, nil, func(t *testing.T, sources []colexecbase.Operator) { + RunTestsWithFn(t, testAllocator, inputs, nil, func(t *testing.T, sources []colexecop.Operator) { out := NewOpTestOutput(sources[0], inputs[0]) if err := out.Verify(); err != nil { @@ -53,7 +53,7 @@ func TestRepeatableBatchSource(t *testing.T) { batchLen = coldata.BatchSize() } batch.SetLength(batchLen) - input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + input := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) b := input.Next(context.Background()) b.SetLength(0) @@ -83,7 +83,7 @@ func TestRepeatableBatchSourceWithFixedSel(t *testing.T) { batch.SetLength(batchLen) batch.SetSelection(true) copy(batch.Selection(), sel) - input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + input := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) b := input.Next(context.Background()) b.SetLength(0) diff --git a/pkg/sql/colexec/colexecutils/BUILD.bazel b/pkg/sql/colexec/colexecutils/BUILD.bazel index 00088406333c..501a93394bf0 100644 --- a/pkg/sql/colexec/colexecutils/BUILD.bazel +++ b/pkg/sql/colexec/colexecutils/BUILD.bazel @@ -15,8 +15,8 @@ go_library( deps = [ "//pkg/col/coldata", "//pkg/sql/colcontainer", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/types", "//pkg/util", @@ -46,8 +46,8 @@ go_test( "//pkg/settings/cluster", "//pkg/sql/colcontainer", "//pkg/sql/colexec/colexectestutils", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/sem/tree", diff --git a/pkg/sql/colexec/colexecutils/bool_vec_to_sel.go b/pkg/sql/colexec/colexecutils/bool_vec_to_sel.go index 3d64ddbb42f3..61d2b9d5d44b 100644 --- a/pkg/sql/colexec/colexecutils/bool_vec_to_sel.go +++ b/pkg/sql/colexec/colexecutils/bool_vec_to_sel.go @@ -14,7 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" ) @@ -22,8 +22,8 @@ import ( // BoolOrUnknownToSelOp plans an infrastructure necessary to convert a column // of either Bool or Unknown type into a selection vector on the input batches. func BoolOrUnknownToSelOp( - input colexecbase.Operator, typs []*types.T, vecIdx int, -) (colexecbase.Operator, error) { + input colexecop.Operator, typs []*types.T, vecIdx int, +) (colexecop.Operator, error) { switch typs[vecIdx].Family() { case types.BoolFamily: return NewBoolVecToSelOp(input, vecIdx), nil @@ -40,17 +40,17 @@ func BoolOrUnknownToSelOp( // BoolVecToSelOp transforms a boolean column into a selection vector by adding // an index to the selection for each true value in the boolean column. type BoolVecToSelOp struct { - colexecbase.OneInputNode - colexecbase.NonExplainable + colexecop.OneInputNode + colexecop.NonExplainable // OutputCol is the boolean output column. It should be shared by other // operators that write to it. OutputCol []bool } -var _ colexecbase.ResettableOperator = &BoolVecToSelOp{} +var _ colexecop.ResettableOperator = &BoolVecToSelOp{} -// Next implements the colexecbase.Operator interface. +// Next implements the colexecop.Operator interface. func (p *BoolVecToSelOp) Next(ctx context.Context) coldata.Batch { // Loop until we have non-zero amount of output to return, or our input's been // exhausted. @@ -104,14 +104,14 @@ func (p *BoolVecToSelOp) Next(ctx context.Context) coldata.Batch { } } -// Init implements the colexecbase.Operator interface. +// Init implements the colexecop.Operator interface. func (p *BoolVecToSelOp) Init() { p.Input.Init() } -// Reset implements the colexecbase.Resetter interface. +// Reset implements the colexecop.Resetter interface. func (p *BoolVecToSelOp) Reset(ctx context.Context) { - if r, ok := p.Input.(colexecbase.Resetter); ok { + if r, ok := p.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -125,9 +125,9 @@ func (p *BoolVecToSelOp) Reset(ctx context.Context) { // // NOTE: if the column can be of a type other than boolean, // BoolOrUnknownToSelOp *must* be used instead. -func NewBoolVecToSelOp(input colexecbase.Operator, colIdx int) colexecbase.Operator { - d := selBoolOp{OneInputNode: colexecbase.NewOneInputNode(input), colIdx: colIdx} - ret := &BoolVecToSelOp{OneInputNode: colexecbase.NewOneInputNode(&d)} +func NewBoolVecToSelOp(input colexecop.Operator, colIdx int) colexecop.Operator { + d := selBoolOp{OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx} + ret := &BoolVecToSelOp{OneInputNode: colexecop.NewOneInputNode(&d)} d.boolVecToSelOp = ret return ret } @@ -135,8 +135,8 @@ func NewBoolVecToSelOp(input colexecbase.Operator, colIdx int) colexecbase.Opera // selBoolOp is a small helper operator that transforms a BoolVecToSelOp into // an operator that can see the inside of its input batch for NewBoolVecToSelOp. type selBoolOp struct { - colexecbase.OneInputNode - colexecbase.NonExplainable + colexecop.OneInputNode + colexecop.NonExplainable boolVecToSelOp *BoolVecToSelOp colIdx int } diff --git a/pkg/sql/colexec/colexecutils/bool_vec_to_sel_test.go b/pkg/sql/colexec/colexecutils/bool_vec_to_sel_test.go index ff06f2abc3d1..b06a3d13b41f 100644 --- a/pkg/sql/colexec/colexecutils/bool_vec_to_sel_test.go +++ b/pkg/sql/colexec/colexecutils/bool_vec_to_sel_test.go @@ -14,7 +14,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -34,7 +34,7 @@ func TestBoolVecToSelOp(t *testing.T) { }, } for _, tc := range tcs { - colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.expected, colexectestutils.OrderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { + colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.expected, colexectestutils.OrderedVerifier, func(input []colexecop.Operator) (colexecop.Operator, error) { return NewBoolVecToSelOp(input[0], 0), nil }) } diff --git a/pkg/sql/colexec/colexecutils/cancel_checker.go b/pkg/sql/colexec/colexecutils/cancel_checker.go index 4ef6461526b8..4b8341d36fbe 100644 --- a/pkg/sql/colexec/colexecutils/cancel_checker.go +++ b/pkg/sql/colexec/colexecutils/cancel_checker.go @@ -14,16 +14,16 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/util/cancelchecker" ) // CancelChecker is an Operator that checks whether query cancellation has // occurred. The check happens on every batch. type CancelChecker struct { - colexecbase.OneInputNode - colexecbase.NonExplainable + colexecop.OneInputNode + colexecop.NonExplainable // Number of times check() has been called since last context cancellation // check. @@ -35,11 +35,11 @@ func (c *CancelChecker) Init() { c.Input.Init() } -var _ colexecbase.Operator = &CancelChecker{} +var _ colexecop.Operator = &CancelChecker{} // NewCancelChecker creates a new CancelChecker. -func NewCancelChecker(op colexecbase.Operator) *CancelChecker { - return &CancelChecker{OneInputNode: colexecbase.NewOneInputNode(op)} +func NewCancelChecker(op colexecop.Operator) *CancelChecker { + return &CancelChecker{OneInputNode: colexecop.NewOneInputNode(op)} } // Next is part of Operator interface. diff --git a/pkg/sql/colexec/colexecutils/cancel_checker_test.go b/pkg/sql/colexec/colexecutils/cancel_checker_test.go index 06d2baf5fa03..5329a06265b4 100644 --- a/pkg/sql/colexec/colexecutils/cancel_checker_test.go +++ b/pkg/sql/colexec/colexecutils/cancel_checker_test.go @@ -14,8 +14,8 @@ import ( "context" "testing" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/cancelchecker" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -32,7 +32,7 @@ func TestCancelChecker(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) typs := []*types.T{types.Int} batch := testAllocator.NewMemBatchWithMaxCapacity(typs) - op := NewCancelChecker(colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs)) + op := NewCancelChecker(colexecop.NewRepeatableBatchSource(testAllocator, batch, typs)) cancel() err := colexecerror.CatchVectorizedRuntimeError(func() { op.Next(ctx) diff --git a/pkg/sql/colexec/colexecutils/deselector.go b/pkg/sql/colexec/colexecutils/deselector.go index 27e2202b78cb..fb6f1db1dec2 100644 --- a/pkg/sql/colexec/colexecutils/deselector.go +++ b/pkg/sql/colexec/colexecutils/deselector.go @@ -15,7 +15,7 @@ import ( "math" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -25,23 +25,23 @@ import ( // or omitted according to the selection vector). If the batches come with no // selection vector, it is a noop. type deselectorOp struct { - colexecbase.OneInputNode - colexecbase.NonExplainable + colexecop.OneInputNode + colexecop.NonExplainable allocator *colmem.Allocator inputTypes []*types.T output coldata.Batch } -var _ colexecbase.Operator = &deselectorOp{} +var _ colexecop.Operator = &deselectorOp{} // NewDeselectorOp creates a new deselector operator on the given input // operator with the given column types. func NewDeselectorOp( - allocator *colmem.Allocator, input colexecbase.Operator, typs []*types.T, -) colexecbase.Operator { + allocator *colmem.Allocator, input colexecop.Operator, typs []*types.T, +) colexecop.Operator { return &deselectorOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, inputTypes: typs, } diff --git a/pkg/sql/colexec/colexecutils/deselector_test.go b/pkg/sql/colexec/colexecutils/deselector_test.go index f0f3cbba50b5..2cbbab3b98af 100644 --- a/pkg/sql/colexec/colexecutils/deselector_test.go +++ b/pkg/sql/colexec/colexecutils/deselector_test.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -67,7 +67,7 @@ func TestDeselector(t *testing.T) { } for _, tc := range tcs { - colexectestutils.RunTestsWithFixedSel(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.typs, tc.sel, func(t *testing.T, input []colexecbase.Operator) { + colexectestutils.RunTestsWithFixedSel(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.typs, tc.sel, func(t *testing.T, input []colexecop.Operator) { op := NewDeselectorOp(testAllocator, input[0], tc.typs) out := colexectestutils.NewOpTestOutput(op, tc.expected) @@ -108,7 +108,7 @@ func BenchmarkDeselector(b *testing.B) { batch.SetSelection(true) copy(batch.Selection(), sel) batch.SetLength(batchLen) - input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, inputTypes) + input := colexecop.NewRepeatableBatchSource(testAllocator, batch, inputTypes) op := NewDeselectorOp(testAllocator, input, inputTypes) op.Init() b.ResetTimer() diff --git a/pkg/sql/colexec/colexecutils/operator.go b/pkg/sql/colexec/colexecutils/operator.go index 4f62b0c65f4d..914be3725623 100644 --- a/pkg/sql/colexec/colexecutils/operator.go +++ b/pkg/sql/colexec/colexecutils/operator.go @@ -14,23 +14,23 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" ) type zeroOperator struct { - colexecbase.OneInputNode - colexecbase.NonExplainable + colexecop.OneInputNode + colexecop.NonExplainable } -var _ colexecbase.Operator = &zeroOperator{} +var _ colexecop.Operator = &zeroOperator{} // NewZeroOp creates a new operator which just returns an empty batch. -func NewZeroOp(input colexecbase.Operator) colexecbase.Operator { - return &zeroOperator{OneInputNode: colexecbase.NewOneInputNode(input)} +func NewZeroOp(input colexecop.Operator) colexecop.Operator { + return &zeroOperator{OneInputNode: colexecop.NewOneInputNode(input)} } func (s *zeroOperator) Init() { @@ -42,18 +42,18 @@ func (s *zeroOperator) Next(context.Context) coldata.Batch { } type fixedNumTuplesNoInputOp struct { - colexecbase.ZeroInputNode - colexecbase.NonExplainable + colexecop.ZeroInputNode + colexecop.NonExplainable batch coldata.Batch numTuplesLeft int } -var _ colexecbase.Operator = &fixedNumTuplesNoInputOp{} +var _ colexecop.Operator = &fixedNumTuplesNoInputOp{} // NewFixedNumTuplesNoInputOp creates a new Operator which returns batches with // no actual columns that have specified number of tuples as the sum of their // lengths. -func NewFixedNumTuplesNoInputOp(allocator *colmem.Allocator, numTuples int) colexecbase.Operator { +func NewFixedNumTuplesNoInputOp(allocator *colmem.Allocator, numTuples int) colexecop.Operator { capacity := numTuples if capacity > coldata.BatchSize() { capacity = coldata.BatchSize() @@ -106,22 +106,22 @@ func (s *fixedNumTuplesNoInputOp) Next(context.Context) coldata.Batch { // --------------------- in column at position of N+1) // type vectorTypeEnforcer struct { - colexecbase.OneInputCloserHelper - colexecbase.NonExplainable + colexecop.OneInputCloserHelper + colexecop.NonExplainable allocator *colmem.Allocator typ *types.T idx int } -var _ colexecbase.ResettableOperator = &vectorTypeEnforcer{} +var _ colexecop.ResettableOperator = &vectorTypeEnforcer{} // NewVectorTypeEnforcer returns a new vectorTypeEnforcer. func NewVectorTypeEnforcer( - allocator *colmem.Allocator, input colexecbase.Operator, typ *types.T, idx int, -) colexecbase.Operator { + allocator *colmem.Allocator, input colexecop.Operator, typ *types.T, idx int, +) colexecop.Operator { return &vectorTypeEnforcer{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, typ: typ, idx: idx, @@ -142,7 +142,7 @@ func (e *vectorTypeEnforcer) Next(ctx context.Context) coldata.Batch { } func (e *vectorTypeEnforcer) Reset(ctx context.Context) { - if r, ok := e.Input.(colexecbase.Resetter); ok { + if r, ok := e.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -161,15 +161,15 @@ func (e *vectorTypeEnforcer) Reset(ctx context.Context) { // NOTE: the type schema passed into BatchSchemaSubsetEnforcer *must* include // the output type of the Operator that the enforcer will be the input to. type BatchSchemaSubsetEnforcer struct { - colexecbase.OneInputCloserHelper - colexecbase.NonExplainable + colexecop.OneInputCloserHelper + colexecop.NonExplainable allocator *colmem.Allocator typs []*types.T subsetStartIdx, subsetEndIdx int } -var _ colexecbase.Operator = &BatchSchemaSubsetEnforcer{} +var _ colexecop.Operator = &BatchSchemaSubsetEnforcer{} // NewBatchSchemaSubsetEnforcer creates a new BatchSchemaSubsetEnforcer. // - subsetStartIdx and subsetEndIdx define the boundaries of the range of @@ -177,12 +177,12 @@ var _ colexecbase.Operator = &BatchSchemaSubsetEnforcer{} // own. func NewBatchSchemaSubsetEnforcer( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, typs []*types.T, subsetStartIdx, subsetEndIdx int, ) *BatchSchemaSubsetEnforcer { return &BatchSchemaSubsetEnforcer{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), allocator: allocator, typs: typs, subsetStartIdx: subsetStartIdx, @@ -190,7 +190,7 @@ func NewBatchSchemaSubsetEnforcer( } } -// Init implements the colexecbase.Operator interface. +// Init implements the colexecop.Operator interface. func (e *BatchSchemaSubsetEnforcer) Init() { e.Input.Init() if e.subsetStartIdx >= e.subsetEndIdx { @@ -198,7 +198,7 @@ func (e *BatchSchemaSubsetEnforcer) Init() { } } -// Next implements the colexecbase.Operator interface. +// Next implements the colexecop.Operator interface. func (e *BatchSchemaSubsetEnforcer) Next(ctx context.Context) coldata.Batch { b := e.Input.Next(ctx) if b.Length() == 0 { diff --git a/pkg/sql/colexec/colexecutils/spilling_queue_test.go b/pkg/sql/colexec/colexecutils/spilling_queue_test.go index 3c19a43c91d4..cea0bde12611 100644 --- a/pkg/sql/colexec/colexecutils/spilling_queue_test.go +++ b/pkg/sql/colexec/colexecutils/spilling_queue_test.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils/colcontainerutils" @@ -125,7 +125,7 @@ func TestSpillingQueue(t *testing.T) { Types: typs, MemoryLimit: memoryLimit, DiskQueueCfg: queueCfg, - FDSemaphore: colexecbase.NewTestingSemaphore(2), + FDSemaphore: colexecop.NewTestingSemaphore(2), DiskAcc: testDiskAcc, }, ) @@ -136,7 +136,7 @@ func TestSpillingQueue(t *testing.T) { Types: typs, MemoryLimit: memoryLimit, DiskQueueCfg: queueCfg, - FDSemaphore: colexecbase.NewTestingSemaphore(2), + FDSemaphore: colexecop.NewTestingSemaphore(2), DiskAcc: testDiskAcc, }, ) @@ -292,7 +292,7 @@ func TestSpillingQueueDidntSpill(t *testing.T) { Types: typs, MemoryLimit: memoryLimit, DiskQueueCfg: queueCfg, - FDSemaphore: colexecbase.NewTestingSemaphore(2), + FDSemaphore: colexecop.NewTestingSemaphore(2), DiskAcc: testDiskAcc, }, ) @@ -360,7 +360,7 @@ func TestSpillingQueueMemoryAccounting(t *testing.T) { Types: typs, MemoryLimit: defaultMemoryLimit, DiskQueueCfg: queueCfg, - FDSemaphore: colexecbase.NewTestingSemaphore(2), + FDSemaphore: colexecop.NewTestingSemaphore(2), DiskAcc: testDiskAcc, } var q *SpillingQueue @@ -463,7 +463,7 @@ func TestSpillingQueueMovingTailWhenSpilling(t *testing.T) { Types: typs, MemoryLimit: memoryLimit, DiskQueueCfg: queueCfg, - FDSemaphore: colexecbase.NewTestingSemaphore(2), + FDSemaphore: colexecop.NewTestingSemaphore(2), DiskAcc: testDiskAcc, } q := NewSpillingQueue(newQueueArgs) diff --git a/pkg/sql/colexec/colexecwindow/BUILD.bazel b/pkg/sql/colexec/colexecwindow/BUILD.bazel index 659a4f977288..759510c1e380 100644 --- a/pkg/sql/colexec/colexecwindow/BUILD.bazel +++ b/pkg/sql/colexec/colexecwindow/BUILD.bazel @@ -1,5 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//pkg/sql/colexecbase:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") go_library( name = "colexecwindow", @@ -15,8 +15,8 @@ go_library( "//pkg/sql/colcontainer", # keep "//pkg/sql/colexec/colexecmisc", "//pkg/sql/colexec/colexecutils", # keep - "//pkg/sql/colexecbase", # keep "//pkg/sql/colexecerror", # keep + "//pkg/sql/colexecop", # keep "//pkg/sql/colmem", # keep "//pkg/sql/execinfrapb", # keep "//pkg/sql/sem/tree", # keep @@ -43,8 +43,8 @@ go_test( "//pkg/sql/colexec/colbuilder", "//pkg/sql/colexec/colexecargs", "//pkg/sql/colexec/colexectestutils", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", diff --git a/pkg/sql/colexec/colexecwindow/partitioner.go b/pkg/sql/colexec/colexecwindow/partitioner.go index f7813097827b..322d79dd5eea 100644 --- a/pkg/sql/colexec/colexecwindow/partitioner.go +++ b/pkg/sql/colexec/colexecwindow/partitioner.go @@ -16,26 +16,26 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecmisc" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" ) -// NewWindowSortingPartitioner creates a new colexecbase.Operator that orders +// NewWindowSortingPartitioner creates a new colexecop.Operator that orders // input first based on the partitionIdxs columns and second on ordCols (i.e. it // handles both PARTITION BY and ORDER BY clauses of a window function) and puts // true in partitionColIdx'th column (which is appended if needed) for every // tuple that is the first within its partition. func NewWindowSortingPartitioner( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, inputTyps []*types.T, partitionIdxs []uint32, ordCols []execinfrapb.Ordering_Column, partitionColIdx int, - createDiskBackedSorter func(input colexecbase.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column) (colexecbase.Operator, error), -) (op colexecbase.Operator, err error) { + createDiskBackedSorter func(input colexecop.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column) (colexecop.Operator, error), +) (op colexecop.Operator, err error) { partitionAndOrderingCols := make([]execinfrapb.Ordering_Column, len(partitionIdxs)+len(ordCols)) for i, idx := range partitionIdxs { partitionAndOrderingCols[i] = execinfrapb.Ordering_Column{ColIdx: idx} @@ -54,7 +54,7 @@ func NewWindowSortingPartitioner( input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.Bool, partitionColIdx) return &windowSortingPartitioner{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, distinctCol: distinctCol, partitionColIdx: partitionColIdx, @@ -62,7 +62,7 @@ func NewWindowSortingPartitioner( } type windowSortingPartitioner struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator // distinctCol is the output column of the chain of ordered distinct diff --git a/pkg/sql/colexec/colexecwindow/rank.eg.go b/pkg/sql/colexec/colexecwindow/rank.eg.go index 7041cd1f6cb7..2b1e8789e71b 100644 --- a/pkg/sql/colexec/colexecwindow/rank.eg.go +++ b/pkg/sql/colexec/colexecwindow/rank.eg.go @@ -15,8 +15,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecmisc" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -35,19 +35,19 @@ var _ = colexecerror.InternalError // output (if there is no such column, a new column is appended). func NewRankOperator( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, windowFn execinfrapb.WindowerSpec_WindowFunc, orderingCols []execinfrapb.Ordering_Column, outputColIdx int, partitionColIdx int, peersColIdx int, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { if len(orderingCols) == 0 { return colexecmisc.NewConstOp(allocator, input, types.Int, int64(1), outputColIdx) } input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.Int, outputColIdx) initFields := rankInitFields{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, outputColIdx: outputColIdx, partitionColIdx: partitionColIdx, @@ -70,7 +70,7 @@ func NewRankOperator( } type rankInitFields struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator outputColIdx int @@ -88,7 +88,7 @@ type rankNoPartitionOp struct { rankIncrement int64 } -var _ colexecbase.Operator = &rankNoPartitionOp{} +var _ colexecop.Operator = &rankNoPartitionOp{} func (r *rankNoPartitionOp) Init() { r.Input.Init() @@ -155,7 +155,7 @@ type rankWithPartitionOp struct { rankIncrement int64 } -var _ colexecbase.Operator = &rankWithPartitionOp{} +var _ colexecop.Operator = &rankWithPartitionOp{} func (r *rankWithPartitionOp) Init() { r.Input.Init() @@ -241,7 +241,7 @@ type denseRankNoPartitionOp struct { rankIncrement int64 } -var _ colexecbase.Operator = &denseRankNoPartitionOp{} +var _ colexecop.Operator = &denseRankNoPartitionOp{} func (r *denseRankNoPartitionOp) Init() { r.Input.Init() @@ -306,7 +306,7 @@ type denseRankWithPartitionOp struct { rankIncrement int64 } -var _ colexecbase.Operator = &denseRankWithPartitionOp{} +var _ colexecop.Operator = &denseRankWithPartitionOp{} func (r *denseRankWithPartitionOp) Init() { r.Input.Init() diff --git a/pkg/sql/colexec/colexecwindow/rank_tmpl.go b/pkg/sql/colexec/colexecwindow/rank_tmpl.go index 815835b196bd..a4aa7620a089 100644 --- a/pkg/sql/colexec/colexecwindow/rank_tmpl.go +++ b/pkg/sql/colexec/colexecwindow/rank_tmpl.go @@ -25,8 +25,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecmisc" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -45,19 +45,19 @@ var _ = colexecerror.InternalError // output (if there is no such column, a new column is appended). func NewRankOperator( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, windowFn execinfrapb.WindowerSpec_WindowFunc, orderingCols []execinfrapb.Ordering_Column, outputColIdx int, partitionColIdx int, peersColIdx int, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { if len(orderingCols) == 0 { return colexecmisc.NewConstOp(allocator, input, types.Int, int64(1), outputColIdx) } input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.Int, outputColIdx) initFields := rankInitFields{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, outputColIdx: outputColIdx, partitionColIdx: partitionColIdx, @@ -96,7 +96,7 @@ func _UPDATE_RANK_INCREMENT() { // */}} type rankInitFields struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator outputColIdx int @@ -154,7 +154,7 @@ type _RANK_STRINGOp struct { rankIncrement int64 } -var _ colexecbase.Operator = &_RANK_STRINGOp{} +var _ colexecop.Operator = &_RANK_STRINGOp{} func (r *_RANK_STRINGOp) Init() { r.Input.Init() diff --git a/pkg/sql/colexec/colexecwindow/relative_rank.eg.go b/pkg/sql/colexec/colexecwindow/relative_rank.eg.go index 596e1cba063b..3450f724c55f 100644 --- a/pkg/sql/colexec/colexecwindow/relative_rank.eg.go +++ b/pkg/sql/colexec/colexecwindow/relative_rank.eg.go @@ -16,8 +16,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecmisc" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -38,7 +38,7 @@ func NewRelativeRankOperator( memoryLimit int64, diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, - input colexecbase.Operator, + input colexecop.Operator, inputTypes []*types.T, windowFn execinfrapb.WindowerSpec_WindowFunc, orderingCols []execinfrapb.Ordering_Column, @@ -46,7 +46,7 @@ func NewRelativeRankOperator( partitionColIdx int, peersColIdx int, diskAcc *mon.BoundAccount, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { if len(orderingCols) == 0 { constValue := float64(0) if windowFn == execinfrapb.WindowerSpec_CUME_DIST { @@ -56,7 +56,7 @@ func NewRelativeRankOperator( } rrInitFields := relativeRankInitFields{ rankInitFields: rankInitFields{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: unlimitedAllocator, outputColIdx: outputColIdx, partitionColIdx: partitionColIdx, @@ -98,7 +98,7 @@ func NewRelativeRankOperator( // clause (or both) is used - we need 3 FDs for each of the spilling queues used // by the operator directly plus we use an external sort to handle PARTITION BY // and/or ORDER BY clauses. -const relativeRankNumRequiredFDs = 3 + colexecbase.ExternalSorterMinPartitions +const relativeRankNumRequiredFDs = 3 + colexecop.ExternalSorterMinPartitions // NOTE: in the context of window functions "partitions" mean a different thing // from "partition" in the context of external algorithms and some disk @@ -131,7 +131,7 @@ const ( type relativeRankInitFields struct { rankInitFields - colexecbase.CloserHelper + colexecop.CloserHelper state relativeRankState memoryLimit int64 @@ -180,7 +180,7 @@ type percentRankNoPartitionOp struct { output coldata.Batch } -var _ colexecbase.ClosableOperator = &percentRankNoPartitionOp{} +var _ colexecop.ClosableOperator = &percentRankNoPartitionOp{} func (r *percentRankNoPartitionOp) Init() { r.Input.Init() @@ -382,7 +382,7 @@ type percentRankWithPartitionOp struct { output coldata.Batch } -var _ colexecbase.ClosableOperator = &percentRankWithPartitionOp{} +var _ colexecop.ClosableOperator = &percentRankWithPartitionOp{} func (r *percentRankWithPartitionOp) Init() { r.Input.Init() @@ -689,7 +689,7 @@ type cumeDistNoPartitionOp struct { output coldata.Batch } -var _ colexecbase.ClosableOperator = &cumeDistNoPartitionOp{} +var _ colexecop.ClosableOperator = &cumeDistNoPartitionOp{} func (r *cumeDistNoPartitionOp) Init() { r.Input.Init() @@ -981,7 +981,7 @@ type cumeDistWithPartitionOp struct { output coldata.Batch } -var _ colexecbase.ClosableOperator = &cumeDistWithPartitionOp{} +var _ colexecop.ClosableOperator = &cumeDistWithPartitionOp{} func (r *cumeDistWithPartitionOp) Init() { r.Input.Init() diff --git a/pkg/sql/colexec/colexecwindow/relative_rank_tmpl.go b/pkg/sql/colexec/colexecwindow/relative_rank_tmpl.go index 55dc1dd7a86d..8bf1381c65b8 100644 --- a/pkg/sql/colexec/colexecwindow/relative_rank_tmpl.go +++ b/pkg/sql/colexec/colexecwindow/relative_rank_tmpl.go @@ -26,8 +26,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecmisc" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -48,7 +48,7 @@ func NewRelativeRankOperator( memoryLimit int64, diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, - input colexecbase.Operator, + input colexecop.Operator, inputTypes []*types.T, windowFn execinfrapb.WindowerSpec_WindowFunc, orderingCols []execinfrapb.Ordering_Column, @@ -56,7 +56,7 @@ func NewRelativeRankOperator( partitionColIdx int, peersColIdx int, diskAcc *mon.BoundAccount, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { if len(orderingCols) == 0 { constValue := float64(0) if windowFn == execinfrapb.WindowerSpec_CUME_DIST { @@ -66,7 +66,7 @@ func NewRelativeRankOperator( } rrInitFields := relativeRankInitFields{ rankInitFields: rankInitFields{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: unlimitedAllocator, outputColIdx: outputColIdx, partitionColIdx: partitionColIdx, @@ -108,7 +108,7 @@ func NewRelativeRankOperator( // clause (or both) is used - we need 3 FDs for each of the spilling queues used // by the operator directly plus we use an external sort to handle PARTITION BY // and/or ORDER BY clauses. -const relativeRankNumRequiredFDs = 3 + colexecbase.ExternalSorterMinPartitions +const relativeRankNumRequiredFDs = 3 + colexecop.ExternalSorterMinPartitions // NOTE: in the context of window functions "partitions" mean a different thing // from "partition" in the context of external algorithms and some disk @@ -209,7 +209,7 @@ func _COMPUTE_PEER_GROUPS_SIZES(_HAS_SEL bool) { // */}} type relativeRankInitFields struct { rankInitFields - colexecbase.CloserHelper + colexecop.CloserHelper state relativeRankState memoryLimit int64 @@ -275,7 +275,7 @@ type _RELATIVE_RANK_STRINGOp struct { output coldata.Batch } -var _ colexecbase.ClosableOperator = &_RELATIVE_RANK_STRINGOp{} +var _ colexecop.ClosableOperator = &_RELATIVE_RANK_STRINGOp{} func (r *_RELATIVE_RANK_STRINGOp) Init() { r.Input.Init() diff --git a/pkg/sql/colexec/colexecwindow/row_number.eg.go b/pkg/sql/colexec/colexecwindow/row_number.eg.go index c029596daaf0..733698f33cfa 100644 --- a/pkg/sql/colexec/colexecwindow/row_number.eg.go +++ b/pkg/sql/colexec/colexecwindow/row_number.eg.go @@ -14,7 +14,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -25,11 +25,11 @@ import ( // ROW_NUMBER. outputColIdx specifies in which coldata.Vec the operator should // put its output (if there is no such column, a new column is appended). func NewRowNumberOperator( - allocator *colmem.Allocator, input colexecbase.Operator, outputColIdx int, partitionColIdx int, -) colexecbase.Operator { + allocator *colmem.Allocator, input colexecop.Operator, outputColIdx int, partitionColIdx int, +) colexecop.Operator { input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.Int, outputColIdx) base := rowNumberBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, outputColIdx: outputColIdx, partitionColIdx: partitionColIdx, @@ -44,7 +44,7 @@ func NewRowNumberOperator( // variations of row number operators. Note that it is not an operator itself // and should not be used directly. type rowNumberBase struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator outputColIdx int partitionColIdx int @@ -60,7 +60,7 @@ type rowNumberNoPartitionOp struct { rowNumberBase } -var _ colexecbase.Operator = &rowNumberNoPartitionOp{} +var _ colexecop.Operator = &rowNumberNoPartitionOp{} func (r *rowNumberNoPartitionOp) Next(ctx context.Context) coldata.Batch { batch := r.Input.Next(ctx) @@ -97,7 +97,7 @@ type rowNumberWithPartitionOp struct { rowNumberBase } -var _ colexecbase.Operator = &rowNumberWithPartitionOp{} +var _ colexecop.Operator = &rowNumberWithPartitionOp{} func (r *rowNumberWithPartitionOp) Next(ctx context.Context) coldata.Batch { batch := r.Input.Next(ctx) diff --git a/pkg/sql/colexec/colexecwindow/row_number_tmpl.go b/pkg/sql/colexec/colexecwindow/row_number_tmpl.go index 3b8347cb0e51..ed7dc288ce9c 100644 --- a/pkg/sql/colexec/colexecwindow/row_number_tmpl.go +++ b/pkg/sql/colexec/colexecwindow/row_number_tmpl.go @@ -24,7 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -35,11 +35,11 @@ import ( // ROW_NUMBER. outputColIdx specifies in which coldata.Vec the operator should // put its output (if there is no such column, a new column is appended). func NewRowNumberOperator( - allocator *colmem.Allocator, input colexecbase.Operator, outputColIdx int, partitionColIdx int, -) colexecbase.Operator { + allocator *colmem.Allocator, input colexecop.Operator, outputColIdx int, partitionColIdx int, +) colexecop.Operator { input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.Int, outputColIdx) base := rowNumberBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, outputColIdx: outputColIdx, partitionColIdx: partitionColIdx, @@ -54,7 +54,7 @@ func NewRowNumberOperator( // variations of row number operators. Note that it is not an operator itself // and should not be used directly. type rowNumberBase struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator outputColIdx int partitionColIdx int @@ -96,7 +96,7 @@ type _ROW_NUMBER_STRINGOp struct { rowNumberBase } -var _ colexecbase.Operator = &_ROW_NUMBER_STRINGOp{} +var _ colexecop.Operator = &_ROW_NUMBER_STRINGOp{} func (r *_ROW_NUMBER_STRINGOp) Next(ctx context.Context) coldata.Batch { batch := r.Input.Next(ctx) diff --git a/pkg/sql/colexec/colexecwindow/window_functions_test.go b/pkg/sql/colexec/colexecwindow/window_functions_test.go index 6b0bd025688f..d76c56c473b2 100644 --- a/pkg/sql/colexec/colexecwindow/window_functions_test.go +++ b/pkg/sql/colexec/colexecwindow/window_functions_test.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -274,7 +274,7 @@ func TestWindowFunctions(t *testing.T) { } { log.Infof(ctx, "spillForced=%t/%s", spillForced, tc.windowerSpec.WindowFns[0].Func.String()) var semsToCheck []semaphore.Semaphore - colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.expected, colexectestutils.UnorderedVerifier, func(inputs []colexecbase.Operator) (colexecbase.Operator, error) { + colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.expected, colexectestutils.UnorderedVerifier, func(inputs []colexecop.Operator) (colexecop.Operator, error) { tc.init() ct := make([]*types.T, len(tc.tuples[0])) for i := range ct { @@ -294,7 +294,7 @@ func TestWindowFunctions(t *testing.T) { } // Relative rank operators currently require the most number of // FDs. - sem := colexecbase.NewTestingSemaphore(relativeRankNumRequiredFDs) + sem := colexecop.NewTestingSemaphore(relativeRankNumRequiredFDs) args := &colexecargs.NewColOperatorArgs{ Spec: spec, Inputs: inputs, diff --git a/pkg/sql/colexec/colexecwindow/window_peer_grouper.eg.go b/pkg/sql/colexec/colexecwindow/window_peer_grouper.eg.go index ec7adda4c55e..322d9e91aa0c 100644 --- a/pkg/sql/colexec/colexecwindow/window_peer_grouper.eg.go +++ b/pkg/sql/colexec/colexecwindow/window_peer_grouper.eg.go @@ -15,7 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecmisc" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -32,12 +32,12 @@ import ( // NOTE: the input *must* already be ordered on ordCols. func NewWindowPeerGrouper( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, typs []*types.T, orderingCols []execinfrapb.Ordering_Column, partitionColIdx int, outputColIdx int, -) (op colexecbase.Operator, err error) { +) (op colexecop.Operator, err error) { allPeers := len(orderingCols) == 0 var distinctCol []bool if !allPeers { @@ -54,7 +54,7 @@ func NewWindowPeerGrouper( } input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.Bool, outputColIdx) initFields := windowPeerGrouperInitFields{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, partitionColIdx: partitionColIdx, distinctCol: distinctCol, @@ -81,7 +81,7 @@ func NewWindowPeerGrouper( } type windowPeerGrouperInitFields struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator partitionColIdx int @@ -96,7 +96,7 @@ type windowPeerGrouperNoPartitionOp struct { windowPeerGrouperInitFields } -var _ colexecbase.Operator = &windowPeerGrouperNoPartitionOp{} +var _ colexecop.Operator = &windowPeerGrouperNoPartitionOp{} func (p *windowPeerGrouperNoPartitionOp) Init() { p.Input.Init() @@ -135,7 +135,7 @@ type windowPeerGrouperWithPartitionOp struct { windowPeerGrouperInitFields } -var _ colexecbase.Operator = &windowPeerGrouperWithPartitionOp{} +var _ colexecop.Operator = &windowPeerGrouperWithPartitionOp{} func (p *windowPeerGrouperWithPartitionOp) Init() { p.Input.Init() @@ -185,7 +185,7 @@ type windowPeerGrouperAllPeersNoPartitionOp struct { seenFirstTuple bool } -var _ colexecbase.Operator = &windowPeerGrouperAllPeersNoPartitionOp{} +var _ colexecop.Operator = &windowPeerGrouperAllPeersNoPartitionOp{} func (p *windowPeerGrouperAllPeersNoPartitionOp) Init() { p.Input.Init() @@ -226,7 +226,7 @@ type windowPeerGrouperAllPeersWithPartitionOp struct { windowPeerGrouperInitFields } -var _ colexecbase.Operator = &windowPeerGrouperAllPeersWithPartitionOp{} +var _ colexecop.Operator = &windowPeerGrouperAllPeersWithPartitionOp{} func (p *windowPeerGrouperAllPeersWithPartitionOp) Init() { p.Input.Init() diff --git a/pkg/sql/colexec/colexecwindow/window_peer_grouper_tmpl.go b/pkg/sql/colexec/colexecwindow/window_peer_grouper_tmpl.go index 9be9db41a36e..ebd8c71e9753 100644 --- a/pkg/sql/colexec/colexecwindow/window_peer_grouper_tmpl.go +++ b/pkg/sql/colexec/colexecwindow/window_peer_grouper_tmpl.go @@ -25,7 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecmisc" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -42,12 +42,12 @@ import ( // NOTE: the input *must* already be ordered on ordCols. func NewWindowPeerGrouper( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, typs []*types.T, orderingCols []execinfrapb.Ordering_Column, partitionColIdx int, outputColIdx int, -) (op colexecbase.Operator, err error) { +) (op colexecop.Operator, err error) { allPeers := len(orderingCols) == 0 var distinctCol []bool if !allPeers { @@ -64,7 +64,7 @@ func NewWindowPeerGrouper( } input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.Bool, outputColIdx) initFields := windowPeerGrouperInitFields{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, partitionColIdx: partitionColIdx, distinctCol: distinctCol, @@ -91,7 +91,7 @@ func NewWindowPeerGrouper( } type windowPeerGrouperInitFields struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator partitionColIdx int @@ -111,7 +111,7 @@ type _PEER_GROUPER_STRINGOp struct { // {{end}} } -var _ colexecbase.Operator = &_PEER_GROUPER_STRINGOp{} +var _ colexecop.Operator = &_PEER_GROUPER_STRINGOp{} func (p *_PEER_GROUPER_STRINGOp) Init() { p.Input.Init() diff --git a/pkg/sql/colexec/columnarizer.go b/pkg/sql/colexec/columnarizer.go index 5a681425e2b8..c46f77b1a90d 100644 --- a/pkg/sql/colexec/columnarizer.go +++ b/pkg/sql/colexec/columnarizer.go @@ -14,8 +14,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -45,13 +45,13 @@ const ( // chunk into a coldata.Batch column by column. type Columnarizer struct { execinfra.ProcessorBase - colexecbase.NonExplainable + colexecop.NonExplainable mode columnarizerMode allocator *colmem.Allocator input execinfra.RowSource da rowenc.DatumAlloc - initStatus colexecbase.OperatorInitStatus + initStatus colexecop.OperatorInitStatus buffered rowenc.EncDatumRows batch coldata.Batch @@ -61,7 +61,7 @@ type Columnarizer struct { typs []*types.T } -var _ colexecbase.Operator = &Columnarizer{} +var _ colexecop.Operator = &Columnarizer{} // NewBufferingColumnarizer returns a new Columnarizer that will be buffering up // rows before emitting them as output batches. @@ -130,10 +130,10 @@ func (c *Columnarizer) Init() { // We don't want to call Start on the input to columnarizer and allocating // internal objects several times if Init method is called more than once, so // we have this check in place. - if c.initStatus == colexecbase.OperatorNotInitialized { + if c.initStatus == colexecop.OperatorNotInitialized { c.accumulatedMeta = make([]execinfrapb.ProducerMetadata, 0, 1) c.input.Start(c.ctx) - c.initStatus = colexecbase.OperatorInitialized + c.initStatus = colexecop.OperatorInitialized } } @@ -215,9 +215,9 @@ func (c *Columnarizer) Run(context.Context) { } var ( - _ colexecbase.Operator = &Columnarizer{} + _ colexecop.Operator = &Columnarizer{} _ execinfrapb.MetadataSource = &Columnarizer{} - _ colexecbase.Closer = &Columnarizer{} + _ colexecop.Closer = &Columnarizer{} ) // DrainMeta is part of the MetadataSource interface. diff --git a/pkg/sql/colexec/count.go b/pkg/sql/colexec/count.go index 5d53528631ae..09d8aec9d5fd 100644 --- a/pkg/sql/colexec/count.go +++ b/pkg/sql/colexec/count.go @@ -14,7 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -24,19 +24,19 @@ import ( // column containing a single integer, the count of rows received from the // upstream. type countOp struct { - colexecbase.OneInputNode + colexecop.OneInputNode internalBatch coldata.Batch done bool count int64 } -var _ colexecbase.Operator = &countOp{} +var _ colexecop.Operator = &countOp{} // NewCountOp returns a new count operator that counts the rows in its input. -func NewCountOp(allocator *colmem.Allocator, input colexecbase.Operator) colexecbase.Operator { +func NewCountOp(allocator *colmem.Allocator, input colexecop.Operator) colexecop.Operator { c := &countOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), } c.internalBatch = allocator.NewMemBatchWithFixedCapacity( []*types.T{types.Int}, 1, /* capacity */ diff --git a/pkg/sql/colexec/count_test.go b/pkg/sql/colexec/count_test.go index 754bf3e7c01b..5497aba9ab36 100644 --- a/pkg/sql/colexec/count_test.go +++ b/pkg/sql/colexec/count_test.go @@ -14,7 +14,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -38,7 +38,7 @@ func TestCount(t *testing.T) { for _, tc := range tcs { // The tuples consisting of all nulls still count as separate rows, so if // we replace all values with nulls, we should get the same output. - colexectestutils.RunTestsWithoutAllNullsInjection(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, nil, tc.expected, colexectestutils.OrderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { + colexectestutils.RunTestsWithoutAllNullsInjection(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, nil, tc.expected, colexectestutils.OrderedVerifier, func(input []colexecop.Operator) (colexecop.Operator, error) { return NewCountOp(testAllocator, input[0]), nil }) } diff --git a/pkg/sql/colexec/crossjoiner_test.go b/pkg/sql/colexec/crossjoiner_test.go index ebc8478e36e8..ad3d4e32430d 100644 --- a/pkg/sql/colexec/crossjoiner_test.go +++ b/pkg/sql/colexec/crossjoiner_test.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -369,14 +369,14 @@ func TestCrossJoiner(t *testing.T) { for _, tc := range getCJTestCases() { for _, tc := range tc.mutateTypes() { log.Infof(ctx, "spillForced=%t", spillForced) - runHashJoinTestCase(t, tc, func(sources []colexecbase.Operator) (colexecbase.Operator, error) { + runHashJoinTestCase(t, tc, func(sources []colexecop.Operator) (colexecop.Operator, error) { spec := createSpecForHashJoiner(tc) args := &colexecargs.NewColOperatorArgs{ Spec: spec, Inputs: sources, StreamingMemAccount: testMemAcc, DiskQueueCfg: queueCfg, - FDSemaphore: colexecbase.NewTestingSemaphore(externalHJMinPartitions), + FDSemaphore: colexecop.NewTestingSemaphore(externalHJMinPartitions), } result, err := colexecargs.TestNewColOperator(ctx, flowCtx, args) if err != nil { @@ -443,10 +443,10 @@ func BenchmarkCrossJoiner(b *testing.B) { args := &colexecargs.NewColOperatorArgs{ Spec: spec, // Inputs will be set below. - Inputs: []colexecbase.Operator{nil, nil}, + Inputs: []colexecop.Operator{nil, nil}, StreamingMemAccount: testMemAcc, DiskQueueCfg: queueCfg, - FDSemaphore: colexecbase.NewTestingSemaphore(VecMaxOpenFDsLimit), + FDSemaphore: colexecop.NewTestingSemaphore(VecMaxOpenFDsLimit), } b.Run(fmt.Sprintf("spillForced=%t/type=%s/rows=%d", spillForced, joinType, nRows), func(b *testing.B) { var nOutputRows int diff --git a/pkg/sql/colexec/default_agg_test.go b/pkg/sql/colexec/default_agg_test.go index 3d45ea04e9ce..6719263cefb2 100644 --- a/pkg/sql/colexec/default_agg_test.go +++ b/pkg/sql/colexec/default_agg_test.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -147,7 +147,7 @@ func TestDefaultAggregateFunc(t *testing.T) { ) require.NoError(t, err) colexectestutils.RunTestsWithTyps(t, testAllocator, []colexectestutils.Tuples{tc.input}, [][]*types.T{tc.typs}, tc.expected, colexectestutils.UnorderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return agg.new(&colexecagg.NewAggregatorArgs{ Allocator: testAllocator, MemAccount: testMemAcc, diff --git a/pkg/sql/colexec/disk_spiller.go b/pkg/sql/colexec/disk_spiller.go index a114b2041551..28f42b9cbe78 100644 --- a/pkg/sql/colexec/disk_spiller.go +++ b/pkg/sql/colexec/disk_spiller.go @@ -15,8 +15,8 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/errors" @@ -72,15 +72,15 @@ import ( // - spillingCallbackFn will be called when the spilling from in-memory to disk // backed operator occurs. It should only be set in tests. func NewOneInputDiskSpiller( - input colexecbase.Operator, - inMemoryOp colexecbase.BufferingInMemoryOperator, + input colexecop.Operator, + inMemoryOp colexecop.BufferingInMemoryOperator, inMemoryMemMonitorName string, - diskBackedOpConstructor func(input colexecbase.Operator) colexecbase.Operator, + diskBackedOpConstructor func(input colexecop.Operator) colexecop.Operator, spillingCallbackFn func(), -) colexecbase.Operator { +) colexecop.Operator { diskBackedOpInput := newBufferExportingOperator(inMemoryOp, input) return &diskSpillerBase{ - inputs: []colexecbase.Operator{input}, + inputs: []colexecop.Operator{input}, inMemoryOp: inMemoryOp, inMemoryMemMonitorName: inMemoryMemMonitorName, diskBackedOp: diskBackedOpConstructor(diskBackedOpInput), @@ -139,21 +139,21 @@ func NewOneInputDiskSpiller( // - spillingCallbackFn will be called when the spilling from in-memory to disk // backed operator occurs. It should only be set in tests. func NewTwoInputDiskSpiller( - inputOne, inputTwo colexecbase.Operator, - inMemoryOp colexecbase.BufferingInMemoryOperator, + inputOne, inputTwo colexecop.Operator, + inMemoryOp colexecop.BufferingInMemoryOperator, inMemoryMemMonitorName string, - diskBackedOpConstructor func(inputOne, inputTwo colexecbase.Operator) colexecbase.Operator, + diskBackedOpConstructor func(inputOne, inputTwo colexecop.Operator) colexecop.Operator, spillingCallbackFn func(), -) colexecbase.Operator { +) colexecop.Operator { diskBackedOpInputOne := newBufferExportingOperator(inMemoryOp, inputOne) diskBackedOpInputTwo := newBufferExportingOperator(inMemoryOp, inputTwo) return &diskSpillerBase{ - inputs: []colexecbase.Operator{inputOne, inputTwo}, + inputs: []colexecop.Operator{inputOne, inputTwo}, inMemoryOp: inMemoryOp, - inMemoryOpInitStatus: colexecbase.OperatorNotInitialized, + inMemoryOpInitStatus: colexecop.OperatorNotInitialized, inMemoryMemMonitorName: inMemoryMemMonitorName, diskBackedOp: diskBackedOpConstructor(diskBackedOpInputOne, diskBackedOpInputTwo), - distBackedOpInitStatus: colexecbase.OperatorNotInitialized, + distBackedOpInitStatus: colexecop.OperatorNotInitialized, spillingCallbackFn: spillingCallbackFn, } } @@ -161,25 +161,25 @@ func NewTwoInputDiskSpiller( // diskSpillerBase is the common base for the one-input and two-input disk // spillers. type diskSpillerBase struct { - colexecbase.NonExplainable + colexecop.NonExplainable - colexecbase.CloserHelper + colexecop.CloserHelper - inputs []colexecbase.Operator + inputs []colexecop.Operator spilled bool - inMemoryOp colexecbase.BufferingInMemoryOperator - inMemoryOpInitStatus colexecbase.OperatorInitStatus + inMemoryOp colexecop.BufferingInMemoryOperator + inMemoryOpInitStatus colexecop.OperatorInitStatus inMemoryMemMonitorName string - diskBackedOp colexecbase.Operator - distBackedOpInitStatus colexecbase.OperatorInitStatus + diskBackedOp colexecop.Operator + distBackedOpInitStatus colexecop.OperatorInitStatus spillingCallbackFn func() } -var _ colexecbase.ResettableOperator = &diskSpillerBase{} +var _ colexecop.ResettableOperator = &diskSpillerBase{} func (d *diskSpillerBase) Init() { - if d.inMemoryOpInitStatus == colexecbase.OperatorInitialized { + if d.inMemoryOpInitStatus == colexecop.OperatorInitialized { return } // It is possible that Init() call below will hit an out of memory error, @@ -188,7 +188,7 @@ func (d *diskSpillerBase) Init() { // Also note that d.input is the input to d.inMemoryOp, so calling Init() // only on the latter is sufficient. d.inMemoryOp.Init() - d.inMemoryOpInitStatus = colexecbase.OperatorInitialized + d.inMemoryOpInitStatus = colexecop.OperatorInitialized } func (d *diskSpillerBase) Next(ctx context.Context) coldata.Batch { @@ -207,12 +207,12 @@ func (d *diskSpillerBase) Next(ctx context.Context) coldata.Batch { if d.spillingCallbackFn != nil { d.spillingCallbackFn() } - if d.distBackedOpInitStatus == colexecbase.OperatorNotInitialized { + if d.distBackedOpInitStatus == colexecop.OperatorNotInitialized { // The disk spiller might be reset for reuse in which case the // the disk-backed operator has already been initialized and we // don't want to perform the initialization again. d.diskBackedOp.Init() - d.distBackedOpInitStatus = colexecbase.OperatorInitialized + d.distBackedOpInitStatus = colexecop.OperatorInitialized } return d.diskBackedOp.Next(ctx) } @@ -225,17 +225,17 @@ func (d *diskSpillerBase) Next(ctx context.Context) coldata.Batch { func (d *diskSpillerBase) Reset(ctx context.Context) { for _, input := range d.inputs { - if r, ok := input.(colexecbase.Resetter); ok { + if r, ok := input.(colexecop.Resetter); ok { r.Reset(ctx) } } - if d.inMemoryOpInitStatus == colexecbase.OperatorInitialized { - if r, ok := d.inMemoryOp.(colexecbase.Resetter); ok { + if d.inMemoryOpInitStatus == colexecop.OperatorInitialized { + if r, ok := d.inMemoryOp.(colexecop.Resetter); ok { r.Reset(ctx) } } - if d.distBackedOpInitStatus == colexecbase.OperatorInitialized { - if r, ok := d.diskBackedOp.(colexecbase.Resetter); ok { + if d.distBackedOpInitStatus == colexecop.OperatorInitialized { + if r, ok := d.diskBackedOp.(colexecop.Resetter); ok { r.Reset(ctx) } } @@ -248,10 +248,10 @@ func (d *diskSpillerBase) Close(ctx context.Context) error { return nil } var retErr error - if c, ok := d.inMemoryOp.(colexecbase.Closer); ok { + if c, ok := d.inMemoryOp.(colexecop.Closer); ok { retErr = c.Close(ctx) } - if c, ok := d.diskBackedOp.(colexecbase.Closer); ok { + if c, ok := d.diskBackedOp.(colexecop.Closer); ok { if err := c.Close(ctx); err != nil { retErr = err } @@ -299,19 +299,19 @@ func (d *diskSpillerBase) Child(nth int, verbose bool) execinfra.OpNode { // initialized when bufferExportingOperator.Init() is called. // NOTE: it is assumed that secondSource is the input to firstSource. type bufferExportingOperator struct { - colexecbase.ZeroInputNode - colexecbase.NonExplainable + colexecop.ZeroInputNode + colexecop.NonExplainable - firstSource colexecbase.BufferingInMemoryOperator - secondSource colexecbase.Operator + firstSource colexecop.BufferingInMemoryOperator + secondSource colexecop.Operator firstSourceDone bool } -var _ colexecbase.ResettableOperator = &bufferExportingOperator{} +var _ colexecop.ResettableOperator = &bufferExportingOperator{} func newBufferExportingOperator( - firstSource colexecbase.BufferingInMemoryOperator, secondSource colexecbase.Operator, -) colexecbase.Operator { + firstSource colexecop.BufferingInMemoryOperator, secondSource colexecop.Operator, +) colexecop.Operator { return &bufferExportingOperator{ firstSource: firstSource, secondSource: secondSource, @@ -336,10 +336,10 @@ func (b *bufferExportingOperator) Next(ctx context.Context) coldata.Batch { } func (b *bufferExportingOperator) Reset(ctx context.Context) { - if r, ok := b.firstSource.(colexecbase.Resetter); ok { + if r, ok := b.firstSource.(colexecop.Resetter); ok { r.Reset(ctx) } - if r, ok := b.secondSource.(colexecbase.Resetter); ok { + if r, ok := b.secondSource.(colexecop.Resetter); ok { r.Reset(ctx) } b.firstSourceDone = false diff --git a/pkg/sql/colexec/distinct_test.go b/pkg/sql/colexec/distinct_test.go index a47ce5d7115f..32df9a2d50fd 100644 --- a/pkg/sql/colexec/distinct_test.go +++ b/pkg/sql/colexec/distinct_test.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecmisc" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -191,7 +191,7 @@ func TestDistinct(t *testing.T) { for _, tc := range distinctTestCases { log.Infof(context.Background(), "unordered") colexectestutils.RunTestsWithTyps(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, [][]*types.T{tc.typs}, tc.expected, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return NewUnorderedDistinct( testAllocator, input[0], tc.distinctCols, tc.typs, ), nil @@ -204,7 +204,7 @@ func TestDistinct(t *testing.T) { orderedCols[i] = tc.distinctCols[j] } colexectestutils.RunTestsWithTyps(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, [][]*types.T{tc.typs}, tc.expected, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return newPartiallyOrderedDistinct( testAllocator, input[0], tc.distinctCols, orderedCols, tc.typs, ) @@ -212,7 +212,7 @@ func TestDistinct(t *testing.T) { } log.Info(context.Background(), "ordered") colexectestutils.RunTestsWithTyps(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, [][]*types.T{tc.typs}, tc.expected, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return colexecmisc.NewOrderedDistinct(input[0], tc.distinctCols, tc.typs) }) } @@ -243,7 +243,7 @@ func TestUnorderedDistinctRandom(t *testing.T) { } tups, expected := generateRandomDataForUnorderedDistinct(rng, nTuples, nCols, newTupleProbability) colexectestutils.RunTestsWithTyps(t, testAllocator, []colexectestutils.Tuples{tups}, [][]*types.T{typs}, expected, colexectestutils.UnorderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return NewUnorderedDistinct(testAllocator, input[0], distinctCols, typs), nil }, ) @@ -265,7 +265,7 @@ func getNewValueProbabilityForDistinct(newTupleProbability float64, nCols int) f func runDistinctBenchmarks( ctx context.Context, b *testing.B, - distinctConstructor func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecbase.Operator, error), + distinctConstructor func(allocator *colmem.Allocator, input colexecop.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecop.Operator, error), getNumOrderedCols func(nCols int) int, namePrefix string, isExternal bool, @@ -344,14 +344,14 @@ func runDistinctBenchmarks( func BenchmarkDistinct(b *testing.B) { ctx := context.Background() - distinctConstructors := []func(*colmem.Allocator, colexecbase.Operator, []uint32, int, []*types.T) (colexecbase.Operator, error){ - func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecbase.Operator, error) { + distinctConstructors := []func(*colmem.Allocator, colexecop.Operator, []uint32, int, []*types.T) (colexecop.Operator, error){ + func(allocator *colmem.Allocator, input colexecop.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecop.Operator, error) { return NewUnorderedDistinct(allocator, input, distinctCols, typs), nil }, - func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecbase.Operator, error) { + func(allocator *colmem.Allocator, input colexecop.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecop.Operator, error) { return newPartiallyOrderedDistinct(allocator, input, distinctCols, distinctCols[:numOrderedCols], typs) }, - func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecbase.Operator, error) { + func(allocator *colmem.Allocator, input colexecop.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecop.Operator, error) { return colexecmisc.NewOrderedDistinct(input, distinctCols, typs) }, } diff --git a/pkg/sql/colexec/execgen/cmd/execgen/distinct_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/distinct_gen.go index c7adb7c32004..1f94211066c9 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/distinct_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/distinct_gen.go @@ -46,7 +46,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" diff --git a/pkg/sql/colexec/external_distinct.go b/pkg/sql/colexec/external_distinct.go index 283ab23321d3..7ed4c72f7bf9 100644 --- a/pkg/sql/colexec/external_distinct.go +++ b/pkg/sql/colexec/external_distinct.go @@ -13,8 +13,8 @@ package colexec import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecmisc" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -30,15 +30,15 @@ func NewExternalDistinct( unlimitedAllocator *colmem.Allocator, flowCtx *execinfra.FlowCtx, args *colexecargs.NewColOperatorArgs, - input colexecbase.Operator, + input colexecop.Operator, inputTypes []*types.T, createDiskBackedSorter DiskBackedSorterConstructor, - inMemUnorderedDistinct colexecbase.Operator, + inMemUnorderedDistinct colexecop.Operator, diskAcc *mon.BoundAccount, -) colexecbase.Operator { +) colexecop.Operator { distinctSpec := args.Spec.Core.Distinct distinctCols := distinctSpec.DistinctColumns - inMemMainOpConstructor := func(partitionedInputs []*partitionerToOperator) colexecbase.ResettableOperator { + inMemMainOpConstructor := func(partitionedInputs []*partitionerToOperator) colexecop.ResettableOperator { // Note that the hash-based partitioner will make sure that partitions // to process using the in-memory unordered distinct fit under the // limit, so we use an unlimited allocator. @@ -51,7 +51,7 @@ func NewExternalDistinct( partitionedInputs []*partitionerToOperator, maxNumberActivePartitions int, _ semaphore.Semaphore, - ) colexecbase.ResettableOperator { + ) colexecop.ResettableOperator { // The distinct operator *must* keep the first tuple from the input // among all that are identical on distinctCols. In order to guarantee // such behavior in the fallback, we append an ordinality column to @@ -83,16 +83,16 @@ func NewExternalDistinct( // in-memory operator tuples, so we plan a special filterer operator to // remove all such tuples. input = &unorderedDistinctFilterer{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), ht: inMemUnorderedDistinct.(*unorderedDistinct).ht, } - numRequiredActivePartitions := colexecbase.ExternalSorterMinPartitions + numRequiredActivePartitions := colexecop.ExternalSorterMinPartitions ed := newHashBasedPartitioner( unlimitedAllocator, flowCtx, args, "external unordered distinct", /* name */ - []colexecbase.Operator{input}, + []colexecop.Operator{input}, [][]*types.T{inputTypes}, [][]uint32{distinctCols}, inMemMainOpConstructor, diff --git a/pkg/sql/colexec/external_distinct_test.go b/pkg/sql/colexec/external_distinct_test.go index d21f80d86aeb..e9a782bf5753 100644 --- a/pkg/sql/colexec/external_distinct_test.go +++ b/pkg/sql/colexec/external_distinct_test.go @@ -21,7 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -77,11 +77,11 @@ func TestExternalDistinct(t *testing.T) { // unordered distinct is free to change the order of the tuples // when exporting them into an external distinct. colexectestutils.UnorderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { // A sorter should never exceed ExternalSorterMinPartitions, even // during repartitioning. A panic will happen if a sorter requests // more than this number of file descriptors. - sem := colexecbase.NewTestingSemaphore(colexecbase.ExternalSorterMinPartitions) + sem := colexecop.NewTestingSemaphore(colexecop.ExternalSorterMinPartitions) semsToCheck = append(semsToCheck, sem) var outputOrdering execinfrapb.Ordering if tc.isOrderedOnDistinctCols { @@ -200,11 +200,11 @@ func TestExternalDistinctSpilling(t *testing.T) { // tups and expected are in an arbitrary order, so we use an unordered // verifier. colexectestutils.UnorderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { // Since we're giving very low memory limit to the operator, in // order to make the test run faster, we'll use an unlimited number // of file descriptors. - sem := colexecbase.NewTestingSemaphore(0 /* limit */) + sem := colexecop.NewTestingSemaphore(0 /* limit */) semsToCheck = append(semsToCheck, sem) var outputOrdering execinfrapb.Ordering distinct, newAccounts, newMonitors, closers, err := createExternalDistinct( @@ -318,14 +318,14 @@ func BenchmarkExternalDistinct(b *testing.B) { runDistinctBenchmarks( ctx, b, - func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecbase.Operator, error) { + func(allocator *colmem.Allocator, input colexecop.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecop.Operator, error) { var outputOrdering execinfrapb.Ordering if maintainOrdering { outputOrdering = convertDistinctColsToOrdering(distinctCols) } op, accs, mons, _, err := createExternalDistinct( - ctx, flowCtx, []colexecbase.Operator{input}, typs, - distinctCols, outputOrdering, queueCfg, &colexecbase.TestingSemaphore{}, + ctx, flowCtx, []colexecop.Operator{input}, typs, + distinctCols, outputOrdering, queueCfg, &colexecop.TestingSemaphore{}, nil /* spillingCallbackFn */, 0, /* numForcedRepartitions */ ) memAccounts = append(memAccounts, accs...) @@ -355,7 +355,7 @@ func BenchmarkExternalDistinct(b *testing.B) { func createExternalDistinct( ctx context.Context, flowCtx *execinfra.FlowCtx, - input []colexecbase.Operator, + input []colexecop.Operator, typs []*types.T, distinctCols []uint32, outputOrdering execinfrapb.Ordering, @@ -363,7 +363,7 @@ func createExternalDistinct( testingSemaphore semaphore.Semaphore, spillingCallbackFn func(), numForcedRepartitions int, -) (colexecbase.Operator, []*mon.BoundAccount, []*mon.BytesMonitor, []colexecbase.Closer, error) { +) (colexecop.Operator, []*mon.BoundAccount, []*mon.BytesMonitor, []colexecop.Closer, error) { distinctSpec := &execinfrapb.DistinctSpec{ DistinctColumns: distinctCols, OutputOrdering: outputOrdering, diff --git a/pkg/sql/colexec/external_hash_aggregator.go b/pkg/sql/colexec/external_hash_aggregator.go index a1b6bdf7cb8a..2f07895cecf7 100644 --- a/pkg/sql/colexec/external_hash_aggregator.go +++ b/pkg/sql/colexec/external_hash_aggregator.go @@ -14,8 +14,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -25,7 +25,7 @@ import ( const ( // This limit comes from the fallback strategy where we are using an // external sort. - ehaNumRequiredActivePartitions = colexecbase.ExternalSorterMinPartitions + ehaNumRequiredActivePartitions = colexecop.ExternalSorterMinPartitions // ehaNumRequiredFDs is the minimum number of file descriptors that are // needed for the machinery of the external aggregator (plus 1 is needed for // the in-memory hash aggregator in order to track tuples in a spilling @@ -42,8 +42,8 @@ func NewExternalHashAggregator( newAggArgs *colexecagg.NewAggregatorArgs, createDiskBackedSorter DiskBackedSorterConstructor, diskAcc *mon.BoundAccount, -) colexecbase.Operator { - inMemMainOpConstructor := func(partitionedInputs []*partitionerToOperator) colexecbase.ResettableOperator { +) colexecop.Operator { + inMemMainOpConstructor := func(partitionedInputs []*partitionerToOperator) colexecop.ResettableOperator { newAggArgs := *newAggArgs newAggArgs.Input = partitionedInputs[0] // We don't need to track the input tuples when we have already spilled. @@ -59,7 +59,7 @@ func NewExternalHashAggregator( partitionedInputs []*partitionerToOperator, maxNumberActivePartitions int, _ semaphore.Semaphore, - ) colexecbase.ResettableOperator { + ) colexecop.ResettableOperator { newAggArgs := *newAggArgs newAggArgs.Input = createDiskBackedSorter( partitionedInputs[0], newAggArgs.InputTypes, @@ -76,7 +76,7 @@ func NewExternalHashAggregator( flowCtx, args, "external hash aggregator", /* name */ - []colexecbase.Operator{newAggArgs.Input}, + []colexecop.Operator{newAggArgs.Input}, [][]*types.T{newAggArgs.InputTypes}, [][]uint32{spec.GroupCols}, inMemMainOpConstructor, diff --git a/pkg/sql/colexec/external_hash_aggregator_test.go b/pkg/sql/colexec/external_hash_aggregator_test.go index 346e50251e2e..a57df5a04ad0 100644 --- a/pkg/sql/colexec/external_hash_aggregator_test.go +++ b/pkg/sql/colexec/external_hash_aggregator_test.go @@ -21,7 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -93,8 +93,8 @@ func TestExternalHashAggregator(t *testing.T) { [][]*types.T{tc.typs}, tc.expected, colexectestutils.UnorderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { - sem := colexecbase.NewTestingSemaphore(ehaNumRequiredFDs) + func(input []colexecop.Operator) (colexecop.Operator, error) { + sem := colexecop.NewTestingSemaphore(ehaNumRequiredFDs) semsToCheck = append(semsToCheck, sem) op, accs, mons, closers, err := createExternalHashAggregator( ctx, flowCtx, &colexecagg.NewAggregatorArgs{ @@ -179,10 +179,10 @@ func BenchmarkExternalHashAggregator(b *testing.B) { for _, groupSize := range groupSizes { benchmarkAggregateFunction( b, aggType{ - new: func(args *colexecagg.NewAggregatorArgs) (colexecbase.ResettableOperator, error) { + new: func(args *colexecagg.NewAggregatorArgs) (colexecop.ResettableOperator, error) { op, accs, mons, _, err := createExternalHashAggregator( ctx, flowCtx, args, queueCfg, - &colexecbase.TestingSemaphore{}, 0, /* numForcedRepartitions */ + &colexecop.TestingSemaphore{}, 0, /* numForcedRepartitions */ ) memAccounts = append(memAccounts, accs...) memMonitors = append(memMonitors, mons...) @@ -191,7 +191,7 @@ func BenchmarkExternalHashAggregator(b *testing.B) { // signatures of the aggregator constructors, we // wrap it with a noop operator. It is ok for the // purposes of this benchmark. - return colexecbase.NewNoop(op), err + return colexecop.NewNoop(op), err }, name: fmt.Sprintf("spilled=%t", spillForced), }, @@ -221,7 +221,7 @@ func createExternalHashAggregator( diskQueueCfg colcontainer.DiskQueueCfg, testingSemaphore semaphore.Semaphore, numForcedRepartitions int, -) (colexecbase.Operator, []*mon.BoundAccount, []*mon.BytesMonitor, []colexecbase.Closer, error) { +) (colexecop.Operator, []*mon.BoundAccount, []*mon.BytesMonitor, []colexecop.Closer, error) { spec := &execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{{ColumnTypes: newAggArgs.InputTypes}}, Core: execinfrapb.ProcessorCoreUnion{ @@ -232,7 +232,7 @@ func createExternalHashAggregator( } args := &colexecargs.NewColOperatorArgs{ Spec: spec, - Inputs: []colexecbase.Operator{newAggArgs.Input}, + Inputs: []colexecop.Operator{newAggArgs.Input}, StreamingMemAccount: testMemAcc, DiskQueueCfg: diskQueueCfg, FDSemaphore: testingSemaphore, diff --git a/pkg/sql/colexec/external_hash_joiner.go b/pkg/sql/colexec/external_hash_joiner.go index 3833430d416c..df8441905a96 100644 --- a/pkg/sql/colexec/external_hash_joiner.go +++ b/pkg/sql/colexec/external_hash_joiner.go @@ -14,8 +14,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecjoin" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -41,7 +41,7 @@ const ( // sorter emits its first batch, it must be the case that the input to it // has returned a zero batch, and thus the FD has been closed. sortMergeNonSortMinFDsOpen = 2 - externalHJMinPartitions = sortMergeNonSortMinFDsOpen + (colexecbase.ExternalSorterMinPartitions * 2) + externalHJMinPartitions = sortMergeNonSortMinFDsOpen + (colexecop.ExternalSorterMinPartitions * 2) ) // externalHashJoiner is an operator that performs Grace hash join algorithm @@ -87,10 +87,10 @@ func NewExternalHashJoiner( flowCtx *execinfra.FlowCtx, args *colexecargs.NewColOperatorArgs, spec colexecjoin.HashJoinerSpec, - leftInput, rightInput colexecbase.Operator, + leftInput, rightInput colexecop.Operator, createDiskBackedSorter DiskBackedSorterConstructor, diskAcc *mon.BoundAccount, -) colexecbase.Operator { +) colexecop.Operator { // This memory limit will restrict the size of the batches output by the // in-memory hash joiner in the main strategy as well as by the merge joiner // in the fallback strategy. @@ -99,9 +99,9 @@ func NewExternalHashJoiner( // If memory limit is 1, we're likely in a "force disk spill" // scenario, but we don't want to artificially limit batches when we // have already spilled, so we'll use a larger limit. - memoryLimit = colexecbase.DefaultMemoryLimit + memoryLimit = colexecop.DefaultMemoryLimit } - inMemMainOpConstructor := func(partitionedInputs []*partitionerToOperator) colexecbase.ResettableOperator { + inMemMainOpConstructor := func(partitionedInputs []*partitionerToOperator) colexecop.ResettableOperator { // Note that the hash-based partitioner will make sure that partitions // to join using in-memory hash joiner fit under the limit, so we use // the same unlimited allocator for both buildSideAllocator and @@ -117,7 +117,7 @@ func NewExternalHashJoiner( partitionedInputs []*partitionerToOperator, maxNumberActivePartitions int, fdSemaphore semaphore.Semaphore, - ) colexecbase.ResettableOperator { + ) colexecop.ResettableOperator { // We need to allocate 2 FDs for reading the partitions (reused by the merge // joiner) that we need to join using sort + merge join strategy, and all // others are divided between the two inputs. @@ -145,7 +145,7 @@ func NewExternalHashJoiner( flowCtx, args, "external hash joiner", /* name */ - []colexecbase.Operator{leftInput, rightInput}, + []colexecop.Operator{leftInput, rightInput}, [][]*types.T{spec.Left.SourceTypes, spec.Right.SourceTypes}, [][]uint32{spec.Left.EqCols, spec.Right.EqCols}, inMemMainOpConstructor, diff --git a/pkg/sql/colexec/external_hash_joiner_test.go b/pkg/sql/colexec/external_hash_joiner_test.go index 0074f48a477b..3b9104ceddbe 100644 --- a/pkg/sql/colexec/external_hash_joiner_test.go +++ b/pkg/sql/colexec/external_hash_joiner_test.go @@ -21,7 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -79,8 +79,8 @@ func TestExternalHashJoiner(t *testing.T) { // allNullsInjection test for now. tc.skipAllNullsInjection = true } - runHashJoinTestCase(t, tc, func(sources []colexecbase.Operator) (colexecbase.Operator, error) { - sem := colexecbase.NewTestingSemaphore(externalHJMinPartitions) + runHashJoinTestCase(t, tc, func(sources []colexecop.Operator) (colexecop.Operator, error) { + sem := colexecop.NewTestingSemaphore(externalHJMinPartitions) semsToCheck = append(semsToCheck, sem) spec := createSpecForHashJoiner(tc) // TODO(asubiotto): Pass in the testing.T of the caller to this @@ -158,12 +158,12 @@ func TestExternalHashJoinerFallbackToSortMergeJoin(t *testing.T) { var spilled bool queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(t, true /* inMem */) defer cleanup() - sem := colexecbase.NewTestingSemaphore(externalHJMinPartitions) + sem := colexecop.NewTestingSemaphore(externalHJMinPartitions) // Ignore closers since the sorter should close itself when it is drained of // all tuples. We assert this by checking that the semaphore reports a count // of 0. hj, accounts, monitors, _, err := createDiskBackedHashJoiner( - ctx, flowCtx, spec, []colexecbase.Operator{leftSource, rightSource}, + ctx, flowCtx, spec, []colexecop.Operator{leftSource, rightSource}, func() { spilled = true }, queueCfg, // Force a repartition so that the recursive repartitioning always // occurs. @@ -262,9 +262,9 @@ func BenchmarkExternalHashJoiner(b *testing.B) { leftSource := colexectestutils.NewChunkingBatchSource(testAllocator, sourceTypes, cols, nRows) rightSource := colexectestutils.NewChunkingBatchSource(testAllocator, sourceTypes, cols, nRows) hj, accounts, monitors, _, err := createDiskBackedHashJoiner( - ctx, flowCtx, spec, []colexecbase.Operator{leftSource, rightSource}, + ctx, flowCtx, spec, []colexecop.Operator{leftSource, rightSource}, func() {}, queueCfg, 0 /* numForcedRepartitions */, false, /* delegateFDAcquisitions */ - colexecbase.NewTestingSemaphore(VecMaxOpenFDsLimit), + colexecop.NewTestingSemaphore(VecMaxOpenFDsLimit), ) memAccounts = append(memAccounts, accounts...) memMonitors = append(memMonitors, monitors...) @@ -294,13 +294,13 @@ func createDiskBackedHashJoiner( ctx context.Context, flowCtx *execinfra.FlowCtx, spec *execinfrapb.ProcessorSpec, - inputs []colexecbase.Operator, + inputs []colexecop.Operator, spillingCallbackFn func(), diskQueueCfg colcontainer.DiskQueueCfg, numForcedRepartitions int, delegateFDAcquisitions bool, testingSemaphore semaphore.Semaphore, -) (colexecbase.Operator, []*mon.BoundAccount, []*mon.BytesMonitor, []colexecbase.Closer, error) { +) (colexecop.Operator, []*mon.BoundAccount, []*mon.BytesMonitor, []colexecop.Closer, error) { args := &colexecargs.NewColOperatorArgs{ Spec: spec, Inputs: inputs, diff --git a/pkg/sql/colexec/external_sort.go b/pkg/sql/colexec/external_sort.go index 4a8189f16f15..3a0ef556bb11 100644 --- a/pkg/sql/colexec/external_sort.go +++ b/pkg/sql/colexec/external_sort.go @@ -17,8 +17,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -108,9 +108,9 @@ const ( // some amount of RAM for its buffer. This is determined by // maxNumberPartitions variable. type externalSorter struct { - colexecbase.OneInputNode - colexecbase.NonExplainable - colexecbase.CloserHelper + colexecop.OneInputNode + colexecop.NonExplainable + colexecop.CloserHelper // mergeUnlimitedAllocator is used to track the memory under the batches // dequeued from partitions during the merge operation. @@ -128,7 +128,7 @@ type externalSorter struct { ordering execinfrapb.Ordering // columnOrdering is the same as ordering used when creating mergers. columnOrdering colinfo.ColumnOrdering - inMemSorter colexecbase.ResettableOperator + inMemSorter colexecop.ResettableOperator inMemSorterInput *inputPartitioningOperator partitioner colcontainer.PartitionedQueue partitionerCreator func() colcontainer.PartitionedQueue @@ -168,7 +168,7 @@ type externalSorter struct { acquiredFDs int } - emitter colexecbase.Operator + emitter colexecop.Operator testingKnobs struct { // delegateFDAcquisitions if true, means that a test wants to force the @@ -179,8 +179,8 @@ type externalSorter struct { } } -var _ colexecbase.ResettableOperator = &externalSorter{} -var _ colexecbase.ClosableOperator = &externalSorter{} +var _ colexecop.ResettableOperator = &externalSorter{} +var _ colexecop.ClosableOperator = &externalSorter{} // NewExternalSorter returns a disk-backed general sort operator. // - unlimitedAllocators must have been created with a memory account derived @@ -199,7 +199,7 @@ func NewExternalSorter( sortUnlimitedAllocator *colmem.Allocator, mergeUnlimitedAllocator *colmem.Allocator, outputUnlimitedAllocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, inputTypes []*types.T, ordering execinfrapb.Ordering, memoryLimit int64, @@ -209,7 +209,7 @@ func NewExternalSorter( diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, diskAcc *mon.BoundAccount, -) colexecbase.Operator { +) colexecop.Operator { // The cache mode is chosen to reuse the cache to have a smaller cache per // partition without affecting performance. diskQueueCfg.CacheMode = colcontainer.DiskQueueCacheModeReuseCache @@ -221,14 +221,14 @@ func NewExternalSorter( // TODO(asubiotto): this number should be tuned. maxNumberPartitions = fdSemaphore.GetLimit() / 16 } - if maxNumberPartitions < colexecbase.ExternalSorterMinPartitions { - maxNumberPartitions = colexecbase.ExternalSorterMinPartitions + if maxNumberPartitions < colexecop.ExternalSorterMinPartitions { + maxNumberPartitions = colexecop.ExternalSorterMinPartitions } if memoryLimit == 1 { // If memory limit is 1, we're likely in a "force disk spill" // scenario, but we don't want to artificially limit batches when we // have already spilled, so we'll use a larger limit. - memoryLimit = colexecbase.DefaultMemoryLimit + memoryLimit = colexecop.DefaultMemoryLimit } // Each disk queue will use up to BufferSizeBytes of RAM, so we reduce the // memoryLimit of the partitions to sort in memory by those cache sizes. @@ -262,7 +262,7 @@ func NewExternalSorter( partitionedDiskQueueSemaphore = nil } es := &externalSorter{ - OneInputNode: colexecbase.NewOneInputNode(inMemSorter), + OneInputNode: colexecop.NewOneInputNode(inMemSorter), mergeUnlimitedAllocator: mergeUnlimitedAllocator, outputUnlimitedAllocator: outputUnlimitedAllocator, mergeMemoryLimit: mergeMemoryLimit, @@ -492,7 +492,7 @@ func (s *externalSorter) shouldMergeAllPartitions() bool { } func (s *externalSorter) Reset(ctx context.Context) { - if r, ok := s.Input.(colexecbase.Resetter); ok { + if r, ok := s.Input.(colexecop.Resetter); ok { r.Reset(ctx) } s.state = externalSorterNewPartition @@ -552,7 +552,7 @@ func (s *externalSorter) createPartitionerToOperators() { // partitions in [firstPartitionIdx, firstPartitionIdx+numPartitions) range. func (s *externalSorter) createMergerForPartitions( ctx context.Context, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { s.createPartitionerToOperators() syncInputs := make([]SynchronizerInput, s.numPartitions) for i := range syncInputs { @@ -592,10 +592,10 @@ func (s *externalSorter) createMergerForPartitions( } func newInputPartitioningOperator( - input colexecbase.Operator, memoryLimit int64, -) colexecbase.ResettableOperator { + input colexecop.Operator, memoryLimit int64, +) colexecop.ResettableOperator { return &inputPartitioningOperator{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), memoryLimit: memoryLimit, } } @@ -605,8 +605,8 @@ func newInputPartitioningOperator( // limit. From that point, the operator returns a zero-length batch (until it is // reset). type inputPartitioningOperator struct { - colexecbase.OneInputNode - colexecbase.NonExplainable + colexecop.OneInputNode + colexecop.NonExplainable // memoryLimit determines the size of each partition. memoryLimit int64 @@ -632,7 +632,7 @@ type inputPartitioningOperator struct { interceptReset bool } -var _ colexecbase.ResettableOperator = &inputPartitioningOperator{} +var _ colexecop.ResettableOperator = &inputPartitioningOperator{} func (o *inputPartitioningOperator) Init() { o.Input.Init() @@ -658,7 +658,7 @@ func (o *inputPartitioningOperator) Next(ctx context.Context) coldata.Batch { func (o *inputPartitioningOperator) Reset(ctx context.Context) { if !o.interceptReset { - if r, ok := o.Input.(colexecbase.Resetter); ok { + if r, ok := o.Input.(colexecop.Resetter); ok { r.Reset(ctx) } } diff --git a/pkg/sql/colexec/external_sort_test.go b/pkg/sql/colexec/external_sort_test.go index 5f8d9459f65f..4bcf4d52864d 100644 --- a/pkg/sql/colexec/external_sort_test.go +++ b/pkg/sql/colexec/external_sort_test.go @@ -20,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -84,11 +84,11 @@ func TestExternalSort(t *testing.T) { [][]*types.T{tc.typs}, tc.expected, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { // A sorter should never exceed ExternalSorterMinPartitions, even // during repartitioning. A panic will happen if a sorter requests // more than this number of file descriptors. - sem := colexecbase.NewTestingSemaphore(colexecbase.ExternalSorterMinPartitions) + sem := colexecop.NewTestingSemaphore(colexecop.ExternalSorterMinPartitions) // If a limit is satisfied before the sorter is drained of all its // tuples, the sorter will not close its partitioner. During a // flow this will happen in a downstream materializer/outbox, @@ -171,7 +171,7 @@ func TestExternalSortRandomized(t *testing.T) { // limit. With a maximum number of partitions of 2 this will result in // repartitioning twice. To make this a total amount of memory, we also need // to add the cache sizes of the queues. - partitionSize := int64(memoryToSort/4) + int64(colexecbase.ExternalSorterMinPartitions*queueCfg.BufferSizeBytes) + partitionSize := int64(memoryToSort/4) + int64(colexecop.ExternalSorterMinPartitions*queueCfg.BufferSizeBytes) for _, tk := range []execinfra.TestingKnobs{{ForceDiskSpill: true}, {MemoryLimitBytes: partitionSize}} { flowCtx.Cfg.TestingKnobs = tk for nCols := 1; nCols <= maxCols; nCols++ { @@ -203,8 +203,8 @@ func TestExternalSortRandomized(t *testing.T) { []colexectestutils.Tuples{tups}, expected, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { - sem := colexecbase.NewTestingSemaphore(colexecbase.ExternalSorterMinPartitions) + func(input []colexecop.Operator) (colexecop.Operator, error) { + sem := colexecop.NewTestingSemaphore(colexecop.ExternalSorterMinPartitions) semsToCheck = append(semsToCheck, sem) sorter, newAccounts, newMonitors, closers, err := createDiskBackedSorter( ctx, flowCtx, input, typs[:nCols], ordCols, @@ -284,7 +284,7 @@ func TestExternalSortMemoryAccounting(t *testing.T) { batch.ColVec(0).Bytes().Set(i, singleTupleValue) } batch.SetLength(batchLength) - numFDs := colexecbase.ExternalSorterMinPartitions + rng.Intn(3) + numFDs := colexecop.ExternalSorterMinPartitions + rng.Intn(3) // The memory limit in the external sorter is divided as follows: // - BufferSizeBytes for each of the disk queues is subtracted right away // - the remaining part is divided evenly between the sorter and the merger. @@ -293,9 +293,9 @@ func TestExternalSortMemoryAccounting(t *testing.T) { input := colexectestutils.NewFiniteBatchSource(testAllocator, batch, typs, numTotalBatches) var spilled bool - sem := colexecbase.NewTestingSemaphore(numFDs) + sem := colexecop.NewTestingSemaphore(numFDs) sorter, accounts, monitors, closers, err := createDiskBackedSorter( - ctx, flowCtx, []colexecbase.Operator{input}, typs, ordCols, + ctx, flowCtx, []colexecop.Operator{input}, typs, ordCols, 0 /* matchLen */, 0 /* k */, func() { spilled = true }, 0 /* numForcedRepartitions */, false, /* delegateFDAcquisition */ queueCfg, sem, @@ -419,9 +419,9 @@ func BenchmarkExternalSort(b *testing.B) { source := colexectestutils.NewFiniteBatchSource(testAllocator, batch, typs, nBatches) var spilled bool sorter, accounts, monitors, _, err := createDiskBackedSorter( - ctx, flowCtx, []colexecbase.Operator{source}, typs, ordCols, + ctx, flowCtx, []colexecop.Operator{source}, typs, ordCols, 0 /* matchLen */, 0 /* k */, func() { spilled = true }, - 0 /* numForcedRepartitions */, false /* delegateFDAcquisitions */, queueCfg, &colexecbase.TestingSemaphore{}, + 0 /* numForcedRepartitions */, false /* delegateFDAcquisitions */, queueCfg, &colexecop.TestingSemaphore{}, ) memAccounts = append(memAccounts, accounts...) memMonitors = append(memMonitors, monitors...) @@ -455,7 +455,7 @@ func BenchmarkExternalSort(b *testing.B) { func createDiskBackedSorter( ctx context.Context, flowCtx *execinfra.FlowCtx, - input []colexecbase.Operator, + input []colexecop.Operator, typs []*types.T, ordCols []execinfrapb.Ordering_Column, matchLen int, @@ -465,7 +465,7 @@ func createDiskBackedSorter( delegateFDAcquisitions bool, diskQueueCfg colcontainer.DiskQueueCfg, testingSemaphore semaphore.Semaphore, -) (colexecbase.Operator, []*mon.BoundAccount, []*mon.BytesMonitor, []colexecbase.Closer, error) { +) (colexecop.Operator, []*mon.BoundAccount, []*mon.BytesMonitor, []colexecop.Closer, error) { sorterSpec := &execinfrapb.SorterSpec{ OutputOrdering: execinfrapb.Ordering{Columns: ordCols}, OrderingMatchLen: uint32(matchLen), diff --git a/pkg/sql/colexec/hash_aggregator.go b/pkg/sql/colexec/hash_aggregator.go index dc9af0ea25b7..8b2069dfebfb 100644 --- a/pkg/sql/colexec/hash_aggregator.go +++ b/pkg/sql/colexec/hash_aggregator.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -63,7 +63,7 @@ const ( // Note that throughout this file "buckets" and "groups" mean the same thing // and are used interchangeably. type hashAggregator struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator spec *execinfrapb.AggregatorSpec @@ -125,12 +125,12 @@ type hashAggregator struct { aggFnsAlloc *colexecagg.AggregateFuncsAlloc hashAlloc aggBucketAlloc datumAlloc rowenc.DatumAlloc - toClose colexecbase.Closers + toClose colexecop.Closers } -var _ colexecbase.ResettableOperator = &hashAggregator{} -var _ colexecbase.BufferingInMemoryOperator = &hashAggregator{} -var _ colexecbase.ClosableOperator = &hashAggregator{} +var _ colexecop.ResettableOperator = &hashAggregator{} +var _ colexecop.BufferingInMemoryOperator = &hashAggregator{} +var _ colexecop.ClosableOperator = &hashAggregator{} // hashAggregatorAllocSize determines the allocation size used by the hash // aggregator's allocators. This number was chosen after running benchmarks of @@ -148,7 +148,7 @@ const hashAggregatorAllocSize = 128 // tuples. func NewHashAggregator( args *colexecagg.NewAggregatorArgs, newSpillingQueueArgs *colexecutils.NewSpillingQueueArgs, -) (colexecbase.ResettableOperator, error) { +) (colexecop.ResettableOperator, error) { aggFnsAlloc, inputArgsConverter, toClose, err := colexecagg.NewAggregateFuncsAlloc( args, hashAggregatorAllocSize, true, /* isHashAgg */ ) @@ -163,7 +163,7 @@ func NewHashAggregator( maxBuffered = coldata.MaxBatchSize } hashAgg := &hashAggregator{ - OneInputNode: colexecbase.NewOneInputNode(args.Input), + OneInputNode: colexecop.NewOneInputNode(args.Input), allocator: args.Allocator, spec: args.Spec, state: hashAggregatorBuffering, @@ -476,9 +476,7 @@ func (op *hashAggregator) onlineAgg(ctx context.Context, b coldata.Batch) { } } -func (op *hashAggregator) ExportBuffered( - ctx context.Context, _ colexecbase.Operator, -) coldata.Batch { +func (op *hashAggregator) ExportBuffered(ctx context.Context, _ colexecop.Operator) coldata.Batch { if !op.inputTrackingState.zeroBatchEnqueued { // Per the contract of the spilling queue, we need to append a // zero-length batch. @@ -495,7 +493,7 @@ func (op *hashAggregator) ExportBuffered( } func (op *hashAggregator) Reset(ctx context.Context) { - if r, ok := op.Input.(colexecbase.Resetter); ok { + if r, ok := op.Input.(colexecop.Resetter); ok { r.Reset(ctx) } op.bufferingState.tuples.ResetInternalBatch() diff --git a/pkg/sql/colexec/hash_aggregator_test.go b/pkg/sql/colexec/hash_aggregator_test.go index 253d0a116997..16919b177e58 100644 --- a/pkg/sql/colexec/hash_aggregator_test.go +++ b/pkg/sql/colexec/hash_aggregator_test.go @@ -20,8 +20,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -158,7 +158,7 @@ func TestHashAggregator(t *testing.T) { &evalCtx, nil /* semaCtx */, tc.spec.Aggregations, tc.typs, ) require.NoError(t, err) - colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.input}, tc.expected, colexectestutils.UnorderedVerifier, func(sources []colexecbase.Operator) (colexecbase.Operator, error) { + colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.input}, tc.expected, colexectestutils.UnorderedVerifier, func(sources []colexecop.Operator) (colexecop.Operator, error) { return NewHashAggregator(&colexecagg.NewAggregatorArgs{ Allocator: testAllocator, MemAccount: testMemAcc, @@ -201,21 +201,21 @@ func BenchmarkHashAggregatorInputTuplesTracking(b *testing.B) { for _, groupSize := range groupSizes { for _, agg := range []aggType{ { - new: func(args *colexecagg.NewAggregatorArgs) (colexecbase.ResettableOperator, error) { + new: func(args *colexecagg.NewAggregatorArgs) (colexecop.ResettableOperator, error) { return NewHashAggregator(args, nil /* newSpillingQueueArgs */) }, name: "tracking=false", }, { - new: func(args *colexecagg.NewAggregatorArgs) (colexecbase.ResettableOperator, error) { + new: func(args *colexecagg.NewAggregatorArgs) (colexecop.ResettableOperator, error) { spillingQueueMemAcc := testMemMonitor.MakeBoundAccount() memAccounts = append(memAccounts, &spillingQueueMemAcc) return NewHashAggregator(args, &colexecutils.NewSpillingQueueArgs{ UnlimitedAllocator: colmem.NewAllocator(ctx, &spillingQueueMemAcc, testColumnFactory), Types: args.InputTypes, - MemoryLimit: colexecbase.DefaultMemoryLimit, + MemoryLimit: colexecop.DefaultMemoryLimit, DiskQueueCfg: queueCfg, - FDSemaphore: &colexecbase.TestingSemaphore{}, + FDSemaphore: &colexecop.TestingSemaphore{}, DiskAcc: testDiskAcc, }) }, diff --git a/pkg/sql/colexec/hash_based_partitioner.go b/pkg/sql/colexec/hash_based_partitioner.go index fbe68827f6f6..0164a099f058 100644 --- a/pkg/sql/colexec/hash_based_partitioner.go +++ b/pkg/sql/colexec/hash_based_partitioner.go @@ -18,8 +18,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -113,17 +113,17 @@ const ( // "fallback" disk-backed strategy (when the recursive repartitioning doesn't // seem to make progress in reducing the size of the partitions). type hashBasedPartitioner struct { - colexecbase.NonExplainable - colexecbase.CloserHelper + colexecop.NonExplainable + colexecop.CloserHelper unlimitedAllocator *colmem.Allocator name string state hashBasedPartitionerState - inputs []colexecbase.Operator + inputs []colexecop.Operator inputTypes [][]*types.T hashCols [][]uint32 - inMemMainOp colexecbase.ResettableOperator - diskBackedFallbackOp colexecbase.ResettableOperator + inMemMainOp colexecop.ResettableOperator + diskBackedFallbackOp colexecop.ResettableOperator maxPartitionSizeToProcessUsingMain int64 // fdState is used to acquire file descriptors up front. fdState struct { @@ -181,7 +181,7 @@ type hashBasedPartitioner struct { } } -var _ colexecbase.ClosableOperator = &hashBasedPartitioner{} +var _ colexecop.ClosableOperator = &hashBasedPartitioner{} // hbpPartitionInfo is a helper struct that tracks the memory usage of a // partition. Note that if the hash-based partitioner has two inputs, we take @@ -194,7 +194,7 @@ type hbpPartitionInfo struct { // DiskBackedSorterConstructor is used by the external operators to instantiate // a disk-backed sorter used in the fallback strategies. -type DiskBackedSorterConstructor func(input colexecbase.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column, maxNumberPartitions int) colexecbase.Operator +type DiskBackedSorterConstructor func(input colexecop.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column, maxNumberPartitions int) colexecop.Operator // newHashBasedPartitioner returns a disk-backed operator that utilizes // partitioning by hash approach to divide up the input set into separate @@ -207,15 +207,15 @@ func newHashBasedPartitioner( flowCtx *execinfra.FlowCtx, args *colexecargs.NewColOperatorArgs, name string, - inputs []colexecbase.Operator, + inputs []colexecop.Operator, inputTypes [][]*types.T, hashCols [][]uint32, - inMemMainOpConstructor func([]*partitionerToOperator) colexecbase.ResettableOperator, + inMemMainOpConstructor func([]*partitionerToOperator) colexecop.ResettableOperator, diskBackedFallbackOpConstructor func( partitionedInputs []*partitionerToOperator, maxNumberActivePartitions int, fdSemaphore semaphore.Semaphore, - ) colexecbase.ResettableOperator, + ) colexecop.ResettableOperator, diskAcc *mon.BoundAccount, numRequiredActivePartitions int, ) *hashBasedPartitioner { @@ -251,7 +251,7 @@ func newHashBasedPartitioner( // If memory limit is 1, we're likely in a "force disk spill" // scenario, but we don't want to artificially limit batches when we // have already spilled, so we'll use a larger limit. - memoryLimit = colexecbase.DefaultMemoryLimit + memoryLimit = colexecop.DefaultMemoryLimit } maxPartitionSizeToProcessUsingMain := memoryLimit - int64(diskQueuesMemUsed) if maxPartitionSizeToProcessUsingMain < hbpMinimalMaxPartitionSizeForMain { @@ -633,7 +633,7 @@ func (op *hashBasedPartitioner) Close(ctx context.Context) error { } // The in-memory main operator might be a Closer (e.g. the in-memory hash // aggregator), and we need to close it if so. - if c, ok := op.inMemMainOp.(colexecbase.Closer); ok { + if c, ok := op.inMemMainOp.(colexecop.Closer); ok { if err := c.Close(ctx); err != nil { retErr = err } @@ -641,7 +641,7 @@ func (op *hashBasedPartitioner) Close(ctx context.Context) error { // Note that it is ok if the disk-backed fallback operator is not a Closer - // it will still be closed appropriately because we accumulate all closers // in NewColOperatorResult. - if c, ok := op.diskBackedFallbackOp.(colexecbase.Closer); ok { + if c, ok := op.diskBackedFallbackOp.(colexecop.Closer); ok { if err := c.Close(ctx); err != nil { retErr = err } diff --git a/pkg/sql/colexec/hashjoiner_test.go b/pkg/sql/colexec/hashjoiner_test.go index 34a8aadbb40c..afdb25dd5eab 100644 --- a/pkg/sql/colexec/hashjoiner_test.go +++ b/pkg/sql/colexec/hashjoiner_test.go @@ -22,8 +22,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecjoin" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -978,7 +978,7 @@ func createSpecForHashJoiner(tc *joinTestCase) *execinfrapb.ProcessorSpec { func runHashJoinTestCase( t *testing.T, tc *joinTestCase, - hjOpConstructor func(sources []colexecbase.Operator) (colexecbase.Operator, error), + hjOpConstructor func(sources []colexecop.Operator) (colexecop.Operator, error), ) { tc.init() inputs := []colexectestutils.Tuples{tc.leftTuples, tc.rightTuples} @@ -1011,7 +1011,7 @@ func TestHashJoiner(t *testing.T) { for _, tcs := range [][]*joinTestCase{getHJTestCases(), getMJTestCases()} { for _, tc := range tcs { for _, tc := range tc.mutateTypes() { - runHashJoinTestCase(t, tc, func(sources []colexecbase.Operator) (colexecbase.Operator, error) { + runHashJoinTestCase(t, tc, func(sources []colexecop.Operator) (colexecop.Operator, error) { spec := createSpecForHashJoiner(tc) args := &colexecargs.NewColOperatorArgs{ Spec: spec, @@ -1077,7 +1077,7 @@ func BenchmarkHashJoiner(b *testing.B) { b.SetBytes(int64(8 * nBatches * coldata.BatchSize() * nCols * 2)) b.ResetTimer() for i := 0; i < b.N; i++ { - leftSource := colexecbase.NewRepeatableBatchSource(testAllocator, batch, sourceTypes) + leftSource := colexecop.NewRepeatableBatchSource(testAllocator, batch, sourceTypes) rightSource := colexectestutils.NewFiniteBatchSource(testAllocator, batch, sourceTypes, nBatches) joinType := descpb.InnerJoin if fullOuter { @@ -1092,7 +1092,7 @@ func BenchmarkHashJoiner(b *testing.B) { hj := colexecjoin.NewHashJoiner( testAllocator, testAllocator, hjSpec, leftSource, rightSource, - colexecjoin.HashJoinerInitialNumBuckets, colexecbase.DefaultMemoryLimit, + colexecjoin.HashJoinerInitialNumBuckets, colexecop.DefaultMemoryLimit, ) hj.Init() @@ -1163,7 +1163,7 @@ func TestHashJoinerProjection(t *testing.T) { rightSource := colexectestutils.NewOpTestInput(testAllocator, 1, rightTuples, rightTypes) args := &colexecargs.NewColOperatorArgs{ Spec: spec, - Inputs: []colexecbase.Operator{leftSource, rightSource}, + Inputs: []colexecop.Operator{leftSource, rightSource}, StreamingMemAccount: testMemAcc, } args.TestingKnobs.UseStreamingMemAccountForBuffering = true diff --git a/pkg/sql/colexec/is_null_ops.eg.go b/pkg/sql/colexec/is_null_ops.eg.go index 7a413a3e0a6f..ea893e517be0 100644 --- a/pkg/sql/colexec/is_null_ops.eg.go +++ b/pkg/sql/colexec/is_null_ops.eg.go @@ -15,14 +15,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" ) type isNullProjBase struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator colIdx int outputIdx int @@ -36,14 +36,14 @@ type isNullProjBase struct { // (we either have IS NULL or IS NOT NULL with tuple type as the input vector). func NewIsNullProjOp( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, colIdx, outputIdx int, negate bool, isTupleNull bool, -) colexecbase.Operator { +) colexecop.Operator { input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.Bool, outputIdx) base := isNullProjBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: outputIdx, @@ -64,7 +64,7 @@ type isNullProjOp struct { isNullProjBase } -var _ colexecbase.Operator = &isNullProjOp{} +var _ colexecop.Operator = &isNullProjOp{} func (o *isNullProjOp) Init() { o.Input.Init() @@ -128,7 +128,7 @@ type isTupleNullProjOp struct { isNullProjBase } -var _ colexecbase.Operator = &isTupleNullProjOp{} +var _ colexecop.Operator = &isTupleNullProjOp{} func (o *isTupleNullProjOp) Init() { o.Input.Init() @@ -199,7 +199,7 @@ func (o *isTupleNullProjOp) Next(ctx context.Context) coldata.Batch { } type isNullSelBase struct { - colexecbase.OneInputNode + colexecop.OneInputNode colIdx int negate bool } @@ -210,10 +210,10 @@ type isNullSelBase struct { // - isTupleNull indicates whether special "is tuple null" version is needed // (we either have IS NULL or IS NOT NULL with tuple type as the input vector). func NewIsNullSelOp( - input colexecbase.Operator, colIdx int, negate bool, isTupleNull bool, -) colexecbase.Operator { + input colexecop.Operator, colIdx int, negate bool, isTupleNull bool, +) colexecop.Operator { base := isNullSelBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, negate: negate, } @@ -230,7 +230,7 @@ type isNullSelOp struct { isNullSelBase } -var _ colexecbase.Operator = &isNullSelOp{} +var _ colexecop.Operator = &isNullSelOp{} func (o *isNullSelOp) Init() { o.Input.Init() @@ -293,7 +293,7 @@ type isTupleNullSelOp struct { isNullSelBase } -var _ colexecbase.Operator = &isTupleNullSelOp{} +var _ colexecop.Operator = &isTupleNullSelOp{} func (o *isTupleNullSelOp) Init() { o.Input.Init() diff --git a/pkg/sql/colexec/is_null_ops_test.go b/pkg/sql/colexec/is_null_ops_test.go index cb7e58c26877..0a0816bef2a2 100644 --- a/pkg/sql/colexec/is_null_ops_test.go +++ b/pkg/sql/colexec/is_null_ops_test.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -123,7 +123,7 @@ func TestIsNullProjOp(t *testing.T) { for _, c := range testCases { log.Infof(ctx, "%s", c.desc) - opConstructor := func(input []colexecbase.Operator) (colexecbase.Operator, error) { + opConstructor := func(input []colexecop.Operator) (colexecop.Operator, error) { return colexectestutils.CreateTestProjectingOperator( ctx, flowCtx, input[0], []*types.T{types.Int}, fmt.Sprintf("@1 %s", c.projExpr), false /* canFallbackToRowexec */, testMemAcc, @@ -229,7 +229,7 @@ func TestIsNullSelOp(t *testing.T) { for _, c := range testCases { log.Infof(ctx, "%s", c.desc) - opConstructor := func(input []colexecbase.Operator) (colexecbase.Operator, error) { + opConstructor := func(input []colexecop.Operator) (colexecop.Operator, error) { typs := []*types.T{types.Int} spec := &execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{{ColumnTypes: typs}}, diff --git a/pkg/sql/colexec/is_null_ops_tmpl.go b/pkg/sql/colexec/is_null_ops_tmpl.go index 7f982268d37b..fa79f48ae73d 100644 --- a/pkg/sql/colexec/is_null_ops_tmpl.go +++ b/pkg/sql/colexec/is_null_ops_tmpl.go @@ -25,14 +25,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" ) type isNullProjBase struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator colIdx int outputIdx int @@ -46,14 +46,14 @@ type isNullProjBase struct { // (we either have IS NULL or IS NOT NULL with tuple type as the input vector). func NewIsNullProjOp( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, colIdx, outputIdx int, negate bool, isTupleNull bool, -) colexecbase.Operator { +) colexecop.Operator { input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.Bool, outputIdx) base := isNullProjBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: outputIdx, @@ -76,7 +76,7 @@ type is_KINDNullProjOp struct { isNullProjBase } -var _ colexecbase.Operator = &is_KINDNullProjOp{} +var _ colexecop.Operator = &is_KINDNullProjOp{} func (o *is_KINDNullProjOp) Init() { o.Input.Init() @@ -159,7 +159,7 @@ func _COMPUTE_IS_NULL( } // */}} type isNullSelBase struct { - colexecbase.OneInputNode + colexecop.OneInputNode colIdx int negate bool } @@ -170,10 +170,10 @@ type isNullSelBase struct { // - isTupleNull indicates whether special "is tuple null" version is needed // (we either have IS NULL or IS NOT NULL with tuple type as the input vector). func NewIsNullSelOp( - input colexecbase.Operator, colIdx int, negate bool, isTupleNull bool, -) colexecbase.Operator { + input colexecop.Operator, colIdx int, negate bool, isTupleNull bool, +) colexecop.Operator { base := isNullSelBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, negate: negate, } @@ -192,7 +192,7 @@ type is_KINDNullSelOp struct { isNullSelBase } -var _ colexecbase.Operator = &is_KINDNullSelOp{} +var _ colexecop.Operator = &is_KINDNullSelOp{} func (o *is_KINDNullSelOp) Init() { o.Input.Init() diff --git a/pkg/sql/colexec/limit.go b/pkg/sql/colexec/limit.go index fd0d2a81f9d0..29f9b6a39005 100644 --- a/pkg/sql/colexec/limit.go +++ b/pkg/sql/colexec/limit.go @@ -14,13 +14,13 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" ) // limitOp is an operator that implements limit, returning only the first n // tuples from its input. type limitOp struct { - colexecbase.OneInputCloserHelper + colexecop.OneInputCloserHelper limit uint64 @@ -30,13 +30,13 @@ type limitOp struct { done bool } -var _ colexecbase.Operator = &limitOp{} -var _ colexecbase.ClosableOperator = &limitOp{} +var _ colexecop.Operator = &limitOp{} +var _ colexecop.ClosableOperator = &limitOp{} // NewLimitOp returns a new limit operator with the given limit. -func NewLimitOp(input colexecbase.Operator, limit uint64) colexecbase.Operator { +func NewLimitOp(input colexecop.Operator, limit uint64) colexecop.Operator { c := &limitOp{ - OneInputCloserHelper: colexecbase.MakeOneInputCloserHelper(input), + OneInputCloserHelper: colexecop.MakeOneInputCloserHelper(input), limit: limit, } return c diff --git a/pkg/sql/colexec/limit_test.go b/pkg/sql/colexec/limit_test.go index d5fd749510a5..1c0460d0e817 100644 --- a/pkg/sql/colexec/limit_test.go +++ b/pkg/sql/colexec/limit_test.go @@ -14,7 +14,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -67,7 +67,7 @@ func TestLimit(t *testing.T) { for _, tc := range tcs { // The tuples consisting of all nulls still count as separate rows, so if // we replace all values with nulls, we should get the same output. - colexectestutils.RunTestsWithoutAllNullsInjection(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, nil, tc.expected, colexectestutils.OrderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { + colexectestutils.RunTestsWithoutAllNullsInjection(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, nil, tc.expected, colexectestutils.OrderedVerifier, func(input []colexecop.Operator) (colexecop.Operator, error) { return NewLimitOp(input[0], tc.limit), nil }) } diff --git a/pkg/sql/colexec/materializer.go b/pkg/sql/colexec/materializer.go index 838a631e054d..eb5f5a85885f 100644 --- a/pkg/sql/colexec/materializer.go +++ b/pkg/sql/colexec/materializer.go @@ -16,8 +16,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -28,9 +28,9 @@ import ( // Materializer converts an Operator input into a execinfra.RowSource. type Materializer struct { execinfra.ProcessorBase - colexecbase.NonExplainable + colexecop.NonExplainable - input colexecbase.Operator + input colexecop.Operator typs []*types.T drainHelper *drainHelper @@ -62,7 +62,7 @@ type Materializer struct { cancelFlow func() context.CancelFunc // closers is a slice of Closers that should be Closed on termination. - closers colexecbase.Closers + closers colexecop.Closers } // drainHelper is a utility struct that wraps MetadataSources in a RowSource @@ -163,11 +163,11 @@ var materializerEmptyPostProcessSpec = &execinfrapb.PostProcessSpec{} func NewMaterializer( flowCtx *execinfra.FlowCtx, processorID int32, - input colexecbase.Operator, + input colexecop.Operator, typs []*types.T, output execinfra.RowReceiver, metadataSourcesQueue []execinfrapb.MetadataSource, - toClose []colexecbase.Closer, + toClose []colexecop.Closer, execStatsForTrace func() *execinfrapb.ComponentStats, cancelFlow func() context.CancelFunc, ) (*Materializer, error) { diff --git a/pkg/sql/colexec/materializer_test.go b/pkg/sql/colexec/materializer_test.go index eca6e825ed90..bf587e86d061 100644 --- a/pkg/sql/colexec/materializer_test.go +++ b/pkg/sql/colexec/materializer_test.go @@ -20,8 +20,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -204,7 +204,7 @@ func TestMaterializerNextErrorAfterConsumerDone(t *testing.T) { m, err := NewMaterializer( flowCtx, 0, /* processorID */ - &colexecbase.CallbackOperator{}, + &colexecop.CallbackOperator{}, nil, /* typ */ nil, /* output */ []execinfrapb.MetadataSource{metadataSource}, diff --git a/pkg/sql/colexec/mergejoiner_test.go b/pkg/sql/colexec/mergejoiner_test.go index 4bf8c49d9af5..475e67cdd15d 100644 --- a/pkg/sql/colexec/mergejoiner_test.go +++ b/pkg/sql/colexec/mergejoiner_test.go @@ -21,7 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecjoin" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -1655,19 +1655,19 @@ func TestMergeJoiner(t *testing.T) { } // We test all cases with the default memory limit (regular scenario) and a // limit of 1 byte (to force the buffered groups to spill to disk). - for _, memoryLimit := range []int64{1, colexecbase.DefaultMemoryLimit} { + for _, memoryLimit := range []int64{1, colexecop.DefaultMemoryLimit} { log.Infof(context.Background(), "MemoryLimit=%s/%s", humanizeutil.IBytes(memoryLimit), tc.description) runner(t, testAllocator, []colexectestutils.Tuples{tc.leftTuples, tc.rightTuples}, [][]*types.T{tc.leftTypes, tc.rightTypes}, tc.expected, verifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { spec := createSpecForMergeJoiner(tc) args := &colexecargs.NewColOperatorArgs{ Spec: spec, Inputs: input, StreamingMemAccount: testMemAcc, DiskQueueCfg: queueCfg, - FDSemaphore: colexecbase.NewTestingSemaphore(mjFDLimit), + FDSemaphore: colexecop.NewTestingSemaphore(mjFDLimit), } flowCtx.Cfg.TestingKnobs.MemoryLimitBytes = memoryLimit result, err := colexecargs.TestNewColOperator(ctx, flowCtx, args) @@ -1717,8 +1717,8 @@ func TestFullOuterMergeJoinWithMaximumNumberOfGroups(t *testing.T) { leftSource := colexectestutils.NewChunkingBatchSource(testAllocator, typs, colsLeft, nTuples) rightSource := colexectestutils.NewChunkingBatchSource(testAllocator, typs, colsRight, nTuples) a, err := colexecjoin.NewMergeJoinOp( - testAllocator, colexecbase.DefaultMemoryLimit, queueCfg, - colexecbase.NewTestingSemaphore(mjFDLimit), descpb.FullOuterJoin, + testAllocator, colexecop.DefaultMemoryLimit, queueCfg, + colexecop.NewTestingSemaphore(mjFDLimit), descpb.FullOuterJoin, leftSource, rightSource, typs, typs, []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, @@ -1788,8 +1788,8 @@ func TestMergeJoinerMultiBatch(t *testing.T) { leftSource := colexectestutils.NewChunkingBatchSource(testAllocator, typs, cols, nTuples) rightSource := colexectestutils.NewChunkingBatchSource(testAllocator, typs, cols, nTuples) a, err := colexecjoin.NewMergeJoinOp( - testAllocator, colexecbase.DefaultMemoryLimit, - queueCfg, colexecbase.NewTestingSemaphore(mjFDLimit), descpb.InnerJoin, + testAllocator, colexecop.DefaultMemoryLimit, + queueCfg, colexecop.NewTestingSemaphore(mjFDLimit), descpb.InnerJoin, leftSource, rightSource, typs, typs, []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, @@ -1864,8 +1864,8 @@ func TestMergeJoinerMultiBatchRuns(t *testing.T) { leftSource := colexectestutils.NewChunkingBatchSource(testAllocator, typs, cols, nTuples) rightSource := colexectestutils.NewChunkingBatchSource(testAllocator, typs, cols, nTuples) a, err := colexecjoin.NewMergeJoinOp( - testAllocator, colexecbase.DefaultMemoryLimit, - queueCfg, colexecbase.NewTestingSemaphore(mjFDLimit), descpb.InnerJoin, + testAllocator, colexecop.DefaultMemoryLimit, + queueCfg, colexecop.NewTestingSemaphore(mjFDLimit), descpb.InnerJoin, leftSource, rightSource, typs, typs, []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}, {ColIdx: 1, Direction: execinfrapb.Ordering_Column_ASC}}, []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}, {ColIdx: 1, Direction: execinfrapb.Ordering_Column_ASC}}, @@ -1992,8 +1992,8 @@ func TestMergeJoinerRandomized(t *testing.T) { rightSource := colexectestutils.NewChunkingBatchSource(testAllocator, typs, rCols, nTuples) a, err := colexecjoin.NewMergeJoinOp( - testAllocator, colexecbase.DefaultMemoryLimit, - queueCfg, colexecbase.NewTestingSemaphore(mjFDLimit), descpb.InnerJoin, + testAllocator, colexecop.DefaultMemoryLimit, + queueCfg, colexecop.NewTestingSemaphore(mjFDLimit), descpb.InnerJoin, leftSource, rightSource, typs, typs, []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, diff --git a/pkg/sql/colexec/offset.go b/pkg/sql/colexec/offset.go index 0a1cc9b15d8c..d8ddfacdefa0 100644 --- a/pkg/sql/colexec/offset.go +++ b/pkg/sql/colexec/offset.go @@ -14,13 +14,13 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" ) // offsetOp is an operator that implements offset, returning everything // after the first n tuples in its input. type offsetOp struct { - colexecbase.OneInputNode + colexecop.OneInputNode offset uint64 @@ -28,12 +28,12 @@ type offsetOp struct { seen uint64 } -var _ colexecbase.Operator = &offsetOp{} +var _ colexecop.Operator = &offsetOp{} // NewOffsetOp returns a new offset operator with the given offset. -func NewOffsetOp(input colexecbase.Operator, offset uint64) colexecbase.Operator { +func NewOffsetOp(input colexecop.Operator, offset uint64) colexecop.Operator { c := &offsetOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), offset: offset, } return c diff --git a/pkg/sql/colexec/offset_test.go b/pkg/sql/colexec/offset_test.go index a5a9e37e5235..2267a36a239d 100644 --- a/pkg/sql/colexec/offset_test.go +++ b/pkg/sql/colexec/offset_test.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -60,7 +60,7 @@ func TestOffset(t *testing.T) { for _, tc := range tcs { // The tuples consisting of all nulls still count as separate rows, so if // we replace all values with nulls, we should get the same output. - colexectestutils.RunTestsWithoutAllNullsInjection(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, nil, tc.expected, colexectestutils.UnorderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { + colexectestutils.RunTestsWithoutAllNullsInjection(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, nil, tc.expected, colexectestutils.UnorderedVerifier, func(input []colexecop.Operator) (colexecop.Operator, error) { return NewOffsetOp(input[0], tc.offset), nil }) } @@ -71,7 +71,7 @@ func BenchmarkOffset(b *testing.B) { typs := []*types.T{types.Int, types.Int, types.Int} batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) - source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) source.Init() o := NewOffsetOp(source, 1) diff --git a/pkg/sql/colexec/ordered_aggregator.go b/pkg/sql/colexec/ordered_aggregator.go index c52ee6d4bbe5..cc13fe3cba6c 100644 --- a/pkg/sql/colexec/ordered_aggregator.go +++ b/pkg/sql/colexec/ordered_aggregator.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecmisc" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -82,7 +82,7 @@ const ( // output batch if a worst case input batch is encountered (one where every // value is part of a new group). type orderedAggregator struct { - colexecbase.OneInputNode + colexecop.OneInputNode state orderedAggregatorState @@ -133,16 +133,16 @@ type orderedAggregator struct { // observed. seenNonEmptyBatch bool datumAlloc rowenc.DatumAlloc - toClose colexecbase.Closers + toClose colexecop.Closers } -var _ colexecbase.ResettableOperator = &orderedAggregator{} -var _ colexecbase.ClosableOperator = &orderedAggregator{} +var _ colexecop.ResettableOperator = &orderedAggregator{} +var _ colexecop.ClosableOperator = &orderedAggregator{} // NewOrderedAggregator creates an ordered aggregator. func NewOrderedAggregator( args *colexecagg.NewAggregatorArgs, -) (colexecbase.ResettableOperator, error) { +) (colexecop.ResettableOperator, error) { for _, aggFn := range args.Spec.Aggregations { if aggFn.FilterColIdx != nil { return nil, errors.AssertionFailedf("filtering ordered aggregation is not supported") @@ -165,7 +165,7 @@ func NewOrderedAggregator( } a := &orderedAggregator{ - OneInputNode: colexecbase.NewOneInputNode(op), + OneInputNode: colexecop.NewOneInputNode(op), allocator: args.Allocator, spec: args.Spec, groupCol: groupCol, @@ -399,7 +399,7 @@ func (a *orderedAggregator) Next(ctx context.Context) coldata.Batch { } func (a *orderedAggregator) Reset(ctx context.Context) { - if r, ok := a.Input.(colexecbase.Resetter); ok { + if r, ok := a.Input.(colexecop.Resetter); ok { r.Reset(ctx) } a.state = orderedAggregatorAggregating diff --git a/pkg/sql/colexec/ordered_synchronizer.eg.go b/pkg/sql/colexec/ordered_synchronizer.eg.go index 66190b485f93..d02170ed99c8 100644 --- a/pkg/sql/colexec/ordered_synchronizer.eg.go +++ b/pkg/sql/colexec/ordered_synchronizer.eg.go @@ -16,8 +16,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -78,8 +78,8 @@ type OrderedSynchronizer struct { } var ( - _ colexecbase.Operator = &OrderedSynchronizer{} - _ colexecbase.Closer = &OrderedSynchronizer{} + _ colexecop.Operator = &OrderedSynchronizer{} + _ colexecop.Closer = &OrderedSynchronizer{} ) // ChildCount implements the execinfrapb.OpNode interface. diff --git a/pkg/sql/colexec/ordered_synchronizer_test.go b/pkg/sql/colexec/ordered_synchronizer_test.go index 3fbbabc58dcf..b0b33ba34a92 100644 --- a/pkg/sql/colexec/ordered_synchronizer_test.go +++ b/pkg/sql/colexec/ordered_synchronizer_test.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -146,8 +146,8 @@ func TestOrderedSync(t *testing.T) { for i := range typs { typs[i] = types.Int } - colexectestutils.RunTests(t, testAllocator, tc.sources, tc.expected, colexectestutils.OrderedVerifier, func(inputs []colexecbase.Operator) (colexecbase.Operator, error) { - return NewOrderedSynchronizer(testAllocator, colexecbase.DefaultMemoryLimit, operatorsToSynchronizerInputs(inputs), typs, tc.ordering) + colexectestutils.RunTests(t, testAllocator, tc.sources, tc.expected, colexectestutils.OrderedVerifier, func(inputs []colexecop.Operator) (colexecop.Operator, error) { + return NewOrderedSynchronizer(testAllocator, colexecop.DefaultMemoryLimit, operatorsToSynchronizerInputs(inputs), typs, tc.ordering) }) } } @@ -188,7 +188,7 @@ func TestOrderedSyncRandomInput(t *testing.T) { inputs[i].Op = colexectestutils.NewOpTestInput(testAllocator, batchSize, sources[i], typs) } ordering := colinfo.ColumnOrdering{{ColIdx: 0, Direction: encoding.Ascending}} - op, err := NewOrderedSynchronizer(testAllocator, colexecbase.DefaultMemoryLimit, inputs, typs, ordering) + op, err := NewOrderedSynchronizer(testAllocator, colexecop.DefaultMemoryLimit, inputs, typs, ordering) require.NoError(t, err) op.Init() out := colexectestutils.NewOpTestOutput(op, expected) @@ -214,11 +214,11 @@ func BenchmarkOrderedSynchronizer(b *testing.B) { inputs := make([]SynchronizerInput, len(batches)) for i := range batches { - inputs[i].Op = colexecbase.NewRepeatableBatchSource(testAllocator, batches[i], typs) + inputs[i].Op = colexecop.NewRepeatableBatchSource(testAllocator, batches[i], typs) } ordering := colinfo.ColumnOrdering{{ColIdx: 0, Direction: encoding.Ascending}} - op, err := NewOrderedSynchronizer(testAllocator, colexecbase.DefaultMemoryLimit, inputs, typs, ordering) + op, err := NewOrderedSynchronizer(testAllocator, colexecop.DefaultMemoryLimit, inputs, typs, ordering) require.NoError(b, err) op.Init() diff --git a/pkg/sql/colexec/ordered_synchronizer_tmpl.go b/pkg/sql/colexec/ordered_synchronizer_tmpl.go index 6bb4dfc53863..e3ed8c22d5a7 100644 --- a/pkg/sql/colexec/ordered_synchronizer_tmpl.go +++ b/pkg/sql/colexec/ordered_synchronizer_tmpl.go @@ -27,8 +27,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -98,8 +98,8 @@ type OrderedSynchronizer struct { } var ( - _ colexecbase.Operator = &OrderedSynchronizer{} - _ colexecbase.Closer = &OrderedSynchronizer{} + _ colexecop.Operator = &OrderedSynchronizer{} + _ colexecop.Closer = &OrderedSynchronizer{} ) // ChildCount implements the execinfrapb.OpNode interface. diff --git a/pkg/sql/colexec/parallel_unordered_synchronizer.go b/pkg/sql/colexec/parallel_unordered_synchronizer.go index 503e289d4017..0ee10827edfe 100644 --- a/pkg/sql/colexec/parallel_unordered_synchronizer.go +++ b/pkg/sql/colexec/parallel_unordered_synchronizer.go @@ -16,8 +16,8 @@ import ( "sync/atomic" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/errors" @@ -33,7 +33,7 @@ type unorderedSynchronizerMsg struct { meta []execinfrapb.ProducerMetadata } -var _ colexecbase.Operator = &ParallelUnorderedSynchronizer{} +var _ colexecop.Operator = &ParallelUnorderedSynchronizer{} var _ execinfra.OpNode = &ParallelUnorderedSynchronizer{} type parallelUnorderedSynchronizerState int @@ -107,22 +107,22 @@ func (s *ParallelUnorderedSynchronizer) Child(nth int, verbose bool) execinfra.O return s.inputs[nth].Op } -// SynchronizerInput is a wrapper over a colexecbase.Operator that a +// SynchronizerInput is a wrapper over a colexecop.Operator that a // synchronizer goroutine will be calling Next on. An accompanying // []execinfrapb.MetadataSource may also be specified, in which case // DrainMeta will be called from the same goroutine. type SynchronizerInput struct { // Op is the input Operator. - Op colexecbase.Operator + Op colexecop.Operator // MetadataSources are metadata sources in the input tree that should be // drained in the same goroutine as Op. MetadataSources execinfrapb.MetadataSources // ToClose are Closers in the input tree that should be closed in the same // goroutine as Op. - ToClose colexecbase.Closers + ToClose colexecop.Closers } -func operatorsToSynchronizerInputs(ops []colexecbase.Operator) []SynchronizerInput { +func operatorsToSynchronizerInputs(ops []colexecop.Operator) []SynchronizerInput { result := make([]SynchronizerInput, len(ops)) for i := range result { result[i].Op = ops[i] diff --git a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go index dd1b2506cecf..5856cd027ff3 100644 --- a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go +++ b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go @@ -20,8 +20,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -51,7 +51,7 @@ func TestParallelUnorderedSynchronizer(t *testing.T) { inputs := make([]SynchronizerInput, numInputs) for i := range inputs { - source := colexecbase.NewRepeatableBatchSource( + source := colexecop.NewRepeatableBatchSource( testAllocator, coldatatestutils.RandomBatch(testAllocator, rng, typs, coldata.BatchSize(), 0 /* length */, rng.Float64()), typs, @@ -151,7 +151,7 @@ func TestUnorderedSynchronizerNoLeaksOnError(t *testing.T) { ctx := context.Background() inputs := make([]SynchronizerInput, 6) - inputs[0].Op = &colexecbase.CallbackOperator{NextCb: func(context.Context) coldata.Batch { + inputs[0].Op = &colexecop.CallbackOperator{NextCb: func(context.Context) coldata.Batch { colexecerror.InternalError(errors.New(expectedErr)) // This code is unreachable, but the compiler cannot infer that. return nil @@ -160,7 +160,7 @@ func TestUnorderedSynchronizerNoLeaksOnError(t *testing.T) { acc := testMemMonitor.MakeBoundAccount() defer acc.Close(ctx) func(allocator *colmem.Allocator) { - inputs[i].Op = &colexecbase.CallbackOperator{ + inputs[i].Op = &colexecop.CallbackOperator{ NextCb: func(ctx context.Context) coldata.Batch { // All inputs that do not encounter an error will continue to return // batches. @@ -199,7 +199,7 @@ func BenchmarkParallelUnorderedSynchronizer(b *testing.B) { for i := range inputs { batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) - inputs[i].Op = colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + inputs[i].Op = colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) } var wg sync.WaitGroup ctx, cancelFn := context.WithCancel(context.Background()) diff --git a/pkg/sql/colexec/partially_ordered_distinct.go b/pkg/sql/colexec/partially_ordered_distinct.go index 42ddb8f651b6..dd0972cd94fe 100644 --- a/pkg/sql/colexec/partially_ordered_distinct.go +++ b/pkg/sql/colexec/partially_ordered_distinct.go @@ -14,8 +14,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -27,11 +27,11 @@ import ( // columns. func newPartiallyOrderedDistinct( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, distinctCols []uint32, orderedCols []uint32, typs []*types.T, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { if len(orderedCols) == 0 || len(orderedCols) == len(distinctCols) { return nil, errors.AssertionFailedf( "partially ordered distinct wrongfully planned: numDistinctCols=%d "+ @@ -62,7 +62,7 @@ func newPartiallyOrderedDistinct( distinct := NewUnorderedDistinct(allocator, chunkerOperator, distinctUnorderedCols, typs) return &partiallyOrderedDistinct{ input: chunkerOperator, - distinct: distinct.(colexecbase.ResettableOperator), + distinct: distinct.(colexecop.ResettableOperator), }, nil } @@ -72,10 +72,10 @@ func newPartiallyOrderedDistinct( // (where "chunk" is all tuples that are equal on the ordered columns). type partiallyOrderedDistinct struct { input *chunkerOperator - distinct colexecbase.ResettableOperator + distinct colexecop.ResettableOperator } -var _ colexecbase.Operator = &partiallyOrderedDistinct{} +var _ colexecop.Operator = &partiallyOrderedDistinct{} func (p *partiallyOrderedDistinct) ChildCount(bool) int { return 1 @@ -152,7 +152,7 @@ type chunkerOperator struct { windowedBatch coldata.Batch } -var _ colexecbase.ResettableOperator = &chunkerOperator{} +var _ colexecop.ResettableOperator = &chunkerOperator{} func (c *chunkerOperator) ChildCount(bool) int { return 1 diff --git a/pkg/sql/colexec/select_in.eg.go b/pkg/sql/colexec/select_in.eg.go index d1dbbc649dc5..8ced09d2e1b4 100644 --- a/pkg/sql/colexec/select_in.eg.go +++ b/pkg/sql/colexec/select_in.eg.go @@ -21,8 +21,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -55,12 +55,12 @@ const ( func GetInProjectionOperator( allocator *colmem.Allocator, t *types.T, - input colexecbase.Operator, + input colexecop.Operator, colIdx int, resultIdx int, datumTuple *tree.DTuple, negate bool, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.Bool, resultIdx) switch typeconv.TypeFamilyToCanonicalTypeFamily(t.Family()) { case types.BoolFamily: @@ -68,7 +68,7 @@ func GetInProjectionOperator( case -1: default: obj := &projectInOpBool{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -82,7 +82,7 @@ func GetInProjectionOperator( case -1: default: obj := &projectInOpBytes{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -96,7 +96,7 @@ func GetInProjectionOperator( case -1: default: obj := &projectInOpDecimal{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -109,7 +109,7 @@ func GetInProjectionOperator( switch t.Width() { case 16: obj := &projectInOpInt16{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -119,7 +119,7 @@ func GetInProjectionOperator( return obj, nil case 32: obj := &projectInOpInt32{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -130,7 +130,7 @@ func GetInProjectionOperator( case -1: default: obj := &projectInOpInt64{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -144,7 +144,7 @@ func GetInProjectionOperator( case -1: default: obj := &projectInOpFloat64{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -158,7 +158,7 @@ func GetInProjectionOperator( case -1: default: obj := &projectInOpTimestamp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -172,7 +172,7 @@ func GetInProjectionOperator( case -1: default: obj := &projectInOpInterval{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -186,7 +186,7 @@ func GetInProjectionOperator( case -1: default: obj := &projectInOpDatum{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -200,15 +200,15 @@ func GetInProjectionOperator( } func GetInOperator( - t *types.T, input colexecbase.Operator, colIdx int, datumTuple *tree.DTuple, negate bool, -) (colexecbase.Operator, error) { + t *types.T, input colexecop.Operator, colIdx int, datumTuple *tree.DTuple, negate bool, +) (colexecop.Operator, error) { switch typeconv.TypeFamilyToCanonicalTypeFamily(t.Family()) { case types.BoolFamily: switch t.Width() { case -1: default: obj := &selectInOpBool{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, negate: negate, } @@ -220,7 +220,7 @@ func GetInOperator( case -1: default: obj := &selectInOpBytes{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, negate: negate, } @@ -232,7 +232,7 @@ func GetInOperator( case -1: default: obj := &selectInOpDecimal{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, negate: negate, } @@ -243,7 +243,7 @@ func GetInOperator( switch t.Width() { case 16: obj := &selectInOpInt16{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, negate: negate, } @@ -251,7 +251,7 @@ func GetInOperator( return obj, nil case 32: obj := &selectInOpInt32{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, negate: negate, } @@ -260,7 +260,7 @@ func GetInOperator( case -1: default: obj := &selectInOpInt64{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, negate: negate, } @@ -272,7 +272,7 @@ func GetInOperator( case -1: default: obj := &selectInOpFloat64{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, negate: negate, } @@ -284,7 +284,7 @@ func GetInOperator( case -1: default: obj := &selectInOpTimestamp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, negate: negate, } @@ -296,7 +296,7 @@ func GetInOperator( case -1: default: obj := &selectInOpInterval{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, negate: negate, } @@ -308,7 +308,7 @@ func GetInOperator( case -1: default: obj := &selectInOpDatum{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, negate: negate, } @@ -320,17 +320,17 @@ func GetInOperator( } type selectInOpBool struct { - colexecbase.OneInputNode + colexecop.OneInputNode colIdx int filterRow []bool hasNulls bool negate bool } -var _ colexecbase.Operator = &selectInOpBool{} +var _ colexecop.Operator = &selectInOpBool{} type projectInOpBool struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator colIdx int outputIdx int @@ -339,7 +339,7 @@ type projectInOpBool struct { negate bool } -var _ colexecbase.Operator = &projectInOpBool{} +var _ colexecop.Operator = &projectInOpBool{} func fillDatumRowBool(t *types.T, datumTuple *tree.DTuple) ([]bool, bool) { conv := colconv.GetDatumToPhysicalFn(t) @@ -562,17 +562,17 @@ func (pi *projectInOpBool) Next(ctx context.Context) coldata.Batch { } type selectInOpBytes struct { - colexecbase.OneInputNode + colexecop.OneInputNode colIdx int filterRow [][]byte hasNulls bool negate bool } -var _ colexecbase.Operator = &selectInOpBytes{} +var _ colexecop.Operator = &selectInOpBytes{} type projectInOpBytes struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator colIdx int outputIdx int @@ -581,7 +581,7 @@ type projectInOpBytes struct { negate bool } -var _ colexecbase.Operator = &projectInOpBytes{} +var _ colexecop.Operator = &projectInOpBytes{} func fillDatumRowBytes(t *types.T, datumTuple *tree.DTuple) ([][]byte, bool) { conv := colconv.GetDatumToPhysicalFn(t) @@ -792,17 +792,17 @@ func (pi *projectInOpBytes) Next(ctx context.Context) coldata.Batch { } type selectInOpDecimal struct { - colexecbase.OneInputNode + colexecop.OneInputNode colIdx int filterRow []apd.Decimal hasNulls bool negate bool } -var _ colexecbase.Operator = &selectInOpDecimal{} +var _ colexecop.Operator = &selectInOpDecimal{} type projectInOpDecimal struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator colIdx int outputIdx int @@ -811,7 +811,7 @@ type projectInOpDecimal struct { negate bool } -var _ colexecbase.Operator = &projectInOpDecimal{} +var _ colexecop.Operator = &projectInOpDecimal{} func fillDatumRowDecimal(t *types.T, datumTuple *tree.DTuple) ([]apd.Decimal, bool) { conv := colconv.GetDatumToPhysicalFn(t) @@ -1026,17 +1026,17 @@ func (pi *projectInOpDecimal) Next(ctx context.Context) coldata.Batch { } type selectInOpInt16 struct { - colexecbase.OneInputNode + colexecop.OneInputNode colIdx int filterRow []int16 hasNulls bool negate bool } -var _ colexecbase.Operator = &selectInOpInt16{} +var _ colexecop.Operator = &selectInOpInt16{} type projectInOpInt16 struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator colIdx int outputIdx int @@ -1045,7 +1045,7 @@ type projectInOpInt16 struct { negate bool } -var _ colexecbase.Operator = &projectInOpInt16{} +var _ colexecop.Operator = &projectInOpInt16{} func fillDatumRowInt16(t *types.T, datumTuple *tree.DTuple) ([]int16, bool) { conv := colconv.GetDatumToPhysicalFn(t) @@ -1271,17 +1271,17 @@ func (pi *projectInOpInt16) Next(ctx context.Context) coldata.Batch { } type selectInOpInt32 struct { - colexecbase.OneInputNode + colexecop.OneInputNode colIdx int filterRow []int32 hasNulls bool negate bool } -var _ colexecbase.Operator = &selectInOpInt32{} +var _ colexecop.Operator = &selectInOpInt32{} type projectInOpInt32 struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator colIdx int outputIdx int @@ -1290,7 +1290,7 @@ type projectInOpInt32 struct { negate bool } -var _ colexecbase.Operator = &projectInOpInt32{} +var _ colexecop.Operator = &projectInOpInt32{} func fillDatumRowInt32(t *types.T, datumTuple *tree.DTuple) ([]int32, bool) { conv := colconv.GetDatumToPhysicalFn(t) @@ -1516,17 +1516,17 @@ func (pi *projectInOpInt32) Next(ctx context.Context) coldata.Batch { } type selectInOpInt64 struct { - colexecbase.OneInputNode + colexecop.OneInputNode colIdx int filterRow []int64 hasNulls bool negate bool } -var _ colexecbase.Operator = &selectInOpInt64{} +var _ colexecop.Operator = &selectInOpInt64{} type projectInOpInt64 struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator colIdx int outputIdx int @@ -1535,7 +1535,7 @@ type projectInOpInt64 struct { negate bool } -var _ colexecbase.Operator = &projectInOpInt64{} +var _ colexecop.Operator = &projectInOpInt64{} func fillDatumRowInt64(t *types.T, datumTuple *tree.DTuple) ([]int64, bool) { conv := colconv.GetDatumToPhysicalFn(t) @@ -1761,17 +1761,17 @@ func (pi *projectInOpInt64) Next(ctx context.Context) coldata.Batch { } type selectInOpFloat64 struct { - colexecbase.OneInputNode + colexecop.OneInputNode colIdx int filterRow []float64 hasNulls bool negate bool } -var _ colexecbase.Operator = &selectInOpFloat64{} +var _ colexecop.Operator = &selectInOpFloat64{} type projectInOpFloat64 struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator colIdx int outputIdx int @@ -1780,7 +1780,7 @@ type projectInOpFloat64 struct { negate bool } -var _ colexecbase.Operator = &projectInOpFloat64{} +var _ colexecop.Operator = &projectInOpFloat64{} func fillDatumRowFloat64(t *types.T, datumTuple *tree.DTuple) ([]float64, bool) { conv := colconv.GetDatumToPhysicalFn(t) @@ -2014,17 +2014,17 @@ func (pi *projectInOpFloat64) Next(ctx context.Context) coldata.Batch { } type selectInOpTimestamp struct { - colexecbase.OneInputNode + colexecop.OneInputNode colIdx int filterRow []time.Time hasNulls bool negate bool } -var _ colexecbase.Operator = &selectInOpTimestamp{} +var _ colexecop.Operator = &selectInOpTimestamp{} type projectInOpTimestamp struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator colIdx int outputIdx int @@ -2033,7 +2033,7 @@ type projectInOpTimestamp struct { negate bool } -var _ colexecbase.Operator = &projectInOpTimestamp{} +var _ colexecop.Operator = &projectInOpTimestamp{} func fillDatumRowTimestamp(t *types.T, datumTuple *tree.DTuple) ([]time.Time, bool) { conv := colconv.GetDatumToPhysicalFn(t) @@ -2255,17 +2255,17 @@ func (pi *projectInOpTimestamp) Next(ctx context.Context) coldata.Batch { } type selectInOpInterval struct { - colexecbase.OneInputNode + colexecop.OneInputNode colIdx int filterRow []duration.Duration hasNulls bool negate bool } -var _ colexecbase.Operator = &selectInOpInterval{} +var _ colexecop.Operator = &selectInOpInterval{} type projectInOpInterval struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator colIdx int outputIdx int @@ -2274,7 +2274,7 @@ type projectInOpInterval struct { negate bool } -var _ colexecbase.Operator = &projectInOpInterval{} +var _ colexecop.Operator = &projectInOpInterval{} func fillDatumRowInterval(t *types.T, datumTuple *tree.DTuple) ([]duration.Duration, bool) { conv := colconv.GetDatumToPhysicalFn(t) @@ -2293,7 +2293,10 @@ func fillDatumRowInterval(t *types.T, datumTuple *tree.DTuple) ([]duration.Durat } func cmpInInterval( - targetElem duration.Duration, targetCol coldata.Durations, filterRow []duration.Duration, hasNulls bool, + targetElem duration.Duration, + targetCol coldata.Durations, + filterRow []duration.Duration, + hasNulls bool, ) comparisonResult { // Filter row input is already sorted due to normalization, so we can use a // binary search right away. @@ -2489,17 +2492,17 @@ func (pi *projectInOpInterval) Next(ctx context.Context) coldata.Batch { } type selectInOpDatum struct { - colexecbase.OneInputNode + colexecop.OneInputNode colIdx int filterRow []interface{} hasNulls bool negate bool } -var _ colexecbase.Operator = &selectInOpDatum{} +var _ colexecop.Operator = &selectInOpDatum{} type projectInOpDatum struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator colIdx int outputIdx int @@ -2508,7 +2511,7 @@ type projectInOpDatum struct { negate bool } -var _ colexecbase.Operator = &projectInOpDatum{} +var _ colexecop.Operator = &projectInOpDatum{} func fillDatumRowDatum(t *types.T, datumTuple *tree.DTuple) ([]interface{}, bool) { conv := colconv.GetDatumToPhysicalFn(t) diff --git a/pkg/sql/colexec/select_in_test.go b/pkg/sql/colexec/select_in_test.go index 127e1d9c1a51..5fecede5e1c3 100644 --- a/pkg/sql/colexec/select_in_test.go +++ b/pkg/sql/colexec/select_in_test.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -74,9 +74,9 @@ func TestSelectInInt64(t *testing.T) { for _, c := range testCases { log.Infof(context.Background(), "%s", c.desc) - opConstructor := func(input []colexecbase.Operator) (colexecbase.Operator, error) { + opConstructor := func(input []colexecop.Operator) (colexecop.Operator, error) { op := selectInOpInt64{ - OneInputNode: colexecbase.NewOneInputNode(input[0]), + OneInputNode: colexecop.NewOneInputNode(input[0]), colIdx: 0, filterRow: c.filterRow, negate: c.negate, @@ -130,10 +130,10 @@ func benchmarkSelectInInt64(b *testing.B, useSelectionVector bool, hasNulls bool } } - source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) source.Init() inOp := &selectInOpInt64{ - OneInputNode: colexecbase.NewOneInputNode(source), + OneInputNode: colexecop.NewOneInputNode(source), colIdx: 0, filterRow: []int64{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}, } @@ -216,7 +216,7 @@ func TestProjectInInt64(t *testing.T) { for _, c := range testCases { log.Infof(ctx, "%s", c.desc) colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{c.inputTuples}, c.outputTuples, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return colexectestutils.CreateTestProjectingOperator( ctx, flowCtx, input[0], []*types.T{types.Int}, fmt.Sprintf("@1 %s", c.inClause), false /* canFallbackToRowexec */, testMemAcc, diff --git a/pkg/sql/colexec/select_in_tmpl.go b/pkg/sql/colexec/select_in_tmpl.go index 65590a0becb4..f2aaff803a94 100644 --- a/pkg/sql/colexec/select_in_tmpl.go +++ b/pkg/sql/colexec/select_in_tmpl.go @@ -28,8 +28,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -80,12 +80,12 @@ const ( func GetInProjectionOperator( allocator *colmem.Allocator, t *types.T, - input colexecbase.Operator, + input colexecop.Operator, colIdx int, resultIdx int, datumTuple *tree.DTuple, negate bool, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { input = colexecutils.NewVectorTypeEnforcer(allocator, input, types.Bool, resultIdx) switch typeconv.TypeFamilyToCanonicalTypeFamily(t.Family()) { // {{range .}} @@ -94,7 +94,7 @@ func GetInProjectionOperator( // {{range .WidthOverloads}} case _TYPE_WIDTH: obj := &projectInOp_TYPE{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, @@ -110,8 +110,8 @@ func GetInProjectionOperator( } func GetInOperator( - t *types.T, input colexecbase.Operator, colIdx int, datumTuple *tree.DTuple, negate bool, -) (colexecbase.Operator, error) { + t *types.T, input colexecop.Operator, colIdx int, datumTuple *tree.DTuple, negate bool, +) (colexecop.Operator, error) { switch typeconv.TypeFamilyToCanonicalTypeFamily(t.Family()) { // {{range .}} case _CANONICAL_TYPE_FAMILY: @@ -119,7 +119,7 @@ func GetInOperator( // {{range .WidthOverloads}} case _TYPE_WIDTH: obj := &selectInOp_TYPE{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), colIdx: colIdx, negate: negate, } @@ -136,17 +136,17 @@ func GetInOperator( // {{range .WidthOverloads}} type selectInOp_TYPE struct { - colexecbase.OneInputNode + colexecop.OneInputNode colIdx int filterRow []_GOTYPE hasNulls bool negate bool } -var _ colexecbase.Operator = &selectInOp_TYPE{} +var _ colexecop.Operator = &selectInOp_TYPE{} type projectInOp_TYPE struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator colIdx int outputIdx int @@ -155,7 +155,7 @@ type projectInOp_TYPE struct { negate bool } -var _ colexecbase.Operator = &projectInOp_TYPE{} +var _ colexecop.Operator = &projectInOp_TYPE{} func fillDatumRow_TYPE(t *types.T, datumTuple *tree.DTuple) ([]_GOTYPE, bool) { conv := colconv.GetDatumToPhysicalFn(t) diff --git a/pkg/sql/colexec/serial_unordered_synchronizer.go b/pkg/sql/colexec/serial_unordered_synchronizer.go index 7c3134dd10c1..44faf47cb91c 100644 --- a/pkg/sql/colexec/serial_unordered_synchronizer.go +++ b/pkg/sql/colexec/serial_unordered_synchronizer.go @@ -14,7 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" ) @@ -32,9 +32,9 @@ type SerialUnorderedSynchronizer struct { } var ( - _ colexecbase.Operator = &SerialUnorderedSynchronizer{} - _ execinfra.OpNode = &SerialUnorderedSynchronizer{} - _ colexecbase.Closer = &SerialUnorderedSynchronizer{} + _ colexecop.Operator = &SerialUnorderedSynchronizer{} + _ execinfra.OpNode = &SerialUnorderedSynchronizer{} + _ colexecop.Closer = &SerialUnorderedSynchronizer{} ) // ChildCount implements the execinfra.OpNode interface. diff --git a/pkg/sql/colexec/serial_unordered_synchronizer_test.go b/pkg/sql/colexec/serial_unordered_synchronizer_test.go index 6980e18cf774..6af2e7fad5d5 100644 --- a/pkg/sql/colexec/serial_unordered_synchronizer_test.go +++ b/pkg/sql/colexec/serial_unordered_synchronizer_test.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -39,7 +39,7 @@ func TestSerialUnorderedSynchronizer(t *testing.T) { inputs := make([]SynchronizerInput, numInputs) for i := range inputs { batch := coldatatestutils.RandomBatch(testAllocator, rng, typs, coldata.BatchSize(), 0 /* length */, rng.Float64()) - source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) source.ResetBatchesToReturn(numBatches) inputIdx := i inputs[i] = SynchronizerInput{ diff --git a/pkg/sql/colexec/sort.go b/pkg/sql/colexec/sort.go index 05e9c4f1d6d5..0ef3b72c33d2 100644 --- a/pkg/sql/colexec/sort.go +++ b/pkg/sql/colexec/sort.go @@ -16,8 +16,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -30,10 +30,10 @@ import ( // in the input operator. func NewSorter( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { return newSorter(allocator, newAllSpooler(allocator, input, inputTypes), inputTypes, orderingCols) } @@ -42,7 +42,7 @@ func newSorter( input spooler, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column, -) (colexecbase.ResettableOperator, error) { +) (colexecop.ResettableOperator, error) { partitioners := make([]partitioner, len(orderingCols)-1) var err error @@ -99,8 +99,8 @@ type spooler interface { // allSpooler is the spooler that spools all tuples from the input. It is used // by the general sorter over the whole input. type allSpooler struct { - colexecbase.OneInputNode - colexecbase.NonExplainable + colexecop.OneInputNode + colexecop.NonExplainable allocator *colmem.Allocator // inputTypes contains the types of all of the columns from the input. @@ -114,13 +114,13 @@ type allSpooler struct { } var _ spooler = &allSpooler{} -var _ colexecbase.Resetter = &allSpooler{} +var _ colexecop.Resetter = &allSpooler{} func newAllSpooler( - allocator *colmem.Allocator, input colexecbase.Operator, inputTypes []*types.T, + allocator *colmem.Allocator, input colexecop.Operator, inputTypes []*types.T, ) spooler { return &allSpooler{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, inputTypes: inputTypes, } @@ -176,7 +176,7 @@ func (p *allSpooler) getWindowedBatch(startIdx, endIdx int) coldata.Batch { } func (p *allSpooler) Reset(ctx context.Context) { - if r, ok := p.Input.(colexecbase.Resetter); ok { + if r, ok := p.Input.(colexecop.Resetter); ok { r.Reset(ctx) } p.spooled = false @@ -216,8 +216,8 @@ type sortOp struct { exported int } -var _ colexecbase.BufferingInMemoryOperator = &sortOp{} -var _ colexecbase.Resetter = &sortOp{} +var _ colexecop.BufferingInMemoryOperator = &sortOp{} +var _ colexecop.Resetter = &sortOp{} // colSorter is a single-column sorter, specialized on a particular type. type colSorter interface { @@ -405,7 +405,7 @@ func (p *sortOp) sort(ctx context.Context) { } func (p *sortOp) Reset(ctx context.Context) { - if r, ok := p.input.(colexecbase.Resetter); ok { + if r, ok := p.input.(colexecop.Resetter); ok { r.Reset(ctx) } p.emitted = 0 @@ -426,7 +426,7 @@ func (p *sortOp) Child(nth int, verbose bool) execinfra.OpNode { return nil } -func (p *sortOp) ExportBuffered(context.Context, colexecbase.Operator) coldata.Batch { +func (p *sortOp) ExportBuffered(context.Context, colexecop.Operator) coldata.Batch { if p.exported == p.input.getNumTuples() { return coldata.ZeroBatch } diff --git a/pkg/sql/colexec/sort_chunks.go b/pkg/sql/colexec/sort_chunks.go index 0693a722e73c..27f372536ed6 100644 --- a/pkg/sql/colexec/sort_chunks.go +++ b/pkg/sql/colexec/sort_chunks.go @@ -15,8 +15,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -30,11 +30,11 @@ import ( // matchLen columns. func NewSortChunks( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column, matchLen int, -) (colexecbase.Operator, error) { +) (colexecop.Operator, error) { if matchLen < 1 || matchLen == len(orderingCols) { colexecerror.InternalError(errors.AssertionFailedf( "sort chunks should only be used when the input is "+ @@ -59,15 +59,15 @@ func NewSortChunks( type sortChunksOp struct { allocator *colmem.Allocator input *chunker - sorter colexecbase.ResettableOperator + sorter colexecop.ResettableOperator exportedFromBuffer int exportedFromBatch int windowedBatch coldata.Batch } -var _ colexecbase.Operator = &sortChunksOp{} -var _ colexecbase.BufferingInMemoryOperator = &sortChunksOp{} +var _ colexecop.Operator = &sortChunksOp{} +var _ colexecop.BufferingInMemoryOperator = &sortChunksOp{} func (c *sortChunksOp) ChildCount(verbose bool) int { return 1 @@ -111,7 +111,7 @@ func (c *sortChunksOp) Next(ctx context.Context) coldata.Batch { } } -func (c *sortChunksOp) ExportBuffered(context.Context, colexecbase.Operator) coldata.Batch { +func (c *sortChunksOp) ExportBuffered(context.Context, colexecop.Operator) coldata.Batch { // First, we check whether chunker has buffered up any tuples, and if so, // whether we have exported them all. if c.input.bufferedTuples.Length() > 0 { @@ -199,8 +199,8 @@ const ( // in the middle of processing the input). Instead, sortChunksOp will empty the // buffer when appropriate. type chunker struct { - colexecbase.OneInputNode - colexecbase.NonExplainable + colexecop.OneInputNode + colexecop.NonExplainable allocator *colmem.Allocator // inputTypes contains the types of all of the columns from input. @@ -254,7 +254,7 @@ var _ spooler = &chunker{} func newChunker( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, inputTypes []*types.T, alreadySortedCols []uint32, ) (*chunker, error) { @@ -268,7 +268,7 @@ func newChunker( } deselector := colexecutils.NewDeselectorOp(allocator, input, inputTypes) return &chunker{ - OneInputNode: colexecbase.NewOneInputNode(deselector), + OneInputNode: colexecop.NewOneInputNode(deselector), allocator: allocator, inputTypes: inputTypes, alreadySortedCols: alreadySortedCols, diff --git a/pkg/sql/colexec/sort_chunks_test.go b/pkg/sql/colexec/sort_chunks_test.go index 6608566913ad..c266036b5777 100644 --- a/pkg/sql/colexec/sort_chunks_test.go +++ b/pkg/sql/colexec/sort_chunks_test.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -187,7 +187,7 @@ func TestSortChunks(t *testing.T) { defer log.Scope(t).Close(t) for _, tc := range sortChunksTestCases { - colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.expected, colexectestutils.OrderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { + colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.expected, colexectestutils.OrderedVerifier, func(input []colexecop.Operator) (colexecop.Operator, error) { return NewSortChunks(testAllocator, input[0], tc.typs, tc.ordCols, tc.matchLen) }) } @@ -229,7 +229,7 @@ func TestSortChunksRandomized(t *testing.T) { copy(expected, tups) sort.Slice(expected, less(expected, ordCols)) - colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{sortedTups}, expected, colexectestutils.OrderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { + colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{sortedTups}, expected, colexectestutils.OrderedVerifier, func(input []colexecop.Operator) (colexecop.Operator, error) { return NewSortChunks(testAllocator, input[0], typs[:nCols], ordCols, matchLen) }) } @@ -241,9 +241,9 @@ func BenchmarkSortChunks(b *testing.B) { rng, _ := randutil.NewPseudoRand() ctx := context.Background() - sorterConstructors := []func(*colmem.Allocator, colexecbase.Operator, []*types.T, []execinfrapb.Ordering_Column, int) (colexecbase.Operator, error){ + sorterConstructors := []func(*colmem.Allocator, colexecop.Operator, []*types.T, []execinfrapb.Ordering_Column, int) (colexecop.Operator, error){ NewSortChunks, - func(allocator *colmem.Allocator, input colexecbase.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column, _ int) (colexecbase.Operator, error) { + func(allocator *colmem.Allocator, input colexecop.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column, _ int) (colexecop.Operator, error) { return NewSorter(allocator, input, inputTypes, orderingCols) }, } diff --git a/pkg/sql/colexec/sort_test.go b/pkg/sql/colexec/sort_test.go index 231bcc5717d9..faae7064a022 100644 --- a/pkg/sql/colexec/sort_test.go +++ b/pkg/sql/colexec/sort_test.go @@ -20,8 +20,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -143,7 +143,7 @@ func TestSort(t *testing.T) { defer log.Scope(t).Close(t) for _, tc := range sortAllTestCases { colexectestutils.RunTestsWithTyps(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, [][]*types.T{tc.typs}, tc.expected, colexectestutils.OrderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { + func(input []colexecop.Operator) (colexecop.Operator, error) { return NewSorter(testAllocator, input[0], tc.typs, tc.ordCols) }) } @@ -170,7 +170,7 @@ func TestSortRandomized(t *testing.T) { if topK { expected = expected[:k] } - colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tups}, expected, colexectestutils.OrderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { + colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tups}, expected, colexectestutils.OrderedVerifier, func(input []colexecop.Operator) (colexecop.Operator, error) { if topK { return NewTopKSorter(testAllocator, input[0], typs[:nCols], ordCols, uint64(k)), nil } @@ -256,7 +256,7 @@ func TestAllSpooler(t *testing.T) { }, } for _, tc := range tcs { - colexectestutils.RunTestsWithFn(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, nil, func(t *testing.T, input []colexecbase.Operator) { + colexectestutils.RunTestsWithFn(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, nil, func(t *testing.T, input []colexecop.Operator) { allSpooler := newAllSpooler(testAllocator, input[0], tc.typ) allSpooler.init() allSpooler.spool(context.Background()) @@ -311,7 +311,7 @@ func BenchmarkSort(b *testing.B) { b.ResetTimer() for n := 0; n < b.N; n++ { source := colexectestutils.NewFiniteBatchSource(testAllocator, batch, typs, nBatches) - var sorter colexecbase.Operator + var sorter colexecop.Operator if topK { sorter = NewTopKSorter(testAllocator, source, typs, ordCols, k) } else { diff --git a/pkg/sql/colexec/sorttopk.go b/pkg/sql/colexec/sorttopk.go index 22ff59671f66..ef1607ca16f7 100644 --- a/pkg/sql/colexec/sorttopk.go +++ b/pkg/sql/colexec/sorttopk.go @@ -17,8 +17,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -35,21 +35,21 @@ const ( // must correspond 1-1 with the columns in the input operator. func NewTopKSorter( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column, k uint64, -) colexecbase.Operator { +) colexecop.Operator { return &topKSorter{ allocator: allocator, - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), inputTypes: inputTypes, orderingCols: orderingCols, k: k, } } -var _ colexecbase.BufferingInMemoryOperator = &topKSorter{} +var _ colexecop.BufferingInMemoryOperator = &topKSorter{} // topKSortState represents the state of the sort operator. type topKSortState int @@ -67,7 +67,7 @@ const ( ) type topKSorter struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator orderingCols []execinfrapb.Ordering_Column @@ -269,7 +269,7 @@ func (t *topKSorter) updateComparators(vecIdx int, batch coldata.Batch) { } } -func (t *topKSorter) ExportBuffered(context.Context, colexecbase.Operator) coldata.Batch { +func (t *topKSorter) ExportBuffered(context.Context, colexecop.Operator) coldata.Batch { topKLen := t.topK.Length() // First, we check whether we have exported all tuples from the topK vector. if t.exportedFromTopK < topKLen { diff --git a/pkg/sql/colexec/sorttopk_test.go b/pkg/sql/colexec/sorttopk_test.go index cbd40c64421c..955b47766253 100644 --- a/pkg/sql/colexec/sorttopk_test.go +++ b/pkg/sql/colexec/sorttopk_test.go @@ -15,7 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -70,7 +70,7 @@ func TestTopKSorter(t *testing.T) { for _, tc := range topKSortTestCases { log.Infof(context.Background(), "%s", tc.description) - colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.expected, colexectestutils.OrderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { + colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.tuples}, tc.expected, colexectestutils.OrderedVerifier, func(input []colexecop.Operator) (colexecop.Operator, error) { return NewTopKSorter(testAllocator, input[0], tc.typs, tc.ordCols, tc.k), nil }) } diff --git a/pkg/sql/colexec/substring.eg.go b/pkg/sql/colexec/substring.eg.go index 1edcde69df55..f38afbf96ca2 100644 --- a/pkg/sql/colexec/substring.eg.go +++ b/pkg/sql/colexec/substring.eg.go @@ -13,8 +13,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" @@ -25,12 +25,12 @@ func newSubstringOperator( typs []*types.T, argumentCols []int, outputIdx int, - input colexecbase.Operator, -) colexecbase.Operator { + input colexecop.Operator, +) colexecop.Operator { startType := typs[argumentCols[1]] lengthType := typs[argumentCols[2]] base := substringFunctionBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, argumentCols: argumentCols, outputIdx: outputIdx, @@ -80,7 +80,7 @@ func newSubstringOperator( } type substringFunctionBase struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator argumentCols []int outputIdx int @@ -94,7 +94,7 @@ type substringInt64Int16Operator struct { substringFunctionBase } -var _ colexecbase.Operator = &substringInt64Int16Operator{} +var _ colexecop.Operator = &substringInt64Int16Operator{} func (s *substringInt64Int16Operator) Next(ctx context.Context) coldata.Batch { batch := s.Input.Next(ctx) @@ -176,7 +176,7 @@ type substringInt64Int32Operator struct { substringFunctionBase } -var _ colexecbase.Operator = &substringInt64Int32Operator{} +var _ colexecop.Operator = &substringInt64Int32Operator{} func (s *substringInt64Int32Operator) Next(ctx context.Context) coldata.Batch { batch := s.Input.Next(ctx) @@ -258,7 +258,7 @@ type substringInt64Int64Operator struct { substringFunctionBase } -var _ colexecbase.Operator = &substringInt64Int64Operator{} +var _ colexecop.Operator = &substringInt64Int64Operator{} func (s *substringInt64Int64Operator) Next(ctx context.Context) coldata.Batch { batch := s.Input.Next(ctx) @@ -340,7 +340,7 @@ type substringInt16Int16Operator struct { substringFunctionBase } -var _ colexecbase.Operator = &substringInt16Int16Operator{} +var _ colexecop.Operator = &substringInt16Int16Operator{} func (s *substringInt16Int16Operator) Next(ctx context.Context) coldata.Batch { batch := s.Input.Next(ctx) @@ -422,7 +422,7 @@ type substringInt16Int32Operator struct { substringFunctionBase } -var _ colexecbase.Operator = &substringInt16Int32Operator{} +var _ colexecop.Operator = &substringInt16Int32Operator{} func (s *substringInt16Int32Operator) Next(ctx context.Context) coldata.Batch { batch := s.Input.Next(ctx) @@ -504,7 +504,7 @@ type substringInt16Int64Operator struct { substringFunctionBase } -var _ colexecbase.Operator = &substringInt16Int64Operator{} +var _ colexecop.Operator = &substringInt16Int64Operator{} func (s *substringInt16Int64Operator) Next(ctx context.Context) coldata.Batch { batch := s.Input.Next(ctx) @@ -586,7 +586,7 @@ type substringInt32Int16Operator struct { substringFunctionBase } -var _ colexecbase.Operator = &substringInt32Int16Operator{} +var _ colexecop.Operator = &substringInt32Int16Operator{} func (s *substringInt32Int16Operator) Next(ctx context.Context) coldata.Batch { batch := s.Input.Next(ctx) @@ -668,7 +668,7 @@ type substringInt32Int32Operator struct { substringFunctionBase } -var _ colexecbase.Operator = &substringInt32Int32Operator{} +var _ colexecop.Operator = &substringInt32Int32Operator{} func (s *substringInt32Int32Operator) Next(ctx context.Context) coldata.Batch { batch := s.Input.Next(ctx) @@ -750,7 +750,7 @@ type substringInt32Int64Operator struct { substringFunctionBase } -var _ colexecbase.Operator = &substringInt32Int64Operator{} +var _ colexecop.Operator = &substringInt32Int64Operator{} func (s *substringInt32Int64Operator) Next(ctx context.Context) coldata.Batch { batch := s.Input.Next(ctx) diff --git a/pkg/sql/colexec/substring_tmpl.go b/pkg/sql/colexec/substring_tmpl.go index b296ae6c38ce..c0c79cb2f7e8 100644 --- a/pkg/sql/colexec/substring_tmpl.go +++ b/pkg/sql/colexec/substring_tmpl.go @@ -23,8 +23,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" @@ -45,12 +45,12 @@ func newSubstringOperator( typs []*types.T, argumentCols []int, outputIdx int, - input colexecbase.Operator, -) colexecbase.Operator { + input colexecop.Operator, +) colexecop.Operator { startType := typs[argumentCols[1]] lengthType := typs[argumentCols[2]] base := substringFunctionBase{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, argumentCols: argumentCols, outputIdx: outputIdx, @@ -78,7 +78,7 @@ func newSubstringOperator( } type substringFunctionBase struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator argumentCols []int outputIdx int @@ -95,7 +95,7 @@ type substring_StartType_LengthTypeOperator struct { substringFunctionBase } -var _ colexecbase.Operator = &substring_StartType_LengthTypeOperator{} +var _ colexecop.Operator = &substring_StartType_LengthTypeOperator{} func (s *substring_StartType_LengthTypeOperator) Next(ctx context.Context) coldata.Batch { batch := s.Input.Next(ctx) diff --git a/pkg/sql/colexec/tuple_proj_op.go b/pkg/sql/colexec/tuple_proj_op.go index 577a9f02bb6a..99a8db66f2c9 100644 --- a/pkg/sql/colexec/tuple_proj_op.go +++ b/pkg/sql/colexec/tuple_proj_op.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -30,12 +30,12 @@ func NewTupleProjOp( inputTypes []*types.T, tupleContentsIdxs []int, outputType *types.T, - input colexecbase.Operator, + input colexecop.Operator, outputIdx int, -) colexecbase.Operator { +) colexecop.Operator { input = colexecutils.NewVectorTypeEnforcer(allocator, input, outputType, outputIdx) return &tupleProjOp{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), allocator: allocator, converter: colconv.NewVecToDatumConverter(len(inputTypes), tupleContentsIdxs), tupleContentsIdxs: tupleContentsIdxs, @@ -45,7 +45,7 @@ func NewTupleProjOp( } type tupleProjOp struct { - colexecbase.OneInputNode + colexecop.OneInputNode allocator *colmem.Allocator converter *colconv.VecToDatumConverter @@ -54,7 +54,7 @@ type tupleProjOp struct { outputIdx int } -var _ colexecbase.Operator = &tupleProjOp{} +var _ colexecop.Operator = &tupleProjOp{} func (t *tupleProjOp) Init() { t.Input.Init() diff --git a/pkg/sql/colexec/types_integration_test.go b/pkg/sql/colexec/types_integration_test.go index 36ee97cb987e..da85e431590a 100644 --- a/pkg/sql/colexec/types_integration_test.go +++ b/pkg/sql/colexec/types_integration_test.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/colserde" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -121,7 +121,7 @@ func TestSQLTypesIntegration(t *testing.T) { // - converting from Arrow format // and returns the resulting batch. type arrowTestOperator struct { - colexecbase.OneInputNode + colexecop.OneInputNode c *colserde.ArrowBatchConverter r *colserde.RecordBatchSerializer @@ -129,16 +129,16 @@ type arrowTestOperator struct { typs []*types.T } -var _ colexecbase.Operator = &arrowTestOperator{} +var _ colexecop.Operator = &arrowTestOperator{} func newArrowTestOperator( - input colexecbase.Operator, + input colexecop.Operator, c *colserde.ArrowBatchConverter, r *colserde.RecordBatchSerializer, typs []*types.T, -) colexecbase.Operator { +) colexecop.Operator { return &arrowTestOperator{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), c: c, r: r, typs: typs, diff --git a/pkg/sql/colexec/unordered_distinct.go b/pkg/sql/colexec/unordered_distinct.go index b2845f61a559..0cd7e8bac7d1 100644 --- a/pkg/sql/colexec/unordered_distinct.go +++ b/pkg/sql/colexec/unordered_distinct.go @@ -15,7 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -23,8 +23,8 @@ import ( // NewUnorderedDistinct creates an unordered distinct on the given distinct // columns. func NewUnorderedDistinct( - allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, typs []*types.T, -) colexecbase.ResettableOperator { + allocator *colmem.Allocator, input colexecop.Operator, distinctCols []uint32, typs []*types.T, +) colexecop.ResettableOperator { // These numbers were chosen after running the micro-benchmarks. const hashTableLoadFactor = 2.0 const hashTableNumBuckets = 128 @@ -40,7 +40,7 @@ func NewUnorderedDistinct( ) return &unorderedDistinct{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), ht: ht, } } @@ -50,7 +50,7 @@ func NewUnorderedDistinct( // distinct tuples from each input batch. Once at least one tuple is appended, // all of the distinct tuples from the batch are emitted in the output. type unorderedDistinct struct { - colexecbase.OneInputNode + colexecop.OneInputNode ht *colexechash.HashTable // lastInputBatch tracks the last input batch read from the input and not @@ -60,8 +60,8 @@ type unorderedDistinct struct { lastInputBatch coldata.Batch } -var _ colexecbase.BufferingInMemoryOperator = &unorderedDistinct{} -var _ colexecbase.ResettableOperator = &unorderedDistinct{} +var _ colexecop.BufferingInMemoryOperator = &unorderedDistinct{} +var _ colexecop.ResettableOperator = &unorderedDistinct{} func (op *unorderedDistinct) Init() { op.Input.Init() @@ -89,7 +89,7 @@ func (op *unorderedDistinct) Next(ctx context.Context) coldata.Batch { } } -func (op *unorderedDistinct) ExportBuffered(context.Context, colexecbase.Operator) coldata.Batch { +func (op *unorderedDistinct) ExportBuffered(context.Context, colexecop.Operator) coldata.Batch { if op.lastInputBatch != nil { batch := op.lastInputBatch op.lastInputBatch = nil @@ -103,7 +103,7 @@ func (op *unorderedDistinct) ExportBuffered(context.Context, colexecbase.Operato // reset resets the unorderedDistinct. func (op *unorderedDistinct) Reset(ctx context.Context) { - if r, ok := op.Input.(colexecbase.Resetter); ok { + if r, ok := op.Input.(colexecop.Resetter); ok { r.Reset(ctx) } op.ht.Reset(ctx) @@ -112,8 +112,8 @@ func (op *unorderedDistinct) Reset(ctx context.Context) { // unorderedDistinctFilterer filters out tuples that are duplicates of the // tuples already emitted by the unordered distinct. type unorderedDistinctFilterer struct { - colexecbase.OneInputNode - colexecbase.NonExplainable + colexecop.OneInputNode + colexecop.NonExplainable ht *colexechash.HashTable // seenBatch tracks whether the operator has already read at least one @@ -121,7 +121,7 @@ type unorderedDistinctFilterer struct { seenBatch bool } -var _ colexecbase.Operator = &unorderedDistinctFilterer{} +var _ colexecop.Operator = &unorderedDistinctFilterer{} func (f *unorderedDistinctFilterer) Init() { f.Input.Init() diff --git a/pkg/sql/colexec/utils.go b/pkg/sql/colexec/utils.go index 9863bd78bd3b..d39a5c7872fa 100644 --- a/pkg/sql/colexec/utils.go +++ b/pkg/sql/colexec/utils.go @@ -15,8 +15,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -38,8 +38,8 @@ func newPartitionerToOperator( // partition on every call to Next. It is a converter from filled in // PartitionedQueue to Operator. type partitionerToOperator struct { - colexecbase.ZeroInputNode - colexecbase.NonExplainable + colexecop.ZeroInputNode + colexecop.NonExplainable allocator *colmem.Allocator types []*types.T @@ -48,7 +48,7 @@ type partitionerToOperator struct { batch coldata.Batch } -var _ colexecbase.Operator = &partitionerToOperator{} +var _ colexecop.Operator = &partitionerToOperator{} func (p *partitionerToOperator) Init() { if p.batch == nil { diff --git a/pkg/sql/colexecbase/BUILD.bazel b/pkg/sql/colexecop/BUILD.bazel similarity index 87% rename from pkg/sql/colexecbase/BUILD.bazel rename to pkg/sql/colexecop/BUILD.bazel index d57fd4b21090..73f3a4b28cab 100644 --- a/pkg/sql/colexecbase/BUILD.bazel +++ b/pkg/sql/colexecop/BUILD.bazel @@ -1,13 +1,13 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( - name = "colexecbase", + name = "colexecop", srcs = [ "constants.go", "operator.go", "testutils.go", ], - importpath = "github.com/cockroachdb/cockroach/pkg/sql/colexecbase", + importpath = "github.com/cockroachdb/cockroach/pkg/sql/colexecop", visibility = ["//visibility:public"], deps = [ "//pkg/col/coldata", @@ -22,10 +22,10 @@ go_library( ) go_test( - name = "colexecbase_test", + name = "colexecop_test", size = "small", srcs = ["dep_test.go"], - embed = [":colexecbase"], + embed = [":colexecop"], deps = [ "//pkg/testutils/buildutil", "//pkg/util/leaktest", diff --git a/pkg/sql/colexecbase/EXECGEN.bzl b/pkg/sql/colexecop/EXECGEN.bzl similarity index 100% rename from pkg/sql/colexecbase/EXECGEN.bzl rename to pkg/sql/colexecop/EXECGEN.bzl diff --git a/pkg/sql/colexecbase/constants.go b/pkg/sql/colexecop/constants.go similarity index 97% rename from pkg/sql/colexecbase/constants.go rename to pkg/sql/colexecop/constants.go index 061ba7118f64..1e9faa899877 100644 --- a/pkg/sql/colexecbase/constants.go +++ b/pkg/sql/colexecop/constants.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package colexecbase +package colexecop // ExternalSorterMinPartitions determines the minimum number of file descriptors // that the external sorter needs to have in order to make progress (when diff --git a/pkg/sql/colexecbase/dep_test.go b/pkg/sql/colexecop/dep_test.go similarity index 91% rename from pkg/sql/colexecbase/dep_test.go rename to pkg/sql/colexecop/dep_test.go index f9e2b8431674..c134353e58d7 100644 --- a/pkg/sql/colexecbase/dep_test.go +++ b/pkg/sql/colexecop/dep_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package colexecbase +package colexecop import ( "testing" @@ -21,7 +21,7 @@ func TestNoLinkForbidden(t *testing.T) { defer leaktest.AfterTest(t)() buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase", true, + "github.com/cockroachdb/cockroach/pkg/sql/colexecop", true, []string{ "github.com/cockroachdb/cockroach/pkg/sql/colcontainer", "github.com/cockroachdb/cockroach/pkg/sql/colexec", diff --git a/pkg/sql/colexecbase/operator.go b/pkg/sql/colexecop/operator.go similarity index 98% rename from pkg/sql/colexecbase/operator.go rename to pkg/sql/colexecop/operator.go index 69c618e3d860..502299d72665 100644 --- a/pkg/sql/colexecbase/operator.go +++ b/pkg/sql/colexecop/operator.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package colexecbase +package colexecop import ( "context" @@ -184,10 +184,10 @@ func NewFeedOperator() *FeedOperator { return &FeedOperator{} } -// Init implements the colexecbase.Operator interface. +// Init implements the colexecop.Operator interface. func (FeedOperator) Init() {} -// Next implements the colexecbase.Operator interface. +// Next implements the colexecop.Operator interface. func (o *FeedOperator) Next(context.Context) coldata.Batch { return o.batch } diff --git a/pkg/sql/colexecbase/testutils.go b/pkg/sql/colexecop/testutils.go similarity index 99% rename from pkg/sql/colexecbase/testutils.go rename to pkg/sql/colexecop/testutils.go index e56105c45440..8064d00e0a58 100644 --- a/pkg/sql/colexecbase/testutils.go +++ b/pkg/sql/colexecop/testutils.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package colexecbase +package colexecop import ( "context" diff --git a/pkg/sql/colfetcher/BUILD.bazel b/pkg/sql/colfetcher/BUILD.bazel index cb6c220b2f3b..4b399b28a9ab 100644 --- a/pkg/sql/colfetcher/BUILD.bazel +++ b/pkg/sql/colfetcher/BUILD.bazel @@ -21,8 +21,8 @@ go_library( "//pkg/sql/catalog/tabledesc", "//pkg/sql/colconv", "//pkg/sql/colencoding", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", diff --git a/pkg/sql/colfetcher/colbatch_scan.go b/pkg/sql/colfetcher/colbatch_scan.go index 18bd51c56282..cae59582b83e 100644 --- a/pkg/sql/colfetcher/colbatch_scan.go +++ b/pkg/sql/colfetcher/colbatch_scan.go @@ -21,8 +21,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -45,7 +45,7 @@ import ( // ColBatchScan is the exec.Operator implementation of TableReader. It reads a table // from kv, presenting it as coldata.Batches via the exec.Operator interface. type ColBatchScan struct { - colexecbase.ZeroInputNode + colexecop.ZeroInputNode spans roachpb.Spans flowCtx *execinfra.FlowCtx rf *cFetcher @@ -68,8 +68,8 @@ type ColBatchScan struct { var _ execinfra.KVReader = &ColBatchScan{} var _ execinfra.Releasable = &ColBatchScan{} -var _ colexecbase.Closer = &ColBatchScan{} -var _ colexecbase.Operator = &ColBatchScan{} +var _ colexecop.Closer = &ColBatchScan{} +var _ colexecop.Operator = &ColBatchScan{} // Init initializes a ColBatchScan. func (s *ColBatchScan) Init() { @@ -308,7 +308,7 @@ func (s *ColBatchScan) Release() { colBatchScanPool.Put(s) } -// Close implements the colexecbase.Closer interface. +// Close implements the colexecop.Closer interface. func (s *ColBatchScan) Close(context.Context) error { if s.tracingSpan != nil { s.tracingSpan.Finish() diff --git a/pkg/sql/colflow/BUILD.bazel b/pkg/sql/colflow/BUILD.bazel index e575231a89eb..97a1218e2c7e 100644 --- a/pkg/sql/colflow/BUILD.bazel +++ b/pkg/sql/colflow/BUILD.bazel @@ -23,8 +23,8 @@ go_library( "//pkg/sql/colexec/colexecargs", "//pkg/sql/colexec/colexechash", "//pkg/sql/colexec/colexecutils", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colflow/colrpc", "//pkg/sql/colmem", "//pkg/sql/execinfra", @@ -85,8 +85,8 @@ go_test( "//pkg/sql/colexec/colexecjoin", "//pkg/sql/colexec/colexectestutils", "//pkg/sql/colexec/colexecutils", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colfetcher", "//pkg/sql/colflow/colrpc", "//pkg/sql/colmem", diff --git a/pkg/sql/colflow/colrpc/BUILD.bazel b/pkg/sql/colflow/colrpc/BUILD.bazel index dbe65e4437ad..053d25172540 100644 --- a/pkg/sql/colflow/colrpc/BUILD.bazel +++ b/pkg/sql/colflow/colrpc/BUILD.bazel @@ -13,8 +13,8 @@ go_library( "//pkg/col/colserde", "//pkg/roachpb", "//pkg/sql/colexec/colexecutils", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", @@ -45,8 +45,8 @@ go_test( "//pkg/roachpb", "//pkg/settings/cluster", "//pkg/sql/colexec/colexecutils", - "//pkg/sql/colexecbase", "//pkg/sql/colexecerror", + "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", diff --git a/pkg/sql/colflow/colrpc/colrpc_test.go b/pkg/sql/colflow/colrpc/colrpc_test.go index 6d145a15dec7..3a9c5fabf566 100644 --- a/pkg/sql/colflow/colrpc/colrpc_test.go +++ b/pkg/sql/colflow/colrpc/colrpc_test.go @@ -24,8 +24,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -190,7 +190,7 @@ func TestOutboxInbox(t *testing.T) { t.Run(fmt.Sprintf("cancellationScenario=%s", cancellationScenarioName), func(t *testing.T) { var ( typs = []*types.T{types.Int} - inputBuffer = colexecbase.NewBatchBuffer() + inputBuffer = colexecop.NewBatchBuffer() // Generate some random behavior before passing the random number // generator to be used in the Outbox goroutine (to avoid races). These // sleep variables enable a sleep for up to half a millisecond with a .25 @@ -272,7 +272,7 @@ func TestOutboxInbox(t *testing.T) { colmem.NewAllocator(ctx, &deselectorMemAcc, coldata.StandardColumnFactory), inputBuffer, typs, ) inputBatches.Init() - outputBatches := colexecbase.NewBatchBuffer() + outputBatches := colexecop.NewBatchBuffer() var readerErr error for { var outputBatch coldata.Batch @@ -573,7 +573,7 @@ func BenchmarkOutboxInbox(b *testing.B) { batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) - input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + input := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) outboxMemAcc := testMemMonitor.MakeBoundAccount() defer outboxMemAcc.Close(ctx) @@ -630,7 +630,7 @@ func TestOutboxStreamIDPropagation(t *testing.T) { var inTags *logtags.Buffer nextDone := make(chan struct{}) - input := &colexecbase.CallbackOperator{NextCb: func(ctx context.Context) coldata.Batch { + input := &colexecop.CallbackOperator{NextCb: func(ctx context.Context) coldata.Batch { b := testAllocator.NewMemBatchWithFixedCapacity(typs, 0) inTags = logtags.FromContext(ctx) nextDone <- struct{}{} diff --git a/pkg/sql/colflow/colrpc/inbox.go b/pkg/sql/colflow/colrpc/inbox.go index 8f4f3b029f96..837ecd8ef0c8 100644 --- a/pkg/sql/colflow/colrpc/inbox.go +++ b/pkg/sql/colflow/colrpc/inbox.go @@ -20,8 +20,8 @@ import ( "github.com/apache/arrow/go/arrow/array" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/colserde" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -48,7 +48,7 @@ type flowStreamServer interface { // RunWithStream (or more specifically, the RPC handler) will unblock Next by // closing the stream. type Inbox struct { - colexecbase.ZeroInputNode + colexecop.ZeroInputNode typs []*types.T allocator *colmem.Allocator @@ -122,7 +122,7 @@ type Inbox struct { } } -var _ colexecbase.Operator = &Inbox{} +var _ colexecop.Operator = &Inbox{} // NewInbox creates a new Inbox. func NewInbox( diff --git a/pkg/sql/colflow/colrpc/outbox.go b/pkg/sql/colflow/colrpc/outbox.go index 2b990a2647e4..5d9f6c660839 100644 --- a/pkg/sql/colflow/colrpc/outbox.go +++ b/pkg/sql/colflow/colrpc/outbox.go @@ -20,8 +20,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/colserde" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -42,7 +42,7 @@ type flowStreamClient interface { // be called with the necessary information to establish a connection to a // given remote endpoint. type Outbox struct { - colexecbase.OneInputNode + colexecop.OneInputNode typs []*types.T @@ -53,7 +53,7 @@ type Outbox struct { draining uint32 metadataSources []execinfrapb.MetadataSource // closers is a slice of Closers that need to be Closed on termination. - closers colexecbase.Closers + closers colexecop.Closers scratch struct { buf *bytes.Buffer @@ -68,10 +68,10 @@ type Outbox struct { // NewOutbox creates a new Outbox. func NewOutbox( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, typs []*types.T, metadataSources []execinfrapb.MetadataSource, - toClose []colexecbase.Closer, + toClose []colexecop.Closer, ) (*Outbox, error) { c, err := colserde.NewArrowBatchConverter(typs) if err != nil { @@ -84,7 +84,7 @@ func NewOutbox( o := &Outbox{ // Add a deselector as selection vectors are not serialized (nor should they // be). - OneInputNode: colexecbase.NewOneInputNode(colexecutils.NewDeselectorOp(allocator, input, typs)), + OneInputNode: colexecop.NewOneInputNode(colexecutils.NewDeselectorOp(allocator, input, typs)), typs: typs, converter: c, serializer: s, diff --git a/pkg/sql/colflow/colrpc/outbox_test.go b/pkg/sql/colflow/colrpc/outbox_test.go index 07f7f57cd089..b5bbf4e64930 100644 --- a/pkg/sql/colflow/colrpc/outbox_test.go +++ b/pkg/sql/colflow/colrpc/outbox_test.go @@ -17,8 +17,8 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -33,7 +33,7 @@ func TestOutboxCatchesPanics(t *testing.T) { ctx := context.Background() var ( - input = colexecbase.NewBatchBuffer() + input = colexecop.NewBatchBuffer() typs = []*types.T{types.Int} rpcLayer = makeMockFlowStreamRPCLayer() ) @@ -83,7 +83,7 @@ func TestOutboxDrainsMetadataSources(t *testing.T) { ctx := context.Background() var ( - input = colexecbase.NewBatchBuffer() + input = colexecop.NewBatchBuffer() typs = []*types.T{types.Int} ) diff --git a/pkg/sql/colflow/invariants_checker.go b/pkg/sql/colflow/invariants_checker.go index fdf811d65da4..a1cf3d8715e2 100644 --- a/pkg/sql/colflow/invariants_checker.go +++ b/pkg/sql/colflow/invariants_checker.go @@ -14,8 +14,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" ) @@ -24,15 +24,15 @@ import ( // are present in the vectorized engine are maintained on all batches. It // should be planned between other Operators in tests. type invariantsChecker struct { - colexecbase.OneInputNode + colexecop.OneInputNode } -var _ colexecbase.Operator = invariantsChecker{} +var _ colexecop.Operator = invariantsChecker{} // newInvariantsChecker creates a new invariantsChecker. -func newInvariantsChecker(input colexecbase.Operator) colexecbase.Operator { +func newInvariantsChecker(input colexecop.Operator) colexecop.Operator { return &invariantsChecker{ - OneInputNode: colexecbase.OneInputNode{Input: input}, + OneInputNode: colexecop.OneInputNode{Input: input}, } } diff --git a/pkg/sql/colflow/panic_injector.go b/pkg/sql/colflow/panic_injector.go index b7ca671d5779..ac10b856e2be 100644 --- a/pkg/sql/colflow/panic_injector.go +++ b/pkg/sql/colflow/panic_injector.go @@ -15,8 +15,8 @@ import ( "math/rand" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/errors" ) @@ -24,11 +24,11 @@ import ( // panicInjector is a helper Operator that will randomly inject panics into // Init and Next methods of the wrapped operator. type panicInjector struct { - colexecbase.OneInputNode + colexecop.OneInputNode rng *rand.Rand } -var _ colexecbase.Operator = panicInjector{} +var _ colexecop.Operator = panicInjector{} const ( // These constants were chosen arbitrarily with the guiding thought that @@ -41,10 +41,10 @@ const ( ) // newPanicInjector creates a new panicInjector. -func newPanicInjector(input colexecbase.Operator) colexecbase.Operator { +func newPanicInjector(input colexecop.Operator) colexecop.Operator { rng, _ := randutil.NewPseudoRand() return &panicInjector{ - OneInputNode: colexecbase.OneInputNode{Input: input}, + OneInputNode: colexecop.OneInputNode{Input: input}, rng: rng, } } diff --git a/pkg/sql/colflow/routers.go b/pkg/sql/colflow/routers.go index a57d7bc8ee8e..8fa5c212e7cb 100644 --- a/pkg/sql/colflow/routers.go +++ b/pkg/sql/colflow/routers.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -140,7 +140,7 @@ func (o *routerOutputOp) Child(nth int, verbose bool) execinfra.OpNode { return nil } -var _ colexecbase.Operator = &routerOutputOp{} +var _ colexecop.Operator = &routerOutputOp{} type routerOutputOpTestingKnobs struct { // blockedThreshold is the number of buffered values above which we consider @@ -402,7 +402,7 @@ const ( // destination for each row. These destinations are exposed as Operators // returned by the constructor. type HashRouter struct { - colexecbase.OneInputNode + colexecop.OneInputNode // hashCols is a slice of indices of the columns used for hashing. hashCols []uint32 @@ -413,7 +413,7 @@ type HashRouter struct { metadataSources execinfrapb.MetadataSources // closers is a slice of Closers that need to be closed when the hash router // terminates. - closers colexecbase.Closers + closers colexecop.Closers // unblockedEventsChan is a channel shared between the HashRouter and its // outputs. outputs send events on this channel when they are unblocked by a @@ -453,7 +453,7 @@ type HashRouter struct { // needs to have a separate disk account. func NewHashRouter( unlimitedAllocators []*colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, types []*types.T, hashCols []uint32, memoryLimit int64, @@ -461,13 +461,13 @@ func NewHashRouter( fdSemaphore semaphore.Semaphore, diskAccounts []*mon.BoundAccount, toDrain []execinfrapb.MetadataSource, - toClose []colexecbase.Closer, -) (*HashRouter, []colexecbase.DrainableOperator) { + toClose []colexecop.Closer, +) (*HashRouter, []colexecop.DrainableOperator) { if diskQueueCfg.CacheMode != colcontainer.DiskQueueCacheModeDefault { colexecerror.InternalError(errors.Errorf("hash router instantiated with incompatible disk queue cache mode: %d", diskQueueCfg.CacheMode)) } outputs := make([]routerOutput, len(unlimitedAllocators)) - outputsAsOps := make([]colexecbase.DrainableOperator, len(unlimitedAllocators)) + outputsAsOps := make([]colexecop.DrainableOperator, len(unlimitedAllocators)) // unblockEventsChan is buffered to 2*numOutputs as we don't want the outputs // writing to it to block. // Unblock events only happen after a corresponding block event. Since these @@ -496,15 +496,15 @@ func NewHashRouter( } func newHashRouterWithOutputs( - input colexecbase.Operator, + input colexecop.Operator, hashCols []uint32, unblockEventsChan <-chan struct{}, outputs []routerOutput, toDrain []execinfrapb.MetadataSource, - toClose []colexecbase.Closer, + toClose []colexecop.Closer, ) *HashRouter { r := &HashRouter{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), hashCols: hashCols, outputs: outputs, closers: toClose, @@ -650,7 +650,7 @@ func (r *HashRouter) processNextBatch(ctx context.Context) bool { // resetForTests resets the HashRouter for a test or benchmark run. func (r *HashRouter) resetForTests(ctx context.Context) { - if i, ok := r.Input.(colexecbase.Resetter); ok { + if i, ok := r.Input.(colexecop.Resetter); ok { i.Reset(ctx) } r.setDrainState(hashRouterDrainStateRunning) diff --git a/pkg/sql/colflow/routers_test.go b/pkg/sql/colflow/routers_test.go index 8f5c9264732c..7879bc3d082b 100644 --- a/pkg/sql/colflow/routers_test.go +++ b/pkg/sql/colflow/routers_test.go @@ -27,8 +27,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -223,7 +223,7 @@ func TestRouterOutputAddBatch(t *testing.T) { memoryLimit: mtc.bytes, diskAcc: tu.testDiskAcc, cfg: queueCfg, - fdSemaphore: colexecbase.NewTestingSemaphore(2), + fdSemaphore: colexecop.NewTestingSemaphore(2), unblockedEventsChan: unblockEventsChan, testingKnobs: routerOutputOpTestingKnobs{ blockedThreshold: tc.blockedThreshold, @@ -269,14 +269,14 @@ func TestRouterOutputNext(t *testing.T) { data, typs, fullSelection := getDataAndFullSelection() testCases := []struct { - unblockEvent func(in colexecbase.Operator, o *routerOutputOp) + unblockEvent func(in colexecop.Operator, o *routerOutputOp) expected colexectestutils.Tuples name string }{ { // ReaderWaitsForData verifies that a reader blocks in Next(ctx) until there // is data available. - unblockEvent: func(in colexecbase.Operator, o *routerOutputOp) { + unblockEvent: func(in colexecop.Operator, o *routerOutputOp) { for { b := in.Next(ctx) pushSelectionIntoBatch(b, fullSelection) @@ -292,7 +292,7 @@ func TestRouterOutputNext(t *testing.T) { { // ReaderWaitsForZeroBatch verifies that a reader blocking on Next will // also get unblocked with no data other than the zero batch. - unblockEvent: func(_ colexecbase.Operator, o *routerOutputOp) { + unblockEvent: func(_ colexecop.Operator, o *routerOutputOp) { o.addBatch(ctx, coldata.ZeroBatch) }, expected: colexectestutils.Tuples{}, @@ -301,7 +301,7 @@ func TestRouterOutputNext(t *testing.T) { { // CancelUnblocksReader verifies that calling cancel on an output unblocks // a reader. - unblockEvent: func(_ colexecbase.Operator, o *routerOutputOp) { + unblockEvent: func(_ colexecop.Operator, o *routerOutputOp) { o.cancel(ctx, nil /* err */) }, expected: colexectestutils.Tuples{}, @@ -334,7 +334,7 @@ func TestRouterOutputNext(t *testing.T) { memoryLimit: mtc.bytes, diskAcc: tu.testDiskAcc, cfg: queueCfg, - fdSemaphore: colexecbase.NewTestingSemaphore(2), + fdSemaphore: colexecop.NewTestingSemaphore(2), unblockedEventsChan: unblockedEventsChan, }, ) @@ -364,7 +364,7 @@ func TestRouterOutputNext(t *testing.T) { tc.unblockEvent(in, o) // Should have data available, pushed by our reader goroutine. - batches := colexecbase.NewBatchBuffer() + batches := colexecop.NewBatchBuffer() out := colexectestutils.NewOpTestOutput(batches, tc.expected) for { b := <-batchChan @@ -394,7 +394,7 @@ func TestRouterOutputNext(t *testing.T) { memoryLimit: mtc.bytes, diskAcc: tu.testDiskAcc, cfg: queueCfg, - fdSemaphore: colexecbase.NewTestingSemaphore(2), + fdSemaphore: colexecop.NewTestingSemaphore(2), unblockedEventsChan: unblockedEventsChan, }, ) @@ -443,7 +443,7 @@ func TestRouterOutputNext(t *testing.T) { memoryLimit: mtc.bytes, diskAcc: tu.testDiskAcc, cfg: queueCfg, - fdSemaphore: colexecbase.NewTestingSemaphore(2), + fdSemaphore: colexecop.NewTestingSemaphore(2), unblockedEventsChan: ch, testingKnobs: routerOutputOpTestingKnobs{ blockedThreshold: blockThreshold, @@ -525,7 +525,7 @@ func TestRouterOutputRandom(t *testing.T) { ) for _, mtc := range memoryTestCases { t.Run(fmt.Sprintf("%s/memoryLimit=%s", testName, humanizeutil.IBytes(mtc.bytes)), func(t *testing.T) { - colexectestutils.RunTestsWithFn(t, tu.testAllocator, []colexectestutils.Tuples{data}, nil, func(t *testing.T, inputs []colexecbase.Operator) { + colexectestutils.RunTestsWithFn(t, tu.testAllocator, []colexectestutils.Tuples{data}, nil, func(t *testing.T, inputs []colexecop.Operator) { var wg sync.WaitGroup unblockedEventsChans := make(chan struct{}, 2) o := newRouterOutputOp( @@ -535,7 +535,7 @@ func TestRouterOutputRandom(t *testing.T) { memoryLimit: mtc.bytes, diskAcc: tu.testDiskAcc, cfg: queueCfg, - fdSemaphore: colexecbase.NewTestingSemaphore(2), + fdSemaphore: colexecop.NewTestingSemaphore(2), unblockedEventsChan: unblockedEventsChans, testingKnobs: routerOutputOpTestingKnobs{ blockedThreshold: blockedThreshold, @@ -612,7 +612,7 @@ func TestRouterOutputRandom(t *testing.T) { } }() - actual := colexecbase.NewBatchBuffer() + actual := colexecop.NewBatchBuffer() // Consumer. wg.Add(1) @@ -646,7 +646,7 @@ func TestRouterOutputRandom(t *testing.T) { } type callbackRouterOutput struct { - colexecbase.ZeroInputNode + colexecop.ZeroInputNode addBatchCb func(coldata.Batch) bool cancelCb func() forwardedErr error @@ -777,7 +777,7 @@ func TestHashRouterCancellation(t *testing.T) { // Never-ending input of 0s. batch := tu.testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) - in := colexecbase.NewRepeatableBatchSource(tu.testAllocator, batch, typs) + in := colexecop.NewRepeatableBatchSource(tu.testAllocator, batch, typs) unbufferedCh := make(chan struct{}) r := newHashRouterWithOutputs(in, []uint32{0}, unbufferedCh, routerOutputs, nil /* toDrain */, nil /* toClose */) @@ -881,7 +881,7 @@ func TestHashRouterOneOutput(t *testing.T) { tu.testAllocator.ReleaseMemory(tu.testAllocator.Used()) diskAcc := tu.testDiskMonitor.MakeBoundAccount() defer diskAcc.Close(ctx) - r, routerOutputs := NewHashRouter([]*colmem.Allocator{tu.testAllocator}, colexectestutils.NewOpFixedSelTestInput(tu.testAllocator, sel, len(sel), data, typs), typs, []uint32{0}, mtc.bytes, queueCfg, colexecbase.NewTestingSemaphore(2), []*mon.BoundAccount{&diskAcc}, nil /* toDrain */, nil /* toClose */) + r, routerOutputs := NewHashRouter([]*colmem.Allocator{tu.testAllocator}, colexectestutils.NewOpFixedSelTestInput(tu.testAllocator, sel, len(sel), data, typs), typs, []uint32{0}, mtc.bytes, queueCfg, colexecop.NewTestingSemaphore(2), []*mon.BoundAccount{&diskAcc}, nil /* toDrain */, nil /* toClose */) if len(routerOutputs) != 1 { t.Fatalf("expected 1 router output but got %d", len(routerOutputs)) @@ -1006,10 +1006,10 @@ func TestHashRouterRandom(t *testing.T) { var expectedDistribution []int for _, mtc := range memoryTestCases { t.Run(fmt.Sprintf(testName+"/memoryLimit=%s", humanizeutil.IBytes(mtc.bytes)), func(t *testing.T) { - colexectestutils.RunTestsWithFn(t, tu.testAllocator, []colexectestutils.Tuples{data}, nil, func(t *testing.T, inputs []colexecbase.Operator) { + colexectestutils.RunTestsWithFn(t, tu.testAllocator, []colexectestutils.Tuples{data}, nil, func(t *testing.T, inputs []colexecop.Operator) { unblockEventsChan := make(chan struct{}, 2*numOutputs) outputs := make([]routerOutput, numOutputs) - outputsAsOps := make([]colexecbase.DrainableOperator, numOutputs) + outputsAsOps := make([]colexecop.DrainableOperator, numOutputs) memoryLimitPerOutput := mtc.bytes / int64(len(outputs)) for i := range outputs { // Create separate monitoring infrastructure as well as @@ -1027,7 +1027,7 @@ func TestHashRouterRandom(t *testing.T) { memoryLimit: memoryLimitPerOutput, diskAcc: &diskAcc, cfg: queueCfg, - fdSemaphore: colexecbase.NewTestingSemaphore(len(outputs) * 2), + fdSemaphore: colexecop.NewTestingSemaphore(len(outputs) * 2), unblockedEventsChan: unblockEventsChan, testingKnobs: routerOutputOpTestingKnobs{ blockedThreshold: blockedThreshold, @@ -1276,7 +1276,7 @@ func BenchmarkHashRouter(b *testing.B) { typs := []*types.T{types.Int} batch := tu.testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) - input := colexecbase.NewRepeatableBatchSource(tu.testAllocator, batch, typs) + input := colexecop.NewRepeatableBatchSource(tu.testAllocator, batch, typs) queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(b, true /* inMem */) defer cleanup() @@ -1295,7 +1295,7 @@ func BenchmarkHashRouter(b *testing.B) { diskAccounts[i] = &diskAcc defer diskAcc.Close(ctx) } - r, outputs := NewHashRouter(allocators, input, typs, []uint32{0}, 64<<20, queueCfg, &colexecbase.TestingSemaphore{}, diskAccounts, nil /* toDrain */, nil /* toClose */) + r, outputs := NewHashRouter(allocators, input, typs, []uint32{0}, 64<<20, queueCfg, &colexecop.TestingSemaphore{}, diskAccounts, nil /* toDrain */, nil /* toClose */) b.SetBytes(8 * int64(coldata.BatchSize()) * int64(numInputBatches)) // We expect distribution to not change. This is a sanity check that // we're resetting properly. diff --git a/pkg/sql/colflow/stats.go b/pkg/sql/colflow/stats.go index a6fec1b5c4b4..0e724ff88d51 100644 --- a/pkg/sql/colflow/stats.go +++ b/pkg/sql/colflow/stats.go @@ -16,8 +16,8 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colflow/colrpc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -29,7 +29,7 @@ import ( // vectorizedStatsCollector is the common interface implemented by collectors. type vectorizedStatsCollector interface { - colexecbase.Operator + colexecop.Operator outputStats(ctx context.Context) } @@ -45,8 +45,8 @@ type childStatsCollector interface { // Next on the underlying Operator and how many batches and tuples were // returned. type batchInfoCollector struct { - colexecbase.Operator - colexecbase.NonExplainable + colexecop.Operator + colexecop.NonExplainable componentID execinfrapb.ComponentID numBatches, numTuples uint64 @@ -62,10 +62,10 @@ type batchInfoCollector struct { childStatsCollectors []childStatsCollector } -var _ colexecbase.Operator = &batchInfoCollector{} +var _ colexecop.Operator = &batchInfoCollector{} func makeBatchInfoCollector( - op colexecbase.Operator, + op colexecop.Operator, id execinfrapb.ComponentID, inputWatch *timeutil.StopWatch, childStatsCollectors []childStatsCollector, @@ -117,7 +117,7 @@ func (bic *batchInfoCollector) getElapsedTime() time.Duration { // (either an operator or a wrapped processor) that performs KV reads that is // present in the chain of operators rooted at 'op'. func newVectorizedStatsCollector( - op colexecbase.Operator, + op colexecop.Operator, kvReader execinfra.KVReader, id execinfrapb.ComponentID, inputWatch *timeutil.StopWatch, @@ -201,7 +201,7 @@ func (vsc *vectorizedStatsCollectorImpl) outputStats(ctx context.Context) { // for streams. In addition to the base stats, newNetworkVectorizedStatsCollector // collects the network latency for a stream. func newNetworkVectorizedStatsCollector( - op colexecbase.Operator, + op colexecop.Operator, id execinfrapb.ComponentID, inputWatch *timeutil.StopWatch, inbox *colrpc.Inbox, diff --git a/pkg/sql/colflow/stats_test.go b/pkg/sql/colflow/stats_test.go index 5f91b68000bf..b766e98fc4c0 100644 --- a/pkg/sql/colflow/stats_test.go +++ b/pkg/sql/colflow/stats_test.go @@ -12,14 +12,14 @@ package colflow import ( "context" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecjoin" "testing" "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecjoin" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -38,7 +38,7 @@ func TestNumBatches(t *testing.T) { tu := newTestUtils(ctx) defer tu.cleanup(ctx) nBatches := 10 - noop := colexecbase.NewNoop(makeFiniteChunksSourceWithBatchSize(tu.testAllocator, nBatches, coldata.BatchSize())) + noop := colexecop.NewNoop(makeFiniteChunksSourceWithBatchSize(tu.testAllocator, nBatches, coldata.BatchSize())) vsc := newVectorizedStatsCollector( noop, nil /* kvReader */, execinfrapb.ComponentID{}, timeutil.NewStopWatch(), nil /* memMonitors */, nil, /* diskMonitors */ @@ -64,7 +64,7 @@ func TestNumTuples(t *testing.T) { defer tu.cleanup(ctx) nBatches := 10 for _, batchSize := range []int{1, 16, 1024} { - noop := colexecbase.NewNoop(makeFiniteChunksSourceWithBatchSize(tu.testAllocator, nBatches, batchSize)) + noop := colexecop.NewNoop(makeFiniteChunksSourceWithBatchSize(tu.testAllocator, nBatches, batchSize)) vsc := newVectorizedStatsCollector( noop, nil /* kvReader */, execinfrapb.ComponentID{}, timeutil.NewStopWatch(), nil /* memMonitors */, nil, /* diskMonitors */ @@ -98,7 +98,7 @@ func TestVectorizedStatsCollector(t *testing.T) { timeSource := timeutil.NewTestTimeSource() mjInputWatch := timeutil.NewTestStopWatch(timeSource.Now) leftSource := &timeAdvancingOperator{ - OneInputNode: colexecbase.NewOneInputNode(makeFiniteChunksSourceWithBatchSize(tu.testAllocator, nBatches, coldata.BatchSize())), + OneInputNode: colexecop.NewOneInputNode(makeFiniteChunksSourceWithBatchSize(tu.testAllocator, nBatches, coldata.BatchSize())), timeSource: timeSource, } leftInput := newVectorizedStatsCollector( @@ -107,7 +107,7 @@ func TestVectorizedStatsCollector(t *testing.T) { nil, /* inputStatsCollectors */ ) rightSource := &timeAdvancingOperator{ - OneInputNode: colexecbase.NewOneInputNode(makeFiniteChunksSourceWithBatchSize(tu.testAllocator, nBatches, coldata.BatchSize())), + OneInputNode: colexecop.NewOneInputNode(makeFiniteChunksSourceWithBatchSize(tu.testAllocator, nBatches, coldata.BatchSize())), timeSource: timeSource, } rightInput := newVectorizedStatsCollector( @@ -116,8 +116,8 @@ func TestVectorizedStatsCollector(t *testing.T) { nil, /* inputStatsCollectors */ ) mergeJoiner, err := colexecjoin.NewMergeJoinOp( - tu.testAllocator, colexecbase.DefaultMemoryLimit, queueCfg, - colexecbase.NewTestingSemaphore(4), descpb.InnerJoin, leftInput, rightInput, + tu.testAllocator, colexecop.DefaultMemoryLimit, queueCfg, + colexecop.NewTestingSemaphore(4), descpb.InnerJoin, leftInput, rightInput, []*types.T{types.Int}, []*types.T{types.Int}, []execinfrapb.Ordering_Column{{ColIdx: 0}}, []execinfrapb.Ordering_Column{{ColIdx: 0}}, @@ -127,7 +127,7 @@ func TestVectorizedStatsCollector(t *testing.T) { t.Fatal(err) } timeAdvancingMergeJoiner := &timeAdvancingOperator{ - OneInputNode: colexecbase.NewOneInputNode(mergeJoiner), + OneInputNode: colexecop.NewOneInputNode(mergeJoiner), timeSource: timeSource, } @@ -162,7 +162,7 @@ func TestVectorizedStatsCollector(t *testing.T) { func makeFiniteChunksSourceWithBatchSize( testAllocator *colmem.Allocator, nBatches int, batchSize int, -) colexecbase.Operator { +) colexecop.Operator { typs := []*types.T{types.Int} batch := testAllocator.NewMemBatchWithFixedCapacity(typs, batchSize) vec := batch.ColVec(0).Int64() @@ -176,12 +176,12 @@ func makeFiniteChunksSourceWithBatchSize( // timeAdvancingOperator is an Operator that advances the time source upon // receiving a non-empty batch from its input. It is used for testing only. type timeAdvancingOperator struct { - colexecbase.OneInputNode + colexecop.OneInputNode timeSource *timeutil.TestTimeSource } -var _ colexecbase.Operator = &timeAdvancingOperator{} +var _ colexecop.Operator = &timeAdvancingOperator{} func (o *timeAdvancingOperator) Init() { o.Input.Init() diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index c2f5d2e20d36..4d4ff31abd48 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -29,8 +29,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colbuilder" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colflow/colrpc" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" @@ -315,9 +315,9 @@ func (f *vectorizedFlow) Cleanup(ctx context.Context) { // created wrapper with those corresponding to operators in inputs (the latter // must have already been wrapped). func (s *vectorizedFlowCreator) wrapWithVectorizedStatsCollectorBase( - op colexecbase.Operator, + op colexecop.Operator, kvReader execinfra.KVReader, - inputs []colexecbase.Operator, + inputs []colexecop.Operator, component execinfrapb.ComponentID, monitors []*mon.BytesMonitor, ) (vectorizedStatsCollector, error) { @@ -352,7 +352,7 @@ func (s *vectorizedFlowCreator) wrapWithNetworkVectorizedStatsCollector( inbox *colrpc.Inbox, component execinfrapb.ComponentID, latency time.Duration, ) (vectorizedStatsCollector, error) { inputWatch := timeutil.NewStopWatch() - op := colexecbase.Operator(inbox) + op := colexecop.Operator(inbox) nvsc := newNetworkVectorizedStatsCollector(op, component, inputWatch, inbox, latency) s.vectorizedStatsCollectorsQueue = append(s.vectorizedStatsCollectorsQueue, nvsc) return nvsc, nil @@ -393,9 +393,9 @@ type flowCreatorHelper interface { // as the metadataSources and closers in this DAG that need to be drained and // closed. type opDAGWithMetaSources struct { - rootOperator colexecbase.Operator + rootOperator colexecop.Operator metadataSources []execinfrapb.MetadataSource - toClose []colexecbase.Closer + toClose []colexecop.Closer } // remoteComponentCreator is an interface that abstracts the constructors for @@ -403,10 +403,10 @@ type opDAGWithMetaSources struct { type remoteComponentCreator interface { newOutbox( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, typs []*types.T, metadataSources []execinfrapb.MetadataSource, - toClose []colexecbase.Closer, + toClose []colexecop.Closer, ) (*colrpc.Outbox, error) newInbox(ctx context.Context, allocator *colmem.Allocator, typs []*types.T, streamID execinfrapb.StreamID) (*colrpc.Inbox, error) } @@ -415,10 +415,10 @@ type vectorizedRemoteComponentCreator struct{} func (vectorizedRemoteComponentCreator) newOutbox( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, typs []*types.T, metadataSources []execinfrapb.MetadataSource, - toClose []colexecbase.Closer, + toClose []colexecop.Closer, ) (*colrpc.Outbox, error) { return colrpc.NewOutbox(allocator, input, typs, metadataSources, toClose) } @@ -489,7 +489,7 @@ type vectorizedFlowCreator struct { numClosers int32 numClosed int32 - inputsScratch []colexecbase.Operator + inputsScratch []colexecop.Operator } var _ execinfra.Releasable = &vectorizedFlowCreator{} @@ -631,11 +631,11 @@ func (s *vectorizedFlowCreator) newStreamingMemAccount( func (s *vectorizedFlowCreator) setupRemoteOutputStream( ctx context.Context, flowCtx *execinfra.FlowCtx, - op colexecbase.Operator, + op colexecop.Operator, outputTyps []*types.T, stream *execinfrapb.StreamEndpointSpec, metadataSourcesQueue []execinfrapb.MetadataSource, - toClose []colexecbase.Closer, + toClose []colexecop.Closer, factory coldata.ColumnFactory, ) (execinfra.OpNode, error) { outbox, err := s.remoteComponentCreator.newOutbox( @@ -690,11 +690,11 @@ func (s *vectorizedFlowCreator) setupRemoteOutputStream( func (s *vectorizedFlowCreator) setupRouter( ctx context.Context, flowCtx *execinfra.FlowCtx, - input colexecbase.Operator, + input colexecop.Operator, outputTyps []*types.T, output *execinfrapb.OutputRouterSpec, metadataSourcesQueue []execinfrapb.MetadataSource, - toClose []colexecbase.Closer, + toClose []colexecop.Closer, factory coldata.ColumnFactory, ) error { if output.Type != execinfrapb.OutputRouterSpec_BY_HASH { @@ -742,7 +742,7 @@ func (s *vectorizedFlowCreator) setupRouter( } case execinfrapb.StreamEndpointSpec_LOCAL: foundLocalOutput = true - localOp := colexecbase.Operator(op) + localOp := colexecop.Operator(op) if s.recordingStats { mons := []*mon.BytesMonitor{hashRouterMemMonitor, diskMon} // Wrap local outputs with vectorized stats collectors when recording @@ -785,7 +785,7 @@ func (s *vectorizedFlowCreator) setupInput( input execinfrapb.InputSyncSpec, opt flowinfra.FuseOpt, factory coldata.ColumnFactory, -) (colexecbase.Operator, []execinfrapb.MetadataSource, []colexecbase.Closer, error) { +) (colexecop.Operator, []execinfrapb.MetadataSource, []colexecop.Closer, error) { inputStreamOps := make([]colexec.SynchronizerInput, 0, len(input.Streams)) // Before we can safely use types we received over the wire in the // operators, we need to make sure they are hydrated. In row execution @@ -830,7 +830,7 @@ func (s *vectorizedFlowCreator) setupInput( return nil, nil, nil, err } s.addStreamEndpoint(inputStream.StreamID, inbox, s.waitGroup) - op := colexecbase.Operator(inbox) + op := colexecop.Operator(inbox) if s.recordingStats { // Note: we can't use flowCtx.StreamComponentID because the stream does // not originate from this node (we are the target node). @@ -863,13 +863,13 @@ func (s *vectorizedFlowCreator) setupInput( } op = os metaSources = []execinfrapb.MetadataSource{os} - toClose = []colexecbase.Closer{os} + toClose = []colexecop.Closer{os} } else { if opt == flowinfra.FuseAggressively { sync := colexec.NewSerialUnorderedSynchronizer(inputStreamOps) op = sync metaSources = []execinfrapb.MetadataSource{sync} - toClose = []colexecbase.Closer{sync} + toClose = []colexecop.Closer{sync} } else { sync := colexec.NewParallelUnorderedSynchronizer(inputStreamOps, s.waitGroup) op = sync @@ -886,7 +886,7 @@ func (s *vectorizedFlowCreator) setupInput( statsInputs = nil } if s.recordingStats { - statsInputsAsOps := make([]colexecbase.Operator, len(statsInputs)) + statsInputsAsOps := make([]colexecop.Operator, len(statsInputs)) for i := range statsInputs { statsInputsAsOps[i] = statsInputs[i].Op } @@ -913,10 +913,10 @@ func (s *vectorizedFlowCreator) setupOutput( ctx context.Context, flowCtx *execinfra.FlowCtx, pspec *execinfrapb.ProcessorSpec, - op colexecbase.Operator, + op colexecop.Operator, opOutputTypes []*types.T, metadataSourcesQueue []execinfrapb.MetadataSource, - toClose []colexecbase.Closer, + toClose []colexecop.Closer, factory coldata.ColumnFactory, ) error { output := &pspec.Output[0] @@ -1030,7 +1030,7 @@ type callbackCloser struct { closeCb func(context.Context) error } -var _ colexecbase.Closer = &callbackCloser{} +var _ colexecop.Closer = &callbackCloser{} // Close implements the Closer interface. func (c *callbackCloser) Close(ctx context.Context) error { @@ -1083,7 +1083,7 @@ func (s *vectorizedFlowCreator) setupFlow( // toClose is similar to metadataSourcesQueue with the difference that these // components do not produce metadata and should be Closed even during // non-graceful termination. - var toClose []colexecbase.Closer + var toClose []colexecop.Closer inputs := s.inputsScratch[:0] for i := range pspec.Input { input, metadataSources, closers, localErr := s.setupInput(ctx, flowCtx, pspec.Input[i], opt, factory) @@ -1136,7 +1136,7 @@ func (s *vectorizedFlowCreator) setupFlow( metadataSourcesQueue = append(metadataSourcesQueue, result.MetadataSources...) if flowCtx.Cfg != nil && flowCtx.Cfg.TestingKnobs.CheckVectorizedFlowIsClosedCorrectly { for _, closer := range result.ToClose { - func(c colexecbase.Closer) { + func(c colexecop.Closer) { closed := false toClose = append(toClose, &callbackCloser{closeCb: func(ctx context.Context) error { if !closed { diff --git a/pkg/sql/colflow/vectorized_flow_shutdown_test.go b/pkg/sql/colflow/vectorized_flow_shutdown_test.go index de0fca9bf47e..189a9500f79b 100644 --- a/pkg/sql/colflow/vectorized_flow_shutdown_test.go +++ b/pkg/sql/colflow/vectorized_flow_shutdown_test.go @@ -23,7 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colflow" "github.com/cockroachdb/cockroach/pkg/sql/colflow/colrpc" "github.com/cockroachdb/cockroach/pkg/sql/colmem" @@ -200,7 +200,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { []uint32{0}, 64<<20, queueCfg, - &colexecbase.TestingSemaphore{}, + &colexecop.TestingSemaphore{}, diskAccounts, toDrain, nil, /* toClose */ @@ -219,7 +219,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { synchronizerInputs = append( synchronizerInputs, colexec.SynchronizerInput{ - Op: colexecbase.Operator(inbox), + Op: colexecop.Operator(inbox), MetadataSources: []execinfrapb.MetadataSource{inbox}, }, ) @@ -238,7 +238,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { ctx context.Context, cancelFn context.CancelFunc, outboxMemAcc *mon.BoundAccount, - outboxInput colexecbase.Operator, + outboxInput colexecop.Operator, inbox *colrpc.Inbox, id int, outboxMetadataSources []execinfrapb.MetadataSource, @@ -248,7 +248,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { idToClosed.Unlock() outbox, err := colrpc.NewOutbox( colmem.NewAllocator(ctx, outboxMemAcc, testColumnFactory), outboxInput, typs, outboxMetadataSources, - []colexecbase.Closer{callbackCloser{closeCb: func() error { + []colexecop.Closer{callbackCloser{closeCb: func() error { idToClosed.Lock() idToClosed.mapping[id] = true idToClosed.Unlock() @@ -304,7 +304,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { ctxRemote, cancelRemote, &outboxMemAccount, - colexecbase.NewRepeatableBatchSource(remoteAllocator, batch, typs), + colexecop.NewRepeatableBatchSource(remoteAllocator, batch, typs), inboxes[i], streamID, []execinfrapb.MetadataSource{createMetadataSourceForID(streamID)}, @@ -313,7 +313,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { streamID++ } - var materializerInput colexecbase.Operator + var materializerInput colexecop.Operator ctxAnotherRemote, cancelAnotherRemote := context.WithCancel(context.Background()) if addAnotherRemote { // Add another "remote" node to the flow. @@ -356,7 +356,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { typs, nil, /* output */ []execinfrapb.MetadataSource{materializerMetadataSource}, - []colexecbase.Closer{callbackCloser{closeCb: func() error { + []colexecop.Closer{callbackCloser{closeCb: func() error { materializerCalledClose = true return nil }}}, /* toClose */ diff --git a/pkg/sql/colflow/vectorized_flow_space_test.go b/pkg/sql/colflow/vectorized_flow_space_test.go index 28e3e69a5549..99ed19dc4b55 100644 --- a/pkg/sql/colflow/vectorized_flow_space_test.go +++ b/pkg/sql/colflow/vectorized_flow_space_test.go @@ -20,8 +20,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colbuilder" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -73,7 +73,7 @@ func TestVectorizeInternalMemorySpaceError(t *testing.T) { for _, tc := range testCases { for _, success := range []bool{true, false} { t.Run(fmt.Sprintf("%s-success-expected-%t", tc.desc, success), func(t *testing.T) { - inputs := []colexecbase.Operator{colexecutils.NewFixedNumTuplesNoInputOp(testAllocator, 0 /* numTuples */)} + inputs := []colexecop.Operator{colexecutils.NewFixedNumTuplesNoInputOp(testAllocator, 0 /* numTuples */)} if len(tc.spec.Input) > 1 { inputs = append(inputs, colexecutils.NewFixedNumTuplesNoInputOp(testAllocator, 0 /* numTuples */)) } @@ -195,9 +195,9 @@ func TestVectorizeAllocatorSpaceError(t *testing.T) { for _, success := range []bool{true, false} { expectNoMemoryError := success || tc.spillingSupported t.Run(fmt.Sprintf("%s-success-expected-%t", tc.desc, expectNoMemoryError), func(t *testing.T) { - inputs := []colexecbase.Operator{colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs)} + inputs := []colexecop.Operator{colexecop.NewRepeatableBatchSource(testAllocator, batch, typs)} if len(tc.spec.Input) > 1 { - inputs = append(inputs, colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs)) + inputs = append(inputs, colexecop.NewRepeatableBatchSource(testAllocator, batch, typs)) } memMon := mon.NewMonitor("MemoryMonitor", mon.MemoryResource, nil, nil, 0, math.MaxInt64, st) flowCtx.Cfg.TestingKnobs = execinfra.TestingKnobs{} @@ -221,7 +221,7 @@ func TestVectorizeAllocatorSpaceError(t *testing.T) { Spec: tc.spec, Inputs: inputs, StreamingMemAccount: &acc, - FDSemaphore: colexecbase.NewTestingSemaphore(256), + FDSemaphore: colexecop.NewTestingSemaphore(256), } // The disk spilling infrastructure relies on different memory // accounts, so if the spilling is supported, we do *not* want to use diff --git a/pkg/sql/colflow/vectorized_flow_test.go b/pkg/sql/colflow/vectorized_flow_test.go index 22aaddfcf648..1aaad1cbd005 100644 --- a/pkg/sql/colflow/vectorized_flow_test.go +++ b/pkg/sql/colflow/vectorized_flow_test.go @@ -20,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colflow/colrpc" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" @@ -35,16 +35,16 @@ import ( ) type callbackRemoteComponentCreator struct { - newOutboxFn func(*colmem.Allocator, colexecbase.Operator, []*types.T, []execinfrapb.MetadataSource) (*colrpc.Outbox, error) + newOutboxFn func(*colmem.Allocator, colexecop.Operator, []*types.T, []execinfrapb.MetadataSource) (*colrpc.Outbox, error) newInboxFn func(allocator *colmem.Allocator, typs []*types.T, streamID execinfrapb.StreamID) (*colrpc.Inbox, error) } func (c callbackRemoteComponentCreator) newOutbox( allocator *colmem.Allocator, - input colexecbase.Operator, + input colexecop.Operator, typs []*types.T, metadataSources []execinfrapb.MetadataSource, - toClose []colexecbase.Closer, + toClose []colexecop.Closer, ) (*colrpc.Outbox, error) { return c.newOutboxFn(allocator, input, typs, metadataSources) } @@ -190,7 +190,7 @@ func TestDrainOnlyInputDAG(t *testing.T) { componentCreator := callbackRemoteComponentCreator{ newOutboxFn: func( allocator *colmem.Allocator, - op colexecbase.Operator, + op colexecop.Operator, typs []*types.T, sources []execinfrapb.MetadataSource, ) (*colrpc.Outbox, error) { @@ -261,7 +261,7 @@ func TestVectorizedFlowTempDirectory(t *testing.T) { Cfg: &execinfra.ServerConfig{ TempFS: ngn, TempStoragePath: tempPath, - VecFDSemaphore: &colexecbase.TestingSemaphore{}, + VecFDSemaphore: &colexecop.TestingSemaphore{}, Metrics: &execinfra.DistSQLMetrics{}, }, EvalCtx: &evalCtx, diff --git a/pkg/sql/colflow/vectorized_meta_propagation_test.go b/pkg/sql/colflow/vectorized_meta_propagation_test.go index b691785288b6..a99bebd4d068 100644 --- a/pkg/sql/colflow/vectorized_meta_propagation_test.go +++ b/pkg/sql/colflow/vectorized_meta_propagation_test.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -71,7 +71,7 @@ func TestVectorizedMetaPropagation(t *testing.T) { t.Fatal(err) } - noop := colexecbase.NewNoop(col) + noop := colexecop.NewNoop(col) mat, err := colexec.NewMaterializer( &flowCtx, 2, /* processorID */ diff --git a/pkg/sql/colflow/vectorized_panic_propagation_test.go b/pkg/sql/colflow/vectorized_panic_propagation_test.go index c614b1634fb7..b3c704ad5d3a 100644 --- a/pkg/sql/colflow/vectorized_panic_propagation_test.go +++ b/pkg/sql/colflow/vectorized_panic_propagation_test.go @@ -17,8 +17,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -125,15 +125,15 @@ func TestNonVectorizedPanicPropagation(t *testing.T) { // and returns the next batch from the input on every even-numbered (i.e. it // becomes a noop for those iterations). Used for tests only. type testVectorizedInternalPanicEmitter struct { - colexecbase.OneInputNode + colexecop.OneInputNode emitBatch bool } -var _ colexecbase.Operator = &testVectorizedInternalPanicEmitter{} +var _ colexecop.Operator = &testVectorizedInternalPanicEmitter{} -func newTestVectorizedInternalPanicEmitter(input colexecbase.Operator) colexecbase.Operator { +func newTestVectorizedInternalPanicEmitter(input colexecop.Operator) colexecop.Operator { return &testVectorizedInternalPanicEmitter{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), } } @@ -158,15 +158,15 @@ func (e *testVectorizedInternalPanicEmitter) Next(ctx context.Context) coldata.B // function. Used for tests only. It is the only colexec.Operator panics from // which are not caught. type testNonVectorizedPanicEmitter struct { - colexecbase.OneInputNode + colexecop.OneInputNode emitBatch bool } -var _ colexecbase.Operator = &testVectorizedInternalPanicEmitter{} +var _ colexecop.Operator = &testVectorizedInternalPanicEmitter{} -func newTestNonVectorizedPanicEmitter(input colexecbase.Operator) colexecbase.Operator { +func newTestNonVectorizedPanicEmitter(input colexecop.Operator) colexecop.Operator { return &testNonVectorizedPanicEmitter{ - OneInputNode: colexecbase.NewOneInputNode(input), + OneInputNode: colexecop.NewOneInputNode(input), } } diff --git a/pkg/sql/distsql/BUILD.bazel b/pkg/sql/distsql/BUILD.bazel index 55a666f4bdcd..790e21675cda 100644 --- a/pkg/sql/distsql/BUILD.bazel +++ b/pkg/sql/distsql/BUILD.bazel @@ -65,7 +65,7 @@ go_test( "//pkg/sql/colexec/colbuilder", "//pkg/sql/colexec/colexecargs", "//pkg/sql/colexec/colexecwindow", - "//pkg/sql/colexecbase", + "//pkg/sql/colexecop", "//pkg/sql/colflow", "//pkg/sql/colmem", "//pkg/sql/execinfra", diff --git a/pkg/sql/distsql/columnar_utils_test.go b/pkg/sql/distsql/columnar_utils_test.go index 33b33438cdca..ceb24b2cbb3f 100644 --- a/pkg/sql/distsql/columnar_utils_test.go +++ b/pkg/sql/distsql/columnar_utils_test.go @@ -27,7 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colbuilder" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -127,7 +127,7 @@ func verifyColOperator(t *testing.T, args verifyColOperatorArgs) error { acc := evalCtx.Mon.MakeBoundAccount() defer acc.Close(ctx) testAllocator := colmem.NewAllocator(ctx, &acc, coldataext.NewExtendedColumnFactory(&evalCtx)) - columnarizers := make([]colexecbase.Operator, len(args.inputs)) + columnarizers := make([]colexecop.Operator, len(args.inputs)) for i, input := range inputsColOp { c, err := colexec.NewBufferingColumnarizer(ctx, testAllocator, flowCtx, int32(i)+1, input) if err != nil { @@ -144,7 +144,7 @@ func verifyColOperator(t *testing.T, args verifyColOperatorArgs) error { FS: tempFS, GetPather: colcontainer.GetPatherFunc(func(context.Context) string { return "" }), }, - FDSemaphore: colexecbase.NewTestingSemaphore(256), + FDSemaphore: colexecop.NewTestingSemaphore(256), // TODO(yuzefovich): adjust expression generator to not produce // mixed-type timestamp-related expressions and then disallow the diff --git a/pkg/sql/distsql/vectorized_panic_propagation_test.go b/pkg/sql/distsql/vectorized_panic_propagation_test.go index b773c5d64e67..11467d1e363d 100644 --- a/pkg/sql/distsql/vectorized_panic_propagation_test.go +++ b/pkg/sql/distsql/vectorized_panic_propagation_test.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colflow" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" @@ -53,7 +53,7 @@ func TestNonVectorizedPanicDoesntHangServer(t *testing.T) { mat, err := colexec.NewMaterializer( &flowCtx, 0, /* processorID */ - &colexecbase.CallbackOperator{ + &colexecop.CallbackOperator{ NextCb: func(ctx context.Context) coldata.Batch { panic("") }, diff --git a/pkg/sql/explain_vec.go b/pkg/sql/explain_vec.go index 68030310076f..010d12ae91ed 100644 --- a/pkg/sql/explain_vec.go +++ b/pkg/sql/explain_vec.go @@ -17,8 +17,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colflow" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -152,7 +152,7 @@ func newPlanningCtxForExplainPurposes( } func shouldOutput(operator execinfra.OpNode, verbose bool) bool { - _, nonExplainable := operator.(colexecbase.NonExplainable) + _, nonExplainable := operator.(colexecop.NonExplainable) return !nonExplainable || verbose } diff --git a/pkg/sql/sem/tree/eval_test/BUILD.bazel b/pkg/sql/sem/tree/eval_test/BUILD.bazel index 14fcd56a747c..52e2ec9b13c4 100644 --- a/pkg/sql/sem/tree/eval_test/BUILD.bazel +++ b/pkg/sql/sem/tree/eval_test/BUILD.bazel @@ -18,7 +18,7 @@ go_test( "//pkg/sql/colexec", "//pkg/sql/colexec/colbuilder", "//pkg/sql/colexec/colexecargs", - "//pkg/sql/colexecbase", + "//pkg/sql/colexecop", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", "//pkg/sql/parser", diff --git a/pkg/sql/sem/tree/eval_test/eval_test.go b/pkg/sql/sem/tree/eval_test/eval_test.go index f907949e2407..b171177039a0 100644 --- a/pkg/sql/sem/tree/eval_test/eval_test.go +++ b/pkg/sql/sem/tree/eval_test/eval_test.go @@ -24,7 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colbuilder" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -165,8 +165,8 @@ func TestEval(t *testing.T) { }, ResultTypes: []*types.T{typedExpr.ResolvedType()}, }, - Inputs: []colexecbase.Operator{ - &colexecbase.CallbackOperator{ + Inputs: []colexecop.Operator{ + &colexecop.CallbackOperator{ NextCb: func(_ context.Context) coldata.Batch { if batchesReturned > 0 { return coldata.ZeroBatch