From f819fcd86504c69a3d57b65244414822f0e2e6ec Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 17 Apr 2020 10:14:39 -0700 Subject: [PATCH] col...: introduce new package and more code movement Move `coldata/random_testutils.go` into newly created package `coldatatestutils`. Move contents of `colbase/random_testutils.go` into `coldatatestutils` package. Rename `colbase` to `colexecbase`. Also remove templated comments from import sections of `_tmpl` files in favor of adding vars that remove unused warnings (those templated comments work poorly when moving/renaming the dependencies). Rename `vecerror` to `colexecerror`. Move `CopyBatch` from `colexecbase` into `coldatatestutils`. Also remove memory accounting from `CopyBatch`. Move `typeconv` from `colexecbase` into `coltypes` folder. Move `coldata/vec_test.go` into `coldata_test` package to prevent an import cycle. Also move one unit test from `coldata/vec_test.go` into `coldata/bytes_test.go`. Move `colexecbase/allocator.go` into newly created `colmem` package. Release note: None --- pkg/col/coldata/batch.go | 2 +- pkg/col/coldata/batch_test.go | 2 +- pkg/col/coldata/bytes_test.go | 55 ++++++ pkg/col/coldata/random_testutils.go | 129 ------------- pkg/col/coldata/vec.go | 2 +- pkg/col/coldata/vec_test.go | 179 +++++++----------- .../coldatatestutils}/random_testutils.go | 157 +++++++++++++-- .../colbase => col/coldatatestutils}/utils.go | 29 ++- pkg/col/colserde/arrowbatchconverter.go | 2 +- pkg/col/colserde/arrowbatchconverter_test.go | 16 +- pkg/col/colserde/file.go | 2 +- pkg/col/colserde/file_test.go | 6 +- pkg/col/colserde/main_test.go | 6 +- pkg/col/colserde/record_batch.go | 2 +- pkg/col/colserde/record_batch_test.go | 5 +- .../coltypes}/typeconv/typeconv.go | 24 ++- pkg/internal/sqlsmith/type.go | 2 +- pkg/sql/colbase/supported_sql_types.go | 33 ---- pkg/sql/colcontainer/diskqueue_test.go | 11 +- pkg/sql/colcontainer/main_test.go | 6 +- pkg/sql/colcontainer/partitionedqueue.go | 12 +- pkg/sql/colcontainer/partitionedqueue_test.go | 8 +- pkg/sql/colencoding/value_encoding_test.go | 2 +- pkg/sql/colexec/aggregator.go | 17 +- pkg/sql/colexec/aggregator_test.go | 26 +-- pkg/sql/colexec/and_or_projection_test.go | 6 +- pkg/sql/colexec/and_or_projection_tmpl.go | 21 +- pkg/sql/colexec/any_not_null_agg_tmpl.go | 13 +- pkg/sql/colexec/avg_agg_tmpl.go | 8 +- pkg/sql/colexec/bool_and_or_agg_tmpl.go | 13 +- pkg/sql/colexec/bool_vec_to_sel.go | 6 +- pkg/sql/colexec/bool_vec_to_sel_test.go | 4 +- pkg/sql/colexec/buffer.go | 6 +- pkg/sql/colexec/builtin_funcs.go | 21 +- pkg/sql/colexec/builtin_funcs_test.go | 10 +- pkg/sql/colexec/cancel_checker.go | 10 +- pkg/sql/colexec/cancel_checker_test.go | 8 +- pkg/sql/colexec/case.go | 27 +-- pkg/sql/colexec/case_test.go | 4 +- pkg/sql/colexec/cast_test.go | 13 +- pkg/sql/colexec/cast_tmpl.go | 42 ++-- pkg/sql/colexec/cfetcher.go | 16 +- pkg/sql/colexec/colbatch_scan.go | 19 +- pkg/sql/colexec/columnarizer.go | 23 +-- pkg/sql/colexec/const_test.go | 6 +- pkg/sql/colexec/const_tmpl.go | 26 +-- pkg/sql/colexec/count.go | 7 +- pkg/sql/colexec/count_test.go | 4 +- pkg/sql/colexec/deselector.go | 13 +- pkg/sql/colexec/deselector_test.go | 9 +- pkg/sql/colexec/disk_spiller.go | 42 ++-- pkg/sql/colexec/distinct_test.go | 17 +- pkg/sql/colexec/distinct_tmpl.go | 27 +-- .../execgen/cmd/execgen/avg_agg_gen.go | 6 +- .../cmd/execgen/bool_and_or_agg_gen.go | 4 +- pkg/sql/colexec/execgen/cmd/execgen/main.go | 4 +- .../colexec/execgen/cmd/execgen/overloads.go | 92 ++++----- .../cmd/execgen/overloads_test_utils_gen.go | 2 +- .../execgen/cmd/execgen/rowstovec_gen.go | 2 +- pkg/sql/colexec/execgen/placeholders.go | 24 +-- pkg/sql/colexec/execplan.go | 135 ++++++------- pkg/sql/colexec/external_hash_joiner.go | 53 +++--- pkg/sql/colexec/external_hash_joiner_test.go | 18 +- pkg/sql/colexec/external_sort.go | 51 ++--- pkg/sql/colexec/external_sort_test.go | 21 +- pkg/sql/colexec/hash_aggregator.go | 19 +- pkg/sql/colexec/hash_aggregator_tmpl.go | 13 +- pkg/sql/colexec/hash_utils_tmpl.go | 13 +- pkg/sql/colexec/hashjoiner.go | 21 +- pkg/sql/colexec/hashjoiner_test.go | 14 +- pkg/sql/colexec/hashtable.go | 13 +- pkg/sql/colexec/hashtable_tmpl.go | 19 +- pkg/sql/colexec/invariants_checker.go | 6 +- pkg/sql/colexec/is_null_ops.go | 15 +- pkg/sql/colexec/is_null_ops_test.go | 6 +- pkg/sql/colexec/like_ops.go | 13 +- pkg/sql/colexec/like_ops_test.go | 8 +- pkg/sql/colexec/limit.go | 6 +- pkg/sql/colexec/limit_test.go | 4 +- pkg/sql/colexec/main_test.go | 10 +- pkg/sql/colexec/materializer.go | 12 +- pkg/sql/colexec/mergejoinbase_tmpl.go | 13 +- pkg/sql/colexec/mergejoiner.go | 29 +-- pkg/sql/colexec/mergejoiner_test.go | 28 +-- pkg/sql/colexec/mergejoiner_tmpl.go | 39 ++-- pkg/sql/colexec/min_max_agg_tmpl.go | 20 +- pkg/sql/colexec/offset.go | 6 +- pkg/sql/colexec/offset_test.go | 6 +- pkg/sql/colexec/one_shot.go | 6 +- pkg/sql/colexec/operator.go | 59 +++--- pkg/sql/colexec/orderedsynchronizer_test.go | 10 +- pkg/sql/colexec/orderedsynchronizer_tmpl.go | 28 +-- pkg/sql/colexec/ordinality.go | 11 +- pkg/sql/colexec/ordinality_test.go | 12 +- pkg/sql/colexec/overloads_test.go | 68 +++---- .../parallel_unordered_synchronizer.go | 20 +- .../parallel_unordered_synchronizer_test.go | 29 +-- pkg/sql/colexec/partially_ordered_distinct.go | 21 +- pkg/sql/colexec/partitioner.go | 13 +- pkg/sql/colexec/proj_const_ops_tmpl.go | 22 ++- pkg/sql/colexec/proj_non_const_ops_tmpl.go | 30 +-- pkg/sql/colexec/projection_ops_test.go | 37 ++-- pkg/sql/colexec/rank_tmpl.go | 24 +-- pkg/sql/colexec/relative_rank_tmpl.go | 43 +++-- pkg/sql/colexec/routers.go | 41 ++-- pkg/sql/colexec/routers_test.go | 64 +++---- pkg/sql/colexec/row_number_tmpl.go | 11 +- pkg/sql/colexec/rowstovec_tmpl.go | 15 +- pkg/sql/colexec/select_in_test.go | 8 +- pkg/sql/colexec/select_in_tmpl.go | 31 +-- pkg/sql/colexec/selection_ops_test.go | 16 +- pkg/sql/colexec/selection_ops_tmpl.go | 21 +- .../colexec/serial_unordered_synchronizer.go | 8 +- .../serial_unordered_synchronizer_test.go | 9 +- pkg/sql/colexec/simple_project.go | 6 +- pkg/sql/colexec/simple_project_test.go | 10 +- pkg/sql/colexec/sort.go | 33 ++-- pkg/sql/colexec/sort_chunks.go | 43 +++-- pkg/sql/colexec/sort_chunks_test.go | 11 +- pkg/sql/colexec/sort_test.go | 14 +- pkg/sql/colexec/sort_tmpl.go | 19 +- pkg/sql/colexec/sorttopk.go | 21 +- pkg/sql/colexec/sorttopk_test.go | 4 +- pkg/sql/colexec/spilling_queue.go | 18 +- pkg/sql/colexec/spilling_queue_test.go | 11 +- pkg/sql/colexec/stats.go | 12 +- pkg/sql/colexec/stats_test.go | 8 +- pkg/sql/colexec/substring_tmpl.go | 23 +-- pkg/sql/colexec/sum_agg_tmpl.go | 6 +- pkg/sql/colexec/types_integration_test.go | 21 +- pkg/sql/colexec/unordered_distinct.go | 15 +- pkg/sql/colexec/utils.go | 23 +-- pkg/sql/colexec/utils_test.go | 104 +++++----- pkg/sql/colexec/values_differ_tmpl.go | 13 +- pkg/sql/colexec/vec_comparators_tmpl.go | 13 +- pkg/sql/colexec/vec_elem_to_datum.go | 6 +- pkg/sql/colexec/window_functions_test.go | 6 +- pkg/sql/colexec/window_functions_util.go | 4 +- pkg/sql/colexec/window_peer_grouper_tmpl.go | 13 +- .../colexecerror}/error.go | 2 +- pkg/sql/{colbase => colexecbase}/dep_test.go | 4 +- pkg/sql/{colbase => colexecbase}/operator.go | 6 +- pkg/sql/{colbase => colexecbase}/testutils.go | 15 +- pkg/sql/colflow/colrpc/colrpc_test.go | 44 +++-- pkg/sql/colflow/colrpc/inbox.go | 21 +- pkg/sql/colflow/colrpc/inbox_test.go | 15 +- pkg/sql/colflow/colrpc/main_test.go | 6 +- pkg/sql/colflow/colrpc/outbox.go | 11 +- pkg/sql/colflow/colrpc/outbox_test.go | 13 +- pkg/sql/colflow/main_test.go | 6 +- pkg/sql/colflow/vectorized_flow.go | 53 +++--- .../colflow/vectorized_flow_shutdown_test.go | 32 ++-- pkg/sql/colflow/vectorized_flow_space_test.go | 14 +- pkg/sql/colflow/vectorized_flow_test.go | 21 +- .../vectorized_panic_propagation_test.go | 16 +- pkg/sql/{colbase => colmem}/allocator.go | 26 +-- pkg/sql/distsql/columnar_operators_test.go | 2 +- pkg/sql/distsql/columnar_utils_test.go | 9 +- .../vectorized_panic_propagation_test.go | 4 +- pkg/sql/explain_vec.go | 4 +- pkg/sql/sem/tree/eval_test.go | 6 +- pkg/testutils/lint/lint_test.go | 16 +- pkg/workload/dep_test.go | 2 +- pkg/workload/workload.go | 2 +- 164 files changed, 1718 insertions(+), 1647 deletions(-) delete mode 100644 pkg/col/coldata/random_testutils.go rename pkg/{sql/colbase => col/coldatatestutils}/random_testutils.go (62%) rename pkg/{sql/colbase => col/coldatatestutils}/utils.go (57%) rename pkg/{sql/colbase => col/coltypes}/typeconv/typeconv.go (83%) delete mode 100644 pkg/sql/colbase/supported_sql_types.go rename pkg/sql/{colbase/vecerror => colexecbase/colexecerror}/error.go (99%) rename pkg/sql/{colbase => colexecbase}/dep_test.go (91%) rename pkg/sql/{colbase => colexecbase}/operator.go (93%) rename pkg/sql/{colbase => colexecbase}/testutils.go (92%) rename pkg/sql/{colbase => colmem}/allocator.go (95%) diff --git a/pkg/col/coldata/batch.go b/pkg/col/coldata/batch.go index 011e1e7d2d4f..ed1a92a57607 100644 --- a/pkg/col/coldata/batch.go +++ b/pkg/col/coldata/batch.go @@ -17,7 +17,7 @@ import ( "sync/atomic" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" ) diff --git a/pkg/col/coldata/batch_test.go b/pkg/col/coldata/batch_test.go index b8ac1fb2d0db..0509d7a2d350 100644 --- a/pkg/col/coldata/batch_test.go +++ b/pkg/col/coldata/batch_test.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/stretchr/testify/assert" diff --git a/pkg/col/coldata/bytes_test.go b/pkg/col/coldata/bytes_test.go index 597983161edd..bab30da6011f 100644 --- a/pkg/col/coldata/bytes_test.go +++ b/pkg/col/coldata/bytes_test.go @@ -18,6 +18,8 @@ import ( "testing" "unsafe" + "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/pkg/errors" @@ -412,3 +414,56 @@ func TestBytes(t *testing.T) { other.AssertOffsetsAreNonDecreasing(4) }) } + +// TestAppendBytesWithLastNull makes sure that Append handles correctly the +// case when the last element of Bytes vector is NULL. +func TestAppendBytesWithLastNull(t *testing.T) { + src := NewMemColumn(types.Bytes, 4) + sel := []int{0, 2, 3} + src.Bytes().Set(0, []byte("zero")) + src.Nulls().SetNull(1) + src.Bytes().Set(2, []byte("two")) + src.Nulls().SetNull(3) + sliceArgs := SliceArgs{ + Src: src, + ColType: coltypes.Bytes, + DestIdx: 0, + SrcStartIdx: 0, + SrcEndIdx: len(sel), + } + dest := NewMemColumn(types.Bytes, 3) + expected := NewMemColumn(types.Bytes, 3) + for _, withSel := range []bool{false, true} { + t.Run(fmt.Sprintf("AppendBytesWithLastNull/sel=%t", withSel), func(t *testing.T) { + expected.Nulls().UnsetNulls() + expected.Bytes().Reset() + if withSel { + sliceArgs.Sel = sel + for expIdx, srcIdx := range sel { + if src.Nulls().NullAt(srcIdx) { + expected.Nulls().SetNull(expIdx) + } else { + expected.Bytes().Set(expIdx, src.Bytes().Get(srcIdx)) + } + } + } else { + sliceArgs.Sel = nil + for expIdx := 0; expIdx < 3; expIdx++ { + if src.Nulls().NullAt(expIdx) { + expected.Nulls().SetNull(expIdx) + } else { + expected.Bytes().Set(expIdx, src.Bytes().Get(expIdx)) + } + } + } + expected.Bytes().UpdateOffsetsToBeNonDecreasing(3) + // require.Equal checks the "string-ified" versions of the vectors for + // equality. Bytes uses maxSetIndex to print out "truncated" + // representation, so we manually update it (Vec.Append will use + // AppendVal function that updates maxSetIndex itself). + expected.Bytes().maxSetIndex = 2 + dest.Append(sliceArgs) + require.Equal(t, expected, dest) + }) + } +} diff --git a/pkg/col/coldata/random_testutils.go b/pkg/col/coldata/random_testutils.go deleted file mode 100644 index b5d7a2225472..000000000000 --- a/pkg/col/coldata/random_testutils.go +++ /dev/null @@ -1,129 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package coldata - -import ( - "fmt" - "math/rand" - "time" - - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/duration" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" -) - -// maxVarLen specifies a length limit for variable length types (e.g. byte slices). -const maxVarLen = 64 - -var locations []*time.Location - -func init() { - // Load some random time zones. - for _, locationName := range []string{ - "Africa/Addis_Ababa", - "America/Anchorage", - "Antarctica/Davis", - "Asia/Ashkhabad", - "Australia/Sydney", - "Europe/Minsk", - "Pacific/Palau", - } { - loc, err := timeutil.LoadLocation(locationName) - if err == nil { - locations = append(locations, loc) - } - } -} - -// RandomVec populates vec with n random values of typ, setting each value to -// null with a probability of nullProbability. It is assumed that n is in bounds -// of the given vec. -// bytesFixedLength (when greater than zero) specifies the fixed length of the -// bytes slice to be generated. It is used only if typ == coltypes.Bytes. -func RandomVec( - rng *rand.Rand, typ *types.T, bytesFixedLength int, vec Vec, n int, nullProbability float64, -) { - switch typeconv.FromColumnType(typ) { - case coltypes.Bool: - bools := vec.Bool() - for i := 0; i < n; i++ { - if rng.Float64() < 0.5 { - bools[i] = true - } else { - bools[i] = false - } - } - case coltypes.Bytes: - bytes := vec.Bytes() - for i := 0; i < n; i++ { - bytesLen := bytesFixedLength - if bytesLen <= 0 { - bytesLen = rng.Intn(maxVarLen) - } - randBytes := make([]byte, bytesLen) - // Read always returns len(bytes[i]) and nil. - _, _ = rand.Read(randBytes) - bytes.Set(i, randBytes) - } - case coltypes.Decimal: - decs := vec.Decimal() - for i := 0; i < n; i++ { - // int64(rng.Uint64()) to get negative numbers, too - decs[i].SetFinite(int64(rng.Uint64()), int32(rng.Intn(40)-20)) - } - case coltypes.Int16: - ints := vec.Int16() - for i := 0; i < n; i++ { - ints[i] = int16(rng.Uint64()) - } - case coltypes.Int32: - ints := vec.Int32() - for i := 0; i < n; i++ { - ints[i] = int32(rng.Uint64()) - } - case coltypes.Int64: - ints := vec.Int64() - for i := 0; i < n; i++ { - ints[i] = int64(rng.Uint64()) - } - case coltypes.Float64: - floats := vec.Float64() - for i := 0; i < n; i++ { - floats[i] = rng.Float64() - } - case coltypes.Timestamp: - timestamps := vec.Timestamp() - for i := 0; i < n; i++ { - timestamps[i] = timeutil.Unix(rng.Int63n(1000000), rng.Int63n(1000000)) - loc := locations[rng.Intn(len(locations))] - timestamps[i] = timestamps[i].In(loc) - } - case coltypes.Interval: - intervals := vec.Interval() - for i := 0; i < n; i++ { - intervals[i] = duration.FromFloat64(rng.Float64()) - } - default: - panic(fmt.Sprintf("unhandled type %s", typ)) - } - vec.Nulls().UnsetNulls() - if nullProbability == 0 { - return - } - - for i := 0; i < n; i++ { - if rng.Float64() < nullProbability { - vec.Nulls().SetNull(i) - } - } -} diff --git a/pkg/col/coldata/vec.go b/pkg/col/coldata/vec.go index 218e50c7fd1a..a933d2204d29 100644 --- a/pkg/col/coldata/vec.go +++ b/pkg/col/coldata/vec.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" ) diff --git a/pkg/col/coldata/vec_test.go b/pkg/col/coldata/vec_test.go index 1b9566fe336b..a57a1e2c3b07 100644 --- a/pkg/col/coldata/vec_test.go +++ b/pkg/col/coldata/vec_test.go @@ -8,14 +8,16 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package coldata +package coldata_test import ( "fmt" "testing" + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -27,10 +29,10 @@ func TestMemColumnWindow(t *testing.T) { rng, _ := randutil.NewPseudoRand() - c := NewMemColumn(types.Int, BatchSize()) + c := coldata.NewMemColumn(types.Int, coldata.BatchSize()) ints := c.Int64() - for i := 0; i < BatchSize(); i++ { + for i := 0; i < coldata.BatchSize(); i++ { ints[i] = int64(i) if i%2 == 0 { // Set every other value to null. @@ -41,8 +43,8 @@ func TestMemColumnWindow(t *testing.T) { startWindow := 1 endWindow := 0 for startWindow > endWindow { - startWindow = rng.Intn(BatchSize()) - endWindow = 1 + rng.Intn(BatchSize()) + startWindow = rng.Intn(coldata.BatchSize()) + endWindow = 1 + rng.Intn(coldata.BatchSize()) } window := c.Window(coltypes.Int64, startWindow, endWindow) @@ -115,12 +117,12 @@ func TestNullRanges(t *testing.T) { }, } - c := NewMemColumn(types.Int, BatchSize()) + c := coldata.NewMemColumn(types.Int, coldata.BatchSize()) for _, tc := range tcs { c.Nulls().UnsetNulls() c.Nulls().SetNullRange(tc.start, tc.end) - for i := 0; i < BatchSize(); i++ { + for i := 0; i < coldata.BatchSize(); i++ { if i >= tc.start && i < tc.end { if !c.Nulls().NullAt(i) { t.Fatalf("expected null at %d, start: %d end: %d", i, tc.start, tc.end) @@ -138,7 +140,7 @@ func TestAppend(t *testing.T) { // TODO(asubiotto): Test nulls. var typ = types.Int - src := NewMemColumn(typ, BatchSize()) + src := coldata.NewMemColumn(typ, coldata.BatchSize()) sel := make([]int, len(src.Int64())) for i := range sel { sel[i] = i @@ -146,36 +148,36 @@ func TestAppend(t *testing.T) { testCases := []struct { name string - args SliceArgs + args coldata.SliceArgs expectedLength int }{ { name: "AppendSimple", - args: SliceArgs{ + args: coldata.SliceArgs{ // DestIdx must be specified to append to the end of dest. - DestIdx: BatchSize(), + DestIdx: coldata.BatchSize(), }, - expectedLength: BatchSize() * 2, + expectedLength: coldata.BatchSize() * 2, }, { name: "AppendOverwriteSimple", - args: SliceArgs{ + args: coldata.SliceArgs{ // DestIdx 0, the default value, will start appending at index 0. DestIdx: 0, }, - expectedLength: BatchSize(), + expectedLength: coldata.BatchSize(), }, { name: "AppendOverwriteSlice", - args: SliceArgs{ + args: coldata.SliceArgs{ // Start appending at index 10. DestIdx: 10, }, - expectedLength: BatchSize() + 10, + expectedLength: coldata.BatchSize() + 10, }, { name: "AppendSlice", - args: SliceArgs{ + args: coldata.SliceArgs{ DestIdx: 20, SrcStartIdx: 10, SrcEndIdx: 20, @@ -184,7 +186,7 @@ func TestAppend(t *testing.T) { }, { name: "AppendWithSel", - args: SliceArgs{ + args: coldata.SliceArgs{ DestIdx: 5, SrcStartIdx: 10, SrcEndIdx: 20, @@ -194,12 +196,12 @@ func TestAppend(t *testing.T) { }, { name: "AppendWithHalfSel", - args: SliceArgs{ + args: coldata.SliceArgs{ DestIdx: 5, Sel: sel[:len(sel)/2], SrcEndIdx: len(sel) / 2, }, - expectedLength: 5 + (BatchSize())/2, + expectedLength: 5 + (coldata.BatchSize())/2, }, } @@ -208,74 +210,21 @@ func TestAppend(t *testing.T) { tc.args.ColType = typeconv.FromColumnType(typ) if tc.args.SrcEndIdx == 0 { // SrcEndIdx is always required. - tc.args.SrcEndIdx = BatchSize() + tc.args.SrcEndIdx = coldata.BatchSize() } t.Run(tc.name, func(t *testing.T) { - dest := NewMemColumn(typ, BatchSize()) + dest := coldata.NewMemColumn(typ, coldata.BatchSize()) dest.Append(tc.args) require.Equal(t, tc.expectedLength, len(dest.Int64())) }) } } -// TestAppendBytesWithLastNull makes sure that Append handles correctly the -// case when the last element of Bytes vector is NULL. -func TestAppendBytesWithLastNull(t *testing.T) { - src := NewMemColumn(types.Bytes, 4) - sel := []int{0, 2, 3} - src.Bytes().Set(0, []byte("zero")) - src.Nulls().SetNull(1) - src.Bytes().Set(2, []byte("two")) - src.Nulls().SetNull(3) - sliceArgs := SliceArgs{ - Src: src, - ColType: coltypes.Bytes, - DestIdx: 0, - SrcStartIdx: 0, - SrcEndIdx: len(sel), - } - dest := NewMemColumn(types.Bytes, 3) - expected := NewMemColumn(types.Bytes, 3) - for _, withSel := range []bool{false, true} { - t.Run(fmt.Sprintf("AppendBytesWithLastNull/sel=%t", withSel), func(t *testing.T) { - expected.Nulls().UnsetNulls() - expected.Bytes().Reset() - if withSel { - sliceArgs.Sel = sel - for expIdx, srcIdx := range sel { - if src.Nulls().NullAt(srcIdx) { - expected.Nulls().SetNull(expIdx) - } else { - expected.Bytes().Set(expIdx, src.Bytes().Get(srcIdx)) - } - } - } else { - sliceArgs.Sel = nil - for expIdx := 0; expIdx < 3; expIdx++ { - if src.Nulls().NullAt(expIdx) { - expected.Nulls().SetNull(expIdx) - } else { - expected.Bytes().Set(expIdx, src.Bytes().Get(expIdx)) - } - } - } - expected.Bytes().UpdateOffsetsToBeNonDecreasing(3) - // require.Equal checks the "string-ified" versions of the vectors for - // equality. Bytes uses maxSetIndex to print out "truncated" - // representation, so we manually update it (Vec.Append will use - // AppendVal function that updates maxSetIndex itself). - expected.Bytes().maxSetIndex = 2 - dest.Append(sliceArgs) - require.Equal(t, expected, dest) - }) - } -} - func TestCopy(t *testing.T) { // TODO(asubiotto): Test nulls. var typ = types.Int - src := NewMemColumn(typ, BatchSize()) + src := coldata.NewMemColumn(typ, coldata.BatchSize()) srcInts := src.Int64() for i := range srcInts { srcInts[i] = int64(i + 1) @@ -295,30 +244,30 @@ func TestCopy(t *testing.T) { testCases := []struct { name string - args CopySliceArgs + args coldata.CopySliceArgs expectedSum int }{ { name: "CopyNothing", - args: CopySliceArgs{}, + args: coldata.CopySliceArgs{}, expectedSum: 0, }, { name: "CopyBatchSizeMinus1WithOffset1", - args: CopySliceArgs{ - SliceArgs: SliceArgs{ + args: coldata.CopySliceArgs{ + SliceArgs: coldata.SliceArgs{ // Use DestIdx 1 to make sure that it is respected. DestIdx: 1, - SrcEndIdx: BatchSize() - 1, + SrcEndIdx: coldata.BatchSize() - 1, }, }, // expectedSum uses sum of positive integers formula. - expectedSum: (BatchSize() - 1) * BatchSize() / 2, + expectedSum: (coldata.BatchSize() - 1) * coldata.BatchSize() / 2, }, { name: "CopyWithSel", - args: CopySliceArgs{ - SliceArgs: SliceArgs{ + args: coldata.CopySliceArgs{ + SliceArgs: coldata.SliceArgs{ Sel: sel[1:], DestIdx: 25, SrcStartIdx: 1, @@ -334,7 +283,7 @@ func TestCopy(t *testing.T) { tc.args.Src = src tc.args.ColType = typeconv.FromColumnType(typ) t.Run(tc.name, func(t *testing.T) { - dest := NewMemColumn(typ, BatchSize()) + dest := coldata.NewMemColumn(typ, coldata.BatchSize()) dest.Copy(tc.args) destInts := dest.Int64() firstNonZero := 0 @@ -355,7 +304,7 @@ func TestCopyNulls(t *testing.T) { var typ = types.Int // Set up the destination vector. - dst := NewMemColumn(typ, BatchSize()) + dst := coldata.NewMemColumn(typ, coldata.BatchSize()) dstInts := dst.Int64() for i := range dstInts { dstInts[i] = int64(1) @@ -366,7 +315,7 @@ func TestCopyNulls(t *testing.T) { } // Set up the source vector. - src := NewMemColumn(typ, BatchSize()) + src := coldata.NewMemColumn(typ, coldata.BatchSize()) srcInts := src.Int64() for i := range srcInts { srcInts[i] = 2 @@ -376,8 +325,8 @@ func TestCopyNulls(t *testing.T) { src.Nulls().SetNull(i) } - copyArgs := CopySliceArgs{ - SliceArgs: SliceArgs{ + copyArgs := coldata.CopySliceArgs{ + SliceArgs: coldata.SliceArgs{ ColType: typeconv.FromColumnType(typ), Src: src, DestIdx: 3, @@ -401,7 +350,7 @@ func TestCopyNulls(t *testing.T) { } // Verify that the remaining elements in dst have not been touched. - for i := 10; i < BatchSize(); i++ { + for i := 10; i < coldata.BatchSize(); i++ { require.True(t, dstInts[i] == 1, "data in dst outside copy range has been changed") require.True(t, !dst.Nulls().NullAt(i), "no extra nulls were added") } @@ -412,7 +361,7 @@ func TestCopySelOnDestDoesNotUnsetOldNulls(t *testing.T) { // Set up the destination vector. It is all nulls except for a single // non-null at index 0. - dst := NewMemColumn(typ, BatchSize()) + dst := coldata.NewMemColumn(typ, coldata.BatchSize()) dstInts := dst.Int64() for i := range dstInts { dstInts[i] = 1 @@ -421,7 +370,7 @@ func TestCopySelOnDestDoesNotUnsetOldNulls(t *testing.T) { dst.Nulls().UnsetNull(0) // Set up the source vector with two nulls. - src := NewMemColumn(typ, BatchSize()) + src := coldata.NewMemColumn(typ, coldata.BatchSize()) srcInts := src.Int64() for i := range srcInts { srcInts[i] = 2 @@ -431,9 +380,9 @@ func TestCopySelOnDestDoesNotUnsetOldNulls(t *testing.T) { // Using a small selection vector and SelOnDest, perform a copy and verify // that nulls in between the selected tuples weren't unset. - copyArgs := CopySliceArgs{ + copyArgs := coldata.CopySliceArgs{ SelOnDest: true, - SliceArgs: SliceArgs{ + SliceArgs: coldata.SliceArgs{ ColType: typeconv.FromColumnType(typ), Src: src, SrcStartIdx: 1, @@ -458,19 +407,19 @@ func TestCopySelOnDestDoesNotUnsetOldNulls(t *testing.T) { func BenchmarkAppend(b *testing.B) { rng, _ := randutil.NewPseudoRand() - sel := rng.Perm(BatchSize()) + sel := rng.Perm(coldata.BatchSize()) benchCases := []struct { name string - args SliceArgs + args coldata.SliceArgs }{ { name: "AppendSimple", - args: SliceArgs{}, + args: coldata.SliceArgs{}, }, { name: "AppendWithSel", - args: SliceArgs{ + args: coldata.SliceArgs{ Sel: sel, }, }, @@ -478,19 +427,19 @@ func BenchmarkAppend(b *testing.B) { for _, typ := range []types.T{*types.Bytes, *types.Decimal, *types.Int} { for _, nullProbability := range []float64{0, 0.2} { - src := NewMemColumn(&typ, BatchSize()) - RandomVec(rng, &typ, 8 /* bytesFixedLength */, src, BatchSize(), nullProbability) + src := coldata.NewMemColumn(&typ, coldata.BatchSize()) + coldatatestutils.RandomVec(rng, &typ, 8 /* bytesFixedLength */, src, coldata.BatchSize(), nullProbability) for _, bc := range benchCases { bc.args.Src = src bc.args.ColType = typeconv.FromColumnType(&typ) - bc.args.SrcEndIdx = BatchSize() - dest := NewMemColumn(&typ, BatchSize()) + bc.args.SrcEndIdx = coldata.BatchSize() + dest := coldata.NewMemColumn(&typ, coldata.BatchSize()) b.Run(fmt.Sprintf("%s/%s/NullProbability=%.1f", &typ, bc.name, nullProbability), func(b *testing.B) { - b.SetBytes(8 * int64(BatchSize())) + b.SetBytes(8 * int64(coldata.BatchSize())) bc.args.DestIdx = 0 for i := 0; i < b.N; i++ { dest.Append(bc.args) - bc.args.DestIdx += BatchSize() + bc.args.DestIdx += coldata.BatchSize() } }) } @@ -500,20 +449,20 @@ func BenchmarkAppend(b *testing.B) { func BenchmarkCopy(b *testing.B) { rng, _ := randutil.NewPseudoRand() - sel := rng.Perm(BatchSize()) + sel := rng.Perm(coldata.BatchSize()) benchCases := []struct { name string - args CopySliceArgs + args coldata.CopySliceArgs }{ { name: "CopySimple", - args: CopySliceArgs{}, + args: coldata.CopySliceArgs{}, }, { name: "CopyWithSel", - args: CopySliceArgs{ - SliceArgs: SliceArgs{ + args: coldata.CopySliceArgs{ + SliceArgs: coldata.SliceArgs{ Sel: sel, }, }, @@ -522,15 +471,15 @@ func BenchmarkCopy(b *testing.B) { for _, typ := range []types.T{*types.Bytes, *types.Decimal, *types.Int} { for _, nullProbability := range []float64{0, 0.2} { - src := NewMemColumn(&typ, BatchSize()) - RandomVec(rng, &typ, 8 /* bytesFixedLength */, src, BatchSize(), nullProbability) + src := coldata.NewMemColumn(&typ, coldata.BatchSize()) + coldatatestutils.RandomVec(rng, &typ, 8 /* bytesFixedLength */, src, coldata.BatchSize(), nullProbability) for _, bc := range benchCases { bc.args.Src = src bc.args.ColType = typeconv.FromColumnType(&typ) - bc.args.SrcEndIdx = BatchSize() - dest := NewMemColumn(&typ, BatchSize()) + bc.args.SrcEndIdx = coldata.BatchSize() + dest := coldata.NewMemColumn(&typ, coldata.BatchSize()) b.Run(fmt.Sprintf("%s/%s/NullProbability=%.1f", &typ, bc.name, nullProbability), func(b *testing.B) { - b.SetBytes(8 * int64(BatchSize())) + b.SetBytes(8 * int64(coldata.BatchSize())) for i := 0; i < b.N; i++ { dest.Copy(bc.args) if typ.Identical(types.Bytes) { diff --git a/pkg/sql/colbase/random_testutils.go b/pkg/col/coldatatestutils/random_testutils.go similarity index 62% rename from pkg/sql/colbase/random_testutils.go rename to pkg/col/coldatatestutils/random_testutils.go index 137f777df963..014f06782727 100644 --- a/pkg/sql/colbase/random_testutils.go +++ b/pkg/col/coldatatestutils/random_testutils.go @@ -1,4 +1,4 @@ -// Copyright 2019 The Cockroach Authors. +// Copyright 2020 The Cockroach Authors. // // Use of this software is governed by the Business Source License // included in the file licenses/BSL.txt. @@ -8,29 +8,146 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package colbase +package coldatatestutils import ( "context" "fmt" "math/rand" + "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) -func randomType(rng *rand.Rand) coltypes.T { - return coltypes.AllTypes[rng.Intn(len(coltypes.AllTypes))] +// maxVarLen specifies a length limit for variable length types (e.g. byte slices). +const maxVarLen = 64 + +var locations []*time.Location + +func init() { + // Load some random time zones. + for _, locationName := range []string{ + "Africa/Addis_Ababa", + "America/Anchorage", + "Antarctica/Davis", + "Asia/Ashkhabad", + "Australia/Sydney", + "Europe/Minsk", + "Pacific/Palau", + } { + loc, err := timeutil.LoadLocation(locationName) + if err == nil { + locations = append(locations, loc) + } + } +} + +// RandomVec populates vec with n random values of typ, setting each value to +// null with a probability of nullProbability. It is assumed that n is in bounds +// of the given vec. +// bytesFixedLength (when greater than zero) specifies the fixed length of the +// bytes slice to be generated. It is used only if typ == coltypes.Bytes. +func RandomVec( + rng *rand.Rand, + typ *types.T, + bytesFixedLength int, + vec coldata.Vec, + n int, + nullProbability float64, +) { + switch typeconv.FromColumnType(typ) { + case coltypes.Bool: + bools := vec.Bool() + for i := 0; i < n; i++ { + if rng.Float64() < 0.5 { + bools[i] = true + } else { + bools[i] = false + } + } + case coltypes.Bytes: + bytes := vec.Bytes() + for i := 0; i < n; i++ { + bytesLen := bytesFixedLength + if bytesLen <= 0 { + bytesLen = rng.Intn(maxVarLen) + } + randBytes := make([]byte, bytesLen) + // Read always returns len(bytes[i]) and nil. + _, _ = rand.Read(randBytes) + bytes.Set(i, randBytes) + } + case coltypes.Decimal: + decs := vec.Decimal() + for i := 0; i < n; i++ { + // int64(rng.Uint64()) to get negative numbers, too + decs[i].SetFinite(int64(rng.Uint64()), int32(rng.Intn(40)-20)) + } + case coltypes.Int16: + ints := vec.Int16() + for i := 0; i < n; i++ { + ints[i] = int16(rng.Uint64()) + } + case coltypes.Int32: + ints := vec.Int32() + for i := 0; i < n; i++ { + ints[i] = int32(rng.Uint64()) + } + case coltypes.Int64: + ints := vec.Int64() + for i := 0; i < n; i++ { + ints[i] = int64(rng.Uint64()) + } + case coltypes.Float64: + floats := vec.Float64() + for i := 0; i < n; i++ { + floats[i] = rng.Float64() + } + case coltypes.Timestamp: + timestamps := vec.Timestamp() + for i := 0; i < n; i++ { + timestamps[i] = timeutil.Unix(rng.Int63n(1000000), rng.Int63n(1000000)) + loc := locations[rng.Intn(len(locations))] + timestamps[i] = timestamps[i].In(loc) + } + case coltypes.Interval: + intervals := vec.Interval() + for i := 0; i < n; i++ { + intervals[i] = duration.FromFloat64(rng.Float64()) + } + default: + panic(fmt.Sprintf("unhandled type %s", typ)) + } + vec.Nulls().UnsetNulls() + if nullProbability == 0 { + return + } + + for i := 0; i < n; i++ { + if rng.Float64() < nullProbability { + vec.Nulls().SetNull(i) + } + } +} + +func randomType(rng *rand.Rand) *types.T { + return &typeconv.AllSupportedSQLTypes[rng.Intn(len(typeconv.AllSupportedSQLTypes))] } -// randomTypes returns an n-length slice of random coltypes.T. -func randomTypes(rng *rand.Rand, n int) []coltypes.T { - typs := make([]coltypes.T, n) +// randomTypes returns an n-length slice of random types.T. +func randomTypes(rng *rand.Rand, n int) []types.T { + typs := make([]types.T, n) for i := range typs { - typs[i] = randomType(rng) + typs[i] = *randomType(rng) } return typs } @@ -39,7 +156,7 @@ func randomTypes(rng *rand.Rand, n int) []coltypes.T { // random elements equal to length (capacity if length is 0). The values will be // null with a probability of nullProbability. func RandomBatch( - allocator *Allocator, + allocator *colmem.Allocator, rng *rand.Rand, typs []types.T, capacity int, @@ -51,7 +168,7 @@ func RandomBatch( length = capacity } for i, typ := range typs { - coldata.RandomVec(rng, &typ, 0 /* bytesFixedLength */, batch.ColVec(i), length, nullProbability) + RandomVec(rng, &typ, 0 /* bytesFixedLength */, batch.ColVec(i), length, nullProbability) } batch.SetLength(length) return batch @@ -65,7 +182,7 @@ func RandomBatch( // less than batchSize. func RandomSel(rng *rand.Rand, batchSize int, probOfOmitting float64) []int { if probOfOmitting < 0 || probOfOmitting > 1 { - vecerror.InternalError(fmt.Sprintf("probability of omitting a row is %f - outside of [0, 1] range", probOfOmitting)) + colexecerror.InternalError(fmt.Sprintf("probability of omitting a row is %f - outside of [0, 1] range", probOfOmitting)) } sel := make([]int, 0, batchSize) for i := 0; i < batchSize; i++ { @@ -87,7 +204,7 @@ var _ = randomTypes // selProbability is 0, none will. The returned batch will have its length set // to the length of the selection vector, unless selProbability is 0. func RandomBatchWithSel( - allocator *Allocator, + allocator *colmem.Allocator, rng *rand.Rand, typs []types.T, n int, @@ -112,7 +229,7 @@ const ( // RandomDataOpArgs are arguments passed in to RandomDataOp. All arguments are // optional (refer to the constants above this struct definition for the // defaults). Bools are false by default and AvailableTyps defaults to -// coltypes.AllTypes. +// typeconv.AllSupportedSQLTypes. type RandomDataOpArgs struct { // DeterministicTyps, if set, overrides AvailableTyps and MaxSchemaLength, // forcing the RandomDataOp to use this schema. @@ -140,7 +257,7 @@ type RandomDataOpArgs struct { // RandomDataOp is an operator that generates random data according to // RandomDataOpArgs. Call GetBuffer to get all data that was returned. type RandomDataOp struct { - allocator *Allocator + allocator *colmem.Allocator batchAccumulator func(b coldata.Batch, typs []types.T) typs []types.T rng *rand.Rand @@ -151,12 +268,14 @@ type RandomDataOp struct { nulls bool } -var _ Operator = &RandomDataOp{} +var _ colexecbase.Operator = &RandomDataOp{} // NewRandomDataOp creates a new RandomDataOp. -func NewRandomDataOp(allocator *Allocator, rng *rand.Rand, args RandomDataOpArgs) *RandomDataOp { +func NewRandomDataOp( + allocator *colmem.Allocator, rng *rand.Rand, args RandomDataOpArgs, +) *RandomDataOp { var ( - availableTyps = AllSupportedSQLTypes + availableTyps = typeconv.AllSupportedSQLTypes maxSchemaLength = defaultMaxSchemaLength batchSize = coldata.BatchSize() numBatches = defaultNumBatches @@ -243,7 +362,7 @@ func (o *RandomDataOp) ChildCount(verbose bool) int { // Child implements the execinfra.OpNode interface. func (o *RandomDataOp) Child(nth int, verbose bool) execinfra.OpNode { - vecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colbase/utils.go b/pkg/col/coldatatestutils/utils.go similarity index 57% rename from pkg/sql/colbase/utils.go rename to pkg/col/coldatatestutils/utils.go index c4e0ef9b1071..22af95c1d09e 100644 --- a/pkg/sql/colbase/utils.go +++ b/pkg/col/coldatatestutils/utils.go @@ -8,30 +8,29 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package colbase +package coldatatestutils import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/types" ) // CopyBatch copies the original batch and returns that copy. However, note that // the underlying capacity might be different (a new batch is created only with // capacity original.Length()). -func CopyBatch(allocator *Allocator, original coldata.Batch, typs []types.T) coldata.Batch { - b := allocator.NewMemBatchWithSize(typs, original.Length()) +// NOTE: memory accounting is not performed. +func CopyBatch(original coldata.Batch, typs []types.T) coldata.Batch { + b := coldata.NewMemBatchWithSize(typs, original.Length()) b.SetLength(original.Length()) - allocator.PerformOperation(b.ColVecs(), func() { - for colIdx, col := range original.ColVecs() { - b.ColVec(colIdx).Copy(coldata.CopySliceArgs{ - SliceArgs: coldata.SliceArgs{ - ColType: typeconv.FromColumnType(&typs[colIdx]), - Src: col, - SrcEndIdx: original.Length(), - }, - }) - } - }) + for colIdx, col := range original.ColVecs() { + b.ColVec(colIdx).Copy(coldata.CopySliceArgs{ + SliceArgs: coldata.SliceArgs{ + ColType: typeconv.FromColumnType(&typs[colIdx]), + Src: col, + SrcEndIdx: original.Length(), + }, + }) + } return b } diff --git a/pkg/col/colserde/arrowbatchconverter.go b/pkg/col/colserde/arrowbatchconverter.go index 32ffd4929896..38c4a360aa4c 100644 --- a/pkg/col/colserde/arrowbatchconverter.go +++ b/pkg/col/colserde/arrowbatchconverter.go @@ -21,7 +21,7 @@ import ( "github.com/apache/arrow/go/arrow/memory" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/errors" diff --git a/pkg/col/colserde/arrowbatchconverter_test.go b/pkg/col/colserde/arrowbatchconverter_test.go index a3bca9003229..137934c28b03 100644 --- a/pkg/col/colserde/arrowbatchconverter_test.go +++ b/pkg/col/colserde/arrowbatchconverter_test.go @@ -17,26 +17,28 @@ import ( "github.com/apache/arrow/go/arrow/array" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/col/colserde" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/stretchr/testify/require" ) -func randomBatch(allocator *colbase.Allocator) ([]types.T, coldata.Batch) { +func randomBatch(allocator *colmem.Allocator) ([]types.T, coldata.Batch) { const maxTyps = 16 rng, _ := randutil.NewPseudoRand() typs := make([]types.T, rng.Intn(maxTyps)+1) for i := range typs { - typs[i] = colbase.AllSupportedSQLTypes[rng.Intn(len(colbase.AllSupportedSQLTypes))] + typs[i] = typeconv.AllSupportedSQLTypes[rng.Intn(len(typeconv.AllSupportedSQLTypes))] } capacity := rng.Intn(coldata.BatchSize()) + 1 length := rng.Intn(capacity) - b := colbase.RandomBatch(allocator, rng, typs, capacity, length, rng.Float64()) + b := coldatatestutils.RandomBatch(allocator, rng, typs, capacity, length, rng.Float64()) return typs, b } @@ -59,7 +61,7 @@ func TestArrowBatchConverterRandom(t *testing.T) { // Make a copy of the original batch because the converter modifies and casts // data without copying for performance reasons. - expected := colbase.CopyBatch(testAllocator, b, typs) + expected := coldatatestutils.CopyBatch(b, typs) arrowData, err := c.BatchToArrow(b) require.NoError(t, err) @@ -109,7 +111,7 @@ func TestRecordBatchRoundtripThroughBytes(t *testing.T) { // Make a copy of the original batch because the converter modifies and // casts data without copying for performance reasons. - expected := colbase.CopyBatch(testAllocator, b, typs) + expected := coldatatestutils.CopyBatch(b, typs) actual, err := roundTripBatch(b, c, r) require.NoError(t, err) @@ -143,7 +145,7 @@ func BenchmarkArrowBatchConverter(b *testing.B) { } // Run a benchmark on every type we care about. for typIdx, typ := range typs { - batch := colbase.RandomBatch(testAllocator, rng, []types.T{typ}, coldata.BatchSize(), 0 /* length */, 0 /* nullProbability */) + batch := coldatatestutils.RandomBatch(testAllocator, rng, []types.T{typ}, coldata.BatchSize(), 0 /* length */, 0 /* nullProbability */) if batch.Width() != 1 { b.Fatalf("unexpected batch width: %d", batch.Width()) } diff --git a/pkg/col/colserde/file.go b/pkg/col/colserde/file.go index 8d1047105f4f..ae0fc33a92ae 100644 --- a/pkg/col/colserde/file.go +++ b/pkg/col/colserde/file.go @@ -20,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/colserde/arrowserde" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/types" diff --git a/pkg/col/colserde/file_test.go b/pkg/col/colserde/file_test.go index ff02890ba50c..1e150d9751a7 100644 --- a/pkg/col/colserde/file_test.go +++ b/pkg/col/colserde/file_test.go @@ -17,9 +17,9 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/col/colserde" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -33,7 +33,7 @@ func TestFileRoundtrip(t *testing.T) { t.Run(`mem`, func(t *testing.T) { // Make a copy of the original batch because the converter modifies and // casts data without copying for performance reasons. - original := colbase.CopyBatch(testAllocator, b, typs) + original := coldatatestutils.CopyBatch(b, typs) var buf bytes.Buffer s, err := colserde.NewFileSerializer(&buf, typs) @@ -66,7 +66,7 @@ func TestFileRoundtrip(t *testing.T) { // Make a copy of the original batch because the converter modifies and // casts data without copying for performance reasons. - original := colbase.CopyBatch(testAllocator, b, typs) + original := coldatatestutils.CopyBatch(b, typs) f, err := os.Create(path) require.NoError(t, err) diff --git a/pkg/col/colserde/main_test.go b/pkg/col/colserde/main_test.go index f69f7df16ebf..b66e07aacd6a 100644 --- a/pkg/col/colserde/main_test.go +++ b/pkg/col/colserde/main_test.go @@ -16,7 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -25,7 +25,7 @@ import ( var ( // testAllocator is a colexec.Allocator with an unlimited budget for use in // tests. - testAllocator *colbase.Allocator + testAllocator *colmem.Allocator // testMemMonitor and testMemAcc are a test monitor with an unlimited budget // and a memory account bound to it for use in tests. @@ -41,7 +41,7 @@ func TestMain(m *testing.M) { defer testMemMonitor.Stop(ctx) memAcc := testMemMonitor.MakeBoundAccount() testMemAcc = &memAcc - testAllocator = colbase.NewAllocator(ctx, testMemAcc) + testAllocator = colmem.NewAllocator(ctx, testMemAcc) defer testMemAcc.Close(ctx) return m.Run() }()) diff --git a/pkg/col/colserde/record_batch.go b/pkg/col/colserde/record_batch.go index fbe30ce63608..c6348e74c84f 100644 --- a/pkg/col/colserde/record_batch.go +++ b/pkg/col/colserde/record_batch.go @@ -18,7 +18,7 @@ import ( "github.com/apache/arrow/go/arrow/memory" "github.com/cockroachdb/cockroach/pkg/col/colserde/arrowserde" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/types" flatbuffers "github.com/google/flatbuffers/go" "github.com/pkg/errors" diff --git a/pkg/col/colserde/record_batch_test.go b/pkg/col/colserde/record_batch_test.go index c5f1b587ae96..a0019e444247 100644 --- a/pkg/col/colserde/record_batch_test.go +++ b/pkg/col/colserde/record_batch_test.go @@ -27,8 +27,7 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/colserde" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -227,7 +226,7 @@ func TestRecordBatchSerializerSerializeDeserializeRandom(t *testing.T) { ) for i := range typs { - typs[i] = colbase.AllSupportedSQLTypes[rng.Intn(len(colbase.AllSupportedSQLTypes))] + typs[i] = typeconv.AllSupportedSQLTypes[rng.Intn(len(typeconv.AllSupportedSQLTypes))] data[i] = randomDataFromType(rng, &typs[i], dataLen, nullProbability) } diff --git a/pkg/sql/colbase/typeconv/typeconv.go b/pkg/col/coltypes/typeconv/typeconv.go similarity index 83% rename from pkg/sql/colbase/typeconv/typeconv.go rename to pkg/col/coltypes/typeconv/typeconv.go index 5607d8fee551..007ae42f1303 100644 --- a/pkg/sql/colbase/typeconv/typeconv.go +++ b/pkg/col/coltypes/typeconv/typeconv.go @@ -18,9 +18,29 @@ import ( "github.com/pkg/errors" ) +// AllSupportedSQLTypes is a slice of all SQL types that the vectorized engine +// currently supports. It should be kept in sync with FromColumnType(). +var AllSupportedSQLTypes = []types.T{ + *types.Bool, + *types.Bytes, + *types.Date, + *types.Decimal, + *types.Int2, + *types.Int4, + *types.Int, + *types.Oid, + *types.Float, + *types.Float4, + *types.String, + *types.Uuid, + *types.Timestamp, + *types.TimestampTZ, + *types.Interval, +} + // FromColumnType returns the T that corresponds to the input ColumnType. -// Note: if you're adding a new type here, add it to -// colexec.AllSupportedSQLTypes as well. +// Note: if you're adding a new type here, add it to AllSupportedSQLTypes as +// well. func FromColumnType(ct *types.T) coltypes.T { switch ct.Family() { case types.BoolFamily: diff --git a/pkg/internal/sqlsmith/type.go b/pkg/internal/sqlsmith/type.go index 9432b278384c..727231535e65 100644 --- a/pkg/internal/sqlsmith/type.go +++ b/pkg/internal/sqlsmith/type.go @@ -12,7 +12,7 @@ package sqlsmith import ( "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" diff --git a/pkg/sql/colbase/supported_sql_types.go b/pkg/sql/colbase/supported_sql_types.go deleted file mode 100644 index 2f278251313a..000000000000 --- a/pkg/sql/colbase/supported_sql_types.go +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright 2019 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package colbase - -import "github.com/cockroachdb/cockroach/pkg/sql/types" - -// AllSupportedSQLTypes is a slice of all SQL types that the vectorized engine -// currently supports. It should be kept in sync with typeconv.FromColumnType(). -var AllSupportedSQLTypes = []types.T{ - *types.Bool, - *types.Bytes, - *types.Date, - *types.Decimal, - *types.Int2, - *types.Int4, - *types.Int, - *types.Oid, - *types.Float, - *types.Float4, - *types.String, - *types.Uuid, - *types.Timestamp, - *types.TimestampTZ, - *types.Interval, -} diff --git a/pkg/sql/colcontainer/diskqueue_test.go b/pkg/sql/colcontainer/diskqueue_test.go index 35269a2b9394..a6921232b8e8 100644 --- a/pkg/sql/colcontainer/diskqueue_test.go +++ b/pkg/sql/colcontainer/diskqueue_test.go @@ -16,8 +16,9 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "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/types" "github.com/cockroachdb/cockroach/pkg/testutils/colcontainerutils" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" @@ -62,12 +63,12 @@ func TestDiskQueue(t *testing.T) { prefix, diskQueueCacheMode, alwaysCompress, suffix, numBatches), func(t *testing.T) { // Create random input. batches := make([]coldata.Batch, 0, numBatches) - op := colbase.NewRandomDataOp(testAllocator, rng, colbase.RandomDataOpArgs{ + op := coldatatestutils.NewRandomDataOp(testAllocator, rng, coldatatestutils.RandomDataOpArgs{ NumBatches: cap(batches), BatchSize: 1 + rng.Intn(coldata.BatchSize()), Nulls: true, BatchAccumulator: func(b coldata.Batch, typs []types.T) { - batches = append(batches, colbase.CopyBatch(testAllocator, b, typs)) + batches = append(batches, coldatatestutils.CopyBatch(b, typs)) }, }) typs := op.Typs() @@ -200,8 +201,8 @@ func BenchmarkDiskQueue(b *testing.B) { rng, _ := randutil.NewPseudoRand() typs := []types.T{*types.Int} - batch := colbase.RandomBatch(testAllocator, rng, typs, coldata.BatchSize(), 0, 0) - op := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + batch := coldatatestutils.RandomBatch(testAllocator, rng, typs, coldata.BatchSize(), 0, 0) + op := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) ctx := context.Background() for i := 0; i < b.N; i++ { op.ResetBatchesToReturn(numBatches) diff --git a/pkg/sql/colcontainer/main_test.go b/pkg/sql/colcontainer/main_test.go index 2e3bd24f658b..831281f6b49e 100644 --- a/pkg/sql/colcontainer/main_test.go +++ b/pkg/sql/colcontainer/main_test.go @@ -16,7 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -25,7 +25,7 @@ import ( var ( // testAllocator is a colexec.Allocator with an unlimited budget for use in // tests. - testAllocator *colbase.Allocator + testAllocator *colmem.Allocator // testMemMonitor and testMemAcc are a test monitor with an unlimited budget // and a memory account bound to it for use in tests. @@ -46,7 +46,7 @@ func TestMain(m *testing.M) { defer testMemMonitor.Stop(ctx) memAcc := testMemMonitor.MakeBoundAccount() testMemAcc = &memAcc - testAllocator = colbase.NewAllocator(ctx, testMemAcc) + testAllocator = colmem.NewAllocator(ctx, testMemAcc) defer testMemAcc.Close(ctx) testDiskMonitor = execinfra.NewTestDiskMonitor(ctx, cluster.MakeTestingClusterSettings()) diff --git a/pkg/sql/colcontainer/partitionedqueue.go b/pkg/sql/colcontainer/partitionedqueue.go index 7474f89ca26d..b627df6bb0ff 100644 --- a/pkg/sql/colcontainer/partitionedqueue.go +++ b/pkg/sql/colcontainer/partitionedqueue.go @@ -15,7 +15,7 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/errors" @@ -144,7 +144,7 @@ func NewPartitionedDiskQueue( // DiskQueues cannot serialize zero length schemas, so catch this error // early. // TODO(asubiotto): We could support this, but not sure we need to. - vecerror.InternalError("zero length schema unsupported") + colexecerror.InternalError("zero length schema unsupported") } return &PartitionedDiskQueue{ typs: typs, @@ -175,7 +175,7 @@ func (p *PartitionedDiskQueue) closeWritePartition( ctx context.Context, idx int, releaseFDOption closeWritePartitionArgument, ) error { if p.partitions[idx].state != partitionStateWriting { - vecerror.InternalError(fmt.Sprintf("illegal state change from %d to partitionStateClosedForWriting, only partitionStateWriting allowed", p.partitions[idx].state)) + colexecerror.InternalError(fmt.Sprintf("illegal state change from %d to partitionStateClosedForWriting, only partitionStateWriting allowed", p.partitions[idx].state)) } if err := p.partitions[idx].Enqueue(ctx, coldata.ZeroBatch); err != nil { return err @@ -190,7 +190,7 @@ func (p *PartitionedDiskQueue) closeWritePartition( func (p *PartitionedDiskQueue) closeReadPartition(idx int) error { if p.partitions[idx].state != partitionStateReading { - vecerror.InternalError(fmt.Sprintf("illegal state change from %d to partitionStateClosedForReading, only partitionStateReading allowed", p.partitions[idx].state)) + colexecerror.InternalError(fmt.Sprintf("illegal state change from %d to partitionStateClosedForReading, only partitionStateReading allowed", p.partitions[idx].state)) } if err := p.partitions[idx].CloseRead(); err != nil { return err @@ -300,7 +300,7 @@ func (p *PartitionedDiskQueue) Dequeue( case partitionStatePermanentlyClosed: return errors.Errorf("partition at index %d permanently closed, cannot Dequeue", partitionIdx) default: - vecerror.InternalError(fmt.Sprintf("unhandled state %d", state)) + colexecerror.InternalError(fmt.Sprintf("unhandled state %d", state)) } notEmpty, err := p.partitions[idx].Dequeue(ctx, batch) if err != nil { @@ -317,7 +317,7 @@ func (p *PartitionedDiskQueue) Dequeue( // Dequeue but more batches will be added in the future (i.e. a zero batch // was never enqueued). Since we require partitions to be closed for writing // before reading, this state is unexpected. - vecerror.InternalError("DiskQueue unexpectedly returned that more data will be added") + colexecerror.InternalError("DiskQueue unexpectedly returned that more data will be added") } return nil } diff --git a/pkg/sql/colcontainer/partitionedqueue_test.go b/pkg/sql/colcontainer/partitionedqueue_test.go index e621b80fcce8..757acc94f8e0 100644 --- a/pkg/sql/colcontainer/partitionedqueue_test.go +++ b/pkg/sql/colcontainer/partitionedqueue_test.go @@ -16,8 +16,8 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "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.NewMemBatch(typs) - sem = &colbase.TestingSemaphore{} + sem = &colexecbase.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 := colbase.NewTestingSemaphore(maxPartitions + 1) + sem := colexecbase.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 := colbase.NewTestingSemaphore(maxPartitions + 2) + sem := colexecbase.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/colencoding/value_encoding_test.go b/pkg/sql/colencoding/value_encoding_test.go index aed26ac4a22f..005e960a13fb 100644 --- a/pkg/sql/colencoding/value_encoding_test.go +++ b/pkg/sql/colencoding/value_encoding_test.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" diff --git a/pkg/sql/colexec/aggregator.go b/pkg/sql/colexec/aggregator.go index 98b949bd995b..bbae6568f288 100644 --- a/pkg/sql/colexec/aggregator.go +++ b/pkg/sql/colexec/aggregator.go @@ -14,8 +14,9 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" @@ -104,7 +105,7 @@ type aggregateFunc interface { type orderedAggregator struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator done bool aggCols [][]uint32 @@ -149,21 +150,21 @@ type orderedAggregator struct { seenNonEmptyBatch bool } -var _ colbase.Operator = &orderedAggregator{} +var _ colexecbase.Operator = &orderedAggregator{} // NewOrderedAggregator creates an ordered aggregator on the given grouping // columns. aggCols is a slice where each index represents a new aggregation // function. The slice at that index specifies the columns of the input batch // that the aggregate function should work on. func NewOrderedAggregator( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, typs []types.T, aggFns []execinfrapb.AggregatorSpec_Func, groupCols []uint32, aggCols [][]uint32, isScalar bool, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { if len(aggFns) != len(aggCols) { return nil, errors.Errorf( @@ -228,7 +229,7 @@ func NewOrderedAggregator( } func makeAggregateFuncs( - allocator *colbase.Allocator, aggTyps [][]types.T, aggFns []execinfrapb.AggregatorSpec_Func, + allocator *colmem.Allocator, aggTyps [][]types.T, aggFns []execinfrapb.AggregatorSpec_Func, ) ([]aggregateFunc, error) { funcs := make([]aggregateFunc, len(aggFns)) diff --git a/pkg/sql/colexec/aggregator_test.go b/pkg/sql/colexec/aggregator_test.go index fefab8b5d333..a6e755c32bcf 100644 --- a/pkg/sql/colexec/aggregator_test.go +++ b/pkg/sql/colexec/aggregator_test.go @@ -18,7 +18,9 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -58,14 +60,14 @@ type aggregatorTestCase struct { // hash aggregators at the same time. type aggType struct { new func( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, typs []types.T, aggFns []execinfrapb.AggregatorSpec_Func, groupCols []uint32, aggCols [][]uint32, isScalar bool, - ) (colbase.Operator, error) + ) (colexecbase.Operator, error) name string } @@ -74,14 +76,14 @@ var aggTypes = []aggType{ // This is a wrapper around NewHashAggregator so its signature is compatible // with orderedAggregator. new: func( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, typs []types.T, aggFns []execinfrapb.AggregatorSpec_Func, groupCols []uint32, aggCols [][]uint32, _ bool, - ) (colbase.Operator, error) { + ) (colexecbase.Operator, error) { return NewHashAggregator( allocator, input, typs, aggFns, groupCols, aggCols) }, @@ -329,7 +331,7 @@ func TestAggregatorOneFunc(t *testing.T) { } t.Run(agg.name, func(t *testing.T) { runTests(t, []tuples{tc.input}, tc.expected, unorderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return agg.new( testAllocator, input[0], @@ -479,7 +481,7 @@ func TestAggregatorMultiFunc(t *testing.T) { t.Fatal(err) } runTestsWithTyps(t, []tuples{tc.input}, [][]types.T{tc.typs}, tc.expected, unorderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return agg.new(testAllocator, input[0], tc.typs, tc.aggFns, tc.groupCols, tc.aggCols, false /* isScalar */) }) }) @@ -571,7 +573,7 @@ func TestAggregatorAllFunctions(t *testing.T) { []tuples{tc.input}, tc.expected, verifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return agg.new(testAllocator, input[0], tc.typs, tc.aggFns, tc.groupCols, tc.aggCols, false /* isScalar */) }) }) @@ -752,7 +754,7 @@ func BenchmarkAggregator(b *testing.B) { if hasNulls { nullProb = nullProbability } - coldata.RandomVec(rng, &typ, bytesFixedLength, cols[1], nTuples, nullProb) + coldatatestutils.RandomVec(rng, &typ, bytesFixedLength, cols[1], nTuples, nullProb) if typ.Identical(types.Int) && aggFn == execinfrapb.AggregatorSpec_SUM { // Summation of random Int64 values can lead to // overflow, and we will panic. To go around it, we @@ -918,7 +920,7 @@ func TestHashAggregator(t *testing.T) { t.Fatal(err) } t.Run(fmt.Sprintf("numOfHashBuckets=%d", numOfHashBuckets), func(t *testing.T) { - runTests(t, []tuples{tc.input}, tc.expected, unorderedVerifier, func(sources []colbase.Operator) (colbase.Operator, error) { + runTests(t, []tuples{tc.input}, tc.expected, unorderedVerifier, func(sources []colexecbase.Operator) (colexecbase.Operator, error) { a, err := NewHashAggregator(testAllocator, sources[0], tc.typs, tc.aggFns, tc.groupCols, tc.aggCols) a.(*hashAggregator).testingKnobs.numOfHashBuckets = uint64(numOfHashBuckets) return a, err diff --git a/pkg/sql/colexec/and_or_projection_test.go b/pkg/sql/colexec/and_or_projection_test.go index d24b1480d446..19eb9150eee1 100644 --- a/pkg/sql/colexec/and_or_projection_test.go +++ b/pkg/sql/colexec/and_or_projection_test.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -197,7 +197,7 @@ func TestAndOrOps(t *testing.T) { [][]types.T{{*types.Bool, *types.Bool}}, tc.expected, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { projOp, err := createTestProjectingOperator( ctx, flowCtx, input[0], []types.T{*types.Bool, *types.Bool}, fmt.Sprintf("@1 %s @2", test.operation), false, /* canFallbackToRowexec */ @@ -257,7 +257,7 @@ func benchmarkLogicalProjOp( } } typs := []types.T{*types.Bool, *types.Bool} - input := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) logicalProjOp, err := createTestProjectingOperator( ctx, flowCtx, input, typs, fmt.Sprintf("@1 %s @2", operation), false, /* canFallbackToRowexec */ diff --git a/pkg/sql/colexec/and_or_projection_tmpl.go b/pkg/sql/colexec/and_or_projection_tmpl.go index dde24f1ac4c1..fbaa2e181d6c 100644 --- a/pkg/sql/colexec/and_or_projection_tmpl.go +++ b/pkg/sql/colexec/and_or_projection_tmpl.go @@ -24,19 +24,20 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" ) // {{ range .}} type _OP_LOWERProjOp struct { - allocator *colbase.Allocator - input colbase.Operator + allocator *colmem.Allocator + input colexecbase.Operator - leftProjOpChain colbase.Operator - rightProjOpChain colbase.Operator + leftProjOpChain colexecbase.Operator + rightProjOpChain colexecbase.Operator leftFeedOp *feedOperator rightFeedOp *feedOperator @@ -54,11 +55,11 @@ type _OP_LOWERProjOp struct { // the boolean columns at leftIdx and rightIdx, returning the result in // outputIdx. func New_OP_TITLEProjOp( - allocator *colbase.Allocator, - input, leftProjOpChain, rightProjOpChain colbase.Operator, + allocator *colmem.Allocator, + input, leftProjOpChain, rightProjOpChain colexecbase.Operator, leftFeedOp, rightFeedOp *feedOperator, leftIdx, rightIdx, outputIdx int, -) colbase.Operator { +) colexecbase.Operator { return &_OP_LOWERProjOp{ allocator: allocator, input: input, @@ -86,7 +87,7 @@ func (o *_OP_LOWERProjOp) Child(nth int, verbose bool) execinfra.OpNode { case 2: return o.rightProjOpChain default: - vecerror.InternalError(fmt.Sprintf("invalid idx %d", nth)) + colexecerror.InternalError(fmt.Sprintf("invalid idx %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/any_not_null_agg_tmpl.go b/pkg/sql/colexec/any_not_null_agg_tmpl.go index f0285dccc227..08905465ff9d 100644 --- a/pkg/sql/colexec/any_not_null_agg_tmpl.go +++ b/pkg/sql/colexec/any_not_null_agg_tmpl.go @@ -25,17 +25,15 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/pkg/errors" ) -func newAnyNotNullAgg(allocator *colbase.Allocator, t *types.T) (aggregateFunc, error) { +func newAnyNotNullAgg(allocator *colmem.Allocator, t *types.T) (aggregateFunc, error) { switch typeconv.FromColumnType(t) { // {{range .}} case _TYPES_T: @@ -46,6 +44,9 @@ func newAnyNotNullAgg(allocator *colbase.Allocator, t *types.T) (aggregateFunc, } } +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Declarations to make the template compile properly. @@ -75,7 +76,7 @@ const _TYPES_T = coltypes.Unhandled // anyNotNull_TYPEAgg implements the ANY_NOT_NULL aggregate, returning the // first non-null value in the input column. type anyNotNull_TYPEAgg struct { - allocator *colbase.Allocator + allocator *colmem.Allocator done bool groups []bool vec coldata.Vec diff --git a/pkg/sql/colexec/avg_agg_tmpl.go b/pkg/sql/colexec/avg_agg_tmpl.go index 5a790bdea167..7d3fcff6bee0 100644 --- a/pkg/sql/colexec/avg_agg_tmpl.go +++ b/pkg/sql/colexec/avg_agg_tmpl.go @@ -23,8 +23,8 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/pkg/errors" @@ -46,13 +46,13 @@ var _ coltypes.T // input to the result of the second input / the third input, where the third // input is an int64. func _ASSIGN_DIV_INT64(_, _, _ string) { - vecerror.InternalError("") + colexecerror.InternalError("") } // _ASSIGN_ADD is the template addition function for assigning the first input // to the result of the second input + the third input. func _ASSIGN_ADD(_, _, _ string) { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} diff --git a/pkg/sql/colexec/bool_and_or_agg_tmpl.go b/pkg/sql/colexec/bool_and_or_agg_tmpl.go index c0c5a8ab1525..073d5b706853 100644 --- a/pkg/sql/colexec/bool_and_or_agg_tmpl.go +++ b/pkg/sql/colexec/bool_and_or_agg_tmpl.go @@ -21,22 +21,19 @@ package colexec import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" - // {{/* - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // */}} - // HACK: crlfmt removes the "*/}}" comment if it's the last line in the import - // block. This was picked because it sorts after "pkg/sql/colbase/vecerror" and - // has no deps. - _ "github.com/cockroachdb/cockroach/pkg/util/bufalloc" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" ) +// Remove unused warning. +var _ = colexecerror.InternalError + // {{/* // _ASSIGN_BOOL_OP is the template boolean operation function for assigning the // first input to the result of a boolean operation of the second and the third // inputs. func _ASSIGN_BOOL_OP(_, _, _ string) { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} diff --git a/pkg/sql/colexec/bool_vec_to_sel.go b/pkg/sql/colexec/bool_vec_to_sel.go index 57aa6d19b941..f6c003ba23bf 100644 --- a/pkg/sql/colexec/bool_vec_to_sel.go +++ b/pkg/sql/colexec/bool_vec_to_sel.go @@ -14,7 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" ) // boolVecToSelOp transforms a boolean column into a selection vector by adding @@ -28,7 +28,7 @@ type boolVecToSelOp struct { outputCol []bool } -var _ colbase.Operator = &boolVecToSelOp{} +var _ colexecbase.Operator = &boolVecToSelOp{} 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 @@ -102,7 +102,7 @@ func boolVecToSel64(vec []bool, sel []int) []int { // For internal use cases that just need a way to create a selection vector // based on a boolean column that *isn't* in a batch, just create a // boolVecToSelOp directly with the desired boolean slice. -func NewBoolVecToSelOp(input colbase.Operator, colIdx int) colbase.Operator { +func NewBoolVecToSelOp(input colexecbase.Operator, colIdx int) colexecbase.Operator { d := selBoolOp{OneInputNode: NewOneInputNode(input), colIdx: colIdx} ret := &boolVecToSelOp{OneInputNode: NewOneInputNode(&d)} d.boolVecToSelOp = ret diff --git a/pkg/sql/colexec/bool_vec_to_sel_test.go b/pkg/sql/colexec/bool_vec_to_sel_test.go index 4ff2927c58bb..2af97a7220c2 100644 --- a/pkg/sql/colexec/bool_vec_to_sel_test.go +++ b/pkg/sql/colexec/bool_vec_to_sel_test.go @@ -13,7 +13,7 @@ package colexec import ( "testing" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) @@ -31,7 +31,7 @@ func TestBoolVecToSelOp(t *testing.T) { }, } for _, tc := range tcs { - runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, func(input []colbase.Operator) (colbase.Operator, error) { + runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { return NewBoolVecToSelOp(input[0], 0), nil }) } diff --git a/pkg/sql/colexec/buffer.go b/pkg/sql/colexec/buffer.go index 36d0d93f9088..b127eede0374 100644 --- a/pkg/sql/colexec/buffer.go +++ b/pkg/sql/colexec/buffer.go @@ -14,7 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" ) // bufferOp is an operator that buffers a single batch at a time from an input, @@ -28,11 +28,11 @@ type bufferOp struct { batch coldata.Batch } -var _ colbase.Operator = &bufferOp{} +var _ colexecbase.Operator = &bufferOp{} // NewBufferOp returns a new bufferOp, initialized to buffer batches from the // supplied input. -func NewBufferOp(input colbase.Operator) colbase.Operator { +func NewBufferOp(input colexecbase.Operator) colexecbase.Operator { return &bufferOp{ OneInputNode: NewOneInputNode(input), } diff --git a/pkg/sql/colexec/builtin_funcs.go b/pkg/sql/colexec/builtin_funcs.go index 5d5e650cf957..e2fbc12ae688 100644 --- a/pkg/sql/colexec/builtin_funcs.go +++ b/pkg/sql/colexec/builtin_funcs.go @@ -15,9 +15,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -26,7 +27,7 @@ import ( type defaultBuiltinFuncOperator struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator evalCtx *tree.EvalContext funcExpr *tree.FuncExpr columnTypes []types.T @@ -40,7 +41,7 @@ type defaultBuiltinFuncOperator struct { da sqlbase.DatumAlloc } -var _ colbase.Operator = &defaultBuiltinFuncOperator{} +var _ colexecbase.Operator = &defaultBuiltinFuncOperator{} func (b *defaultBuiltinFuncOperator) Init() { b.input.Init() @@ -82,7 +83,7 @@ func (b *defaultBuiltinFuncOperator) Next(ctx context.Context) coldata.Batch { } else { res, err = b.funcExpr.ResolvedOverload().Fn(b.evalCtx, b.row) if err != nil { - vecerror.ExpectedError(err) + colexecerror.ExpectedError(err) } } @@ -92,7 +93,7 @@ func (b *defaultBuiltinFuncOperator) Next(ctx context.Context) coldata.Batch { } else { converted, err := b.converter(res) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } coldata.SetValueAt(output, converted, rowIdx, b.outputPhysType) } @@ -107,14 +108,14 @@ func (b *defaultBuiltinFuncOperator) Next(ctx context.Context) coldata.Batch { // NewBuiltinFunctionOperator returns an operator that applies builtin functions. func NewBuiltinFunctionOperator( - allocator *colbase.Allocator, + allocator *colmem.Allocator, evalCtx *tree.EvalContext, funcExpr *tree.FuncExpr, columnTypes []types.T, argumentCols []int, outputIdx int, - input colbase.Operator, -) (colbase.Operator, error) { + input colexecbase.Operator, +) (colexecbase.Operator, error) { switch funcExpr.ResolvedOverload().SpecializedVecBuiltin { case tree.SubstringStringIntInt: input = newVectorTypeEnforcer(allocator, input, types.Bytes, outputIdx) diff --git a/pkg/sql/colexec/builtin_funcs_test.go b/pkg/sql/colexec/builtin_funcs_test.go index 8dc64d8913b6..b25808b1a8a9 100644 --- a/pkg/sql/colexec/builtin_funcs_test.go +++ b/pkg/sql/colexec/builtin_funcs_test.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" @@ -73,7 +73,7 @@ func TestBasicBuiltinFunctions(t *testing.T) { for _, tc := range testCases { t.Run(tc.desc, func(t *testing.T) { runTests(t, []tuples{tc.inputTuples}, tc.outputTuples, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return createTestProjectingOperator( ctx, flowCtx, input[0], tc.inputTypes, tc.expr, false, /* canFallbackToRowexec */ @@ -125,7 +125,7 @@ func benchmarkBuiltinFunctions(b *testing.B, useSelectionVector bool, hasNulls b } typs := []types.T{*types.Int} - source := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) op, err := createTestProjectingOperator( ctx, flowCtx, source, typs, "abs(@1)" /* projectingExpr */, false, /* canFallbackToRowexec */ @@ -169,8 +169,8 @@ func BenchmarkCompareSpecializedOperators(b *testing.B) { eCol[i] = 4 } batch.SetLength(coldata.BatchSize()) - var source colbase.Operator - source = colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + var source colexecbase.Operator + source = colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) source = newVectorTypeEnforcer(testAllocator, source, types.Bytes, outputIdx) // Set up the default operator. diff --git a/pkg/sql/colexec/cancel_checker.go b/pkg/sql/colexec/cancel_checker.go index 5296e29c71d2..420f807ed5d7 100644 --- a/pkg/sql/colexec/cancel_checker.go +++ b/pkg/sql/colexec/cancel_checker.go @@ -14,8 +14,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" ) @@ -35,10 +35,10 @@ func (c *CancelChecker) Init() { c.input.Init() } -var _ colbase.Operator = &CancelChecker{} +var _ colexecbase.Operator = &CancelChecker{} // NewCancelChecker creates a new CancelChecker. -func NewCancelChecker(op colbase.Operator) *CancelChecker { +func NewCancelChecker(op colexecbase.Operator) *CancelChecker { return &CancelChecker{OneInputNode: NewOneInputNode(op)} } @@ -72,7 +72,7 @@ func (c *CancelChecker) check(ctx context.Context) { func (c *CancelChecker) checkEveryCall(ctx context.Context) { select { case <-ctx.Done(): - vecerror.ExpectedError(sqlbase.QueryCanceledError) + colexecerror.ExpectedError(sqlbase.QueryCanceledError) default: } } diff --git a/pkg/sql/colexec/cancel_checker_test.go b/pkg/sql/colexec/cancel_checker_test.go index ca2679cad5e6..60b9a6dfa8b8 100644 --- a/pkg/sql/colexec/cancel_checker_test.go +++ b/pkg/sql/colexec/cancel_checker_test.go @@ -14,8 +14,8 @@ import ( "context" "testing" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -30,9 +30,9 @@ func TestCancelChecker(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) typs := []types.T{*types.Int} batch := testAllocator.NewMemBatch(typs) - op := NewCancelChecker(NewNoop(colbase.NewRepeatableBatchSource(testAllocator, batch, typs))) + op := NewCancelChecker(NewNoop(colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs))) cancel() - err := vecerror.CatchVectorizedRuntimeError(func() { + err := colexecerror.CatchVectorizedRuntimeError(func() { op.Next(ctx) }) require.True(t, errors.Is(err, sqlbase.QueryCanceledError)) diff --git a/pkg/sql/colexec/case.go b/pkg/sql/colexec/case.go index 9f747feabd22..0970b8b21c36 100644 --- a/pkg/sql/colexec/case.go +++ b/pkg/sql/colexec/case.go @@ -15,19 +15,20 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/types" ) type caseOp struct { - allocator *colbase.Allocator + allocator *colmem.Allocator buffer *bufferOp - caseOps []colbase.Operator - elseOp colbase.Operator + caseOps []colexecbase.Operator + elseOp colexecbase.Operator thenIdxs []int outputIdx int @@ -60,14 +61,14 @@ func (c *caseOp) Child(nth int, verbose bool) execinfra.OpNode { } else if nth == 1+len(c.caseOps) { return c.elseOp } - vecerror.InternalError(fmt.Sprintf("invalid idx %d", nth)) + colexecerror.InternalError(fmt.Sprintf("invalid idx %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } func (c *caseOp) InternalMemoryUsage() int { // We internally use two selection vectors, origSel and prevSel. - return 2 * colbase.SizeOfBatchSizeSelVector + return 2 * colmem.SizeOfBatchSizeSelVector } // NewCaseOp returns an operator that runs a case statement. @@ -81,14 +82,14 @@ func (c *caseOp) InternalMemoryUsage() int { // thenCol is the index into the output batch to write to. // typ is the type of the CASE expression. func NewCaseOp( - allocator *colbase.Allocator, - buffer colbase.Operator, - caseOps []colbase.Operator, - elseOp colbase.Operator, + allocator *colmem.Allocator, + buffer colexecbase.Operator, + caseOps []colexecbase.Operator, + elseOp colexecbase.Operator, thenIdxs []int, outputIdx int, typ *types.T, -) colbase.Operator { +) colexecbase.Operator { return &caseOp{ allocator: allocator, buffer: buffer.(*bufferOp), diff --git a/pkg/sql/colexec/case_test.go b/pkg/sql/colexec/case_test.go index 1654cbf0b6ce..dd75897b60b9 100644 --- a/pkg/sql/colexec/case_test.go +++ b/pkg/sql/colexec/case_test.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -80,7 +80,7 @@ func TestCaseOp(t *testing.T) { inputTypes: []types.T{*types.Int, *types.Int}, }, } { - runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, func(inputs []colbase.Operator) (colbase.Operator, error) { + runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, func(inputs []colexecbase.Operator) (colexecbase.Operator, error) { spec.Input[0].ColumnTypes = tc.inputTypes spec.Post.RenderExprs[0].Expr = tc.renderExpr args := NewColOperatorArgs{ diff --git a/pkg/sql/colexec/cast_test.go b/pkg/sql/colexec/cast_test.go index ab700a003259..3532bee96408 100644 --- a/pkg/sql/colexec/cast_test.go +++ b/pkg/sql/colexec/cast_test.go @@ -16,8 +16,9 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -112,7 +113,7 @@ func TestRandomizedCast(t *testing.T) { output = append(output, tuple{c.fromPhysType(fromDatum), c.toPhysType(toDatum)}) } runTests(t, []tuples{input}, output, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return createTestCastOperator(ctx, flowCtx, input[0], c.fromTyp, c.toTyp) }) }) @@ -151,11 +152,11 @@ func BenchmarkCastOp(b *testing.B) { selectivity = 1.0 } typs := []types.T{typePair[0]} - batch := colbase.RandomBatchWithSel( + batch := coldatatestutils.RandomBatchWithSel( testAllocator, rng, typs, coldata.BatchSize(), nullProbability, selectivity, ) - source := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) op, err := createTestCastOperator(ctx, flowCtx, source, &typePair[0], &typePair[1]) require.NoError(b, err) b.SetBytes(int64(8 * coldata.BatchSize())) @@ -173,10 +174,10 @@ func BenchmarkCastOp(b *testing.B) { func createTestCastOperator( ctx context.Context, flowCtx *execinfra.FlowCtx, - input colbase.Operator, + input colexecbase.Operator, fromTyp *types.T, toTyp *types.T, -) (colbase.Operator, error) { +) (colexecbase.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/cast_tmpl.go b/pkg/sql/colexec/cast_tmpl.go index d1c28e5bd4f2..d56b4a151143 100644 --- a/pkg/sql/colexec/cast_tmpl.go +++ b/pkg/sql/colexec/cast_tmpl.go @@ -27,17 +27,19 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/pkg/errors" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* type _ALLTYPES interface{} @@ -51,22 +53,22 @@ var _ tree.Datum var _ coltypes.T func _ASSIGN_CAST(to, from interface{}) { - vecerror.InternalError("") + colexecerror.InternalError("") } // This will be replaced with execgen.UNSAFEGET func _FROM_TYPE_UNSAFEGET(to, from interface{}) interface{} { - vecerror.InternalError("") + colexecerror.InternalError("") } // This will be replaced with execgen.SET. func _TO_TYPE_SET(to, from interface{}) { - vecerror.InternalError("") + colexecerror.InternalError("") } // This will be replaced with execgen.SLICE. func _FROM_TYPE_SLICE(col, i, j interface{}) interface{} { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} @@ -131,22 +133,22 @@ func cast(fromType, toType *types.T, inputVec, outputVec coldata.Vec, n int, sel // {{end}} // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unhandled cast FROM -> TO type: %s -> %s", fromType, toType)) + colexecerror.InternalError(fmt.Sprintf("unhandled cast FROM -> TO type: %s -> %s", fromType, toType)) } // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unhandled FROM type: %s", fromType)) + colexecerror.InternalError(fmt.Sprintf("unhandled FROM type: %s", fromType)) } } func GetCastOperator( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, colIdx int, resultIdx int, fromType *types.T, toType *types.T, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { input = newVectorTypeEnforcer(allocator, input, toType, resultIdx) if fromType.Family() == types.UnknownFamily { return &castOpNullAny{ @@ -184,12 +186,12 @@ func GetCastOperator( type castOpNullAny struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator colIdx int outputIdx int } -var _ colbase.Operator = &castOpNullAny{} +var _ colexecbase.Operator = &castOpNullAny{} func (c *castOpNullAny) Init() { c.input.Init() @@ -211,7 +213,7 @@ func (c *castOpNullAny) Next(ctx context.Context) coldata.Batch { if vecNulls.NullAt(i) { projNulls.SetNull(i) } else { - vecerror.InternalError(errors.Errorf("unexpected non-null at index %d", i)) + colexecerror.InternalError(errors.Errorf("unexpected non-null at index %d", i)) } } } else { @@ -219,7 +221,7 @@ func (c *castOpNullAny) Next(ctx context.Context) coldata.Batch { if vecNulls.NullAt(i) { projNulls.SetNull(i) } else { - vecerror.InternalError(fmt.Errorf("unexpected non-null at index %d", i)) + colexecerror.InternalError(fmt.Errorf("unexpected non-null at index %d", i)) } } } @@ -228,14 +230,14 @@ func (c *castOpNullAny) Next(ctx context.Context) coldata.Batch { type castOp struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator colIdx int outputIdx int fromType *types.T toType *types.T } -var _ colbase.Operator = &castOp{} +var _ colexecbase.Operator = &castOp{} func (c *castOp) Init() { c.input.Init() diff --git a/pkg/sql/colexec/cfetcher.go b/pkg/sql/colexec/cfetcher.go index 0bc5072f03b2..16236074d33a 100644 --- a/pkg/sql/colexec/cfetcher.go +++ b/pkg/sql/colexec/cfetcher.go @@ -19,13 +19,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" "github.com/cockroachdb/cockroach/pkg/sql/colencoding" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/scrub" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -235,7 +235,7 @@ type cFetcher struct { // adapter is a utility struct that helps with memory accounting. adapter struct { ctx context.Context - allocator *colbase.Allocator + allocator *colmem.Allocator batch coldata.Batch err error } @@ -245,7 +245,7 @@ type cFetcher struct { // non-primary index, tables.ValNeededForCol can only refer to columns in the // index. func (rf *cFetcher) Init( - allocator *colbase.Allocator, + allocator *colmem.Allocator, reverse bool, lockStr sqlbase.ScanLockingStrength, returnRangeInfo bool, @@ -601,7 +601,7 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { case stateInitFetch: moreKeys, kv, newSpan, err := rf.fetcher.NextKV(ctx) if err != nil { - return nil, vecerror.NewStorageError(err) + return nil, colexecerror.NewStorageError(err) } if !moreKeys { rf.machine.state[0] = stateEmitLastBatch @@ -750,7 +750,7 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { for { moreRows, kv, _, err := rf.fetcher.NextKV(ctx) if err != nil { - return nil, vecerror.NewStorageError(err) + return nil, colexecerror.NewStorageError(err) } if debugState { log.Infof(ctx, "found kv %s, seeking to prefix %s", kv.Key, rf.machine.seekPrefix) @@ -782,7 +782,7 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { case stateFetchNextKVWithUnfinishedRow: moreKVs, kv, _, err := rf.fetcher.NextKV(ctx) if err != nil { - return nil, vecerror.NewStorageError(err) + return nil, colexecerror.NewStorageError(err) } if !moreKVs { // No more data. Finalize the row and exit. diff --git a/pkg/sql/colexec/colbatch_scan.go b/pkg/sql/colexec/colbatch_scan.go index 7fdb9a98ecea..c1accb8cb359 100644 --- a/pkg/sql/colexec/colbatch_scan.go +++ b/pkg/sql/colexec/colbatch_scan.go @@ -15,8 +15,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/row" @@ -36,7 +37,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 { - colbase.ZeroInputNode + colexecbase.ZeroInputNode spans roachpb.Spans flowCtx *execinfra.FlowCtx rf *cFetcher @@ -49,7 +50,7 @@ type colBatchScan struct { init bool } -var _ colbase.Operator = &colBatchScan{} +var _ colexecbase.Operator = &colBatchScan{} func (s *colBatchScan) Init() { s.ctx = context.Background() @@ -61,17 +62,17 @@ func (s *colBatchScan) Init() { s.ctx, s.flowCtx.Txn, s.spans, limitBatches, s.limitHint, s.flowCtx.TraceKV, ); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } } func (s *colBatchScan) Next(ctx context.Context) coldata.Batch { bat, err := s.rf.NextBatch(ctx) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } if bat.Selection() != nil { - vecerror.InternalError("unexpectedly a selection vector is set on the batch coming from CFetcher") + colexecerror.InternalError("unexpectedly a selection vector is set on the batch coming from CFetcher") } return bat } @@ -99,7 +100,7 @@ func (s *colBatchScan) DrainMeta(ctx context.Context) []execinfrapb.ProducerMeta // newColBatchScan creates a new colBatchScan operator. func newColBatchScan( - allocator *colbase.Allocator, + allocator *colmem.Allocator, flowCtx *execinfra.FlowCtx, spec *execinfrapb.TableReaderSpec, post *execinfrapb.PostProcessSpec, @@ -149,7 +150,7 @@ func newColBatchScan( // initCRowFetcher initializes a row.cFetcher. See initRowFetcher. func initCRowFetcher( - allocator *colbase.Allocator, + allocator *colmem.Allocator, fetcher *cFetcher, desc *sqlbase.TableDescriptor, indexIdx int, diff --git a/pkg/sql/colexec/columnarizer.go b/pkg/sql/colexec/columnarizer.go index d1e3b9201b56..0d5863bb51a7 100644 --- a/pkg/sql/colexec/columnarizer.go +++ b/pkg/sql/colexec/columnarizer.go @@ -15,9 +15,10 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -38,7 +39,7 @@ type Columnarizer struct { // which will simplify this model. mu syncutil.Mutex - allocator *colbase.Allocator + allocator *colmem.Allocator input execinfra.RowSource da sqlbase.DatumAlloc initStatus OperatorInitStatus @@ -50,12 +51,12 @@ type Columnarizer struct { typs []types.T } -var _ colbase.Operator = &Columnarizer{} +var _ colexecbase.Operator = &Columnarizer{} // NewColumnarizer returns a new Columnarizer. func NewColumnarizer( ctx context.Context, - allocator *colbase.Allocator, + allocator *colmem.Allocator, flowCtx *execinfra.FlowCtx, processorID int32, input execinfra.RowSource, @@ -127,7 +128,7 @@ func (c *Columnarizer) Next(context.Context) coldata.Batch { for idx, ct := range columnTypes { err := EncDatumRowsToColVec(c.allocator, c.buffered[:nRows], c.batch.ColVec(idx), idx, &ct, &c.da) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } } c.batch.SetLength(nRows) @@ -139,10 +140,10 @@ func (c *Columnarizer) Next(context.Context) coldata.Batch { // Columnarizers are not expected to be Run, so we prohibit calling this method // on them. func (c *Columnarizer) Run(context.Context) { - vecerror.InternalError("Columnarizer should not be Run") + colexecerror.InternalError("Columnarizer should not be Run") } -var _ colbase.Operator = &Columnarizer{} +var _ colexecbase.Operator = &Columnarizer{} var _ execinfrapb.MetadataSource = &Columnarizer{} // DrainMeta is part of the MetadataSource interface. @@ -174,9 +175,9 @@ func (c *Columnarizer) Child(nth int, verbose bool) execinfra.OpNode { if n, ok := c.input.(execinfra.OpNode); ok { return n } - vecerror.InternalError("input to Columnarizer is not an execinfra.OpNode") + colexecerror.InternalError("input to Columnarizer is not an execinfra.OpNode") } - vecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/const_test.go b/pkg/sql/colexec/const_test.go index e5c9d5cf2c96..aac5d231c223 100644 --- a/pkg/sql/colexec/const_test.go +++ b/pkg/sql/colexec/const_test.go @@ -15,7 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -49,7 +49,7 @@ func TestConst(t *testing.T) { } for _, tc := range tcs { runTestsWithTyps(t, []tuples{tc.tuples}, [][]types.T{{*types.Int}}, tc.expected, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return createTestProjectingOperator( ctx, flowCtx, input[0], []types.T{*types.Int}, "9" /* projectingExpr */, false, /* canFallbackToRowexec */ @@ -85,7 +85,7 @@ func TestConstNull(t *testing.T) { } for _, tc := range tcs { runTestsWithTyps(t, []tuples{tc.tuples}, [][]types.T{{*types.Int}}, tc.expected, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return createTestProjectingOperator( ctx, flowCtx, input[0], []types.T{*types.Int}, "NULL::INT" /* projectingExpr */, false, /* canFallbackToRowexec */ diff --git a/pkg/sql/colexec/const_tmpl.go b/pkg/sql/colexec/const_tmpl.go index a443fcef25f8..fe688f7da376 100644 --- a/pkg/sql/colexec/const_tmpl.go +++ b/pkg/sql/colexec/const_tmpl.go @@ -26,16 +26,18 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/pkg/errors" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Declarations to make the template compile properly. @@ -63,12 +65,12 @@ type _GOTYPE interface{} // NewConstOp creates a new operator that produces a constant value constVal of // type t at index outputIdx. func NewConstOp( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, t *types.T, constVal interface{}, outputIdx int, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { input = newVectorTypeEnforcer(allocator, input, t, outputIdx) switch typeconv.FromColumnType(t) { // {{range .}} @@ -90,7 +92,7 @@ func NewConstOp( type const_TYPEOp struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator outputIdx int constVal _GOTYPE } @@ -130,8 +132,8 @@ 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 *colbase.Allocator, input colbase.Operator, outputIdx int, typ *types.T, -) colbase.Operator { + allocator *colmem.Allocator, input colexecbase.Operator, outputIdx int, typ *types.T, +) colexecbase.Operator { input = newVectorTypeEnforcer(allocator, input, typ, outputIdx) return &constNullOp{ OneInputNode: NewOneInputNode(input), @@ -142,11 +144,11 @@ func NewConstNullOp( type constNullOp struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator outputIdx int } -var _ colbase.Operator = &constNullOp{} +var _ colexecbase.Operator = &constNullOp{} func (c constNullOp) Init() { c.input.Init() diff --git a/pkg/sql/colexec/count.go b/pkg/sql/colexec/count.go index 673591b17ef8..d6ecdae1b7cf 100644 --- a/pkg/sql/colexec/count.go +++ b/pkg/sql/colexec/count.go @@ -14,7 +14,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -30,10 +31,10 @@ type countOp struct { count int64 } -var _ colbase.Operator = &countOp{} +var _ colexecbase.Operator = &countOp{} // NewCountOp returns a new count operator that counts the rows in its input. -func NewCountOp(allocator *colbase.Allocator, input colbase.Operator) colbase.Operator { +func NewCountOp(allocator *colmem.Allocator, input colexecbase.Operator) colexecbase.Operator { c := &countOp{ OneInputNode: NewOneInputNode(input), } diff --git a/pkg/sql/colexec/count_test.go b/pkg/sql/colexec/count_test.go index 604b4b9c3812..fd242ecd6b91 100644 --- a/pkg/sql/colexec/count_test.go +++ b/pkg/sql/colexec/count_test.go @@ -13,7 +13,7 @@ package colexec import ( "testing" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) @@ -35,7 +35,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. - runTestsWithoutAllNullsInjection(t, []tuples{tc.tuples}, nil /* typs */, tc.expected, orderedVerifier, func(input []colbase.Operator) (colbase.Operator, error) { + runTestsWithoutAllNullsInjection(t, []tuples{tc.tuples}, nil /* typs */, tc.expected, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { return NewCountOp(testAllocator, input[0]), nil }) } diff --git a/pkg/sql/colexec/deselector.go b/pkg/sql/colexec/deselector.go index 46f3ee33d5ef..100fd65f6bca 100644 --- a/pkg/sql/colexec/deselector.go +++ b/pkg/sql/colexec/deselector.go @@ -14,8 +14,9 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -26,19 +27,19 @@ import ( type deselectorOp struct { OneInputNode NonExplainable - allocator *colbase.Allocator + allocator *colmem.Allocator inputTypes []types.T output coldata.Batch } -var _ colbase.Operator = &deselectorOp{} +var _ colexecbase.Operator = &deselectorOp{} // NewDeselectorOp creates a new deselector operator on the given input // operator with the given column coltypes. func NewDeselectorOp( - allocator *colbase.Allocator, input colbase.Operator, typs []types.T, -) colbase.Operator { + allocator *colmem.Allocator, input colexecbase.Operator, typs []types.T, +) colexecbase.Operator { return &deselectorOp{ OneInputNode: NewOneInputNode(input), allocator: allocator, diff --git a/pkg/sql/colexec/deselector_test.go b/pkg/sql/colexec/deselector_test.go index 8b71213aa99e..b2d481e7dccd 100644 --- a/pkg/sql/colexec/deselector_test.go +++ b/pkg/sql/colexec/deselector_test.go @@ -16,7 +16,8 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -63,7 +64,7 @@ func TestDeselector(t *testing.T) { } for _, tc := range tcs { - runTestsWithFixedSel(t, []tuples{tc.tuples}, tc.sel, func(t *testing.T, input []colbase.Operator) { + runTestsWithFixedSel(t, []tuples{tc.tuples}, tc.sel, func(t *testing.T, input []colexecbase.Operator) { op := NewDeselectorOp(testAllocator, input[0], tc.typs) out := newOpTestOutput(op, tc.expected) @@ -94,7 +95,7 @@ func BenchmarkDeselector(b *testing.B) { } } for _, probOfOmitting := range []float64{0.1, 0.9} { - sel := colbase.RandomSel(rng, coldata.BatchSize(), probOfOmitting) + sel := coldatatestutils.RandomSel(rng, coldata.BatchSize(), probOfOmitting) batchLen := len(sel) for _, nBatches := range []int{1 << 1, 1 << 2, 1 << 4, 1 << 8} { @@ -104,7 +105,7 @@ func BenchmarkDeselector(b *testing.B) { batch.SetSelection(true) copy(batch.Selection(), sel) batch.SetLength(batchLen) - input := colbase.NewRepeatableBatchSource(testAllocator, batch, inputTypes) + input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, inputTypes) op := NewDeselectorOp(testAllocator, input, inputTypes) op.Init() b.ResetTimer() diff --git a/pkg/sql/colexec/disk_spiller.go b/pkg/sql/colexec/disk_spiller.go index 443a58369294..e0751cc94b49 100644 --- a/pkg/sql/colexec/disk_spiller.go +++ b/pkg/sql/colexec/disk_spiller.go @@ -16,8 +16,8 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" ) @@ -26,7 +26,7 @@ import ( // in memory and knows how to export them once the memory limit has been // reached. type bufferingInMemoryOperator interface { - colbase.Operator + colexecbase.Operator // ExportBuffered returns all the batches that have been buffered up from the // input and have not yet been processed by the operator. It needs to be @@ -36,7 +36,7 @@ type bufferingInMemoryOperator interface { // // Calling ExportBuffered may invalidate the contents of the last batch // returned by ExportBuffered. - ExportBuffered(input colbase.Operator) coldata.Batch + ExportBuffered(input colexecbase.Operator) coldata.Batch } // oneInputDiskSpiller is an Operator that manages the fallback from a one @@ -89,15 +89,15 @@ type bufferingInMemoryOperator interface { // - 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 colbase.Operator, + input colexecbase.Operator, inMemoryOp bufferingInMemoryOperator, inMemoryMemMonitorName string, - diskBackedOpConstructor func(input colbase.Operator) colbase.Operator, + diskBackedOpConstructor func(input colexecbase.Operator) colexecbase.Operator, spillingCallbackFn func(), -) colbase.Operator { +) colexecbase.Operator { diskBackedOpInput := newBufferExportingOperator(inMemoryOp, input) return &diskSpillerBase{ - inputs: []colbase.Operator{input}, + inputs: []colexecbase.Operator{input}, inMemoryOp: inMemoryOp, inMemoryMemMonitorName: inMemoryMemMonitorName, diskBackedOp: diskBackedOpConstructor(diskBackedOpInput), @@ -156,16 +156,16 @@ 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 colbase.Operator, + inputOne, inputTwo colexecbase.Operator, inMemoryOp bufferingInMemoryOperator, inMemoryMemMonitorName string, - diskBackedOpConstructor func(inputOne, inputTwo colbase.Operator) colbase.Operator, + diskBackedOpConstructor func(inputOne, inputTwo colexecbase.Operator) colexecbase.Operator, spillingCallbackFn func(), -) colbase.Operator { +) colexecbase.Operator { diskBackedOpInputOne := newBufferExportingOperator(inMemoryOp, inputOne) diskBackedOpInputTwo := newBufferExportingOperator(inMemoryOp, inputTwo) return &diskSpillerBase{ - inputs: []colbase.Operator{inputOne, inputTwo}, + inputs: []colexecbase.Operator{inputOne, inputTwo}, inMemoryOp: inMemoryOp, inMemoryOpInitStatus: OperatorNotInitialized, inMemoryMemMonitorName: inMemoryMemMonitorName, @@ -182,13 +182,13 @@ type diskSpillerBase struct { closerHelper - inputs []colbase.Operator + inputs []colexecbase.Operator spilled bool inMemoryOp bufferingInMemoryOperator inMemoryOpInitStatus OperatorInitStatus inMemoryMemMonitorName string - diskBackedOp colbase.Operator + diskBackedOp colexecbase.Operator distBackedOpInitStatus OperatorInitStatus spillingCallbackFn func() } @@ -213,7 +213,7 @@ func (d *diskSpillerBase) Next(ctx context.Context) coldata.Batch { return d.diskBackedOp.Next(ctx) } var batch coldata.Batch - if err := vecerror.CatchVectorizedRuntimeError( + if err := colexecerror.CatchVectorizedRuntimeError( func() { batch = d.inMemoryOp.Next(ctx) }, @@ -230,7 +230,7 @@ func (d *diskSpillerBase) Next(ctx context.Context) coldata.Batch { } // Either not an out of memory error or an OOM error coming from a // different operator, so we propagate it further. - vecerror.InternalError(err) + colexecerror.InternalError(err) } return batch } @@ -295,7 +295,7 @@ func (d *diskSpillerBase) Child(nth int, verbose bool) execinfra.OpNode { case 0: return d.inMemoryOp default: - vecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -309,19 +309,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 { - colbase.ZeroInputNode + colexecbase.ZeroInputNode NonExplainable firstSource bufferingInMemoryOperator - secondSource colbase.Operator + secondSource colexecbase.Operator firstSourceDone bool } var _ resettableOperator = &bufferExportingOperator{} func newBufferExportingOperator( - firstSource bufferingInMemoryOperator, secondSource colbase.Operator, -) colbase.Operator { + firstSource bufferingInMemoryOperator, secondSource colexecbase.Operator, +) colexecbase.Operator { return &bufferExportingOperator{ firstSource: firstSource, secondSource: secondSource, diff --git a/pkg/sql/colexec/distinct_test.go b/pkg/sql/colexec/distinct_test.go index a24647e5d3fa..6f0fc6075e27 100644 --- a/pkg/sql/colexec/distinct_test.go +++ b/pkg/sql/colexec/distinct_test.go @@ -17,7 +17,8 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -162,7 +163,7 @@ func TestDistinct(t *testing.T) { for _, numOfBuckets := range []uint64{1, 3, 5, hashTableNumBuckets} { t.Run(fmt.Sprintf("unordered/numOfBuckets=%d", numOfBuckets), func(t *testing.T) { runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return NewUnorderedDistinct( testAllocator, input[0], tc.distinctCols, tc.typs, numOfBuckets), nil @@ -177,7 +178,7 @@ func TestDistinct(t *testing.T) { orderedCols[i] = tc.distinctCols[j] } runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return newPartiallyOrderedDistinct( testAllocator, input[0], tc.distinctCols, orderedCols, tc.typs, @@ -187,7 +188,7 @@ func TestDistinct(t *testing.T) { } t.Run("ordered", func(t *testing.T) { runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return NewOrderedDistinct(input[0], tc.distinctCols, tc.typs) }) }) @@ -199,14 +200,14 @@ func BenchmarkDistinct(b *testing.B) { rng, _ := randutil.NewPseudoRand() ctx := context.Background() - distinctConstructors := []func(*colbase.Allocator, colbase.Operator, []uint32, int, []types.T) (colbase.Operator, error){ - func(allocator *colbase.Allocator, input colbase.Operator, distinctCols []uint32, numOrderedCols int, typs []types.T) (colbase.Operator, error) { + 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) { return NewUnorderedDistinct(allocator, input, distinctCols, typs, hashTableNumBuckets), nil }, - func(allocator *colbase.Allocator, input colbase.Operator, distinctCols []uint32, numOrderedCols int, typs []types.T) (colbase.Operator, error) { + func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []types.T) (colexecbase.Operator, error) { return newPartiallyOrderedDistinct(allocator, input, distinctCols, distinctCols[:numOrderedCols], typs) }, - func(allocator *colbase.Allocator, input colbase.Operator, distinctCols []uint32, numOrderedCols int, typs []types.T) (colbase.Operator, error) { + func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []types.T) (colexecbase.Operator, error) { return NewOrderedDistinct(input, distinctCols, typs) }, } diff --git a/pkg/sql/colexec/distinct_tmpl.go b/pkg/sql/colexec/distinct_tmpl.go index d0bb61460a26..ec296f4fcf56 100644 --- a/pkg/sql/colexec/distinct_tmpl.go +++ b/pkg/sql/colexec/distinct_tmpl.go @@ -28,12 +28,10 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" @@ -44,8 +42,8 @@ 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 colbase.Operator, distinctCols []uint32, typs []types.T, -) (colbase.Operator, []bool, error) { + input colexecbase.Operator, distinctCols []uint32, typs []types.T, +) (colexecbase.Operator, []bool, error) { distinctCol := make([]bool, coldata.BatchSize()) // zero the boolean column on every iteration. input = fnOp{ @@ -64,7 +62,7 @@ func OrderedDistinctColsToOperators( } } if r, ok = input.(resettableOperator); !ok { - vecerror.InternalError("unexpectedly an ordered distinct is not a resetter") + colexecerror.InternalError("unexpectedly an ordered distinct is not a resetter") } distinctChain := &distinctChainOps{ resettableOperator: r, @@ -81,8 +79,8 @@ var _ resettableOperator = &distinctChainOps{} // NewOrderedDistinct creates a new ordered distinct operator on the given // input columns with the given coltypes. func NewOrderedDistinct( - input colbase.Operator, distinctCols []uint32, typs []types.T, -) (colbase.Operator, error) { + input colexecbase.Operator, distinctCols []uint32, typs []types.T, +) (colexecbase.Operator, error) { op, outputCol, err := OrderedDistinctColsToOperators(input, distinctCols, typs) if err != nil { return nil, err @@ -93,6 +91,9 @@ func NewOrderedDistinct( }, nil } +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Declarations to make the template compile properly. @@ -132,14 +133,14 @@ const _TYPES_T = coltypes.Unhandled // _ASSIGN_NE is the template equality function for assigning the first input // to the result of the second input != the third input. func _ASSIGN_NE(_ bool, _, _ _GOTYPE) bool { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} func newSingleOrderedDistinct( - input colbase.Operator, distinctColIdx int, outputCol []bool, t *types.T, -) (colbase.Operator, error) { + input colexecbase.Operator, distinctColIdx int, outputCol []bool, t *types.T, +) (colexecbase.Operator, error) { switch typeconv.FromColumnType(t) { // {{range .}} case _TYPES_T: diff --git a/pkg/sql/colexec/execgen/cmd/execgen/avg_agg_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/avg_agg_gen.go index 731fad03a59b..94437d3e8641 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/avg_agg_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/avg_agg_gen.go @@ -18,7 +18,7 @@ import ( "text/template" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" ) @@ -37,14 +37,14 @@ func (a avgAggTmplInfo) AssignDivInt64(target, l, r string) string { return fmt.Sprintf( `%s.SetInt64(%s) if _, err := tree.DecimalCtx.Quo(&%s, &%s, &%s); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) }`, target, r, target, l, target, ) case coltypes.Float64: return fmt.Sprintf("%s = %s / float64(%s)", target, l, r) default: - vecerror.InternalError("unsupported avg agg type") + colexecerror.InternalError("unsupported avg agg type") // This code is unreachable, but the compiler cannot infer that. return "" } diff --git a/pkg/sql/colexec/execgen/cmd/execgen/bool_and_or_agg_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/bool_and_or_agg_gen.go index b7ac04cce249..1d554f2e3339 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/bool_and_or_agg_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/bool_and_or_agg_gen.go @@ -17,7 +17,7 @@ import ( "strings" "text/template" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" ) type booleanAggTmplInfo struct { @@ -31,7 +31,7 @@ func (b booleanAggTmplInfo) AssignBoolOp(target, l, r string) string { case false: return fmt.Sprintf("%s = %s || %s", target, l, r) default: - vecerror.InternalError("unsupported boolean agg type") + colexecerror.InternalError("unsupported boolean agg type") // This code is unreachable, but the compiler cannot infer that. return "" } diff --git a/pkg/sql/colexec/execgen/cmd/execgen/main.go b/pkg/sql/colexec/execgen/cmd/execgen/main.go index e565d76c03d4..833f6a77ffdc 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/main.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/main.go @@ -20,7 +20,7 @@ import ( "path/filepath" "regexp" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/pkg/errors" ) @@ -58,7 +58,7 @@ type entry struct { func registerGenerator(g generator, filename, dep string) { if _, ok := generators[filename]; ok { - vecerror.InternalError(fmt.Sprintf("%s generator already registered", filename)) + colexecerror.InternalError(fmt.Sprintf("%s generator already registered", filename)) } generators[filename] = entry{fn: g, dep: dep} } diff --git a/pkg/sql/colexec/execgen/cmd/execgen/overloads.go b/pkg/sql/colexec/execgen/cmd/execgen/overloads.go index 2abb3f8ce4a0..512206ef167e 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/overloads.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/overloads.go @@ -18,7 +18,7 @@ import ( "text/template" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" ) @@ -213,7 +213,7 @@ func floatToInt(intSize int, floatSize int) func(string, string) string { return func(to, from string) string { convStr := ` if math.IsNaN(float64(%[2]s)) || %[2]s <= float%[4]d(math.MinInt%[3]d) || %[2]s >= float%[4]d(math.MaxInt%[3]d) { - vecerror.ExpectedError(tree.ErrIntOutOfRange) + colexecerror.ExpectedError(tree.ErrIntOutOfRange) } %[1]s = int%[3]d(%[2]s) ` @@ -234,7 +234,7 @@ func floatToDecimal(to, from string) string { var tmpDec apd.Decimal _, tmpErr := tmpDec.SetFloat64(float64(%[2]s)) if tmpErr != nil { - vecerror.ExpectedError(tmpErr) + colexecerror.ExpectedError(tmpErr) } %[1]s = tmpDec } @@ -322,7 +322,7 @@ func init() { } `)) if err := t.Execute(&buf, args); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return buf.String() } @@ -615,7 +615,7 @@ func (boolCustomizer) getCmpOpCompareFunc() compareFunc { `)) if err := t.Execute(&buf, args); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return buf.String() } @@ -667,15 +667,15 @@ func (decimalCustomizer) getBinOpAssignFunc() assignFunc { { cond, err := tree.%s.%s(&%s, &%s, &%s) if cond.DivisionByZero() { - vecerror.ExpectedError(tree.ErrDivByZero) + colexecerror.ExpectedError(tree.ErrDivByZero) } if err != nil { - vecerror.ExpectedError(err) + colexecerror.ExpectedError(err) } } `, binaryOpDecCtx[op.BinOp], binaryOpDecMethod[op.BinOp], target, l, r) } - return fmt.Sprintf("if _, err := tree.%s.%s(&%s, &%s, &%s); err != nil { vecerror.ExpectedError(err) }", + return fmt.Sprintf("if _, err := tree.%s.%s(&%s, &%s, &%s); err != nil { colexecerror.ExpectedError(err) }", binaryOpDecCtx[op.BinOp], binaryOpDecMethod[op.BinOp], target, l, r) } } @@ -743,7 +743,7 @@ func getFloatCmpOpCompareFunc(checkLeftNan, checkRightNan bool) compareFunc { `)) if err := t.Execute(&buf, args); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return buf.String() } @@ -795,7 +795,7 @@ func (c intCustomizer) getCmpOpCompareFunc() compareFunc { `)) if err := t.Execute(&buf, args); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return buf.String() } @@ -821,7 +821,7 @@ func (c intCustomizer) getBinOpAssignFunc() assignFunc { { result := {{.Left}} + {{.Right}} if (result < {{.Left}}) != ({{.Right}} < 0) { - vecerror.ExpectedError(tree.ErrIntOutOfRange) + colexecerror.ExpectedError(tree.ErrIntOutOfRange) } {{.Target}} = result } @@ -832,7 +832,7 @@ func (c intCustomizer) getBinOpAssignFunc() assignFunc { { result := {{.Left}} - {{.Right}} if (result < {{.Left}}) != ({{.Right}} > 0) { - vecerror.ExpectedError(tree.ErrIntOutOfRange) + colexecerror.ExpectedError(tree.ErrIntOutOfRange) } {{.Target}} = result } @@ -857,7 +857,7 @@ func (c intCustomizer) getBinOpAssignFunc() assignFunc { upperBound = "math.MaxInt32" lowerBound = "math.MinInt32" default: - vecerror.InternalError(fmt.Sprintf("unhandled integer width %d", c.width)) + colexecerror.InternalError(fmt.Sprintf("unhandled integer width %d", c.width)) } args["UpperBound"] = upperBound @@ -869,9 +869,9 @@ func (c intCustomizer) getBinOpAssignFunc() assignFunc { if {{.Left}} != 0 && {{.Right}} != 0 { sameSign := ({{.Left}} < 0) == ({{.Right}} < 0) if (result < 0) == sameSign { - vecerror.ExpectedError(tree.ErrIntOutOfRange) + colexecerror.ExpectedError(tree.ErrIntOutOfRange) } else if result/{{.Right}} != {{.Left}} { - vecerror.ExpectedError(tree.ErrIntOutOfRange) + colexecerror.ExpectedError(tree.ErrIntOutOfRange) } } } @@ -886,23 +886,23 @@ func (c intCustomizer) getBinOpAssignFunc() assignFunc { t = template.Must(template.New("").Parse(` { if {{.Right}} == 0 { - vecerror.ExpectedError(tree.ErrDivByZero) + colexecerror.ExpectedError(tree.ErrDivByZero) } leftTmpDec, rightTmpDec := &decimalScratch.tmpDec1, &decimalScratch.tmpDec2 leftTmpDec.SetFinite(int64({{.Left}}), 0) rightTmpDec.SetFinite(int64({{.Right}}), 0) if _, err := tree.{{.Ctx}}.Quo(&{{.Target}}, leftTmpDec, rightTmpDec); err != nil { - vecerror.ExpectedError(err) + colexecerror.ExpectedError(err) } } `)) default: - vecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) + colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) } if err := t.Execute(&buf, args); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return buf.String() } @@ -916,13 +916,13 @@ func (c decimalFloatCustomizer) getCmpOpCompareFunc() compareFunc { { tmpDec := &decimalScratch.tmpDec1 if _, err := tmpDec.SetFloat64(float64({{.Right}})); err != nil { - vecerror.ExpectedError(err) + colexecerror.ExpectedError(err) } {{.Target}} = tree.CompareDecimals(&{{.Left}}, tmpDec) } `)) if err := t.Execute(&buf, args); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return buf.String() } @@ -940,7 +940,7 @@ func (c decimalIntCustomizer) getCmpOpCompareFunc() compareFunc { } `)) if err := t.Execute(&buf, args); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return buf.String() } @@ -960,18 +960,18 @@ func (c decimalIntCustomizer) getBinOpAssignFunc() assignFunc { { {{ if .IsDivision }} if {{.Right}} == 0 { - vecerror.ExpectedError(tree.ErrDivByZero) + colexecerror.ExpectedError(tree.ErrDivByZero) } {{ end }} tmpDec := &decimalScratch.tmpDec1 tmpDec.SetFinite(int64({{.Right}}), 0) if _, err := tree.{{.Ctx}}.{{.Op}}(&{{.Target}}, &{{.Left}}, tmpDec); err != nil { - vecerror.ExpectedError(err) + colexecerror.ExpectedError(err) } } `)) if err := t.Execute(&buf, args); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return buf.String() } @@ -985,13 +985,13 @@ func (c floatDecimalCustomizer) getCmpOpCompareFunc() compareFunc { { tmpDec := &decimalScratch.tmpDec1 if _, err := tmpDec.SetFloat64(float64({{.Left}})); err != nil { - vecerror.ExpectedError(err) + colexecerror.ExpectedError(err) } {{.Target}} = tree.CompareDecimals(tmpDec, &{{.Right}}) } `)) if err := t.Execute(&buf, args); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return buf.String() } @@ -1010,7 +1010,7 @@ func (c intDecimalCustomizer) getCmpOpCompareFunc() compareFunc { `)) if err := t.Execute(&buf, args); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return buf.String() } @@ -1033,18 +1033,18 @@ func (c intDecimalCustomizer) getBinOpAssignFunc() assignFunc { {{ if .IsDivision }} cond, err := tree.{{.Ctx}}.{{.Op}}(&{{.Target}}, tmpDec, &{{.Right}}) if cond.DivisionByZero() { - vecerror.ExpectedError(tree.ErrDivByZero) + colexecerror.ExpectedError(tree.ErrDivByZero) } {{ else }} _, err := tree.{{.Ctx}}.{{.Op}}(&{{.Target}}, tmpDec, &{{.Right}}) {{ end }} if err != nil { - vecerror.ExpectedError(err) + colexecerror.ExpectedError(err) } } `)) if err := t.Execute(&buf, args); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return buf.String() } @@ -1077,7 +1077,7 @@ func (c timestampCustomizer) getCmpOpCompareFunc() compareFunc { }`)) if err := t.Execute(&buf, args); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return buf.String() } @@ -1102,7 +1102,7 @@ func (c timestampCustomizer) getBinOpAssignFunc() assignFunc { `, target, l, r) default: - vecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) + colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) } // This code is unreachable, but the compiler cannot infer that. return "" @@ -1136,7 +1136,7 @@ func (c intervalCustomizer) getBinOpAssignFunc() assignFunc { return fmt.Sprintf(`%[1]s = %[2]s.Sub(%[3]s)`, target, l, r) default: - vecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) + colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) } // This code is unreachable, but the compiler cannot infer that. return "" @@ -1153,7 +1153,7 @@ func (c timestampIntervalCustomizer) getBinOpAssignFunc() assignFunc { return fmt.Sprintf(`%[1]s = duration.Add(%[2]s, %[3]s.Mul(-1))`, target, l, r) default: - vecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) + colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) } // This code is unreachable, but the compiler cannot infer that. return "" @@ -1167,7 +1167,7 @@ func (c intervalTimestampCustomizer) getBinOpAssignFunc() assignFunc { return fmt.Sprintf(`%[1]s = duration.Add(%[3]s, %[2]s)`, target, l, r) default: - vecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) + colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) } return "" } @@ -1182,12 +1182,12 @@ func (c intervalIntCustomizer) getBinOpAssignFunc() assignFunc { case tree.Div: return fmt.Sprintf(` if %[3]s == 0 { - vecerror.ExpectedError(tree.ErrDivByZero) + colexecerror.ExpectedError(tree.ErrDivByZero) } %[1]s = %[2]s.Div(int64(%[3]s))`, target, l, r) default: - vecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) + colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) } return "" } @@ -1200,7 +1200,7 @@ func (c intIntervalCustomizer) getBinOpAssignFunc() assignFunc { return fmt.Sprintf(`%[1]s = %[3]s.Mul(int64(%[2]s))`, target, l, r) default: - vecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) + colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) } return "" } @@ -1215,12 +1215,12 @@ func (c intervalFloatCustomizer) getBinOpAssignFunc() assignFunc { case tree.Div: return fmt.Sprintf(` if %[3]s == 0.0 { - vecerror.ExpectedError(tree.ErrDivByZero) + colexecerror.ExpectedError(tree.ErrDivByZero) } %[1]s = %[2]s.DivFloat(float64(%[3]s))`, target, l, r) default: - vecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) + colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) } return "" } @@ -1233,7 +1233,7 @@ func (c floatIntervalCustomizer) getBinOpAssignFunc() assignFunc { return fmt.Sprintf(`%[1]s = %[3]s.MulFloat(float64(%[2]s))`, target, l, r) default: - vecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) + colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) } return "" } @@ -1246,12 +1246,12 @@ func (c intervalDecimalCustomizer) getBinOpAssignFunc() assignFunc { return fmt.Sprintf(` f, err := %[3]s.Float64() if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } %[1]s = %[2]s.MulFloat(f)`, target, l, r) default: - vecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) + colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) } return "" } @@ -1264,13 +1264,13 @@ func (c decimalIntervalCustomizer) getBinOpAssignFunc() assignFunc { return fmt.Sprintf(` f, err := %[2]s.Float64() if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } %[1]s = %[3]s.MulFloat(f)`, target, l, r) default: - vecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) + colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.BinOp.String())) } return "" } diff --git a/pkg/sql/colexec/execgen/cmd/execgen/overloads_test_utils_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/overloads_test_utils_gen.go index 6077c77dc03b..71154fa5ca75 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/overloads_test_utils_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/overloads_test_utils_gen.go @@ -24,7 +24,7 @@ import ( "time" "github.com/cockroachdb/apd" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/duration" ) diff --git a/pkg/sql/colexec/execgen/cmd/execgen/rowstovec_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/rowstovec_gen.go index 4859aa814ac0..593b5c8e8dbd 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/rowstovec_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/rowstovec_gen.go @@ -18,7 +18,7 @@ import ( "text/template" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/types" ) diff --git a/pkg/sql/colexec/execgen/placeholders.go b/pkg/sql/colexec/execgen/placeholders.go index b7951d3a83dd..c62108baae2b 100644 --- a/pkg/sql/colexec/execgen/placeholders.go +++ b/pkg/sql/colexec/execgen/placeholders.go @@ -10,7 +10,7 @@ package execgen -import "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" +import "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" const nonTemplatePanic = "do not call from non-template code" @@ -32,7 +32,7 @@ var ( // UNSAFEGET is a template function. Use this if you are not keeping data around // (including passing it to SET). func UNSAFEGET(target, i interface{}) interface{} { - vecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) return nil } @@ -41,54 +41,54 @@ func UNSAFEGET(target, i interface{}) interface{} { // source is. You must use this on the result of UNSAFEGET if you wish to store // that result past the lifetime of the batch you UNSAFEGET'd from. func COPYVAL(dest, src interface{}) { - vecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) } // SET is a template function. func SET(target, i, new interface{}) { - vecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) } // SLICE is a template function. func SLICE(target, start, end interface{}) interface{} { - vecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) return nil } // COPYSLICE is a template function. func COPYSLICE(target, src, destIdx, srcStartIdx, srcEndIdx interface{}) { - vecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) } // APPENDSLICE is a template function. func APPENDSLICE(target, src, destIdx, srcStartIdx, srcEndIdx interface{}) { - vecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) } // APPENDVAL is a template function. func APPENDVAL(target, v interface{}) { - vecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) } // LEN is a template function. func LEN(target interface{}) interface{} { - vecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) return nil } // ZERO is a template function. func ZERO(target interface{}) { - vecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) } // RANGE is a template function. func RANGE(loopVariableIdent, target, start, end interface{}) bool { - vecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) return false } // WINDOW is a template function. func WINDOW(target, start, end interface{}) interface{} { - vecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) return nil } diff --git a/pkg/sql/colexec/execplan.go b/pkg/sql/colexec/execplan.go index bd71e5637b35..9e11538d987b 100644 --- a/pkg/sql/colexec/execplan.go +++ b/pkg/sql/colexec/execplan.go @@ -17,10 +17,11 @@ import ( "reflect" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -34,7 +35,7 @@ import ( "github.com/marusama/semaphore" ) -func checkNumIn(inputs []colbase.Operator, numIn int) error { +func checkNumIn(inputs []colexecbase.Operator, numIn int) error { if len(inputs) != numIn { return errors.Errorf("expected %d input(s), got %d", numIn, len(inputs)) } @@ -46,7 +47,7 @@ func checkNumIn(inputs []colbase.Operator, numIn int) error { func wrapRowSources( ctx context.Context, flowCtx *execinfra.FlowCtx, - inputs []colbase.Operator, + inputs []colexecbase.Operator, inputTypes [][]types.T, acc *mon.BoundAccount, newToWrap func([]execinfra.RowSource) (execinfra.RowSource, error), @@ -88,14 +89,14 @@ func wrapRowSources( return nil, err } - return NewColumnarizer(ctx, colbase.NewAllocator(ctx, acc), flowCtx, processorID, toWrap) + return NewColumnarizer(ctx, colmem.NewAllocator(ctx, acc), flowCtx, processorID, toWrap) } // NewColOperatorArgs is a helper struct that encompasses all of the input // arguments to NewColOperator call. type NewColOperatorArgs struct { Spec *execinfrapb.ProcessorSpec - Inputs []colbase.Operator + Inputs []colexecbase.Operator StreamingMemAccount *mon.BoundAccount ProcessorConstructor execinfra.ProcessorConstructor DiskQueueCfg colcontainer.DiskQueueCfg @@ -136,7 +137,7 @@ type NewColOperatorArgs struct { // NewColOperatorResult is a helper struct that encompasses all of the return // values of NewColOperator call. type NewColOperatorResult struct { - Op colbase.Operator + Op colexecbase.Operator ColumnTypes []types.T InternalMemUsage int MetadataSources []execinfrapb.MetadataSource @@ -323,7 +324,7 @@ func (r *NewColOperatorResult) createDiskBackedSort( ctx context.Context, flowCtx *execinfra.FlowCtx, args NewColOperatorArgs, - input colbase.Operator, + input colexecbase.Operator, inputTypes []types.T, ordering execinfrapb.Ordering, matchLen uint32, @@ -331,12 +332,12 @@ func (r *NewColOperatorResult) createDiskBackedSort( processorID int32, post *execinfrapb.PostProcessSpec, memMonitorNamePrefix string, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { streamingMemAccount := args.StreamingMemAccount useStreamingMemAccountForBuffering := args.TestingKnobs.UseStreamingMemAccountForBuffering var ( sorterMemMonitorName string - inMemorySorter colbase.Operator + inMemorySorter colexecbase.Operator err error ) if len(ordering.Columns) == int(matchLen) { @@ -356,7 +357,7 @@ func (r *NewColOperatorResult) createDiskBackedSort( ) } inMemorySorter, err = NewSortChunks( - colbase.NewAllocator(ctx, sortChunksMemAccount), input, inputTypes, + colmem.NewAllocator(ctx, sortChunksMemAccount), input, inputTypes, ordering.Columns, int(matchLen), ) } else if post.Limit != 0 && post.Filter.Empty() && post.Limit+post.Offset < math.MaxUint16 { @@ -374,7 +375,7 @@ func (r *NewColOperatorResult) createDiskBackedSort( } k := uint16(post.Limit + post.Offset) inMemorySorter = NewTopKSorter( - colbase.NewAllocator(ctx, topKSorterMemAccount), input, inputTypes, + colmem.NewAllocator(ctx, topKSorterMemAccount), input, inputTypes, ordering.Columns, k, ) } else { @@ -389,7 +390,7 @@ func (r *NewColOperatorResult) createDiskBackedSort( ) } inMemorySorter, err = NewSorter( - colbase.NewAllocator(ctx, sorterMemAccount), input, inputTypes, ordering.Columns, + colmem.NewAllocator(ctx, sorterMemAccount), input, inputTypes, ordering.Columns, ) } if err != nil { @@ -405,12 +406,12 @@ func (r *NewColOperatorResult) createDiskBackedSort( return newOneInputDiskSpiller( input, inMemorySorter.(bufferingInMemoryOperator), sorterMemMonitorName, - func(input colbase.Operator) colbase.Operator { + func(input colexecbase.Operator) colexecbase.Operator { monitorNamePrefix := fmt.Sprintf("%sexternal-sorter", memMonitorNamePrefix) // We are using an unlimited memory monitor here because external // sort itself is responsible for making sure that we stay within // the memory limit. - unlimitedAllocator := colbase.NewAllocator( + unlimitedAllocator := colmem.NewAllocator( ctx, r.createBufferingUnlimitedMemAccount( ctx, flowCtx, monitorNamePrefix, )) @@ -458,7 +459,7 @@ func (r *NewColOperatorResult) createDiskBackedSort( func (r *NewColOperatorResult) createAndWrapRowSource( ctx context.Context, flowCtx *execinfra.FlowCtx, - inputs []colbase.Operator, + inputs []colexecbase.Operator, inputTypes [][]types.T, streamingMemAccount *mon.BoundAccount, spec *execinfrapb.ProcessorSpec, @@ -532,7 +533,7 @@ func NewColOperator( result.OpMonitors = result.OpMonitors[:0] } if panicErr != nil { - vecerror.InternalError(panicErr) + colexecerror.InternalError(panicErr) } }() spec := args.Spec @@ -602,7 +603,7 @@ func NewColOperator( return result, err } var scanOp *colBatchScan - scanOp, err = newColBatchScan(colbase.NewAllocator(ctx, streamingMemAccount), flowCtx, core.TableReader, post) + scanOp, err = newColBatchScan(colmem.NewAllocator(ctx, streamingMemAccount), flowCtx, core.TableReader, post) if err != nil { return result, err } @@ -636,13 +637,13 @@ func NewColOperator( // TODO(solon): The distsql plan for this case includes a TableReader, so // we end up creating an orphaned colBatchScan. We should avoid that. // Ideally the optimizer would not plan a scan in this unusual case. - result.Op, result.IsStreaming, err = NewSingleTupleNoInputOp(colbase.NewAllocator(ctx, streamingMemAccount)), true, nil + result.Op, result.IsStreaming, err = NewSingleTupleNoInputOp(colmem.NewAllocator(ctx, streamingMemAccount)), true, nil // We make ColumnTypes non-nil so that sanity check doesn't panic. result.ColumnTypes = make([]types.T, 0) break } if aggSpec.IsRowCount() { - result.Op, result.IsStreaming, err = NewCountOp(colbase.NewAllocator(ctx, streamingMemAccount), inputs[0]), true, nil + result.Op, result.IsStreaming, err = NewCountOp(colmem.NewAllocator(ctx, streamingMemAccount), inputs[0]), true, nil result.ColumnTypes = []types.T{*types.Int} break } @@ -697,12 +698,12 @@ func NewColOperator( hashAggregatorMemAccount = result.createBufferingUnlimitedMemAccount(ctx, flowCtx, "hash-aggregator") } result.Op, err = NewHashAggregator( - colbase.NewAllocator(ctx, hashAggregatorMemAccount), inputs[0], typs, aggFns, + colmem.NewAllocator(ctx, hashAggregatorMemAccount), inputs[0], typs, aggFns, aggSpec.GroupCols, aggCols, ) } else { result.Op, err = NewOrderedAggregator( - colbase.NewAllocator(ctx, streamingMemAccount), inputs[0], typs, aggFns, + colmem.NewAllocator(ctx, streamingMemAccount), inputs[0], typs, aggFns, aggSpec.GroupCols, aggCols, aggSpec.IsScalar(), ) result.IsStreaming = true @@ -736,7 +737,7 @@ func NewColOperator( // columns and we think that the probability of distinct tuples in the // input is about 0.01 or less. result.Op = NewUnorderedDistinct( - colbase.NewAllocator(ctx, distinctMemAccount), inputs[0], + colmem.NewAllocator(ctx, distinctMemAccount), inputs[0], core.Distinct.DistinctColumns, result.ColumnTypes, hashTableNumBuckets, ) } @@ -747,7 +748,7 @@ func NewColOperator( } outputIdx := len(spec.Input[0].ColumnTypes) result.Op = NewOrdinalityOp( - colbase.NewAllocator(ctx, streamingMemAccount), inputs[0], outputIdx, + colmem.NewAllocator(ctx, streamingMemAccount), inputs[0], outputIdx, ) result.IsStreaming = true result.ColumnTypes = make([]types.T, outputIdx+1) @@ -793,7 +794,7 @@ func NewColOperator( return result, err } inMemoryHashJoiner := newHashJoiner( - colbase.NewAllocator(ctx, hashJoinerMemAccount), hjSpec, inputs[0], inputs[1], + colmem.NewAllocator(ctx, hashJoinerMemAccount), hjSpec, inputs[0], inputs[1], ) if args.TestingKnobs.DiskSpillingDisabled { // We will not be creating a disk-backed hash joiner because we're @@ -805,9 +806,9 @@ func NewColOperator( result.Op = newTwoInputDiskSpiller( inputs[0], inputs[1], inMemoryHashJoiner.(bufferingInMemoryOperator), hashJoinerMemMonitorName, - func(inputOne, inputTwo colbase.Operator) colbase.Operator { + func(inputOne, inputTwo colexecbase.Operator) colexecbase.Operator { monitorNamePrefix := "external-hash-joiner" - unlimitedAllocator := colbase.NewAllocator( + unlimitedAllocator := colmem.NewAllocator( ctx, result.createBufferingUnlimitedMemAccount( ctx, flowCtx, monitorNamePrefix, )) @@ -823,7 +824,7 @@ func NewColOperator( execinfra.GetWorkMemLimit(flowCtx.Cfg), diskQueueCfg, args.FDSemaphore, - func(input colbase.Operator, inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column, maxNumberPartitions int) (colbase.Operator, error) { + func(input colexecbase.Operator, inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column, maxNumberPartitions int) (colexecbase.Operator, error) { sortArgs := args if !args.TestingKnobs.DelegateFDAcquisitions { // Set the FDSemaphore to nil. This indicates that no FDs @@ -888,7 +889,7 @@ func NewColOperator( // We are using an unlimited memory monitor here because merge joiner // itself is responsible for making sure that we stay within the memory // limit, and it will fall back to disk if necessary. - unlimitedAllocator := colbase.NewAllocator( + unlimitedAllocator := colmem.NewAllocator( ctx, result.createBufferingUnlimitedMemAccount( ctx, flowCtx, monitorName, )) @@ -952,9 +953,9 @@ func NewColOperator( // which kind of partitioner to use should come from the optimizer. partitionColIdx = int(wf.OutputColIdx) input, err = NewWindowSortingPartitioner( - colbase.NewAllocator(ctx, streamingMemAccount), input, typs, + colmem.NewAllocator(ctx, streamingMemAccount), input, typs, core.Windower.PartitionBy, wf.Ordering.Columns, int(wf.OutputColIdx), - func(input colbase.Operator, inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column) (colbase.Operator, error) { + func(input colexecbase.Operator, inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column) (colexecbase.Operator, error) { return result.createDiskBackedSort( ctx, flowCtx, args, input, inputTypes, execinfrapb.Ordering{Columns: orderingCols}, 0, /* matchLen */ @@ -980,7 +981,7 @@ func NewColOperator( if windowFnNeedsPeersInfo(*wf.Func.WindowFunc) { peersColIdx = int(wf.OutputColIdx + tempColOffset) input, err = NewWindowPeerGrouper( - colbase.NewAllocator(ctx, streamingMemAccount), + colmem.NewAllocator(ctx, streamingMemAccount), input, typs, wf.Ordering.Columns, partitionColIdx, peersColIdx, ) @@ -993,11 +994,11 @@ func NewColOperator( switch windowFn { case execinfrapb.WindowerSpec_ROW_NUMBER: result.Op = NewRowNumberOperator( - colbase.NewAllocator(ctx, streamingMemAccount), input, outputIdx, partitionColIdx, + colmem.NewAllocator(ctx, streamingMemAccount), input, outputIdx, partitionColIdx, ) case execinfrapb.WindowerSpec_RANK, execinfrapb.WindowerSpec_DENSE_RANK: result.Op, err = NewRankOperator( - colbase.NewAllocator(ctx, streamingMemAccount), input, windowFn, + colmem.NewAllocator(ctx, streamingMemAccount), input, windowFn, wf.Ordering.Columns, outputIdx, partitionColIdx, peersColIdx, ) case execinfrapb.WindowerSpec_PERCENT_RANK, execinfrapb.WindowerSpec_CUME_DIST: @@ -1006,7 +1007,7 @@ func NewColOperator( // making sure that we stay within the memory limit, and // they will fall back to disk if necessary. memAccName := memMonitorsPrefix + "relative-rank" - unlimitedAllocator := colbase.NewAllocator( + unlimitedAllocator := colmem.NewAllocator( ctx, result.createBufferingUnlimitedMemAccount(ctx, flowCtx, memAccName), ) diskAcc := result.createDiskAccount(ctx, flowCtx, memAccName) @@ -1159,7 +1160,7 @@ func (r *NewColOperatorResult) wrapPostProcessSpec( Post: *post, } return r.createAndWrapRowSource( - ctx, flowCtx, []colbase.Operator{r.Op}, [][]types.T{r.ColumnTypes}, streamingMemAccount, noopSpec, processorConstructor, + ctx, flowCtx, []colexecbase.Operator{r.Op}, [][]types.T{r.ColumnTypes}, streamingMemAccount, noopSpec, processorConstructor, ) } @@ -1306,7 +1307,7 @@ func (r *NewColOperatorResult) createDiskAccount( } type postProcessResult struct { - Op colbase.Operator + Op colexecbase.Operator ColumnTypes []types.T InternalMemUsage int } @@ -1374,9 +1375,9 @@ func planSelectionOperators( evalCtx *tree.EvalContext, expr tree.TypedExpr, columnTypes []types.T, - input colbase.Operator, + input colexecbase.Operator, acc *mon.BoundAccount, -) (op colbase.Operator, resultIdx int, typs []types.T, internalMemUsed int, err error) { +) (op colexecbase.Operator, resultIdx int, typs []types.T, internalMemUsed int, err error) { switch t := expr.(type) { case *tree.IndexedVar: return NewBoolVecToSelOp(input, t.Idx), -1, columnTypes, internalMemUsed, nil @@ -1385,7 +1386,7 @@ func planSelectionOperators( // 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 colbase.Operator + var leftOp, rightOp colexecbase.Operator var internalMemUsedLeft, internalMemUsedRight int leftOp, _, typs, internalMemUsedLeft, err = planSelectionOperators( ctx, evalCtx, t.TypedLeft(), columnTypes, input, acc, @@ -1490,12 +1491,12 @@ func planTypedMaybeNullProjectionOperators( expr tree.TypedExpr, exprTyp *types.T, columnTypes []types.T, - input colbase.Operator, + input colexecbase.Operator, acc *mon.BoundAccount, -) (op colbase.Operator, resultIdx int, typs []types.T, internalMemUsed int, err error) { +) (op colexecbase.Operator, resultIdx int, typs []types.T, internalMemUsed int, err error) { if expr == tree.DNull { resultIdx = len(columnTypes) - op = NewConstNullOp(colbase.NewAllocator(ctx, acc), input, resultIdx, exprTyp) + op = NewConstNullOp(colmem.NewAllocator(ctx, acc), input, resultIdx, exprTyp) typs = append(columnTypes, *exprTyp) return op, resultIdx, typs, internalMemUsed, nil } @@ -1523,16 +1524,16 @@ func planCastOperator( ctx context.Context, acc *mon.BoundAccount, columnTypes []types.T, - input colbase.Operator, + input colexecbase.Operator, inputIdx int, fromType *types.T, toType *types.T, -) (op colbase.Operator, resultIdx int, typs []types.T, err error) { +) (op colexecbase.Operator, resultIdx int, typs []types.T, err error) { if err := checkCastSupported(fromType, toType); err != nil { return op, resultIdx, typs, err } outputIdx := len(columnTypes) - op, err = GetCastOperator(colbase.NewAllocator(ctx, acc), input, inputIdx, outputIdx, fromType, toType) + op, err = GetCastOperator(colmem.NewAllocator(ctx, acc), input, inputIdx, outputIdx, fromType, toType) typs = append(columnTypes, *toType) return op, outputIdx, typs, err } @@ -1546,9 +1547,9 @@ func planProjectionOperators( evalCtx *tree.EvalContext, expr tree.TypedExpr, columnTypes []types.T, - input colbase.Operator, + input colexecbase.Operator, acc *mon.BoundAccount, -) (op colbase.Operator, resultIdx int, typs []types.T, internalMemUsed int, err error) { +) (op colexecbase.Operator, resultIdx int, typs []types.T, internalMemUsed int, err error) { resultIdx = -1 switch t := expr.(type) { case *tree.IndexedVar: @@ -1598,7 +1599,7 @@ func planProjectionOperators( resultIdx = len(typs) typs = append(typs, *funcOutputType) op, err = NewBuiltinFunctionOperator( - colbase.NewAllocator(ctx, acc), evalCtx, t, typs, inputCols, resultIdx, op, + colmem.NewAllocator(ctx, acc), evalCtx, t, typs, inputCols, resultIdx, op, ) return op, resultIdx, typs, internalMemUsed, err case tree.Datum: @@ -1613,7 +1614,7 @@ func planProjectionOperators( if err != nil { return nil, resultIdx, typs, internalMemUsed, err } - op, err := NewConstOp(colbase.NewAllocator(ctx, acc), input, datumType, constVal, resultIdx) + op, err := NewConstOp(colmem.NewAllocator(ctx, acc), input, datumType, constVal, resultIdx) if err != nil { return nil, resultIdx, typs, internalMemUsed, err } @@ -1623,12 +1624,12 @@ func planProjectionOperators( return nil, resultIdx, typs, internalMemUsed, errors.New("CASE WHEN expressions unsupported") } - allocator := colbase.NewAllocator(ctx, acc) + allocator := colmem.NewAllocator(ctx, acc) // We don't know the schema yet and will update it below, right before // instantiating caseOp. schemaEnforcer := newBatchSchemaPrefixEnforcer(allocator, input, nil /* typs */) buffer := NewBufferOp(schemaEnforcer) - caseOps := make([]colbase.Operator, len(t.Whens)) + caseOps := make([]colexecbase.Operator, len(t.Whens)) caseOutputType := t.ResolvedType() switch typeconv.FromColumnType(caseOutputType) { case coltypes.Bytes: @@ -1697,7 +1698,7 @@ func planProjectionOperators( } } var elseInternalMemUsed int - var elseOp colbase.Operator + var elseOp colexecbase.Operator elseExpr := t.Else if elseExpr == nil { // If there's no ELSE arm, we write NULLs. @@ -1769,9 +1770,9 @@ func planProjectionExpr( outputType *types.T, left, right tree.TypedExpr, columnTypes []types.T, - input colbase.Operator, + input colexecbase.Operator, acc *mon.BoundAccount, -) (op colbase.Operator, resultIdx int, typs []types.T, internalMemUsed int, err error) { +) (op colexecbase.Operator, resultIdx int, typs []types.T, internalMemUsed int, err error) { if err := checkSupportedProjectionExpr(projOp, left, right); err != nil { return nil, resultIdx, typs, internalMemUsed, err } @@ -1820,7 +1821,7 @@ func planProjectionExpr( // The projection result will be outputted to a new column which is appended // to the input batch. op, err = GetProjectionLConstOperator( - colbase.NewAllocator(ctx, acc), left.ResolvedType(), &typs[rightIdx], actualOutputType, + colmem.NewAllocator(ctx, acc), left.ResolvedType(), &typs[rightIdx], actualOutputType, projOp, input, rightIdx, lConstArg, resultIdx, ) } else { @@ -1843,7 +1844,7 @@ func planProjectionExpr( if projOp == tree.Like || projOp == tree.NotLike { negate := projOp == tree.NotLike op, err = GetLikeProjectionOperator( - colbase.NewAllocator(ctx, acc), evalCtx, input, leftIdx, resultIdx, + colmem.NewAllocator(ctx, acc), evalCtx, input, leftIdx, resultIdx, string(tree.MustBeDString(rConstArg)), negate, ) } else if projOp == tree.In || projOp == tree.NotIn { @@ -1854,7 +1855,7 @@ func planProjectionExpr( return nil, resultIdx, typs, internalMemUsed, err } op, err = GetInProjectionOperator( - colbase.NewAllocator(ctx, acc), &typs[leftIdx], input, leftIdx, + colmem.NewAllocator(ctx, acc), &typs[leftIdx], input, leftIdx, resultIdx, datumTuple, negate, ) } else if projOp == tree.IsDistinctFrom || projOp == tree.IsNotDistinctFrom { @@ -1865,10 +1866,10 @@ func planProjectionExpr( // IS NULL is replaced with IS NOT DISTINCT FROM NULL, so we want to // negate when IS DISTINCT FROM is used. negate := projOp == tree.IsDistinctFrom - op = newIsNullProjOp(colbase.NewAllocator(ctx, acc), input, leftIdx, resultIdx, negate) + op = newIsNullProjOp(colmem.NewAllocator(ctx, acc), input, leftIdx, resultIdx, negate) } else { op, err = GetProjectionRConstOperator( - colbase.NewAllocator(ctx, acc), &typs[leftIdx], right.ResolvedType(), actualOutputType, + colmem.NewAllocator(ctx, acc), &typs[leftIdx], right.ResolvedType(), actualOutputType, projOp, input, leftIdx, rConstArg, resultIdx, ) } @@ -1887,7 +1888,7 @@ func planProjectionExpr( internalMemUsed += internalMemUsedRight resultIdx = len(typs) op, err = GetProjectionOperator( - colbase.NewAllocator(ctx, acc), &typs[leftIdx], &typs[rightIdx], actualOutputType, + colmem.NewAllocator(ctx, acc), &typs[leftIdx], &typs[rightIdx], actualOutputType, projOp, input, leftIdx, rightIdx, resultIdx, ) } @@ -1925,15 +1926,15 @@ func planLogicalProjectionOp( evalCtx *tree.EvalContext, expr tree.TypedExpr, columnTypes []types.T, - input colbase.Operator, + input colexecbase.Operator, acc *mon.BoundAccount, -) (op colbase.Operator, resultIdx int, typs []types.T, internalMemUsed int, err error) { +) (op colexecbase.Operator, resultIdx int, typs []types.T, internalMemUsed int, err error) { // Add a new boolean column that will store the result of the projection. resultIdx = len(columnTypes) typs = append(columnTypes, *types.Bool) var ( typedLeft, typedRight tree.TypedExpr - leftProjOpChain, rightProjOpChain, outputOp colbase.Operator + leftProjOpChain, rightProjOpChain, outputOp colexecbase.Operator leftIdx, rightIdx int internalMemUsedLeft, internalMemUsedRight int leftFeedOp, rightFeedOp feedOperator @@ -1946,7 +1947,7 @@ func planLogicalProjectionOp( typedLeft = t.TypedLeft() typedRight = t.TypedRight() default: - vecerror.InternalError(fmt.Sprintf("unexpected logical expression type %s", t.String())) + colexecerror.InternalError(fmt.Sprintf("unexpected logical expression type %s", t.String())) } leftProjOpChain, leftIdx, typs, internalMemUsedLeft, err = planTypedMaybeNullProjectionOperators( ctx, evalCtx, typedLeft, types.Bool, typs, &leftFeedOp, acc, @@ -1960,7 +1961,7 @@ func planLogicalProjectionOp( if err != nil { return nil, resultIdx, typs, internalMemUsed, err } - allocator := colbase.NewAllocator(ctx, acc) + allocator := colmem.NewAllocator(ctx, acc) input = newBatchSchemaPrefixEnforcer(allocator, input, typs) switch expr.(type) { case *tree.AndExpr: diff --git a/pkg/sql/colexec/external_hash_joiner.go b/pkg/sql/colexec/external_hash_joiner.go index b7cf546558a4..49149288bd6e 100644 --- a/pkg/sql/colexec/external_hash_joiner.go +++ b/pkg/sql/colexec/external_hash_joiner.go @@ -16,10 +16,11 @@ import ( "math" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -167,7 +168,7 @@ type externalHashJoiner struct { mu syncutil.Mutex state externalHashJoinerState - unlimitedAllocator *colbase.Allocator + unlimitedAllocator *colmem.Allocator spec hashJoinerSpec diskQueueCfg colcontainer.DiskQueueCfg @@ -272,19 +273,19 @@ const ( // let the partitioned disk queues acquire file descriptors instead of acquiring // them up front in Next. Should be true only in tests. func newExternalHashJoiner( - unlimitedAllocator *colbase.Allocator, + unlimitedAllocator *colmem.Allocator, spec hashJoinerSpec, - leftInput, rightInput colbase.Operator, + leftInput, rightInput colexecbase.Operator, memoryLimit int64, diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, - createReusableDiskBackedSorter func(input colbase.Operator, inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column, maxNumberPartitions int) (colbase.Operator, error), + createReusableDiskBackedSorter func(input colexecbase.Operator, inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column, maxNumberPartitions int) (colexecbase.Operator, error), numForcedRepartitions int, delegateFDAcquisitions bool, diskAcc *mon.BoundAccount, -) colbase.Operator { +) colexecbase.Operator { if diskQueueCfg.CacheMode != colcontainer.DiskQueueCacheModeClearAndReuseCache { - vecerror.InternalError(errors.Errorf("external hash joiner instantiated with suboptimal disk queue cache mode: %d", diskQueueCfg.CacheMode)) + colexecerror.InternalError(errors.Errorf("external hash joiner instantiated with suboptimal disk queue cache mode: %d", diskQueueCfg.CacheMode)) } partitionedDiskQueueSemaphore := fdSemaphore if !delegateFDAcquisitions { @@ -345,14 +346,14 @@ func newExternalHashJoiner( leftJoinerInput, spec.left.sourceTypes, leftOrdering, externalSorterMaxNumberPartitions, ) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } rightOrdering := makeOrderingCols(spec.right.eqCols) rightPartitionSorter, err := createReusableDiskBackedSorter( rightJoinerInput, spec.right.sourceTypes, rightOrdering, externalSorterMaxNumberPartitions, ) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } diskBackedSortMerge, err := newMergeJoinOp( unlimitedAllocator, memoryLimit, diskQueueCfg, @@ -361,7 +362,7 @@ func newExternalHashJoiner( diskAcc, ) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } ehj := &externalHashJoiner{ twoInputNode: newTwoInputNode(leftInput, rightInput), @@ -471,7 +472,7 @@ func (hj *externalHashJoiner) partitionBatch( scratchBatch.SetLength(len(sel)) }) if err := partitioner.Enqueue(ctx, partitionIdx, scratchBatch); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } partitionInfo, ok := hj.partitionsToJoinUsingInMemHash[partitionIdx] if !ok { @@ -483,7 +484,7 @@ func (hj *externalHashJoiner) partitionBatch( // We cannot use allocator's methods directly because those // look at the capacities of the vectors, and in our case only // first len(sel) tuples belong to the "current" batch. - partitionInfo.rightMemSize += colbase.GetProportionalBatchMemSize(scratchBatch, int64(len(sel))) + partitionInfo.rightMemSize += colmem.GetProportionalBatchMemSize(scratchBatch, int64(len(sel))) } } } @@ -513,10 +514,10 @@ StateChanged: // from before doing that to exempt them from releasing their FDs to // the semaphore. if err := hj.leftPartitioner.CloseAllOpenWriteFileDescriptors(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } if err := hj.rightPartitioner.CloseAllOpenWriteFileDescriptors(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } hj.inMemHashJoiner.Init() hj.partitionIdxOffset += hj.numBuckets @@ -526,7 +527,7 @@ StateChanged: if !hj.testingKnobs.delegateFDAcquisitions && hj.fdState.acquiredFDs == 0 { toAcquire := hj.maxNumberActivePartitions if err := hj.fdState.fdSemaphore.Acquire(ctx, toAcquire); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } hj.fdState.acquiredFDs = toAcquire } @@ -562,7 +563,7 @@ StateChanged: } for { if err := partitioner.Dequeue(ctx, parentPartitionIdx, batch); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } if batch.Length() == 0 { break @@ -572,7 +573,7 @@ StateChanged: // We're done reading from this partition, and it will never be read // from again, so we can close it. if err := partitioner.CloseInactiveReadPartitions(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } // We're done writing to the newly created partitions. // TODO(yuzefovich): we should not release the descriptors here. The @@ -585,7 +586,7 @@ StateChanged: // partitions we want. This'll allow us to remove the call to // CloseAllOpen... in the first state as well. if err := partitioner.CloseAllOpenWriteFileDescriptors(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } } for idx := 0; idx < hj.numBuckets; idx++ { @@ -662,10 +663,10 @@ StateChanged: // We're done joining these partitions, so we close them and transition // to joining new ones. if err := hj.leftPartitioner.CloseInactiveReadPartitions(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } if err := hj.rightPartitioner.CloseInactiveReadPartitions(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } hj.state = externalHJJoinNewPartition continue @@ -694,10 +695,10 @@ StateChanged: // We're done joining these partitions, so we close them and transition // to joining new ones. if err := hj.leftPartitioner.CloseInactiveReadPartitions(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } if err := hj.rightPartitioner.CloseInactiveReadPartitions(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } hj.state = externalHJSortMergeNewPartition continue @@ -706,11 +707,11 @@ StateChanged: case externalHJFinished: if err := hj.idempotentCloseLocked(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return coldata.ZeroBatch default: - vecerror.InternalError(fmt.Sprintf("unexpected externalHashJoinerState %d", hj.state)) + colexecerror.InternalError(fmt.Sprintf("unexpected externalHashJoinerState %d", hj.state)) } } } diff --git a/pkg/sql/colexec/external_hash_joiner_test.go b/pkg/sql/colexec/external_hash_joiner_test.go index 987cec69a040..d87826b4a74b 100644 --- a/pkg/sql/colexec/external_hash_joiner_test.go +++ b/pkg/sql/colexec/external_hash_joiner_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/colbase" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -75,8 +75,8 @@ func TestExternalHashJoiner(t *testing.T) { }(tc.skipAllNullsInjection) tc.skipAllNullsInjection = true } - runHashJoinTestCase(t, tc, func(sources []colbase.Operator) (colbase.Operator, error) { - sem := colbase.NewTestingSemaphore(externalHJMinPartitions) + runHashJoinTestCase(t, tc, func(sources []colexecbase.Operator) (colexecbase.Operator, error) { + sem := colexecbase.NewTestingSemaphore(externalHJMinPartitions) semsToCheck = append(semsToCheck, sem) spec := createSpecForHashJoiner(tc) // TODO(asubiotto): Pass in the testing.T of the caller to this @@ -154,12 +154,12 @@ func TestExternalHashJoinerFallbackToSortMergeJoin(t *testing.T) { var spilled bool queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(t, true /* inMem */) defer cleanup() - sem := colbase.NewTestingSemaphore(externalHJMinPartitions) + sem := colexecbase.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, []colbase.Operator{leftSource, rightSource}, + ctx, flowCtx, spec, []colexecbase.Operator{leftSource, rightSource}, func() { spilled = true }, queueCfg, 0 /* numForcedRepartitions */, true, /* delegateFDAcquisitions */ sem, ) @@ -264,9 +264,9 @@ func BenchmarkExternalHashJoiner(b *testing.B) { leftSource.reset(nBatches) rightSource.reset(nBatches) hj, accounts, monitors, _, err := createDiskBackedHashJoiner( - ctx, flowCtx, spec, []colbase.Operator{leftSource, rightSource}, + ctx, flowCtx, spec, []colexecbase.Operator{leftSource, rightSource}, func() {}, queueCfg, 0 /* numForcedRepartitions */, false, /* delegateFDAcquisitions */ - colbase.NewTestingSemaphore(VecMaxOpenFDsLimit), + colexecbase.NewTestingSemaphore(VecMaxOpenFDsLimit), ) memAccounts = append(memAccounts, accounts...) memMonitors = append(memMonitors, monitors...) @@ -297,13 +297,13 @@ func createDiskBackedHashJoiner( ctx context.Context, flowCtx *execinfra.FlowCtx, spec *execinfrapb.ProcessorSpec, - inputs []colbase.Operator, + inputs []colexecbase.Operator, spillingCallbackFn func(), diskQueueCfg colcontainer.DiskQueueCfg, numForcedRepartitions int, delegateFDAcquisitions bool, testingSemaphore semaphore.Semaphore, -) (colbase.Operator, []*mon.BoundAccount, []*mon.BytesMonitor, []IdempotentCloser, error) { +) (colexecbase.Operator, []*mon.BoundAccount, []*mon.BytesMonitor, []IdempotentCloser, error) { args := NewColOperatorArgs{ Spec: spec, Inputs: inputs, diff --git a/pkg/sql/colexec/external_sort.go b/pkg/sql/colexec/external_sort.go index 3ec810249145..942909c241b9 100644 --- a/pkg/sql/colexec/external_sort.go +++ b/pkg/sql/colexec/external_sort.go @@ -15,9 +15,10 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -119,7 +120,7 @@ type externalSorter struct { // Next, which will simplify this model. mu syncutil.Mutex - unlimitedAllocator *colbase.Allocator + unlimitedAllocator *colmem.Allocator state externalSorterState inputTypes []types.T ordering execinfrapb.Ordering @@ -139,7 +140,7 @@ type externalSorter struct { acquiredFDs int } - emitter colbase.Operator + emitter colexecbase.Operator testingKnobs struct { // delegateFDAcquisitions if true, means that a test wants to force the @@ -172,9 +173,9 @@ var _ closableOperator = &externalSorter{} // them up front in Next. This should only be true in tests. func newExternalSorter( ctx context.Context, - unlimitedAllocator *colbase.Allocator, + unlimitedAllocator *colmem.Allocator, standaloneMemAccount *mon.BoundAccount, - input colbase.Operator, + input colexecbase.Operator, inputTypes []types.T, ordering execinfrapb.Ordering, memoryLimit int64, @@ -183,9 +184,9 @@ func newExternalSorter( diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, diskAcc *mon.BoundAccount, -) colbase.Operator { +) colexecbase.Operator { if diskQueueCfg.CacheMode != colcontainer.DiskQueueCacheModeReuseCache { - vecerror.InternalError(errors.Errorf("external sorter instantiated with suboptimal disk queue cache mode: %d", diskQueueCfg.CacheMode)) + colexecerror.InternalError(errors.Errorf("external sorter instantiated with suboptimal disk queue cache mode: %d", diskQueueCfg.CacheMode)) } if diskQueueCfg.BufferSizeBytes > 0 && maxNumberPartitions == 0 { // With the default limit of 256 file descriptors, this results in 16 @@ -201,7 +202,7 @@ func newExternalSorter( // memoryLimit of the partitions to sort in memory by those cache sizes. To be // safe, we also estimate the size of the output batch and subtract that as // well. - batchMemSize := colbase.EstimateBatchSizeBytesFromSQLTypes(inputTypes, coldata.BatchSize()) + batchMemSize := colmem.EstimateBatchSizeBytesFromSQLTypes(inputTypes, coldata.BatchSize()) // Reserve a certain amount of memory for the partition caches. memoryLimit -= int64((maxNumberPartitions * diskQueueCfg.BufferSizeBytes) + batchMemSize) if memoryLimit < 1 { @@ -215,7 +216,7 @@ func newExternalSorter( inputTypes, ordering.Columns, ) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } partitionedDiskQueueSemaphore := fdSemaphore if !delegateFDAcquisitions { @@ -266,7 +267,7 @@ func (s *externalSorter) Next(ctx context.Context) coldata.Batch { s.partitioner = s.partitionerCreator() } if err := s.partitioner.Enqueue(ctx, newPartitionIdx, b); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } s.state = externalSorterSpillPartition continue @@ -294,12 +295,12 @@ func (s *externalSorter) Next(ctx context.Context) coldata.Batch { if !s.testingKnobs.delegateFDAcquisitions && s.fdState.fdSemaphore != nil && s.fdState.acquiredFDs == 0 { toAcquire := s.maxNumberPartitions if err := s.fdState.fdSemaphore.Acquire(ctx, toAcquire); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } s.fdState.acquiredFDs = toAcquire } if err := s.partitioner.Enqueue(ctx, curPartitionIdx, b); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } continue case externalSorterRepeatedMerging: @@ -314,13 +315,13 @@ func (s *externalSorter) Next(ctx context.Context) coldata.Batch { before := s.unlimitedAllocator.Used() merger, err := s.createMergerForPartitions(s.firstPartitionIdx, s.numPartitions) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } merger.Init() newPartitionIdx := s.firstPartitionIdx + s.numPartitions for b := merger.Next(ctx); b.Length() > 0; b = merger.Next(ctx) { if err := s.partitioner.Enqueue(ctx, newPartitionIdx, b); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } } after := s.unlimitedAllocator.Used() @@ -329,7 +330,7 @@ func (s *externalSorter) Next(ctx context.Context) coldata.Batch { // merger must have exhausted all inputs, this is all the partitions just // read from. if err := s.partitioner.CloseInactiveReadPartitions(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } s.firstPartitionIdx += s.numPartitions s.numPartitions = 1 @@ -347,7 +348,7 @@ func (s *externalSorter) Next(ctx context.Context) coldata.Batch { var err error s.emitter, err = s.createMergerForPartitions(s.firstPartitionIdx, s.numPartitions) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } } s.emitter.Init() @@ -362,11 +363,11 @@ func (s *externalSorter) Next(ctx context.Context) coldata.Batch { return b case externalSorterFinished: if err := s.internalCloseLocked(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return coldata.ZeroBatch default: - vecerror.InternalError(fmt.Sprintf("unexpected externalSorterState %d", s.state)) + colexecerror.InternalError(fmt.Sprintf("unexpected externalSorterState %d", s.state)) } } } @@ -379,7 +380,7 @@ func (s *externalSorter) reset(ctx context.Context) { s.mu.Lock() defer s.mu.Unlock() if err := s.internalCloseLocked(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } s.firstPartitionIdx = 0 s.numPartitions = 0 @@ -414,8 +415,8 @@ func (s *externalSorter) IdempotentClose(ctx context.Context) error { // partitions in [firstIdx, firstIdx+numPartitions) range. func (s *externalSorter) createMergerForPartitions( firstIdx, numPartitions int, -) (colbase.Operator, error) { - syncInputs := make([]colbase.Operator, numPartitions) +) (colexecbase.Operator, error) { + syncInputs := make([]colexecbase.Operator, numPartitions) for i := range syncInputs { syncInputs[i] = newPartitionerToOperator( s.unlimitedAllocator, s.inputTypes, s.partitioner, firstIdx+i, @@ -430,7 +431,7 @@ func (s *externalSorter) createMergerForPartitions( } func newInputPartitioningOperator( - input colbase.Operator, standaloneMemAccount *mon.BoundAccount, memoryLimit int64, + input colexecbase.Operator, standaloneMemAccount *mon.BoundAccount, memoryLimit int64, ) resettableOperator { return &inputPartitioningOperator{ OneInputNode: NewOneInputNode(input), @@ -490,9 +491,9 @@ func (o *inputPartitioningOperator) Next(ctx context.Context) coldata.Batch { // it's ok if we have some deviation. This numbers matter only to understand // when to start a new partition, and the memory will be actually accounted // for correctly.) - batchMemSize := colbase.GetProportionalBatchMemSize(b, int64(b.Length())) + batchMemSize := colmem.GetProportionalBatchMemSize(b, int64(b.Length())) if err := o.standaloneMemAccount.Grow(ctx, batchMemSize); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return b } diff --git a/pkg/sql/colexec/external_sort_test.go b/pkg/sql/colexec/external_sort_test.go index 38181732d5db..1353f9f0a107 100644 --- a/pkg/sql/colexec/external_sort_test.go +++ b/pkg/sql/colexec/external_sort_test.go @@ -17,8 +17,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -74,11 +75,11 @@ func TestExternalSort(t *testing.T) { []tuples{tc.tuples}, tc.expected, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.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 := colbase.NewTestingSemaphore(externalSorterMinPartitions) + sem := colexecbase.NewTestingSemaphore(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, @@ -155,7 +156,7 @@ func TestExternalSortRandomized(t *testing.T) { // memory limit. // memoryToSort is the total amount of memory that will be sorted in this // test. - memoryToSort := (nTups / coldata.BatchSize()) * colbase.EstimateBatchSizeBytesFromSQLTypes(typs, coldata.BatchSize()) + memoryToSort := (nTups / coldata.BatchSize()) * colmem.EstimateBatchSizeBytesFromSQLTypes(typs, coldata.BatchSize()) // partitionSize will be the memory limit passed in to tests with a memory // 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 @@ -191,8 +192,8 @@ func TestExternalSortRandomized(t *testing.T) { []tuples{tups}, expected, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { - sem := colbase.NewTestingSemaphore(externalSorterMinPartitions) + func(input []colexecbase.Operator) (colexecbase.Operator, error) { + sem := colexecbase.NewTestingSemaphore(externalSorterMinPartitions) semsToCheck = append(semsToCheck, sem) sorter, newAccounts, newMonitors, closers, err := createDiskBackedSorter( ctx, flowCtx, input, typs[:nCols], ordCols, @@ -274,9 +275,9 @@ func BenchmarkExternalSort(b *testing.B) { // external sorter figure out that number itself) once we pass in // filled-in disk queue config. sorter, accounts, monitors, _, err := createDiskBackedSorter( - ctx, flowCtx, []colbase.Operator{source}, typs, ordCols, + ctx, flowCtx, []colexecbase.Operator{source}, typs, ordCols, 0 /* matchLen */, 0 /* k */, func() { spilled = true }, - 64 /* maxNumberPartitions */, false /* delegateFDAcquisitions */, queueCfg, &colbase.TestingSemaphore{}, + 64 /* maxNumberPartitions */, false /* delegateFDAcquisitions */, queueCfg, &colexecbase.TestingSemaphore{}, ) memAccounts = append(memAccounts, accounts...) memMonitors = append(memMonitors, monitors...) @@ -310,7 +311,7 @@ func BenchmarkExternalSort(b *testing.B) { func createDiskBackedSorter( ctx context.Context, flowCtx *execinfra.FlowCtx, - input []colbase.Operator, + input []colexecbase.Operator, typs []types.T, ordCols []execinfrapb.Ordering_Column, matchLen int, @@ -320,7 +321,7 @@ func createDiskBackedSorter( delegateFDAcquisitions bool, diskQueueCfg colcontainer.DiskQueueCfg, testingSemaphore semaphore.Semaphore, -) (colbase.Operator, []*mon.BoundAccount, []*mon.BytesMonitor, []IdempotentCloser, error) { +) (colexecbase.Operator, []*mon.BoundAccount, []*mon.BytesMonitor, []IdempotentCloser, 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 7b4399883780..0c3bd4849d1f 100644 --- a/pkg/sql/colexec/hash_aggregator.go +++ b/pkg/sql/colexec/hash_aggregator.go @@ -15,9 +15,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" @@ -58,7 +59,7 @@ const ( type hashAggregator struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator aggCols [][]uint32 aggTypes [][]types.T @@ -162,19 +163,19 @@ type hashAggregator struct { decimalScratch decimalOverloadScratch } -var _ colbase.Operator = &hashAggregator{} +var _ colexecbase.Operator = &hashAggregator{} // NewHashAggregator creates a hash aggregator on the given grouping columns. // The input specifications to this function are the same as that of the // NewOrderedAggregator function. func NewHashAggregator( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, typs []types.T, aggFns []execinfrapb.AggregatorSpec_Func, groupCols []uint32, aggCols [][]uint32, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { aggTyps := extractAggTypes(aggCols, typs) outputTypes, err := makeAggregateFuncsOutputTypes(aggTyps, aggFns) if err != nil { @@ -314,7 +315,7 @@ func (op *hashAggregator) Next(ctx context.Context) coldata.Batch { case hashAggregatorDone: return coldata.ZeroBatch default: - vecerror.InternalError("hash aggregator in unhandled state") + colexecerror.InternalError("hash aggregator in unhandled state") // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/hash_aggregator_tmpl.go b/pkg/sql/colexec/hash_aggregator_tmpl.go index 73653e0ea6e7..87dd71d23a55 100644 --- a/pkg/sql/colexec/hash_aggregator_tmpl.go +++ b/pkg/sql/colexec/hash_aggregator_tmpl.go @@ -26,15 +26,16 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Declarations to make the template compile properly. @@ -54,7 +55,7 @@ var _ coltypes.T // _ASSIGN_NE is the template function for assigning the result of comparing // the second input to the third input into the first input. func _ASSIGN_NE(_, _, _ interface{}) int { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} @@ -161,7 +162,7 @@ func (v hashAggFuncs) match( } // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unhandled type %s", &keyTyp)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", &keyTyp)) } } diff --git a/pkg/sql/colexec/hash_utils_tmpl.go b/pkg/sql/colexec/hash_utils_tmpl.go index c52451dbdc6b..5c8a516e4cb8 100644 --- a/pkg/sql/colexec/hash_utils_tmpl.go +++ b/pkg/sql/colexec/hash_utils_tmpl.go @@ -28,14 +28,15 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/types" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Dummy import to pull in "unsafe" package @@ -56,7 +57,7 @@ type _GOTYPESLICE interface{} // _ASSIGN_HASH is the template equality function for assigning the first input // to the result of the hash value of the second input. func _ASSIGN_HASH(_, _ interface{}) uint64 { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} @@ -137,6 +138,6 @@ func rehash( // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unhandled type %s", t)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", t)) } } diff --git a/pkg/sql/colexec/hashjoiner.go b/pkg/sql/colexec/hashjoiner.go index 210acec51be5..1a7ea6cd77a9 100644 --- a/pkg/sql/colexec/hashjoiner.go +++ b/pkg/sql/colexec/hashjoiner.go @@ -14,9 +14,10 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" @@ -156,7 +157,7 @@ type hashJoinerSourceSpec struct { type hashJoiner struct { twoInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator // spec holds the specification for the current hash join process. spec hashJoinerSpec // state stores the current state of the hash joiner. @@ -252,7 +253,7 @@ func (hj *hashJoiner) Next(ctx context.Context) coldata.Batch { hj.emitUnmatched() return hj.output default: - vecerror.InternalError("hash joiner in unhandled state") + colexecerror.InternalError("hash joiner in unhandled state") // This code is unreachable, but the compiler cannot infer that. return nil } @@ -526,7 +527,7 @@ func (hj *hashJoiner) congregate(nResults int, batch coldata.Batch, batchSize in hj.output.SetLength(nResults) } -func (hj *hashJoiner) ExportBuffered(input colbase.Operator) coldata.Batch { +func (hj *hashJoiner) ExportBuffered(input colexecbase.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 @@ -552,7 +553,7 @@ func (hj *hashJoiner) ExportBuffered(input colbase.Operator) coldata.Batch { hj.exportBufferedState.rightExported = newRightExported return b } else { - vecerror.InternalError(errors.New( + colexecerror.InternalError(errors.New( "unexpectedly ExportBuffered is called with neither left nor right inputs to hash join", )) // This code is unreachable, but the compiler cannot infer that. @@ -573,7 +574,7 @@ func (hj *hashJoiner) resetOutput() { } func (hj *hashJoiner) reset(ctx context.Context) { - for _, input := range []colbase.Operator{hj.inputOne, hj.inputTwo} { + for _, input := range []colexecbase.Operator{hj.inputOne, hj.inputTwo} { if r, ok := input.(resetter); ok { r.reset(ctx) } @@ -653,8 +654,8 @@ func makeHashJoinerSpec( // newHashJoiner creates a new equality hash join operator on the left and // right input tables. func newHashJoiner( - allocator *colbase.Allocator, spec hashJoinerSpec, leftSource, rightSource colbase.Operator, -) colbase.Operator { + allocator *colmem.Allocator, spec hashJoinerSpec, leftSource, rightSource colexecbase.Operator, +) colexecbase.Operator { hj := &hashJoiner{ twoInputNode: newTwoInputNode(leftSource, rightSource), allocator: allocator, diff --git a/pkg/sql/colexec/hashjoiner_test.go b/pkg/sql/colexec/hashjoiner_test.go index b56598d8071f..a3c229aee0a6 100644 --- a/pkg/sql/colexec/hashjoiner_test.go +++ b/pkg/sql/colexec/hashjoiner_test.go @@ -20,8 +20,8 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -43,7 +43,7 @@ func init() { for i, f := range floats { _, err := decs[i].SetFloat64(f) if err != nil { - vecerror.InternalError(fmt.Sprintf("%v", err)) + colexecerror.InternalError(fmt.Sprintf("%v", err)) } } @@ -937,7 +937,7 @@ func createSpecForHashJoiner(tc *joinTestCase) *execinfrapb.ProcessorSpec { func runHashJoinTestCase( t *testing.T, tc *joinTestCase, - hjOpConstructor func(sources []colbase.Operator) (colbase.Operator, error), + hjOpConstructor func(sources []colexecbase.Operator) (colexecbase.Operator, error), ) { tc.init() inputs := []tuples{tc.leftTuples, tc.rightTuples} @@ -975,7 +975,7 @@ func TestHashJoiner(t *testing.T) { for _, tcs := range [][]*joinTestCase{hjTestCases, mjTestCases} { for _, tc := range tcs { for _, tc := range tc.mutateTypes() { - runHashJoinTestCase(t, tc, func(sources []colbase.Operator) (colbase.Operator, error) { + runHashJoinTestCase(t, tc, func(sources []colexecbase.Operator) (colexecbase.Operator, error) { spec := createSpecForHashJoiner(tc) args := NewColOperatorArgs{ Spec: spec, @@ -1045,7 +1045,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 := colbase.NewRepeatableBatchSource(testAllocator, batch, sourceTypes) + leftSource := colexecbase.NewRepeatableBatchSource(testAllocator, batch, sourceTypes) rightSource := newFiniteBatchSource(batch, sourceTypes, nBatches) joinType := sqlbase.JoinType_INNER if fullOuter { @@ -1160,7 +1160,7 @@ func TestHashJoinerProjection(t *testing.T) { rightSource := newOpTestInput(1, rightTuples, rightTypes) args := NewColOperatorArgs{ Spec: spec, - Inputs: []colbase.Operator{leftSource, rightSource}, + Inputs: []colexecbase.Operator{leftSource, rightSource}, StreamingMemAccount: testMemAcc, } args.TestingKnobs.UseStreamingMemAccountForBuffering = true diff --git a/pkg/sql/colexec/hashtable.go b/pkg/sql/colexec/hashtable.go index 4fc72724f51b..fc824d51507e 100644 --- a/pkg/sql/colexec/hashtable.go +++ b/pkg/sql/colexec/hashtable.go @@ -15,8 +15,9 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -106,7 +107,7 @@ type hashTableProbeBuffer struct { // The table can then be probed in column batches to find at most one matching // row per column batch row. type hashTable struct { - allocator *colbase.Allocator + allocator *colmem.Allocator // buildScratch contains the scratch buffers required for the build table. buildScratch hashTableBuildBuffer @@ -156,7 +157,7 @@ type hashTable struct { var _ resetter = &hashTable{} func newHashTable( - allocator *colbase.Allocator, + allocator *colmem.Allocator, numBuckets uint64, sourceTypes []types.T, eqCols []uint32, @@ -207,7 +208,7 @@ func newHashTable( // build executes the entirety of the hash table build phase using the input // as the build source. The input is entirely consumed in the process. -func (ht *hashTable) build(ctx context.Context, input colbase.Operator) { +func (ht *hashTable) build(ctx context.Context, input colexecbase.Operator) { nKeyCols := len(ht.keyCols) switch ht.mode { @@ -278,7 +279,7 @@ func (ht *hashTable) build(ctx context.Context, input colbase.Operator) { ht.buildNextChains(ctx, ht.buildScratch.first, ht.buildScratch.next, numBuffered+1, batch.Length()) } default: - vecerror.InternalError(fmt.Sprintf("hashTable in unhandled state")) + colexecerror.InternalError(fmt.Sprintf("hashTable in unhandled state")) } } diff --git a/pkg/sql/colexec/hashtable_tmpl.go b/pkg/sql/colexec/hashtable_tmpl.go index 0deb0e4f5055..cb5da33751d7 100644 --- a/pkg/sql/colexec/hashtable_tmpl.go +++ b/pkg/sql/colexec/hashtable_tmpl.go @@ -26,15 +26,16 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Dummy import to pull in "tree" package. @@ -49,7 +50,7 @@ var _ = math.MaxInt64 // _ASSIGN_NE is the template equality function for assigning the first input // to the result of the the second input != the third input. func _ASSIGN_NE(_, _, _ interface{}) int { - vecerror.InternalError("") + colexecerror.InternalError("") } // _PROBE_TYPE is the template type variable for coltypes.T. It will be @@ -219,11 +220,11 @@ func (ht *hashTable) checkCol( } // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unhandled type %s", buildType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", buildType)) } // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unhandled type %s", probeType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", probeType)) } } @@ -285,7 +286,7 @@ func (ht *hashTable) checkColForDistinctTuples( // {{end}} // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unhandled type %s", probeType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", probeType)) } } @@ -346,7 +347,7 @@ func (ht *hashTable) checkBuildForDistinct( probeVecs []coldata.Vec, nToCheck uint64, probeSel []int, ) uint64 { if probeSel == nil { - vecerror.InternalError("invalid selection vector") + colexecerror.InternalError("invalid selection vector") } copy(ht.probeScratch.distinct, zeroBoolColumn) diff --git a/pkg/sql/colexec/invariants_checker.go b/pkg/sql/colexec/invariants_checker.go index 99021bbe1842..f585c29b05cc 100644 --- a/pkg/sql/colexec/invariants_checker.go +++ b/pkg/sql/colexec/invariants_checker.go @@ -15,7 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" ) // invariantsChecker is a helper Operator that will check that invariants that @@ -25,10 +25,10 @@ type invariantsChecker struct { OneInputNode } -var _ colbase.Operator = invariantsChecker{} +var _ colexecbase.Operator = invariantsChecker{} // NewInvariantsChecker creates a new invariantsChecker. -func NewInvariantsChecker(input colbase.Operator) colbase.Operator { +func NewInvariantsChecker(input colexecbase.Operator) colexecbase.Operator { return &invariantsChecker{ OneInputNode: OneInputNode{input: input}, } diff --git a/pkg/sql/colexec/is_null_ops.go b/pkg/sql/colexec/is_null_ops.go index 595ab35cac18..faaa03bd4985 100644 --- a/pkg/sql/colexec/is_null_ops.go +++ b/pkg/sql/colexec/is_null_ops.go @@ -14,7 +14,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -23,15 +24,15 @@ import ( // If negate is true, it does the opposite - it performs IS NOT NULL check. type isNullProjOp struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator colIdx int outputIdx int negate bool } func newIsNullProjOp( - allocator *colbase.Allocator, input colbase.Operator, colIdx, outputIdx int, negate bool, -) colbase.Operator { + allocator *colmem.Allocator, input colexecbase.Operator, colIdx, outputIdx int, negate bool, +) colexecbase.Operator { input = newVectorTypeEnforcer(allocator, input, types.Bool, outputIdx) return &isNullProjOp{ OneInputNode: NewOneInputNode(input), @@ -42,7 +43,7 @@ func newIsNullProjOp( } } -var _ colbase.Operator = &isNullProjOp{} +var _ colexecbase.Operator = &isNullProjOp{} func (o *isNullProjOp) Init() { o.input.Init() @@ -96,7 +97,7 @@ type isNullSelOp struct { negate bool } -func newIsNullSelOp(input colbase.Operator, colIdx int, negate bool) colbase.Operator { +func newIsNullSelOp(input colexecbase.Operator, colIdx int, negate bool) colexecbase.Operator { return &isNullSelOp{ OneInputNode: NewOneInputNode(input), colIdx: colIdx, @@ -104,7 +105,7 @@ func newIsNullSelOp(input colbase.Operator, colIdx int, negate bool) colbase.Ope } } -var _ colbase.Operator = &isNullSelOp{} +var _ colexecbase.Operator = &isNullSelOp{} func (o *isNullSelOp) 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 3edb2374cadf..a066d7a657be 100644 --- a/pkg/sql/colexec/is_null_ops_test.go +++ b/pkg/sql/colexec/is_null_ops_test.go @@ -16,7 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -83,7 +83,7 @@ func TestIsNullProjOp(t *testing.T) { for _, c := range testCases { t.Run(c.desc, func(t *testing.T) { - opConstructor := func(input []colbase.Operator) (colbase.Operator, error) { + opConstructor := func(input []colexecbase.Operator) (colexecbase.Operator, error) { projExpr := "IS NULL" if c.negate { projExpr = "IS NOT NULL" @@ -157,7 +157,7 @@ func TestIsNullSelOp(t *testing.T) { for _, c := range testCases { t.Run(c.desc, func(t *testing.T) { - opConstructor := func(input []colbase.Operator) (colbase.Operator, error) { + opConstructor := func(input []colexecbase.Operator) (colexecbase.Operator, error) { selExpr := "IS NULL" if c.negate { selExpr = "IS NOT NULL" diff --git a/pkg/sql/colexec/like_ops.go b/pkg/sql/colexec/like_ops.go index cc234a99c028..51862686e098 100644 --- a/pkg/sql/colexec/like_ops.go +++ b/pkg/sql/colexec/like_ops.go @@ -13,7 +13,8 @@ package colexec import ( "strings" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" @@ -88,8 +89,8 @@ func getLikeOperatorType(pattern string, negate bool) (likeOpType, string, error // 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 colbase.Operator, colIdx int, pattern string, negate bool, -) (colbase.Operator, error) { + ctx *tree.EvalContext, input colexecbase.Operator, colIdx int, pattern string, negate bool, +) (colexecbase.Operator, error) { likeOpType, pattern, err := getLikeOperatorType(pattern, negate) if err != nil { return nil, err @@ -173,14 +174,14 @@ func isWildcard(c byte) bool { // result of the specified LIKE pattern, or NOT LIKE if the negate argument is // true. The implementation varies depending on the complexity of the pattern. func GetLikeProjectionOperator( - allocator *colbase.Allocator, + allocator *colmem.Allocator, ctx *tree.EvalContext, - input colbase.Operator, + input colexecbase.Operator, colIdx int, resultIdx int, pattern string, negate bool, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { likeOpType, pattern, err := getLikeOperatorType(pattern, negate) if err != nil { return nil, err diff --git a/pkg/sql/colexec/like_ops_test.go b/pkg/sql/colexec/like_ops_test.go index 320c09b4f8f9..290f8d584f1c 100644 --- a/pkg/sql/colexec/like_ops_test.go +++ b/pkg/sql/colexec/like_ops_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/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -91,7 +91,7 @@ func TestLikeOperators(t *testing.T) { } { runTests( t, []tuples{tc.tups}, tc.expected, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { ctx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) return GetLikeOperator(&ctx, input[0], 0, tc.pattern, tc.negate) }) @@ -120,7 +120,7 @@ func BenchmarkLikeOps(b *testing.B) { } batch.SetLength(coldata.BatchSize()) - source := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) source.Init() base := selConstOpBase{ @@ -143,7 +143,7 @@ func BenchmarkLikeOps(b *testing.B) { testCases := []struct { name string - op colbase.Operator + op colexecbase.Operator }{ {name: "selPrefixBytesBytesConstOp", op: prefixOp}, {name: "selSuffixBytesBytesConstOp", op: suffixOp}, diff --git a/pkg/sql/colexec/limit.go b/pkg/sql/colexec/limit.go index a06b8bff92e6..ff1287a7713c 100644 --- a/pkg/sql/colexec/limit.go +++ b/pkg/sql/colexec/limit.go @@ -14,7 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" ) // limitOp is an operator that implements limit, returning only the first n @@ -31,11 +31,11 @@ type limitOp struct { done bool } -var _ colbase.Operator = &limitOp{} +var _ colexecbase.Operator = &limitOp{} var _ closableOperator = &limitOp{} // NewLimitOp returns a new limit operator with the given limit. -func NewLimitOp(input colbase.Operator, limit int) colbase.Operator { +func NewLimitOp(input colexecbase.Operator, limit int) colexecbase.Operator { c := &limitOp{ OneInputNode: NewOneInputNode(input), limit: limit, diff --git a/pkg/sql/colexec/limit_test.go b/pkg/sql/colexec/limit_test.go index 1780bc666aa8..8b71838a9e3d 100644 --- a/pkg/sql/colexec/limit_test.go +++ b/pkg/sql/colexec/limit_test.go @@ -13,7 +13,7 @@ package colexec import ( "testing" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) @@ -64,7 +64,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. - runTestsWithoutAllNullsInjection(t, []tuples{tc.tuples}, nil /* typs */, tc.expected, orderedVerifier, func(input []colbase.Operator) (colbase.Operator, error) { + runTestsWithoutAllNullsInjection(t, []tuples{tc.tuples}, nil /* typs */, tc.expected, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { return NewLimitOp(input[0], tc.limit), nil }) } diff --git a/pkg/sql/colexec/main_test.go b/pkg/sql/colexec/main_test.go index cecd77c88dbb..3a678ea8eebe 100644 --- a/pkg/sql/colexec/main_test.go +++ b/pkg/sql/colexec/main_test.go @@ -18,8 +18,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -30,7 +30,7 @@ import ( var ( // testAllocator is an Allocator with an unlimited budget for use in tests. - testAllocator *colbase.Allocator + testAllocator *colmem.Allocator // testMemMonitor and testMemAcc are a test monitor with an unlimited budget // and a memory account bound to it for use in tests. @@ -51,7 +51,7 @@ func TestMain(m *testing.M) { defer testMemMonitor.Stop(ctx) memAcc := testMemMonitor.MakeBoundAccount() testMemAcc = &memAcc - testAllocator = colbase.NewAllocator(ctx, testMemAcc) + testAllocator = colmem.NewAllocator(ctx, testMemAcc) defer testMemAcc.Close(ctx) testDiskMonitor = execinfra.NewTestDiskMonitor(ctx, cluster.MakeTestingClusterSettings()) @@ -65,7 +65,7 @@ func TestMain(m *testing.M) { randomBatchSize := generateBatchSize() fmt.Printf("coldata.BatchSize() is set to %d\n", randomBatchSize) if err := coldata.SetBatchSizeForTests(randomBatchSize); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return m.Run() }()) diff --git a/pkg/sql/colexec/materializer.go b/pkg/sql/colexec/materializer.go index 5b465f92f9cc..b463ec4d126f 100644 --- a/pkg/sql/colexec/materializer.go +++ b/pkg/sql/colexec/materializer.go @@ -15,8 +15,8 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -29,7 +29,7 @@ type Materializer struct { execinfra.ProcessorBase NonExplainable - input colbase.Operator + input colexecbase.Operator da sqlbase.DatumAlloc @@ -77,7 +77,7 @@ const materializerProcName = "materializer" func NewMaterializer( flowCtx *execinfra.FlowCtx, processorID int32, - input colbase.Operator, + input colexecbase.Operator, typs []types.T, post *execinfrapb.PostProcessSpec, output execinfra.RowReceiver, @@ -130,7 +130,7 @@ func (m *Materializer) Child(nth int, verbose bool) execinfra.OpNode { if nth == 0 { return m.input } - vecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -182,7 +182,7 @@ func (m *Materializer) next() (sqlbase.EncDatumRow, *execinfrapb.ProducerMetadat // Next is part of the execinfra.RowSource interface. func (m *Materializer) Next() (sqlbase.EncDatumRow, *execinfrapb.ProducerMetadata) { - if err := vecerror.CatchVectorizedRuntimeError(m.nextAdapter); err != nil { + if err := colexecerror.CatchVectorizedRuntimeError(m.nextAdapter); err != nil { m.MoveToDraining(err) return nil, m.DrainHelper() } diff --git a/pkg/sql/colexec/mergejoinbase_tmpl.go b/pkg/sql/colexec/mergejoinbase_tmpl.go index 400ee751a0b0..b75073426003 100644 --- a/pkg/sql/colexec/mergejoinbase_tmpl.go +++ b/pkg/sql/colexec/mergejoinbase_tmpl.go @@ -28,15 +28,16 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/duration" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Declarations to make the template compile properly. @@ -70,7 +71,7 @@ type _GOTYPE interface{} // _ASSIGN_EQ is the template equality function for assigning the first input // to the result of the the second input == the third input. func _ASSIGN_EQ(_, _, _ interface{}) int { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} @@ -124,7 +125,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( } // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unhandled type %s", &typ)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", &typ)) } } return false diff --git a/pkg/sql/colexec/mergejoiner.go b/pkg/sql/colexec/mergejoiner.go index 32c32ddd676e..634980fa7466 100644 --- a/pkg/sql/colexec/mergejoiner.go +++ b/pkg/sql/colexec/mergejoiner.go @@ -16,10 +16,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -198,11 +199,11 @@ type mergeJoinInput struct { // determine where the current group ends, in the case that the group ended // with a batch. distincterInput *feedOperator - distincter colbase.Operator + distincter colexecbase.Operator distinctOutput []bool // source specifies the input operator to the merge join. - source colbase.Operator + source colexecbase.Operator } // The merge join operator uses a probe and build approach to generate the @@ -225,13 +226,13 @@ type mergeJoinInput struct { // sources, based on the equality columns, assuming both inputs are in sorted // order. func newMergeJoinOp( - unlimitedAllocator *colbase.Allocator, + unlimitedAllocator *colmem.Allocator, memoryLimit int64, diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, joinType sqlbase.JoinType, - left colbase.Operator, - right colbase.Operator, + left colexecbase.Operator, + right colexecbase.Operator, leftTypes []types.T, rightTypes []types.T, leftOrdering []execinfrapb.Ordering_Column, @@ -288,13 +289,13 @@ func (s *mjBuilderCrossProductState) setBuilderColumnState(target mjBuilderCross } func newMergeJoinBase( - unlimitedAllocator *colbase.Allocator, + unlimitedAllocator *colmem.Allocator, memoryLimit int64, diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, joinType sqlbase.JoinType, - left colbase.Operator, - right colbase.Operator, + left colexecbase.Operator, + right colexecbase.Operator, leftTypes []types.T, rightTypes []types.T, leftOrdering []execinfrapb.Ordering_Column, @@ -375,7 +376,7 @@ type mergeJoinBase struct { // Next, which will simplify this model. mu syncutil.Mutex - unlimitedAllocator *colbase.Allocator + unlimitedAllocator *colmem.Allocator memoryLimit int64 diskQueueCfg colcontainer.DiskQueueCfg fdSemaphore semaphore.Semaphore @@ -569,7 +570,7 @@ func (o *mergeJoinBase) appendToBufferedGroup( scratchBatch.SetSelection(false) scratchBatch.SetLength(groupLength) if err := bufferedGroup.enqueue(ctx, scratchBatch); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } } @@ -717,7 +718,7 @@ func (o *mergeJoinBase) IdempotentClose(ctx context.Context) error { return nil } var lastErr error - for _, op := range []colbase.Operator{o.left.source, o.right.source} { + for _, op := range []colexecbase.Operator{o.left.source, o.right.source} { if c, ok := op.(IdempotentCloser); ok { if err := c.IdempotentClose(ctx); err != nil { lastErr = err diff --git a/pkg/sql/colexec/mergejoiner_test.go b/pkg/sql/colexec/mergejoiner_test.go index a816cca71ffd..11c878844f3e 100644 --- a/pkg/sql/colexec/mergejoiner_test.go +++ b/pkg/sql/colexec/mergejoiner_test.go @@ -16,8 +16,10 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -1539,14 +1541,14 @@ func TestMergeJoiner(t *testing.T) { runner(t, []tuples{tc.leftTuples, tc.rightTuples}, [][]types.T{tc.leftTypes, tc.rightTypes}, tc.expected, mergeJoinVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { spec := createSpecForMergeJoiner(tc) args := NewColOperatorArgs{ Spec: spec, Inputs: input, StreamingMemAccount: testMemAcc, DiskQueueCfg: queueCfg, - FDSemaphore: colbase.NewTestingSemaphore(mjFDLimit), + FDSemaphore: colexecbase.NewTestingSemaphore(mjFDLimit), } args.TestingKnobs.UseStreamingMemAccountForBuffering = true flowCtx.Cfg.TestingKnobs.MemoryLimitBytes = memoryLimit @@ -1601,7 +1603,7 @@ func TestFullOuterMergeJoinWithMaximumNumberOfGroups(t *testing.T) { rightSource := newChunkingBatchSource(typs, colsRight, nTuples) a, err := newMergeJoinOp( testAllocator, defaultMemoryLimit, queueCfg, - colbase.NewTestingSemaphore(mjFDLimit), sqlbase.FullOuterJoin, + colexecbase.NewTestingSemaphore(mjFDLimit), sqlbase.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}}, @@ -1694,8 +1696,8 @@ func TestMergeJoinCrossProduct(t *testing.T) { groupsRight[i] = int64(rightGroupIdx) } for i, typ := range typs[1:] { - coldata.RandomVec(rng, &typ, 0 /* bytesFixedLength */, colsLeft[i+1], nTuples, nullProbability) - coldata.RandomVec(rng, &typ, 0 /* bytesFixedLength */, colsRight[i+1], nTuples, nullProbability) + coldatatestutils.RandomVec(rng, &typ, 0 /* bytesFixedLength */, colsLeft[i+1], nTuples, nullProbability) + coldatatestutils.RandomVec(rng, &typ, 0 /* bytesFixedLength */, colsRight[i+1], nTuples, nullProbability) } leftMJSource := newChunkingBatchSource(typs, colsLeft, nTuples) rightMJSource := newChunkingBatchSource(typs, colsRight, nTuples) @@ -1703,7 +1705,7 @@ func TestMergeJoinCrossProduct(t *testing.T) { rightHJSource := newChunkingBatchSource(typs, colsRight, nTuples) mj, err := newMergeJoinOp( testAllocator, defaultMemoryLimit, queueCfg, - colbase.NewTestingSemaphore(mjFDLimit), sqlbase.InnerJoin, + colexecbase.NewTestingSemaphore(mjFDLimit), sqlbase.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}}, @@ -1772,7 +1774,7 @@ func TestMergeJoinerMultiBatch(t *testing.T) { a, err := newMergeJoinOp( testAllocator, defaultMemoryLimit, - queueCfg, colbase.NewTestingSemaphore(mjFDLimit), sqlbase.InnerJoin, + queueCfg, colexecbase.NewTestingSemaphore(mjFDLimit), sqlbase.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}}, @@ -1852,7 +1854,7 @@ func TestMergeJoinerMultiBatchRuns(t *testing.T) { a, err := newMergeJoinOp( testAllocator, defaultMemoryLimit, - queueCfg, colbase.NewTestingSemaphore(mjFDLimit), sqlbase.InnerJoin, + queueCfg, colexecbase.NewTestingSemaphore(mjFDLimit), sqlbase.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}}, @@ -1982,7 +1984,7 @@ func TestMergeJoinerRandomized(t *testing.T) { a, err := newMergeJoinOp( testAllocator, defaultMemoryLimit, - queueCfg, colbase.NewTestingSemaphore(mjFDLimit), sqlbase.InnerJoin, + queueCfg, colexecbase.NewTestingSemaphore(mjFDLimit), sqlbase.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}}, @@ -2085,7 +2087,7 @@ func BenchmarkMergeJoiner(b *testing.B) { benchMemAccount.Clear(ctx) base, err := newMergeJoinBase( - colbase.NewAllocator(ctx, &benchMemAccount), defaultMemoryLimit, queueCfg, colbase.NewTestingSemaphore(mjFDLimit), + colmem.NewAllocator(ctx, &benchMemAccount), defaultMemoryLimit, queueCfg, colexecbase.NewTestingSemaphore(mjFDLimit), sqlbase.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}}, @@ -2116,7 +2118,7 @@ func BenchmarkMergeJoiner(b *testing.B) { benchMemAccount.Clear(ctx) base, err := newMergeJoinBase( - colbase.NewAllocator(ctx, &benchMemAccount), defaultMemoryLimit, queueCfg, colbase.NewTestingSemaphore(mjFDLimit), + colmem.NewAllocator(ctx, &benchMemAccount), defaultMemoryLimit, queueCfg, colexecbase.NewTestingSemaphore(mjFDLimit), sqlbase.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}}, @@ -2149,7 +2151,7 @@ func BenchmarkMergeJoiner(b *testing.B) { benchMemAccount.Clear(ctx) base, err := newMergeJoinBase( - colbase.NewAllocator(ctx, &benchMemAccount), defaultMemoryLimit, queueCfg, colbase.NewTestingSemaphore(mjFDLimit), + colmem.NewAllocator(ctx, &benchMemAccount), defaultMemoryLimit, queueCfg, colexecbase.NewTestingSemaphore(mjFDLimit), sqlbase.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/mergejoiner_tmpl.go b/pkg/sql/colexec/mergejoiner_tmpl.go index 86a9d2551219..4d5e22a46079 100644 --- a/pkg/sql/colexec/mergejoiner_tmpl.go +++ b/pkg/sql/colexec/mergejoiner_tmpl.go @@ -29,15 +29,16 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // {{/* "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/duration" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Declarations to make the template compile properly. @@ -71,13 +72,13 @@ type _GOTYPE interface{} // _ASSIGN_EQ is the template equality function for assigning the first input // to the result of the second input == the third input. func _ASSIGN_EQ(_, _, _ interface{}) int { - vecerror.InternalError("") + colexecerror.InternalError("") } // _ASSIGN_LT is the template equality function for assigning the first input // to the result of the the second input < the third input. func _ASSIGN_LT(_, _, _ interface{}) int { - vecerror.InternalError("") + colexecerror.InternalError("") } // _L_SEL_IND is the template type variable for the loop variable that @@ -271,7 +272,7 @@ func _PROBE_SWITCH( } // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) } // {{end}} // {{/* @@ -292,7 +293,7 @@ func _LEFT_UNMATCHED_GROUP_SWITCH(_JOIN_TYPE joinTypeInfo) { // */}} // {{ if or $.JoinType.IsLeftOuter $.JoinType.IsLeftAnti }} if lGroup.unmatched { if curLIdx+1 != curLLength { - vecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLLength-curLIdx)) + colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLLength-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -332,7 +333,7 @@ func _RIGHT_UNMATCHED_GROUP_SWITCH(_JOIN_TYPE joinTypeInfo) { // */}} // {{ if $.JoinType.IsRightOuter }} if rGroup.unmatched { if curRIdx+1 != curRLength { - vecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curRLength-curRIdx)) + colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curRLength-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -715,7 +716,7 @@ func _LEFT_SWITCH(_JOIN_TYPE joinTypeInfo, _HAS_SELECTION bool, _HAS_NULLS bool) o.builderState.left.groupsIdx = zeroMJCPGroupsIdx // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) } // {{end}} // {{/* @@ -803,7 +804,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildLeftBufferedGroup( if currentBatch == nil { currentBatch, err = bufferedGroup.dequeue(ctx) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } o.builderState.lBufferedGroupBatch = currentBatch o.builderState.left.curSrcStartIdx = 0 @@ -875,7 +876,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildLeftBufferedGroup( } // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) } updatedDestStartIdx = outStartIdx o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -889,7 +890,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildLeftBufferedGroup( o.unlimitedAllocator.ReleaseBatch(currentBatch) currentBatch, err = bufferedGroup.dequeue(ctx) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } o.builderState.lBufferedGroupBatch = currentBatch batchLength = currentBatch.Length() @@ -1009,7 +1010,7 @@ func _RIGHT_SWITCH(_JOIN_TYPE joinTypeInfo, _HAS_SELECTION bool, _HAS_NULLS bool o.builderState.right.groupsIdx = zeroMJCPGroupsIdx // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) } // {{end}} // {{/* @@ -1103,7 +1104,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildRightBufferedGroup( if currentBatch == nil { currentBatch, err = bufferedGroup.dequeue(ctx) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } o.builderState.rBufferedGroupBatch = currentBatch o.builderState.right.curSrcStartIdx = 0 @@ -1149,7 +1150,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildRightBufferedGroup( } // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) } } outStartIdx += toAppend @@ -1165,7 +1166,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildRightBufferedGroup( o.unlimitedAllocator.ReleaseBatch(currentBatch) currentBatch, err = bufferedGroup.dequeue(ctx) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } o.builderState.rBufferedGroupBatch = currentBatch batchLength = currentBatch.Length() @@ -1174,7 +1175,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildRightBufferedGroup( // We have fully processed all the batches from the buffered group, so // we need to rewind it. if err := bufferedGroup.rewind(); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } o.builderState.rBufferedGroupBatch = nil } @@ -1386,7 +1387,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) build(ctx context.Context) { // {{ end }} default: - vecerror.InternalError(fmt.Sprintf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) + colexecerror.InternalError(fmt.Sprintf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) } } } @@ -1506,7 +1507,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) Next(ctx context.Context) coldata.Batch { } return coldata.ZeroBatch default: - vecerror.InternalError(fmt.Sprintf("unexpected merge joiner state in Next: %v", o.state)) + colexecerror.InternalError(fmt.Sprintf("unexpected merge joiner state in Next: %v", o.state)) } } } diff --git a/pkg/sql/colexec/min_max_agg_tmpl.go b/pkg/sql/colexec/min_max_agg_tmpl.go index 63b16e2496a1..a12061c69550 100644 --- a/pkg/sql/colexec/min_max_agg_tmpl.go +++ b/pkg/sql/colexec/min_max_agg_tmpl.go @@ -27,18 +27,22 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - // {{/* - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/pkg/errors" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + +// Remove unused warning. +var _ = colexecerror.InternalError + // {{/* // Declarations to make the template compile properly. @@ -72,7 +76,7 @@ type _GOTYPESLICE interface{} // if the second input compares successfully to the third input. The comparison // operator is tree.LT for MIN and is tree.GT for MAX. func _ASSIGN_CMP(_, _, _ string) bool { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} @@ -82,7 +86,7 @@ func _ASSIGN_CMP(_, _, _ string) bool { // {{/* Capture the aggregation name so we can use it in the inner loop. */}} // {{$agg := .AggNameLower}} -func new_AGG_TITLEAgg(allocator *colbase.Allocator, t *types.T) (aggregateFunc, error) { +func new_AGG_TITLEAgg(allocator *colmem.Allocator, t *types.T) (aggregateFunc, error) { switch typeconv.FromColumnType(t) { // {{range .Overloads}} case _TYPES_T: @@ -96,7 +100,7 @@ func new_AGG_TITLEAgg(allocator *colbase.Allocator, t *types.T) (aggregateFunc, // {{range .Overloads}} type _AGG_TYPEAgg struct { - allocator *colbase.Allocator + allocator *colmem.Allocator done bool groups []bool curIdx int diff --git a/pkg/sql/colexec/offset.go b/pkg/sql/colexec/offset.go index 111805661a99..f90f5c0a0177 100644 --- a/pkg/sql/colexec/offset.go +++ b/pkg/sql/colexec/offset.go @@ -14,7 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" ) // offsetOp is an operator that implements offset, returning everything @@ -28,10 +28,10 @@ type offsetOp struct { seen int } -var _ colbase.Operator = &offsetOp{} +var _ colexecbase.Operator = &offsetOp{} // NewOffsetOp returns a new offset operator with the given offset. -func NewOffsetOp(input colbase.Operator, offset int) colbase.Operator { +func NewOffsetOp(input colexecbase.Operator, offset int) colexecbase.Operator { c := &offsetOp{ OneInputNode: NewOneInputNode(input), offset: offset, diff --git a/pkg/sql/colexec/offset_test.go b/pkg/sql/colexec/offset_test.go index 84705759246c..793a785d1ef9 100644 --- a/pkg/sql/colexec/offset_test.go +++ b/pkg/sql/colexec/offset_test.go @@ -15,7 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) @@ -57,7 +57,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. - runTestsWithoutAllNullsInjection(t, []tuples{tc.tuples}, nil /* typs */, tc.expected, unorderedVerifier, func(input []colbase.Operator) (colbase.Operator, error) { + runTestsWithoutAllNullsInjection(t, []tuples{tc.tuples}, nil /* typs */, tc.expected, unorderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { return NewOffsetOp(input[0], tc.offset), nil }) } @@ -68,7 +68,7 @@ func BenchmarkOffset(b *testing.B) { typs := []types.T{*types.Int, *types.Int, *types.Int} batch := testAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) - source := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) source.Init() o := NewOffsetOp(source, 1) diff --git a/pkg/sql/colexec/one_shot.go b/pkg/sql/colexec/one_shot.go index 356685ad3bc7..835cd997b2f1 100644 --- a/pkg/sql/colexec/one_shot.go +++ b/pkg/sql/colexec/one_shot.go @@ -14,7 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" ) // oneShotOp is an operator that does an arbitrary operation on the first batch @@ -23,12 +23,12 @@ import ( type oneShotOp struct { OneInputNode - outputSourceRef *colbase.Operator + outputSourceRef *colexecbase.Operator fn func(batch coldata.Batch) } -var _ colbase.Operator = &oneShotOp{} +var _ colexecbase.Operator = &oneShotOp{} func (o *oneShotOp) Init() { o.input.Init() diff --git a/pkg/sql/colexec/operator.go b/pkg/sql/colexec/operator.go index 0880dcb38596..394ea088f745 100644 --- a/pkg/sql/colexec/operator.go +++ b/pkg/sql/colexec/operator.go @@ -15,8 +15,9 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -41,13 +42,13 @@ type NonExplainable interface { } // NewOneInputNode returns an execinfra.OpNode with a single Operator input. -func NewOneInputNode(input colbase.Operator) OneInputNode { +func NewOneInputNode(input colexecbase.Operator) OneInputNode { return OneInputNode{input: input} } // OneInputNode is an execinfra.OpNode with a single Operator input. type OneInputNode struct { - input colbase.Operator + input colexecbase.Operator } // ChildCount implements the execinfra.OpNode interface. @@ -60,24 +61,24 @@ func (n OneInputNode) Child(nth int, verbose bool) execinfra.OpNode { if nth == 0 { return n.input } - vecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } // Input returns the single input of this OneInputNode as an Operator. -func (n OneInputNode) Input() colbase.Operator { +func (n OneInputNode) Input() colexecbase.Operator { return n.input } // newTwoInputNode returns an execinfra.OpNode with two Operator inputs. -func newTwoInputNode(inputOne, inputTwo colbase.Operator) twoInputNode { +func newTwoInputNode(inputOne, inputTwo colexecbase.Operator) twoInputNode { return twoInputNode{inputOne: inputOne, inputTwo: inputTwo} } type twoInputNode struct { - inputOne colbase.Operator - inputTwo colbase.Operator + inputOne colexecbase.Operator + inputTwo colexecbase.Operator } func (twoInputNode) ChildCount(verbose bool) int { @@ -91,7 +92,7 @@ func (n *twoInputNode) Child(nth int, verbose bool) execinfra.OpNode { case 1: return n.inputTwo } - vecerror.InternalError(fmt.Sprintf("invalid idx %d", nth)) + colexecerror.InternalError(fmt.Sprintf("invalid idx %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -104,7 +105,7 @@ func (n *twoInputNode) Child(nth int, verbose bool) execinfra.OpNode { // "private" to the operator and is not exposed to the outside; notably, it // does *not* include any coldata.Batch'es and coldata.Vec's. type InternalMemoryOperator interface { - colbase.Operator + colexecbase.Operator // InternalMemoryUsage reports the internal memory usage (in bytes) of an // operator. InternalMemoryUsage() int @@ -118,7 +119,7 @@ type resetter interface { // resettableOperator is an Operator that can be reset. type resettableOperator interface { - colbase.Operator + colexecbase.Operator resetter } @@ -147,7 +148,7 @@ func (c *closerHelper) close() bool { } type closableOperator interface { - colbase.Operator + colexecbase.Operator IdempotentCloser } @@ -156,10 +157,10 @@ type noopOperator struct { NonExplainable } -var _ colbase.Operator = &noopOperator{} +var _ colexecbase.Operator = &noopOperator{} // NewNoop returns a new noop Operator. -func NewNoop(input colbase.Operator) colbase.Operator { +func NewNoop(input colexecbase.Operator) colexecbase.Operator { return &noopOperator{OneInputNode: NewOneInputNode(input)} } @@ -182,10 +183,10 @@ type zeroOperator struct { NonExplainable } -var _ colbase.Operator = &zeroOperator{} +var _ colexecbase.Operator = &zeroOperator{} // NewZeroOp creates a new operator which just returns an empty batch. -func NewZeroOp(input colbase.Operator) colbase.Operator { +func NewZeroOp(input colexecbase.Operator) colexecbase.Operator { return &zeroOperator{OneInputNode: NewOneInputNode(input)} } @@ -200,18 +201,18 @@ func (s *zeroOperator) Next(ctx context.Context) coldata.Batch { } type singleTupleNoInputOperator struct { - colbase.ZeroInputNode + colexecbase.ZeroInputNode NonExplainable batch coldata.Batch nexted bool } -var _ colbase.Operator = &singleTupleNoInputOperator{} +var _ colexecbase.Operator = &singleTupleNoInputOperator{} // NewSingleTupleNoInputOp creates a new Operator which returns a batch of // length 1 with no actual columns on the first call to Next() and zero-length // batches on all consecutive calls. -func NewSingleTupleNoInputOp(allocator *colbase.Allocator) colbase.Operator { +func NewSingleTupleNoInputOp(allocator *colmem.Allocator) colexecbase.Operator { return &singleTupleNoInputOperator{ batch: allocator.NewMemBatchWithSize(nil /* types */, 1 /* size */), } @@ -233,7 +234,7 @@ func (s *singleTupleNoInputOperator) Next(ctx context.Context) coldata.Batch { // feedOperator is used to feed an Operator chain with input by manually // setting the next batch. type feedOperator struct { - colbase.ZeroInputNode + colexecbase.ZeroInputNode NonExplainable batch coldata.Batch } @@ -244,7 +245,7 @@ func (o *feedOperator) Next(context.Context) coldata.Batch { return o.batch } -var _ colbase.Operator = &feedOperator{} +var _ colexecbase.Operator = &feedOperator{} // vectorTypeEnforcer is a utility Operator that on every call to Next // enforces that non-zero length batch from the input has a vector of the @@ -274,16 +275,16 @@ type vectorTypeEnforcer struct { OneInputNode NonExplainable - allocator *colbase.Allocator + allocator *colmem.Allocator typ *types.T idx int } -var _ colbase.Operator = &vectorTypeEnforcer{} +var _ colexecbase.Operator = &vectorTypeEnforcer{} func newVectorTypeEnforcer( - allocator *colbase.Allocator, input colbase.Operator, typ *types.T, idx int, -) colbase.Operator { + allocator *colmem.Allocator, input colexecbase.Operator, typ *types.T, idx int, +) colexecbase.Operator { return &vectorTypeEnforcer{ OneInputNode: NewOneInputNode(input), allocator: allocator, @@ -319,14 +320,14 @@ type batchSchemaPrefixEnforcer struct { OneInputNode NonExplainable - allocator *colbase.Allocator + allocator *colmem.Allocator typs []types.T } -var _ colbase.Operator = &batchSchemaPrefixEnforcer{} +var _ colexecbase.Operator = &batchSchemaPrefixEnforcer{} func newBatchSchemaPrefixEnforcer( - allocator *colbase.Allocator, input colbase.Operator, typs []types.T, + allocator *colmem.Allocator, input colexecbase.Operator, typs []types.T, ) *batchSchemaPrefixEnforcer { return &batchSchemaPrefixEnforcer{ OneInputNode: NewOneInputNode(input), diff --git a/pkg/sql/colexec/orderedsynchronizer_test.go b/pkg/sql/colexec/orderedsynchronizer_test.go index ed42d85ef3aa..1c4c43772266 100644 --- a/pkg/sql/colexec/orderedsynchronizer_test.go +++ b/pkg/sql/colexec/orderedsynchronizer_test.go @@ -17,7 +17,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/encoding" @@ -143,7 +143,7 @@ func TestOrderedSync(t *testing.T) { for i := range typs { typs[i] = *types.Int } - runTests(t, tc.sources, tc.expected, orderedVerifier, func(inputs []colbase.Operator) (colbase.Operator, error) { + runTests(t, tc.sources, tc.expected, orderedVerifier, func(inputs []colexecbase.Operator) (colexecbase.Operator, error) { return NewOrderedSynchronizer(testAllocator, inputs, typs, tc.ordering) }) } @@ -179,7 +179,7 @@ func TestOrderedSyncRandomInput(t *testing.T) { } sources[sourceIdx] = append(sources[sourceIdx], t) } - inputs := make([]colbase.Operator, numInputs) + inputs := make([]colexecbase.Operator, numInputs) for i := range inputs { inputs[i] = newOpTestInput(batchSize, sources[i], typs) } @@ -208,9 +208,9 @@ func BenchmarkOrderedSynchronizer(b *testing.B) { batch.ColVec(0).Int64()[i/numInputs] = i } - inputs := make([]colbase.Operator, len(batches)) + inputs := make([]colexecbase.Operator, len(batches)) for i := range batches { - inputs[i] = colbase.NewRepeatableBatchSource(testAllocator, batches[i], typs) + inputs[i] = colexecbase.NewRepeatableBatchSource(testAllocator, batches[i], typs) } ordering := sqlbase.ColumnOrdering{{ColIdx: 0, Direction: encoding.Ascending}} diff --git a/pkg/sql/colexec/orderedsynchronizer_tmpl.go b/pkg/sql/colexec/orderedsynchronizer_tmpl.go index 33e9a3f6aee7..e9f5c12d908f 100644 --- a/pkg/sql/colexec/orderedsynchronizer_tmpl.go +++ b/pkg/sql/colexec/orderedsynchronizer_tmpl.go @@ -28,12 +28,11 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -41,6 +40,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/encoding" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Declarations to make the template compile properly. @@ -68,8 +70,8 @@ const _TYPES_T = coltypes.Unhandled // stream of rows, ordered according to a set of columns. The rows in each input // stream are assumed to be ordered according to the same set of columns. type OrderedSynchronizer struct { - allocator *colbase.Allocator - inputs []colbase.Operator + allocator *colmem.Allocator + inputs []colexecbase.Operator ordering sqlbase.ColumnOrdering typs []types.T physTypes []coltypes.T @@ -107,7 +109,7 @@ type OrderedSynchronizer struct { outColsMap []int } -var _ colbase.Operator = &OrderedSynchronizer{} +var _ colexecbase.Operator = &OrderedSynchronizer{} // ChildCount implements the execinfrapb.OpNode interface. func (o *OrderedSynchronizer) ChildCount(verbose bool) int { @@ -121,8 +123,8 @@ func (o *OrderedSynchronizer) Child(nth int, verbose bool) execinfra.OpNode { // NewOrderedSynchronizer creates a new OrderedSynchronizer. func NewOrderedSynchronizer( - allocator *colbase.Allocator, - inputs []colbase.Operator, + allocator *colmem.Allocator, + inputs []colexecbase.Operator, typs []types.T, ordering sqlbase.ColumnOrdering, ) (*OrderedSynchronizer, error) { @@ -180,7 +182,7 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { execgen.SET(outCol, outputIdx, v) // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unhandled type %s", physType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", physType)) } } } @@ -222,7 +224,7 @@ func (o *OrderedSynchronizer) Init() { o.out_TYPECols = append(o.out_TYPECols, outVec._TYPE()) // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unhandled type %s", &o.typs[i])) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", &o.typs[i])) } } for i := range o.inputs { @@ -256,7 +258,7 @@ func (o *OrderedSynchronizer) compareRow(batchIdx1 int, batchIdx2 int) int { case encoding.Descending: return -res default: - vecerror.InternalError(fmt.Sprintf("unexpected direction value %d", d)) + colexecerror.InternalError(fmt.Sprintf("unexpected direction value %d", d)) } } } diff --git a/pkg/sql/colexec/ordinality.go b/pkg/sql/colexec/ordinality.go index ba45d89d0101..74243fa189f4 100644 --- a/pkg/sql/colexec/ordinality.go +++ b/pkg/sql/colexec/ordinality.go @@ -14,7 +14,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -23,7 +24,7 @@ import ( type ordinalityOp struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator // outputIdx is the index of the column in which ordinalityOp will write the // ordinal number. outputIdx int @@ -31,12 +32,12 @@ type ordinalityOp struct { counter int64 } -var _ colbase.Operator = &ordinalityOp{} +var _ colexecbase.Operator = &ordinalityOp{} // NewOrdinalityOp returns a new WITH ORDINALITY operator. func NewOrdinalityOp( - allocator *colbase.Allocator, input colbase.Operator, outputIdx int, -) colbase.Operator { + allocator *colmem.Allocator, input colexecbase.Operator, outputIdx int, +) colexecbase.Operator { input = newVectorTypeEnforcer(allocator, input, types.Int, outputIdx) c := &ordinalityOp{ OneInputNode: NewOneInputNode(input), diff --git a/pkg/sql/colexec/ordinality_test.go b/pkg/sql/colexec/ordinality_test.go index 96f16c1b2ae4..c31d6643b22d 100644 --- a/pkg/sql/colexec/ordinality_test.go +++ b/pkg/sql/colexec/ordinality_test.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -66,7 +66,7 @@ func TestOrdinality(t *testing.T) { for _, tc := range tcs { runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return createTestOrdinalityOperator(ctx, flowCtx, input[0], tc.inputTypes) }) } @@ -87,7 +87,7 @@ func BenchmarkOrdinality(b *testing.B) { typs := []types.T{*types.Int, *types.Int, *types.Int} batch := testAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) - source := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) ordinality, err := createTestOrdinalityOperator(ctx, flowCtx, source, []types.T{*types.Int, *types.Int, *types.Int}) require.NoError(b, err) ordinality.Init() @@ -99,8 +99,8 @@ func BenchmarkOrdinality(b *testing.B) { } func createTestOrdinalityOperator( - ctx context.Context, flowCtx *execinfra.FlowCtx, input colbase.Operator, inputTypes []types.T, -) (colbase.Operator, error) { + ctx context.Context, flowCtx *execinfra.FlowCtx, input colexecbase.Operator, inputTypes []types.T, +) (colexecbase.Operator, error) { spec := &execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{{ColumnTypes: inputTypes}}, Core: execinfrapb.ProcessorCoreUnion{ @@ -109,7 +109,7 @@ func createTestOrdinalityOperator( } args := NewColOperatorArgs{ Spec: spec, - Inputs: []colbase.Operator{input}, + Inputs: []colexecbase.Operator{input}, StreamingMemAccount: testMemAcc, } args.TestingKnobs.UseStreamingMemAccountForBuffering = true diff --git a/pkg/sql/colexec/overloads_test.go b/pkg/sql/colexec/overloads_test.go index 44ea40bc9d7b..ade2112c6b82 100644 --- a/pkg/sql/colexec/overloads_test.go +++ b/pkg/sql/colexec/overloads_test.go @@ -15,7 +15,7 @@ import ( "testing" "github.com/cockroachdb/apd" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/errors" @@ -26,10 +26,10 @@ func TestIntegerAddition(t *testing.T) { defer leaktest.AfterTest(t)() // The addition overload is the same for all integer widths, so we only test // one of them. - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performPlusInt16Int16(1, math.MaxInt16) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performPlusInt16Int16(-1, math.MinInt16) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performPlusInt16Int16(math.MaxInt16, 1) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performPlusInt16Int16(math.MinInt16, -1) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performPlusInt16Int16(1, math.MaxInt16) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performPlusInt16Int16(-1, math.MinInt16) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performPlusInt16Int16(math.MaxInt16, 1) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performPlusInt16Int16(math.MinInt16, -1) }), tree.ErrIntOutOfRange)) require.Equal(t, int16(math.MaxInt16), performPlusInt16Int16(1, math.MaxInt16-1)) require.Equal(t, int16(math.MinInt16), performPlusInt16Int16(-1, math.MinInt16+1)) @@ -46,10 +46,10 @@ func TestIntegerSubtraction(t *testing.T) { defer leaktest.AfterTest(t)() // The subtraction overload is the same for all integer widths, so we only // test one of them. - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMinusInt16Int16(1, -math.MaxInt16) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMinusInt16Int16(-2, math.MaxInt16) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMinusInt16Int16(math.MaxInt16, -1) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMinusInt16Int16(math.MinInt16, 1) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMinusInt16Int16(1, -math.MaxInt16) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMinusInt16Int16(-2, math.MaxInt16) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMinusInt16Int16(math.MaxInt16, -1) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMinusInt16Int16(math.MinInt16, 1) }), tree.ErrIntOutOfRange)) require.Equal(t, int16(math.MaxInt16), performMinusInt16Int16(1, -math.MaxInt16+1)) require.Equal(t, int16(math.MinInt16), performMinusInt16Int16(-1, math.MaxInt16)) @@ -78,9 +78,9 @@ func TestIntegerDivision(t *testing.T) { } require.Equal(t, 0, res.Cmp(d)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performDivInt16Int16(10, 0) }), tree.ErrDivByZero)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performDivInt32Int32(10, 0) }), tree.ErrDivByZero)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performDivInt64Int64(10, 0) }), tree.ErrDivByZero)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performDivInt16Int16(10, 0) }), tree.ErrDivByZero)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performDivInt32Int32(10, 0) }), tree.ErrDivByZero)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performDivInt64Int64(10, 0) }), tree.ErrDivByZero)) res = performDivInt16Int16(math.MaxInt16, -1) require.Equal(t, 0, res.Cmp(d.SetFinite(-math.MaxInt16, 0))) @@ -92,24 +92,24 @@ func TestIntegerDivision(t *testing.T) { func TestIntegerMultiplication(t *testing.T) { defer leaktest.AfterTest(t)() - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMultInt16Int16(math.MaxInt16-1, 100) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMultInt16Int16(math.MaxInt16-1, 3) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMultInt16Int16(math.MinInt16+1, 3) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMultInt16Int16(math.MinInt16+1, 100) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMultInt16Int16(math.MaxInt16-1, 100) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMultInt16Int16(math.MaxInt16-1, 3) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMultInt16Int16(math.MinInt16+1, 3) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMultInt16Int16(math.MinInt16+1, 100) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMultInt32Int32(math.MaxInt32-1, 100) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMultInt32Int32(math.MaxInt32-1, 3) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMultInt32Int32(math.MinInt32+1, 3) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMultInt32Int32(math.MinInt32+1, 100) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMultInt32Int32(math.MaxInt32-1, 100) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMultInt32Int32(math.MaxInt32-1, 3) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMultInt32Int32(math.MinInt32+1, 3) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMultInt32Int32(math.MinInt32+1, 100) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMultInt64Int64(math.MaxInt64-1, 100) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMultInt64Int64(math.MaxInt64-1, 3) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMultInt64Int64(math.MinInt64+1, 3) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMultInt64Int64(math.MinInt64+1, 100) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMultInt64Int64(math.MaxInt64-1, 100) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMultInt64Int64(math.MaxInt64-1, 3) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMultInt64Int64(math.MinInt64+1, 3) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMultInt64Int64(math.MinInt64+1, 100) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMultInt16Int16(math.MinInt16, -1) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMultInt32Int32(math.MinInt32, -1) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performMultInt64Int64(math.MinInt64, -1) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMultInt16Int16(math.MinInt16, -1) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMultInt32Int32(math.MinInt32, -1) }), tree.ErrIntOutOfRange)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performMultInt64Int64(math.MinInt64, -1) }), tree.ErrIntOutOfRange)) require.Equal(t, int16(-math.MaxInt16), performMultInt16Int16(math.MaxInt16, -1)) require.Equal(t, int32(-math.MaxInt32), performMultInt32Int32(math.MaxInt32, -1)) @@ -160,15 +160,15 @@ func TestDecimalDivByZero(t *testing.T) { nonZeroDec.SetFinite(4, 0) zeroDec.SetFinite(0, 0) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performDivDecimalInt16(nonZeroDec, 0) }), tree.ErrDivByZero)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performDivDecimalInt32(nonZeroDec, 0) }), tree.ErrDivByZero)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performDivDecimalInt64(nonZeroDec, 0) }), tree.ErrDivByZero)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performDivDecimalInt16(nonZeroDec, 0) }), tree.ErrDivByZero)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performDivDecimalInt32(nonZeroDec, 0) }), tree.ErrDivByZero)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performDivDecimalInt64(nonZeroDec, 0) }), tree.ErrDivByZero)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performDivInt64Decimal(2, zeroDec) }), tree.ErrDivByZero)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performDivInt32Decimal(2, zeroDec) }), tree.ErrDivByZero)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performDivInt16Decimal(2, zeroDec) }), tree.ErrDivByZero)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performDivInt64Decimal(2, zeroDec) }), tree.ErrDivByZero)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performDivInt32Decimal(2, zeroDec) }), tree.ErrDivByZero)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performDivInt16Decimal(2, zeroDec) }), tree.ErrDivByZero)) - require.True(t, errors.Is(vecerror.CatchVectorizedRuntimeError(func() { performDivDecimalDecimal(nonZeroDec, zeroDec) }), tree.ErrDivByZero)) + require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performDivDecimalDecimal(nonZeroDec, zeroDec) }), tree.ErrDivByZero)) } func TestDecimalComparison(t *testing.T) { diff --git a/pkg/sql/colexec/parallel_unordered_synchronizer.go b/pkg/sql/colexec/parallel_unordered_synchronizer.go index 5adf8c4c43bd..398364ab2c4d 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/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/contextutil" @@ -31,13 +31,13 @@ type unorderedSynchronizerMsg struct { b coldata.Batch } -var _ colbase.Operator = &ParallelUnorderedSynchronizer{} +var _ colexecbase.Operator = &ParallelUnorderedSynchronizer{} var _ execinfra.OpNode = &ParallelUnorderedSynchronizer{} // ParallelUnorderedSynchronizer is an Operator that combines multiple Operator streams // into one. type ParallelUnorderedSynchronizer struct { - inputs []colbase.Operator + inputs []colexecbase.Operator // readNextBatch is a slice of channels, where each channel corresponds to the // input at the same index in inputs. It is used as a barrier for input // goroutines to wait on until the Next goroutine signals that it is safe to @@ -91,7 +91,7 @@ func (s *ParallelUnorderedSynchronizer) Child(nth int, verbose bool) execinfra.O // guaranteed that these spawned goroutines will have completed on any error or // zero-length batch received from Next. func NewParallelUnorderedSynchronizer( - inputs []colbase.Operator, typs []types.T, wg *sync.WaitGroup, + inputs []colexecbase.Operator, typs []types.T, wg *sync.WaitGroup, ) *ParallelUnorderedSynchronizer { readNextBatch := make([]chan struct{}, len(inputs)) for i := range readNextBatch { @@ -132,7 +132,7 @@ func (s *ParallelUnorderedSynchronizer) Init() { func (s *ParallelUnorderedSynchronizer) init(ctx context.Context) { ctx, s.cancelFn = contextutil.WithCancel(ctx) for i, input := range s.inputs { - s.nextBatch[i] = func(input colbase.Operator, inputIdx int) func() { + s.nextBatch[i] = func(input colexecbase.Operator, inputIdx int) func() { return func() { s.batches[inputIdx] = input.Next(ctx) } @@ -142,7 +142,7 @@ func (s *ParallelUnorderedSynchronizer) init(ctx context.Context) { // TODO(asubiotto): Most inputs are Inboxes, and these have handler // goroutines just sitting around waiting for cancellation. I wonder if we // could reuse those goroutines to push batches to batchCh directly. - go func(input colbase.Operator, inputIdx int) { + go func(input colexecbase.Operator, inputIdx int) { defer func() { if int(atomic.AddUint32(&s.numFinishedInputs, 1)) == len(s.inputs) { close(s.batchCh) @@ -154,7 +154,7 @@ func (s *ParallelUnorderedSynchronizer) init(ctx context.Context) { inputIdx: inputIdx, } for { - if err := vecerror.CatchVectorizedRuntimeError(s.nextBatch[inputIdx]); err != nil { + if err := colexecerror.CatchVectorizedRuntimeError(s.nextBatch[inputIdx]); err != nil { select { // Non-blocking write to errCh, if an error is present the main // goroutine will use that and cancel all inputs. @@ -217,7 +217,7 @@ func (s *ParallelUnorderedSynchronizer) Next(ctx context.Context) coldata.Batch // propagate this error through a panic. s.cancelFn() s.internalWaitGroup.Wait() - vecerror.InternalError(err) + colexecerror.InternalError(err) } case msg := <-s.batchCh: if msg == nil { @@ -227,7 +227,7 @@ func (s *ParallelUnorderedSynchronizer) Next(ctx context.Context) coldata.Batch select { case err := <-s.errCh: if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } default: } diff --git a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go index f8ee4c4e113f..420287adad0f 100644 --- a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go +++ b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go @@ -18,8 +18,9 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -42,11 +43,11 @@ func TestParallelUnorderedSynchronizer(t *testing.T) { numBatches = rng.Intn(maxBatches) + 1 ) - inputs := make([]colbase.Operator, numInputs) + inputs := make([]colexecbase.Operator, numInputs) for i := range inputs { - source := colbase.NewRepeatableBatchSource( + source := colexecbase.NewRepeatableBatchSource( testAllocator, - colbase.RandomBatch(testAllocator, rng, typs, coldata.BatchSize(), 0 /* length */, rng.Float64()), + coldatatestutils.RandomBatch(testAllocator, rng, typs, coldata.BatchSize(), 0 /* length */, rng.Float64()), typs, ) source.ResetBatchesToReturn(numBatches) @@ -77,7 +78,7 @@ func TestParallelUnorderedSynchronizer(t *testing.T) { batchesReturned := 0 for { var b coldata.Batch - if err := vecerror.CatchVectorizedRuntimeError(func() { b = s.Next(ctx) }); err != nil { + if err := colexecerror.CatchVectorizedRuntimeError(func() { b = s.Next(ctx) }); err != nil { if cancel { require.True(t, testutils.IsError(err, "context canceled"), err) break @@ -101,17 +102,17 @@ func TestUnorderedSynchronizerNoLeaksOnError(t *testing.T) { const expectedErr = "first input error" - inputs := make([]colbase.Operator, 6) - inputs[0] = &colbase.CallbackOperator{NextCb: func(context.Context) coldata.Batch { - vecerror.InternalError(expectedErr) + inputs := make([]colexecbase.Operator, 6) + inputs[0] = &colexecbase.CallbackOperator{NextCb: func(context.Context) coldata.Batch { + colexecerror.InternalError(expectedErr) // This code is unreachable, but the compiler cannot infer that. return nil }} for i := 1; i < len(inputs); i++ { - inputs[i] = &colbase.CallbackOperator{ + inputs[i] = &colexecbase.CallbackOperator{ NextCb: func(ctx context.Context) coldata.Batch { <-ctx.Done() - vecerror.InternalError(ctx.Err()) + colexecerror.InternalError(ctx.Err()) // This code is unreachable, but the compiler cannot infer that. return nil }, @@ -123,7 +124,7 @@ func TestUnorderedSynchronizerNoLeaksOnError(t *testing.T) { wg sync.WaitGroup ) s := NewParallelUnorderedSynchronizer(inputs, []types.T{*types.Int}, &wg) - err := vecerror.CatchVectorizedRuntimeError(func() { _ = s.Next(ctx) }) + err := colexecerror.CatchVectorizedRuntimeError(func() { _ = s.Next(ctx) }) // This is the crux of the test: assert that all inputs have finished. require.Equal(t, len(inputs), int(atomic.LoadUint32(&s.numFinishedInputs))) require.True(t, testutils.IsError(err, expectedErr), err) @@ -133,11 +134,11 @@ func BenchmarkParallelUnorderedSynchronizer(b *testing.B) { const numInputs = 6 typs := []types.T{*types.Int} - inputs := make([]colbase.Operator, numInputs) + inputs := make([]colexecbase.Operator, numInputs) for i := range inputs { batch := testAllocator.NewMemBatchWithSize(typs, coldata.BatchSize()) batch.SetLength(coldata.BatchSize()) - inputs[i] = colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + inputs[i] = colexecbase.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 a4e9c5d62468..859e6b4a74c1 100644 --- a/pkg/sql/colexec/partially_ordered_distinct.go +++ b/pkg/sql/colexec/partially_ordered_distinct.go @@ -15,9 +15,10 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" @@ -30,12 +31,12 @@ const partiallyOrderedDistinctNumHashBuckets = 1024 // distinct columns when we have partial ordering on some of the distinct // columns. func newPartiallyOrderedDistinct( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, distinctCols []uint32, orderedCols []uint32, typs []types.T, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { if len(orderedCols) == 0 || len(orderedCols) == len(distinctCols) { return nil, errors.AssertionFailedf( "partially ordered distinct wrongfully planned: numDistinctCols=%d "+ @@ -82,7 +83,7 @@ type partiallyOrderedDistinct struct { distinct resettableOperator } -var _ colbase.Operator = &partiallyOrderedDistinct{} +var _ colexecbase.Operator = &partiallyOrderedDistinct{} func (p *partiallyOrderedDistinct) ChildCount(bool) int { return 1 @@ -92,7 +93,7 @@ func (p *partiallyOrderedDistinct) Child(nth int, _ bool) execinfra.OpNode { if nth == 0 { return p.input } - vecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -118,7 +119,7 @@ func (p *partiallyOrderedDistinct) Next(ctx context.Context) coldata.Batch { } func newChunkerOperator( - allocator *colbase.Allocator, input *chunker, inputTypes []types.T, + allocator *colmem.Allocator, input *chunker, inputTypes []types.T, ) *chunkerOperator { return &chunkerOperator{ input: input, @@ -169,7 +170,7 @@ func (c *chunkerOperator) Child(nth int, _ bool) execinfra.OpNode { if nth == 0 { return c.input } - vecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/partitioner.go b/pkg/sql/colexec/partitioner.go index 8a5d25d07a96..c8b1ed99ed22 100644 --- a/pkg/sql/colexec/partitioner.go +++ b/pkg/sql/colexec/partitioner.go @@ -14,7 +14,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -25,14 +26,14 @@ import ( // puts true in partitionColIdx'th column (which is appended if needed) for // every tuple that is the first within its partition. func NewWindowSortingPartitioner( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, inputTyps []types.T, partitionIdxs []uint32, ordCols []execinfrapb.Ordering_Column, partitionColIdx int, - createDiskBackedSorter func(input colbase.Operator, inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column) (colbase.Operator, error), -) (op colbase.Operator, err error) { + createDiskBackedSorter func(input colexecbase.Operator, inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column) (colexecbase.Operator, error), +) (op colexecbase.Operator, err error) { partitionAndOrderingCols := make([]execinfrapb.Ordering_Column, len(partitionIdxs)+len(ordCols)) for i, idx := range partitionIdxs { partitionAndOrderingCols[i] = execinfrapb.Ordering_Column{ColIdx: idx} @@ -61,7 +62,7 @@ func NewWindowSortingPartitioner( type windowSortingPartitioner struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator // distinctCol is the output column of the chain of ordered distinct // operators in which true will indicate that a new partition begins with the // corresponding tuple. diff --git a/pkg/sql/colexec/proj_const_ops_tmpl.go b/pkg/sql/colexec/proj_const_ops_tmpl.go index 8a56339898f0..4b5fa0354d9c 100644 --- a/pkg/sql/colexec/proj_const_ops_tmpl.go +++ b/pkg/sql/colexec/proj_const_ops_tmpl.go @@ -28,18 +28,20 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/pkg/errors" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Declarations to make the template compile properly. @@ -67,13 +69,13 @@ var _ coltypes.T // _ASSIGN is the template function for assigning the first input to the result // of computation an operation on the second and the third inputs. func _ASSIGN(_, _, _ interface{}) { - vecerror.InternalError("") + colexecerror.InternalError("") } // _RET_UNSAFEGET is the template function that will be replaced by // "execgen.UNSAFEGET" which uses _RET_TYP. func _RET_UNSAFEGET(_, _ interface{}) interface{} { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} @@ -201,16 +203,16 @@ func _SET_SINGLE_TUPLE_PROJECTION(_HAS_NULLS bool) { // */}} // GetProjection_CONST_SIDEConstOperator returns the appropriate constant // projection operator for the given left and right column types and operation. func GetProjection_CONST_SIDEConstOperator( - allocator *colbase.Allocator, + allocator *colmem.Allocator, leftType *types.T, rightType *types.T, outputType *types.T, op tree.Operator, - input colbase.Operator, + input colexecbase.Operator, colIdx int, constArg tree.Datum, outputIdx int, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { input = newVectorTypeEnforcer(allocator, input, outputType, outputIdx) projConstOpBase := projConstOpBase{ OneInputNode: NewOneInputNode(input), diff --git a/pkg/sql/colexec/proj_non_const_ops_tmpl.go b/pkg/sql/colexec/proj_non_const_ops_tmpl.go index ddf1191fcf1b..749ce3637f07 100644 --- a/pkg/sql/colexec/proj_non_const_ops_tmpl.go +++ b/pkg/sql/colexec/proj_non_const_ops_tmpl.go @@ -26,18 +26,20 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/pkg/errors" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Declarations to make the template compile properly. @@ -59,25 +61,25 @@ var _ duration.Duration // _ASSIGN is the template function for assigning the first input to the result // of computation an operation on the second and the third inputs. func _ASSIGN(_, _, _ interface{}) { - vecerror.InternalError("") + colexecerror.InternalError("") } // _L_UNSAFEGET is the template function that will be replaced by // "execgen.UNSAFEGET" which uses _L_TYP. func _L_UNSAFEGET(_, _ interface{}) interface{} { - vecerror.InternalError("") + colexecerror.InternalError("") } // _R_UNSAFEGET is the template function that will be replaced by // "execgen.UNSAFEGET" which uses _R_TYP. func _R_UNSAFEGET(_, _ interface{}) interface{} { - vecerror.InternalError("") + colexecerror.InternalError("") } // _RET_UNSAFEGET is the template function that will be replaced by // "execgen.UNSAFEGET" which uses _RET_TYP. func _RET_UNSAFEGET(_, _ interface{}) interface{} { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} @@ -89,7 +91,7 @@ func _RET_UNSAFEGET(_, _ interface{}) interface{} { // around the problem we specify it here. type projConstOpBase struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator colIdx int outputIdx int decimalScratch decimalOverloadScratch @@ -98,7 +100,7 @@ type projConstOpBase struct { // projOpBase contains all of the fields for non-constant binary projections. type projOpBase struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator col1Idx int col2Idx int outputIdx int @@ -227,16 +229,16 @@ func _SET_SINGLE_TUPLE_PROJECTION(_HAS_NULLS bool) { // */}} // GetProjectionOperator returns the appropriate projection operator for the // given left and right column types and operation. func GetProjectionOperator( - allocator *colbase.Allocator, + allocator *colmem.Allocator, leftType *types.T, rightType *types.T, outputType *types.T, op tree.Operator, - input colbase.Operator, + input colexecbase.Operator, col1Idx int, col2Idx int, outputIdx int, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { input = newVectorTypeEnforcer(allocator, input, outputType, outputIdx) projOpBase := projOpBase{ OneInputNode: NewOneInputNode(input), diff --git a/pkg/sql/colexec/projection_ops_test.go b/pkg/sql/colexec/projection_ops_test.go index 81772ec236db..3858d1af08d2 100644 --- a/pkg/sql/colexec/projection_ops_test.go +++ b/pkg/sql/colexec/projection_ops_test.go @@ -18,10 +18,11 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -45,7 +46,7 @@ func TestProjPlusInt64Int64ConstOp(t *testing.T) { }, } runTests(t, []tuples{{{1}, {2}, {nil}}}, tuples{{1, 2}, {2, 3}, {nil, nil}}, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return createTestProjectingOperator( ctx, flowCtx, input[0], []types.T{*types.Int}, "@1 + 1" /* projectingExpr */, false, /* canFallbackToRowexec */ @@ -67,7 +68,7 @@ func TestProjPlusInt64Int64Op(t *testing.T) { } runTests(t, []tuples{{{1, 2}, {3, 4}, {5, nil}}}, tuples{{1, 2, 3}, {3, 4, 7}, {5, nil, nil}}, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return createTestProjectingOperator( ctx, flowCtx, input[0], []types.T{*types.Int, *types.Int}, "@1 + @2" /* projectingExpr */, false, /* canFallbackToRowexec */ @@ -89,7 +90,7 @@ func TestProjDivFloat64Float64Op(t *testing.T) { } runTests(t, []tuples{{{1.0, 2.0}, {3.0, 4.0}, {5.0, nil}}}, tuples{{1.0, 2.0, 0.5}, {3.0, 4.0, 0.75}, {5.0, nil, nil}}, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return createTestProjectingOperator( ctx, flowCtx, input[0], []types.T{*types.Float, *types.Float}, "@1 / @2" /* projectingExpr */, false, /* canFallbackToRowexec */ @@ -129,7 +130,7 @@ func benchmarkProjPlusInt64Int64ConstOp(b *testing.B, useSelectionVector bool, h sel[i] = i } } - source := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) plusOp, err := createTestProjectingOperator( ctx, flowCtx, source, []types.T{*types.Int}, "@1 + 1" /* projectingExpr */, false, /* canFallbackToRowexec */ @@ -156,7 +157,7 @@ func BenchmarkProjPlusInt64Int64ConstOp(b *testing.B) { func TestGetProjectionConstOperator(t *testing.T) { defer leaktest.AfterTest(t)() binOp := tree.Mult - var input colbase.Operator + var input colexecbase.Operator colIdx := 3 constVal := 31.37 constArg := tree.NewDFloat(tree.DFloat(constVal)) @@ -185,7 +186,7 @@ func TestGetProjectionConstOperator(t *testing.T) { func TestGetProjectionConstMixedTypeOperator(t *testing.T) { defer leaktest.AfterTest(t)() binOp := tree.GE - var input colbase.Operator + var input colexecbase.Operator colIdx := 3 constVal := int16(31) constArg := tree.NewDInt(tree.DInt(constVal)) @@ -250,8 +251,8 @@ func TestRandomComparisons(t *testing.T) { lVec := b.ColVec(0) rVec := b.ColVec(1) ret := b.ColVec(2) - coldata.RandomVec(rng, typ, bytesFixedLength, lVec, numTuples, 0) - coldata.RandomVec(rng, typ, bytesFixedLength, rVec, numTuples, 0) + coldatatestutils.RandomVec(rng, typ, bytesFixedLength, lVec, numTuples, 0) + coldatatestutils.RandomVec(rng, typ, bytesFixedLength, rVec, numTuples, 0) for i := range lDatums { lDatums[i] = PhysicalTypeColElemToDatum(lVec, i, da, typ) rDatums[i] = PhysicalTypeColElemToDatum(rVec, i, da, typ) @@ -304,7 +305,7 @@ func TestGetProjectionOperator(t *testing.T) { defer leaktest.AfterTest(t)() typ := types.Int2 binOp := tree.Mult - var input colbase.Operator + var input colexecbase.Operator col1Idx := 5 col2Idx := 7 outputIdx := 9 @@ -331,7 +332,7 @@ func TestGetProjectionOperator(t *testing.T) { func benchmarkProjOp( b *testing.B, - makeProjOp func(source *colbase.RepeatableBatchSource, intWidth int32) (colbase.Operator, error), + makeProjOp func(source *colexecbase.RepeatableBatchSource, intWidth int32) (colexecbase.Operator, error), useSelectionVector bool, hasNulls bool, intType *types.T, @@ -376,7 +377,7 @@ func benchmarkProjOp( sel[i] = i } } - source := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) op, err := makeProjOp(source, intType.Width()) require.NoError(b, err) op.Init() @@ -409,26 +410,26 @@ func BenchmarkProjOp(b *testing.B) { return nil } } - projOpMap := map[string]func(*colbase.RepeatableBatchSource, int32) (colbase.Operator, error){ - "projPlusIntIntOp": func(source *colbase.RepeatableBatchSource, width int32) (colbase.Operator, error) { + projOpMap := map[string]func(*colexecbase.RepeatableBatchSource, int32) (colexecbase.Operator, error){ + "projPlusIntIntOp": func(source *colexecbase.RepeatableBatchSource, width int32) (colexecbase.Operator, error) { return createTestProjectingOperator( ctx, flowCtx, source, getInputTypesForIntWidth(width), "@1 + @2" /* projectingExpr */, false, /* canFallbackToRowexec */ ) }, - "projMinusIntIntOp": func(source *colbase.RepeatableBatchSource, width int32) (colbase.Operator, error) { + "projMinusIntIntOp": func(source *colexecbase.RepeatableBatchSource, width int32) (colexecbase.Operator, error) { return createTestProjectingOperator( ctx, flowCtx, source, getInputTypesForIntWidth(width), "@1 - @2" /* projectingExpr */, false, /* canFallbackToRowexec */ ) }, - "projMultIntIntOp": func(source *colbase.RepeatableBatchSource, width int32) (colbase.Operator, error) { + "projMultIntIntOp": func(source *colexecbase.RepeatableBatchSource, width int32) (colexecbase.Operator, error) { return createTestProjectingOperator( ctx, flowCtx, source, getInputTypesForIntWidth(width), "@1 * @2" /* projectingExpr */, false, /* canFallbackToRowexec */ ) }, - "projDivIntIntOp": func(source *colbase.RepeatableBatchSource, width int32) (colbase.Operator, error) { + "projDivIntIntOp": func(source *colexecbase.RepeatableBatchSource, width int32) (colexecbase.Operator, error) { return createTestProjectingOperator( ctx, flowCtx, source, getInputTypesForIntWidth(width), "@1 / @2" /* projectingExpr */, false, /* canFallbackToRowexec */ diff --git a/pkg/sql/colexec/rank_tmpl.go b/pkg/sql/colexec/rank_tmpl.go index 954c4fc5be0d..2533a075ec49 100644 --- a/pkg/sql/colexec/rank_tmpl.go +++ b/pkg/sql/colexec/rank_tmpl.go @@ -23,15 +23,17 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - // {{/* - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" ) +// Remove unused warning. +var _ = colexecerror.InternalError + // TODO(yuzefovich): add benchmarks. // NewRankOperator creates a new Operator that computes window functions RANK @@ -39,14 +41,14 @@ import ( // outputColIdx specifies in which coldata.Vec the operator should put its // output (if there is no such column, a new column is appended). func NewRankOperator( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, windowFn execinfrapb.WindowerSpec_WindowFunc, orderingCols []execinfrapb.Ordering_Column, outputColIdx int, partitionColIdx int, peersColIdx int, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { if len(orderingCols) == 0 { return NewConstOp(allocator, input, types.Int, int64(1), outputColIdx) } @@ -79,13 +81,13 @@ func NewRankOperator( // _UPDATE_RANK is the template function for updating the state of rank // operators. func _UPDATE_RANK() { - vecerror.InternalError("") + colexecerror.InternalError("") } // _UPDATE_RANK_INCREMENT is the template function for updating the state of // rank operators. func _UPDATE_RANK_INCREMENT() { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} @@ -93,7 +95,7 @@ func _UPDATE_RANK_INCREMENT() { type rankInitFields struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator outputColIdx int partitionColIdx int peersColIdx int @@ -111,7 +113,7 @@ type _RANK_STRINGOp struct { rankIncrement int64 } -var _ colbase.Operator = &_RANK_STRINGOp{} +var _ colexecbase.Operator = &_RANK_STRINGOp{} func (r *_RANK_STRINGOp) Init() { r.Input().Init() diff --git a/pkg/sql/colexec/relative_rank_tmpl.go b/pkg/sql/colexec/relative_rank_tmpl.go index cfd85e0cfd0c..126d773108e3 100644 --- a/pkg/sql/colexec/relative_rank_tmpl.go +++ b/pkg/sql/colexec/relative_rank_tmpl.go @@ -23,10 +23,11 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -42,11 +43,11 @@ import ( // outputColIdx specifies in which coldata.Vec the operator should put its // output (if there is no such column, a new column is appended). func NewRelativeRankOperator( - unlimitedAllocator *colbase.Allocator, + unlimitedAllocator *colmem.Allocator, memoryLimit int64, diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, - input colbase.Operator, + input colexecbase.Operator, inputTypes []types.T, windowFn execinfrapb.WindowerSpec_WindowFunc, orderingCols []execinfrapb.Ordering_Column, @@ -54,7 +55,7 @@ func NewRelativeRankOperator( partitionColIdx int, peersColIdx int, diskAcc *mon.BoundAccount, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { if len(orderingCols) == 0 { constValue := float64(0) if windowFn == execinfrapb.WindowerSpec_CUME_DIST { @@ -155,7 +156,7 @@ func _COMPUTE_PARTITIONS_SIZES() { // */}} // We need to flush the vector of partitions sizes. r.partitionsState.runningSizes.SetLength(coldata.BatchSize()) if err := r.partitionsState.enqueue(ctx, r.partitionsState.runningSizes); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } r.partitionsState.runningSizes = nil r.partitionsState.idx = 0 @@ -193,7 +194,7 @@ func _COMPUTE_PEER_GROUPS_SIZES() { // */}} // We need to flush the vector of peer group sizes. r.peerGroupsState.runningSizes.SetLength(coldata.BatchSize()) if err := r.peerGroupsState.enqueue(ctx, r.peerGroupsState.runningSizes); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } r.peerGroupsState.runningSizes = nil r.peerGroupsState.idx = 0 @@ -357,7 +358,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { n := batch.Length() if n == 0 { if err := r.bufferedTuples.enqueue(ctx, coldata.ZeroBatch); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } // {{if .HasPartition}} // We need to flush the last vector of the running partitions @@ -373,10 +374,10 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { r.partitionsState.idx++ r.partitionsState.runningSizes.SetLength(r.partitionsState.idx) if err := r.partitionsState.enqueue(ctx, r.partitionsState.runningSizes); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } if err := r.partitionsState.enqueue(ctx, coldata.ZeroBatch); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } // {{end}} // {{if .IsCumeDist}} @@ -393,10 +394,10 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { r.peerGroupsState.idx++ r.peerGroupsState.runningSizes.SetLength(r.peerGroupsState.idx) if err := r.peerGroupsState.enqueue(ctx, r.peerGroupsState.runningSizes); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } if err := r.peerGroupsState.enqueue(ctx, coldata.ZeroBatch); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } // {{end}} // We have fully consumed the input, so now we can populate the output. @@ -434,7 +435,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { r.scratch.SetLength(n) }) if err := r.bufferedTuples.enqueue(ctx, r.scratch); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } // Then, we need to update the sizes of the partitions. @@ -479,7 +480,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { case relativeRankEmitting: if r.scratch, err = r.bufferedTuples.dequeue(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } n := r.scratch.Length() if n == 0 { @@ -490,7 +491,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { // Get the next batch of partition sizes if we haven't already. if r.partitionsState.dequeuedSizes == nil { if r.partitionsState.dequeuedSizes, err = r.partitionsState.dequeue(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } r.partitionsState.idx = 0 r.numTuplesInPartition = 0 @@ -500,7 +501,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { // Get the next batch of peer group sizes if we haven't already. if r.peerGroupsState.dequeuedSizes == nil { if r.peerGroupsState.dequeuedSizes, err = r.peerGroupsState.dequeue(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } r.peerGroupsState.idx = 0 r.numPeers = 0 @@ -538,7 +539,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { if partitionCol[i] { if r.partitionsState.idx == r.partitionsState.dequeuedSizes.Length() { if r.partitionsState.dequeuedSizes, err = r.partitionsState.dequeue(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } r.partitionsState.idx = 0 } @@ -574,7 +575,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { r.numPrecedingTuples += r.numPeers if r.peerGroupsState.idx == r.peerGroupsState.dequeuedSizes.Length() { if r.peerGroupsState.dequeuedSizes, err = r.peerGroupsState.dequeue(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } r.peerGroupsState.idx = 0 } @@ -603,12 +604,12 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { case relativeRankFinished: if err := r.idempotentCloseLocked(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return coldata.ZeroBatch default: - vecerror.InternalError("percent rank operator in unhandled state") + colexecerror.InternalError("percent rank operator in unhandled state") // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/routers.go b/pkg/sql/colexec/routers.go index f5a3b4c05710..12ec56fa7a41 100644 --- a/pkg/sql/colexec/routers.go +++ b/pkg/sql/colexec/routers.go @@ -16,10 +16,11 @@ import ( "sync" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -108,7 +109,7 @@ type routerOutputOp struct { // In short, batches whose references are retained are also retained in the // allocator, but if any references are overwritten or lost, those batches // are released. - unlimitedAllocator *colbase.Allocator + unlimitedAllocator *colmem.Allocator cond *sync.Cond // pendingBatch is a partially-filled batch with data added through // addBatch. Once this batch reaches capacity, it is flushed to data. The @@ -154,12 +155,12 @@ func (o *routerOutputOp) Child(nth int, verbose bool) execinfra.OpNode { if nth == 0 { return o.input } - vecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } -var _ colbase.Operator = &routerOutputOp{} +var _ colexecbase.Operator = &routerOutputOp{} // newRouterOutputOp creates a new router output. The caller must ensure that // unblockedEventsChan is a buffered channel, as the router output will write to @@ -167,7 +168,7 @@ var _ colbase.Operator = &routerOutputOp{} // memoryLimit will act as a soft limit to allow the router output to use disk // when it is exceeded. func newRouterOutputOp( - unlimitedAllocator *colbase.Allocator, + unlimitedAllocator *colmem.Allocator, types []types.T, unblockedEventsChan chan<- struct{}, memoryLimit int64, @@ -179,7 +180,7 @@ func newRouterOutputOp( } func newRouterOutputOpWithBlockedThresholdAndBatchSize( - unlimitedAllocator *colbase.Allocator, + unlimitedAllocator *colmem.Allocator, types []types.T, unblockedEventsChan chan<- struct{}, memoryLimit int64, @@ -295,12 +296,12 @@ func (o *routerOutputOp) addBatch(ctx context.Context, batch coldata.Batch, sele // We set the error to nil so that it is not propagated again, during // drain() call. o.mu.drainState.err = nil - vecerror.InternalError(err) + colexecerror.InternalError(err) } if batch.Length() == 0 { if o.mu.pendingBatch != nil { if err := o.mu.data.enqueue(ctx, o.mu.pendingBatch); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } } o.mu.pendingBatch = coldata.ZeroBatch @@ -346,7 +347,7 @@ func (o *routerOutputOp) addBatch(ctx context.Context, batch coldata.Batch, sele if o.testingKnobs.alwaysFlush || newLength >= o.outputBatchSize { // The capacity in o.mu.pendingBatch has been filled. if err := o.mu.data.enqueue(ctx, o.mu.pendingBatch); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } o.mu.pendingBatch = nil } @@ -436,20 +437,20 @@ type HashRouter struct { // be called concurrently between different outputs. Similarly, each output // needs to have a separate disk account. func NewHashRouter( - unlimitedAllocators []*colbase.Allocator, - input colbase.Operator, + unlimitedAllocators []*colmem.Allocator, + input colexecbase.Operator, types []types.T, hashCols []uint32, memoryLimit int64, diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, diskAccounts []*mon.BoundAccount, -) (*HashRouter, []colbase.Operator) { +) (*HashRouter, []colexecbase.Operator) { if diskQueueCfg.CacheMode != colcontainer.DiskQueueCacheModeDefault { - vecerror.InternalError(errors.Errorf("hash router instantiated with incompatible disk queue cache mode: %d", diskQueueCfg.CacheMode)) + colexecerror.InternalError(errors.Errorf("hash router instantiated with incompatible disk queue cache mode: %d", diskQueueCfg.CacheMode)) } outputs := make([]routerOutput, len(unlimitedAllocators)) - outputsAsOps := make([]colbase.Operator, len(unlimitedAllocators)) + outputsAsOps := make([]colexecbase.Operator, 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 @@ -472,7 +473,7 @@ func NewHashRouter( } func newHashRouterWithOutputs( - input colbase.Operator, + input colexecbase.Operator, types []types.T, hashCols []uint32, unblockEventsChan <-chan struct{}, @@ -502,7 +503,7 @@ func (r *HashRouter) Run(ctx context.Context) { // make sure that we catch errors in all code paths, so we wrap the whole // method with a catcher. Note that we also have "internal" catchers as // well for more fine-grained control of error propagation. - if err := vecerror.CatchVectorizedRuntimeError(func() { + if err := colexecerror.CatchVectorizedRuntimeError(func() { r.input.Init() // cancelOutputs buffers non-nil error as metadata, cancels all of the // outputs additionally buffering any error if such occurs during the @@ -514,7 +515,7 @@ func (r *HashRouter) Run(ctx context.Context) { bufferErr(err) } for _, o := range r.outputs { - if err := vecerror.CatchVectorizedRuntimeError(func() { + if err := colexecerror.CatchVectorizedRuntimeError(func() { o.cancel(ctx) }); err != nil { bufferErr(err) @@ -557,7 +558,7 @@ func (r *HashRouter) Run(ctx context.Context) { } } - if err := vecerror.CatchVectorizedRuntimeError(processNextBatch); err != nil { + if err := colexecerror.CatchVectorizedRuntimeError(processNextBatch); err != nil { cancelOutputs(err) return } diff --git a/pkg/sql/colexec/routers_test.go b/pkg/sql/colexec/routers_test.go index 7eaa93e4b47e..41d34a5f5685 100644 --- a/pkg/sql/colexec/routers_test.go +++ b/pkg/sql/colexec/routers_test.go @@ -20,8 +20,10 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "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/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -142,7 +144,7 @@ func TestRouterOutputAddBatch(t *testing.T) { t.Run(fmt.Sprintf("%s/memoryLimit=%s", tc.name, humanizeutil.IBytes(mtc.bytes)), func(t *testing.T) { // Clear the testAllocator for use. testAllocator.ReleaseMemory(testAllocator.Used()) - o := newRouterOutputOpWithBlockedThresholdAndBatchSize(testAllocator, typs, unblockEventsChan, mtc.bytes, queueCfg, colbase.NewTestingSemaphore(2), tc.blockedThreshold, tc.outputBatchSize, testDiskAcc) + o := newRouterOutputOpWithBlockedThresholdAndBatchSize(testAllocator, typs, unblockEventsChan, mtc.bytes, queueCfg, colexecbase.NewTestingSemaphore(2), tc.blockedThreshold, tc.outputBatchSize, testDiskAcc) in := newOpTestInput(tc.inputBatchSize, data, nil /* typs */) out := newOpTestOutput(o, data[:len(tc.selection)]) in.Init() @@ -180,14 +182,14 @@ func TestRouterOutputNext(t *testing.T) { data, typs, fullSelection := getDataAndFullSelection() testCases := []struct { - unblockEvent func(in colbase.Operator, o *routerOutputOp) + unblockEvent func(in colexecbase.Operator, o *routerOutputOp) expected tuples name string }{ { // ReaderWaitsForData verifies that a reader blocks in Next(ctx) until there // is data available. - unblockEvent: func(in colbase.Operator, o *routerOutputOp) { + unblockEvent: func(in colexecbase.Operator, o *routerOutputOp) { for { b := in.Next(ctx) o.addBatch(ctx, b, fullSelection) @@ -202,7 +204,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(_ colbase.Operator, o *routerOutputOp) { + unblockEvent: func(_ colexecbase.Operator, o *routerOutputOp) { o.addBatch(ctx, coldata.ZeroBatch, nil /* selection */) }, expected: tuples{}, @@ -211,7 +213,7 @@ func TestRouterOutputNext(t *testing.T) { { // CancelUnblocksReader verifies that calling cancel on an output unblocks // a reader. - unblockEvent: func(_ colbase.Operator, o *routerOutputOp) { + unblockEvent: func(_ colexecbase.Operator, o *routerOutputOp) { o.cancel(ctx) }, expected: tuples{}, @@ -235,7 +237,7 @@ func TestRouterOutputNext(t *testing.T) { if queueCfg.FS == nil { t.Fatal("FS was nil") } - o := newRouterOutputOp(testAllocator, typs, unblockedEventsChan, mtc.bytes, queueCfg, colbase.NewTestingSemaphore(2), testDiskAcc) + o := newRouterOutputOp(testAllocator, typs, unblockedEventsChan, mtc.bytes, queueCfg, colexecbase.NewTestingSemaphore(2), testDiskAcc) in := newOpTestInput(coldata.BatchSize(), data, nil /* typs */) in.Init() wg.Add(1) @@ -262,7 +264,7 @@ func TestRouterOutputNext(t *testing.T) { tc.unblockEvent(in, o) // Should have data available, pushed by our reader goroutine. - batches := colbase.NewBatchBuffer() + batches := colexecbase.NewBatchBuffer() out := newOpTestOutput(batches, tc.expected) for { b := <-batchChan @@ -285,7 +287,7 @@ func TestRouterOutputNext(t *testing.T) { } t.Run(fmt.Sprintf("NextAfterZeroBatchDoesntBlock/memoryLimit=%s", humanizeutil.IBytes(mtc.bytes)), func(t *testing.T) { - o := newRouterOutputOp(testAllocator, typs, unblockedEventsChan, mtc.bytes, queueCfg, colbase.NewTestingSemaphore(2), testDiskAcc) + o := newRouterOutputOp(testAllocator, typs, unblockedEventsChan, mtc.bytes, queueCfg, colexecbase.NewTestingSemaphore(2), testDiskAcc) o.addBatch(ctx, coldata.ZeroBatch, fullSelection) o.Next(ctx) o.Next(ctx) @@ -324,7 +326,7 @@ func TestRouterOutputNext(t *testing.T) { } ch := make(chan struct{}, 2) - o := newRouterOutputOpWithBlockedThresholdAndBatchSize(testAllocator, typs, ch, mtc.bytes, queueCfg, colbase.NewTestingSemaphore(2), blockThreshold, coldata.BatchSize(), testDiskAcc) + o := newRouterOutputOpWithBlockedThresholdAndBatchSize(testAllocator, typs, ch, mtc.bytes, queueCfg, colexecbase.NewTestingSemaphore(2), blockThreshold, coldata.BatchSize(), testDiskAcc) in := newOpTestInput(smallBatchSize, data, nil /* typs */) out := newOpTestOutput(o, expected) in.Init() @@ -395,10 +397,10 @@ 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) { - runTestsWithFn(t, []tuples{data}, nil /* typs */, func(t *testing.T, inputs []colbase.Operator) { + runTestsWithFn(t, []tuples{data}, nil /* typs */, func(t *testing.T, inputs []colexecbase.Operator) { var wg sync.WaitGroup unblockedEventsChans := make(chan struct{}, 2) - o := newRouterOutputOpWithBlockedThresholdAndBatchSize(testAllocator, typs, unblockedEventsChans, mtc.bytes, queueCfg, colbase.NewTestingSemaphore(2), blockedThreshold, outputSize, testDiskAcc) + o := newRouterOutputOpWithBlockedThresholdAndBatchSize(testAllocator, typs, unblockedEventsChans, mtc.bytes, queueCfg, colexecbase.NewTestingSemaphore(2), blockedThreshold, outputSize, testDiskAcc) inputs[0].Init() expected := make(tuples, 0, len(data)) @@ -417,7 +419,7 @@ func TestRouterOutputRandom(t *testing.T) { b := inputs[0].Next(ctx) selection := b.Selection() if selection == nil { - selection = colbase.RandomSel(rng, b.Length(), rng.Float64()) + selection = coldatatestutils.RandomSel(rng, b.Length(), rng.Float64()) } selection = selection[:b.Length()] @@ -468,20 +470,14 @@ func TestRouterOutputRandom(t *testing.T) { } }() - actual := colbase.NewBatchBuffer() + actual := colexecbase.NewBatchBuffer() // Consumer. wg.Add(1) go func() { - // Create a new allocator to copy the resulting batches. We need - // a separate allocator to testAllocator since this is a separate - // goroutine and allocators may not be used concurrently. - acc := testMemMonitor.MakeBoundAccount() - allocator := colbase.NewAllocator(ctx, &acc) - defer acc.Close(ctx) for { b := o.Next(ctx) - actual.Add(colbase.CopyBatch(allocator, b, typs), typs) + actual.Add(coldatatestutils.CopyBatch(b, typs), typs) if b.Length() == 0 { wg.Done() return @@ -508,7 +504,7 @@ func TestRouterOutputRandom(t *testing.T) { } type callbackRouterOutput struct { - colbase.ZeroInputNode + colexecbase.ZeroInputNode addBatchCb func(coldata.Batch, []int) bool cancelCb func() } @@ -628,7 +624,7 @@ func TestHashRouterCancellation(t *testing.T) { // Never-ending input of 0s. batch := testAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) - in := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + in := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) unbufferedCh := make(chan struct{}) r := newHashRouterWithOutputs(in, typs, []uint32{0}, unbufferedCh, outputs) @@ -716,7 +712,7 @@ func TestHashRouterOneOutput(t *testing.T) { rng, _ := randutil.NewPseudoRand() - sel := colbase.RandomSel(rng, coldata.BatchSize(), rng.Float64()) + sel := coldatatestutils.RandomSel(rng, coldata.BatchSize(), rng.Float64()) data, typs, _ := getDataAndFullSelection() @@ -735,8 +731,8 @@ func TestHashRouterOneOutput(t *testing.T) { diskAcc := testDiskMonitor.MakeBoundAccount() defer diskAcc.Close(ctx) r, routerOutputs := NewHashRouter( - []*colbase.Allocator{testAllocator}, newOpFixedSelTestInput(sel, len(sel), data), - typs, []uint32{0}, mtc.bytes, queueCfg, colbase.NewTestingSemaphore(2), + []*colmem.Allocator{testAllocator}, newOpFixedSelTestInput(sel, len(sel), data), + typs, []uint32{0}, mtc.bytes, queueCfg, colexecbase.NewTestingSemaphore(2), []*mon.BoundAccount{&diskAcc}, ) @@ -833,10 +829,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) { - runTestsWithFn(t, []tuples{data}, nil /* typs */, func(t *testing.T, inputs []colbase.Operator) { + runTestsWithFn(t, []tuples{data}, nil /* typs */, func(t *testing.T, inputs []colexecbase.Operator) { unblockEventsChan := make(chan struct{}, 2*numOutputs) outputs := make([]routerOutput, numOutputs) - outputsAsOps := make([]colbase.Operator, numOutputs) + outputsAsOps := make([]colexecbase.Operator, numOutputs) memoryLimitPerOutput := mtc.bytes / int64(len(outputs)) for i := range outputs { // Create separate monitoring infrastructure as well as @@ -846,8 +842,8 @@ func TestHashRouterRandom(t *testing.T) { defer acc.Close(ctx) diskAcc := testDiskMonitor.MakeBoundAccount() defer diskAcc.Close(ctx) - allocator := colbase.NewAllocator(ctx, &acc) - op := newRouterOutputOpWithBlockedThresholdAndBatchSize(allocator, typs, unblockEventsChan, memoryLimitPerOutput, queueCfg, colbase.NewTestingSemaphore(len(outputs)*2), blockedThreshold, outputSize, &diskAcc) + allocator := colmem.NewAllocator(ctx, &acc) + op := newRouterOutputOpWithBlockedThresholdAndBatchSize(allocator, typs, unblockEventsChan, memoryLimitPerOutput, queueCfg, colexecbase.NewTestingSemaphore(len(outputs)*2), blockedThreshold, outputSize, &diskAcc) outputs[i] = op outputsAsOps[i] = op } @@ -934,7 +930,7 @@ func BenchmarkHashRouter(b *testing.B) { typs := []types.T{*types.Int} batch := testAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) - input := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(b, true /* inMem */) defer cleanup() @@ -943,17 +939,17 @@ func BenchmarkHashRouter(b *testing.B) { for _, numOutputs := range []int{2, 4, 8, 16} { for _, numInputBatches := range []int{2, 4, 8, 16} { b.Run(fmt.Sprintf("numOutputs=%d/numInputBatches=%d", numOutputs, numInputBatches), func(b *testing.B) { - allocators := make([]*colbase.Allocator, numOutputs) + allocators := make([]*colmem.Allocator, numOutputs) diskAccounts := make([]*mon.BoundAccount, numOutputs) for i := range allocators { acc := testMemMonitor.MakeBoundAccount() - allocators[i] = colbase.NewAllocator(ctx, &acc) + allocators[i] = colmem.NewAllocator(ctx, &acc) defer acc.Close(ctx) diskAcc := testDiskMonitor.MakeBoundAccount() diskAccounts[i] = &diskAcc defer diskAcc.Close(ctx) } - r, outputs := NewHashRouter(allocators, input, typs, []uint32{0}, 64<<20, queueCfg, &colbase.TestingSemaphore{}, diskAccounts) + r, outputs := NewHashRouter(allocators, input, typs, []uint32{0}, 64<<20, queueCfg, &colexecbase.TestingSemaphore{}, diskAccounts) 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/colexec/row_number_tmpl.go b/pkg/sql/colexec/row_number_tmpl.go index 555b245bc231..b080ceec7baf 100644 --- a/pkg/sql/colexec/row_number_tmpl.go +++ b/pkg/sql/colexec/row_number_tmpl.go @@ -23,7 +23,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -33,8 +34,8 @@ 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 *colbase.Allocator, input colbase.Operator, outputColIdx int, partitionColIdx int, -) colbase.Operator { + allocator *colmem.Allocator, input colexecbase.Operator, outputColIdx int, partitionColIdx int, +) colexecbase.Operator { input = newVectorTypeEnforcer(allocator, input, types.Int, outputColIdx) base := rowNumberBase{ OneInputNode: NewOneInputNode(input), @@ -53,7 +54,7 @@ func NewRowNumberOperator( // and should not be used directly. type rowNumberBase struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator outputColIdx int partitionColIdx int @@ -70,7 +71,7 @@ type _ROW_NUMBER_STRINGOp struct { rowNumberBase } -var _ colbase.Operator = &_ROW_NUMBER_STRINGOp{} +var _ colexecbase.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/rowstovec_tmpl.go b/pkg/sql/colexec/rowstovec_tmpl.go index 28cb26c86496..bbcea313c480 100644 --- a/pkg/sql/colexec/rowstovec_tmpl.go +++ b/pkg/sql/colexec/rowstovec_tmpl.go @@ -25,17 +25,18 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // {{/* "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Dummy import to pull in "apd" package. @@ -89,7 +90,7 @@ func _ROWS_TO_COL_VEC( // EncDatumRowsToColVec converts one column from EncDatumRows to a column // vector. columnIdx is the 0-based index of the column in the EncDatumRows. func EncDatumRowsToColVec( - allocator *colbase.Allocator, + allocator *colmem.Allocator, rows sqlbase.EncDatumRows, vec coldata.Vec, columnIdx int, @@ -110,14 +111,14 @@ func EncDatumRowsToColVec( _ROWS_TO_COL_VEC(rows, vec, columnIdx, typ, alloc) // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unsupported width %d for type %s", typ.Width(), typ.String())) + colexecerror.InternalError(fmt.Sprintf("unsupported width %d for type %s", typ.Width(), typ.String())) } // {{ else }} _ROWS_TO_COL_VEC(rows, vec, columnIdx, typ, alloc) // {{end}} // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unsupported type %s", typ.String())) + colexecerror.InternalError(fmt.Sprintf("unsupported type %s", typ.String())) } }, ) diff --git a/pkg/sql/colexec/select_in_test.go b/pkg/sql/colexec/select_in_test.go index 4be5287155f2..93ae730b0af0 100644 --- a/pkg/sql/colexec/select_in_test.go +++ b/pkg/sql/colexec/select_in_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/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -74,7 +74,7 @@ func TestSelectInInt64(t *testing.T) { for _, c := range testCases { t.Run(c.desc, func(t *testing.T) { - opConstructor := func(input []colbase.Operator) (colbase.Operator, error) { + opConstructor := func(input []colexecbase.Operator) (colexecbase.Operator, error) { op := selectInOpInt64{ OneInputNode: NewOneInputNode(input[0]), colIdx: 0, @@ -131,7 +131,7 @@ func benchmarkSelectInInt64(b *testing.B, useSelectionVector bool, hasNulls bool } } - source := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) source.Init() inOp := &selectInOpInt64{ OneInputNode: NewOneInputNode(source), @@ -216,7 +216,7 @@ func TestProjectInInt64(t *testing.T) { for _, c := range testCases { t.Run(c.desc, func(t *testing.T) { runTests(t, []tuples{c.inputTuples}, c.outputTuples, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { expr, err := parser.ParseExpr(fmt.Sprintf("@1 %s", c.inClause)) if err != nil { return nil, err diff --git a/pkg/sql/colexec/select_in_tmpl.go b/pkg/sql/colexec/select_in_tmpl.go index d6a373d4b981..4c354bf0c0f6 100644 --- a/pkg/sql/colexec/select_in_tmpl.go +++ b/pkg/sql/colexec/select_in_tmpl.go @@ -28,18 +28,23 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - // {{/* - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/pkg/errors" ) +// Remove unused warnings. +var ( + _ = execgen.UNSAFEGET + _ = colexecerror.InternalError +) + // {{/* type _GOTYPE interface{} @@ -64,7 +69,7 @@ var _ bytes.Buffer var _ = math.MaxInt64 func _ASSIGN_EQ(_, _, _ interface{}) int { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} @@ -79,14 +84,14 @@ const ( ) func GetInProjectionOperator( - allocator *colbase.Allocator, + allocator *colmem.Allocator, typ *types.T, - input colbase.Operator, + input colexecbase.Operator, colIdx int, resultIdx int, datumTuple *tree.DTuple, negate bool, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { input = newVectorTypeEnforcer(allocator, input, types.Bool, resultIdx) var err error switch typeconv.FromColumnType(typ) { @@ -111,8 +116,8 @@ func GetInProjectionOperator( } func GetInOperator( - typ *types.T, input colbase.Operator, colIdx int, datumTuple *tree.DTuple, negate bool, -) (colbase.Operator, error) { + typ *types.T, input colexecbase.Operator, colIdx int, datumTuple *tree.DTuple, negate bool, +) (colexecbase.Operator, error) { var err error switch typeconv.FromColumnType(typ) { // {{range .}} @@ -145,7 +150,7 @@ type selectInOp_TYPE struct { type projectInOp_TYPE struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator colIdx int outputIdx int filterRow []_GOTYPE @@ -153,7 +158,7 @@ type projectInOp_TYPE struct { negate bool } -var _ colbase.Operator = &projectInOp_TYPE{} +var _ colexecbase.Operator = &projectInOp_TYPE{} func fillDatumRow_TYPE(typ *types.T, datumTuple *tree.DTuple) ([]_GOTYPE, bool, error) { conv := getDatumToPhysicalFn(typ) diff --git a/pkg/sql/colexec/selection_ops_test.go b/pkg/sql/colexec/selection_ops_test.go index f9ab129ab560..842958a4eb62 100644 --- a/pkg/sql/colexec/selection_ops_test.go +++ b/pkg/sql/colexec/selection_ops_test.go @@ -18,7 +18,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -33,7 +33,7 @@ const ( func TestSelLTInt64Int64ConstOp(t *testing.T) { defer leaktest.AfterTest(t)() tups := tuples{{0}, {1}, {2}, {nil}} - runTests(t, []tuples{tups}, tuples{{0}, {1}}, orderedVerifier, func(input []colbase.Operator) (colbase.Operator, error) { + runTests(t, []tuples{tups}, tuples{{0}, {1}}, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { return &selLTInt64Int64ConstOp{ selConstOpBase: selConstOpBase{ OneInputNode: NewOneInputNode(input[0]), @@ -55,7 +55,7 @@ func TestSelLTInt64Int64(t *testing.T) { {-1, nil}, {nil, nil}, } - runTests(t, []tuples{tups}, tuples{{0, 1}}, orderedVerifier, func(input []colbase.Operator) (colbase.Operator, error) { + runTests(t, []tuples{tups}, tuples{{0, 1}}, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { return &selLTInt64Int64Op{ selOpBase: selOpBase{ OneInputNode: NewOneInputNode(input[0]), @@ -69,7 +69,7 @@ func TestSelLTInt64Int64(t *testing.T) { func TestGetSelectionConstOperator(t *testing.T) { defer leaktest.AfterTest(t)() cmpOp := tree.LT - var input colbase.Operator + var input colexecbase.Operator colIdx := 3 constVal := int64(31) constArg := tree.NewDDate(pgdate.MakeCompatibleDateFromDisk(constVal)) @@ -92,7 +92,7 @@ func TestGetSelectionConstOperator(t *testing.T) { func TestGetSelectionConstMixedTypeOperator(t *testing.T) { defer leaktest.AfterTest(t)() cmpOp := tree.LT - var input colbase.Operator + var input colexecbase.Operator colIdx := 3 constVal := int16(31) constArg := tree.NewDInt(tree.DInt(constVal)) @@ -116,7 +116,7 @@ func TestGetSelectionOperator(t *testing.T) { defer leaktest.AfterTest(t)() ct := types.Int2 cmpOp := tree.GE - var input colbase.Operator + var input colexecbase.Operator col1Idx := 5 col2Idx := 7 op, err := GetSelectionOperator(ct, ct, cmpOp, input, col1Idx, col2Idx) @@ -163,7 +163,7 @@ func benchmarkSelLTInt64Int64ConstOp(b *testing.B, useSelectionVector bool, hasN sel[i] = i } } - source := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) source.Init() plusOp := &selLTInt64Int64ConstOp{ @@ -224,7 +224,7 @@ func benchmarkSelLTInt64Int64Op(b *testing.B, useSelectionVector bool, hasNulls sel[i] = i } } - source := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) source.Init() plusOp := &selLTInt64Int64Op{ diff --git a/pkg/sql/colexec/selection_ops_tmpl.go b/pkg/sql/colexec/selection_ops_tmpl.go index 9639bc6cf7cc..a145a2f3b625 100644 --- a/pkg/sql/colexec/selection_ops_tmpl.go +++ b/pkg/sql/colexec/selection_ops_tmpl.go @@ -28,18 +28,19 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/pkg/errors" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Declarations to make the template compile properly. @@ -67,7 +68,7 @@ var _ coltypes.T // _ASSIGN_CMP is the template function for assigning the result of comparing // the second input to the third input into the first input. func _ASSIGN_CMP(_, _, _ interface{}) int { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} @@ -293,10 +294,10 @@ func GetSelectionConstOperator( leftType *types.T, constType *types.T, cmpOp tree.ComparisonOperator, - input colbase.Operator, + input colexecbase.Operator, colIdx int, constArg tree.Datum, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { c, err := getDatumToPhysicalFn(constType)(constArg) if err != nil { return nil, err @@ -335,10 +336,10 @@ func GetSelectionOperator( leftType *types.T, rightType *types.T, cmpOp tree.ComparisonOperator, - input colbase.Operator, + input colexecbase.Operator, col1Idx int, col2Idx int, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { selOpBase := selOpBase{ OneInputNode: NewOneInputNode(input), col1Idx: col1Idx, diff --git a/pkg/sql/colexec/serial_unordered_synchronizer.go b/pkg/sql/colexec/serial_unordered_synchronizer.go index 4673abcb9b90..f48595aa8e9a 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/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -26,12 +26,12 @@ import ( // undesirable - for example when the whole query is planned on the gateway and // we want to run it in the RootTxn. type SerialUnorderedSynchronizer struct { - inputs []colbase.Operator + inputs []colexecbase.Operator // curSerialInputIdx indicates the index of the current input being consumed. curSerialInputIdx int } -var _ colbase.Operator = &SerialUnorderedSynchronizer{} +var _ colexecbase.Operator = &SerialUnorderedSynchronizer{} var _ execinfra.OpNode = &SerialUnorderedSynchronizer{} // ChildCount implements the execinfra.OpNode interface. @@ -46,7 +46,7 @@ func (s *SerialUnorderedSynchronizer) Child(nth int, verbose bool) execinfra.OpN // NewSerialUnorderedSynchronizer creates a new SerialUnorderedSynchronizer. func NewSerialUnorderedSynchronizer( - inputs []colbase.Operator, typs []types.T, + inputs []colexecbase.Operator, typs []types.T, ) *SerialUnorderedSynchronizer { return &SerialUnorderedSynchronizer{ inputs: inputs, diff --git a/pkg/sql/colexec/serial_unordered_synchronizer_test.go b/pkg/sql/colexec/serial_unordered_synchronizer_test.go index 1e12d92973f9..8e8887abf6ef 100644 --- a/pkg/sql/colexec/serial_unordered_synchronizer_test.go +++ b/pkg/sql/colexec/serial_unordered_synchronizer_test.go @@ -15,7 +15,8 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -31,10 +32,10 @@ func TestSerialUnorderedSynchronizer(t *testing.T) { const numBatches = 4 typs := []types.T{*types.Int} - inputs := make([]colbase.Operator, numInputs) + inputs := make([]colexecbase.Operator, numInputs) for i := range inputs { - batch := colbase.RandomBatch(testAllocator, rng, typs, coldata.BatchSize(), 0 /* length */, rng.Float64()) - source := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + batch := coldatatestutils.RandomBatch(testAllocator, rng, typs, coldata.BatchSize(), 0 /* length */, rng.Float64()) + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) source.ResetBatchesToReturn(numBatches) inputs[i] = source } diff --git a/pkg/sql/colexec/simple_project.go b/pkg/sql/colexec/simple_project.go index 6323f1936cd3..36bbac23a55c 100644 --- a/pkg/sql/colexec/simple_project.go +++ b/pkg/sql/colexec/simple_project.go @@ -14,7 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -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 colbase.Operator, numInputCols int, projection []uint32, -) colbase.Operator { + input colexecbase.Operator, numInputCols int, projection []uint32, +) colexecbase.Operator { if numInputCols == len(projection) { projectionIsRedundant := true for i := range projection { diff --git a/pkg/sql/colexec/simple_project_test.go b/pkg/sql/colexec/simple_project_test.go index aaafd93f94dd..4123fe30faf6 100644 --- a/pkg/sql/colexec/simple_project_test.go +++ b/pkg/sql/colexec/simple_project_test.go @@ -14,7 +14,7 @@ import ( "sync" "testing" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/stretchr/testify/require" @@ -62,7 +62,7 @@ func TestSimpleProjectOp(t *testing.T) { }, } for _, tc := range tcs { - runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, func(input []colbase.Operator) (colbase.Operator, error) { + runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { return NewSimpleProjectOp(input[0], len(tc.tuples[0]), tc.colsToKeep), nil }) } @@ -70,7 +70,7 @@ func TestSimpleProjectOp(t *testing.T) { // Empty projection. The all nulls injection test case will also return // nothing. runTestsWithoutAllNullsInjection(t, []tuples{{{1, 2, 3}, {1, 2, 3}}}, nil /* typs */, tuples{{}, {}}, orderedVerifier, - func(input []colbase.Operator) (colbase.Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return NewSimpleProjectOp(input[0], 3 /* numInputCols */, nil), nil }) @@ -108,8 +108,8 @@ func TestSimpleProjectOpWithUnorderedSynchronizer(t *testing.T) { {"bb", constVal}, } runTestsWithoutAllNullsInjection(t, inputTuples, [][]types.T{inputTypes, inputTypes}, expected, - unorderedVerifier, func(inputs []colbase.Operator) (colbase.Operator, error) { - var input colbase.Operator + unorderedVerifier, func(inputs []colexecbase.Operator) (colexecbase.Operator, error) { + var input colexecbase.Operator input = NewParallelUnorderedSynchronizer(inputs, inputTypes, &wg) input = NewSimpleProjectOp(input, len(inputTypes), []uint32{0}) return NewConstOp(testAllocator, input, types.Int, constVal, 1) diff --git a/pkg/sql/colexec/sort.go b/pkg/sql/colexec/sort.go index 7a5c1a79e474..c88214037a05 100644 --- a/pkg/sql/colexec/sort.go +++ b/pkg/sql/colexec/sort.go @@ -15,9 +15,10 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -28,16 +29,16 @@ import ( // given in orderingCols. The inputTypes must correspond 1-1 with the columns // in the input operator. func NewSorter( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { return newSorter(allocator, newAllSpooler(allocator, input, inputTypes), inputTypes, orderingCols) } func newSorter( - allocator *colbase.Allocator, + allocator *colmem.Allocator, input spooler, inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column, @@ -101,7 +102,7 @@ type allSpooler struct { OneInputNode NonExplainable - allocator *colbase.Allocator + allocator *colmem.Allocator // inputTypes contains the types of all of the columns from the input. inputTypes []types.T // bufferedTuples stores all the values from the input after spooling. Each @@ -116,7 +117,7 @@ var _ spooler = &allSpooler{} var _ resetter = &allSpooler{} func newAllSpooler( - allocator *colbase.Allocator, input colbase.Operator, inputTypes []types.T, + allocator *colmem.Allocator, input colexecbase.Operator, inputTypes []types.T, ) spooler { return &allSpooler{ OneInputNode: NewOneInputNode(input), @@ -135,7 +136,7 @@ func (p *allSpooler) init() { func (p *allSpooler) spool(ctx context.Context) { if p.spooled { - vecerror.InternalError("spool() is called for the second time") + colexecerror.InternalError("spool() is called for the second time") } p.spooled = true for batch := p.input.Next(ctx); batch.Length() != 0; batch = p.input.Next(ctx) { @@ -147,7 +148,7 @@ func (p *allSpooler) spool(ctx context.Context) { func (p *allSpooler) getValues(i int) coldata.Vec { if !p.spooled { - vecerror.InternalError("getValues() is called before spool()") + colexecerror.InternalError("getValues() is called before spool()") } return p.bufferedTuples.ColVec(i) } @@ -158,7 +159,7 @@ func (p *allSpooler) getNumTuples() int { func (p *allSpooler) getPartitionsCol() []bool { if !p.spooled { - vecerror.InternalError("getPartitionsCol() is called before spool()") + colexecerror.InternalError("getPartitionsCol() is called before spool()") } return nil } @@ -186,7 +187,7 @@ func (p *allSpooler) reset(ctx context.Context) { } type sortOp struct { - allocator *colbase.Allocator + allocator *colmem.Allocator input spooler // inputTypes contains the types of all of the columns from input. @@ -295,7 +296,7 @@ func (p *sortOp) Next(ctx context.Context) coldata.Batch { p.emitted = newEmitted return p.output } - vecerror.InternalError(fmt.Sprintf("invalid sort state %v", p.state)) + colexecerror.InternalError(fmt.Sprintf("invalid sort state %v", p.state)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -422,12 +423,12 @@ func (p *sortOp) Child(nth int, verbose bool) execinfra.OpNode { if nth == 0 { return p.input } - vecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } -func (p *sortOp) ExportBuffered(colbase.Operator) coldata.Batch { +func (p *sortOp) ExportBuffered(colexecbase.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 a0ca66eb2531..97a526e1bf11 100644 --- a/pkg/sql/colexec/sort_chunks.go +++ b/pkg/sql/colexec/sort_chunks.go @@ -15,9 +15,10 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -28,14 +29,14 @@ import ( // the columns in the input operator. The input tuples must be sorted on first // matchLen columns. func NewSortChunks( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column, matchLen int, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { if matchLen < 1 || matchLen == len(orderingCols) { - vecerror.InternalError(fmt.Sprintf( + colexecerror.InternalError(fmt.Sprintf( "sort chunks should only be used when the input is "+ "already ordered on at least one column but not fully ordered; "+ "num ordering cols = %d, matchLen = %d", len(orderingCols), matchLen)) @@ -56,7 +57,7 @@ func NewSortChunks( } type sortChunksOp struct { - allocator *colbase.Allocator + allocator *colmem.Allocator input *chunker sorter resettableOperator @@ -65,7 +66,7 @@ type sortChunksOp struct { windowedBatch coldata.Batch } -var _ colbase.Operator = &sortChunksOp{} +var _ colexecbase.Operator = &sortChunksOp{} var _ bufferingInMemoryOperator = &sortChunksOp{} func (c *sortChunksOp) ChildCount(verbose bool) int { @@ -76,7 +77,7 @@ func (c *sortChunksOp) Child(nth int, verbose bool) execinfra.OpNode { if nth == 0 { return c.input } - vecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -110,7 +111,7 @@ func (c *sortChunksOp) Next(ctx context.Context) coldata.Batch { } } -func (c *sortChunksOp) ExportBuffered(colbase.Operator) coldata.Batch { +func (c *sortChunksOp) ExportBuffered(colexecbase.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 { @@ -201,7 +202,7 @@ type chunker struct { OneInputNode NonExplainable - allocator *colbase.Allocator + allocator *colmem.Allocator // inputTypes contains the types of all of the columns from input. inputTypes []types.T // inputDone indicates whether input has been fully consumed. @@ -252,8 +253,8 @@ type chunker struct { var _ spooler = &chunker{} func newChunker( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, inputTypes []types.T, alreadySortedCols []uint32, ) (*chunker, error) { @@ -313,7 +314,7 @@ func (s *chunker) prepareNextChunks(ctx context.Context) chunkerReadingState { if s.batch.Selection() != nil { // We assume that the input has been deselected, so the batch should // never have a selection vector set. - vecerror.InternalError(fmt.Sprintf("unexpected: batch with non-nil selection vector")) + colexecerror.InternalError(fmt.Sprintf("unexpected: batch with non-nil selection vector")) } // First, run the partitioners on our pre-sorted columns to determine the @@ -407,11 +408,11 @@ func (s *chunker) prepareNextChunks(ctx context.Context) chunkerReadingState { if s.inputDone { return chunkerDone } - vecerror.InternalError(fmt.Sprintf("unexpected: chunkerEmittingFromBatch state" + + colexecerror.InternalError(fmt.Sprintf("unexpected: chunkerEmittingFromBatch state" + "when s.chunks is fully processed and input is not done")) } default: - vecerror.InternalError(fmt.Sprintf("invalid chunker spooler state %v", s.state)) + colexecerror.InternalError(fmt.Sprintf("invalid chunker spooler state %v", s.state)) } } } @@ -439,7 +440,7 @@ func (s *chunker) getValues(i int) coldata.Vec { case chunkerReadFromBatch: return s.batch.ColVec(i).Window(typeconv.FromColumnType(&s.inputTypes[i]), s.chunks[s.chunksStartIdx], s.chunks[len(s.chunks)-1]) default: - vecerror.InternalError(fmt.Sprintf("unexpected chunkerReadingState in getValues: %v", s.state)) + colexecerror.InternalError(fmt.Sprintf("unexpected chunkerReadingState in getValues: %v", s.state)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -454,7 +455,7 @@ func (s *chunker) getNumTuples() int { case chunkerDone: return 0 default: - vecerror.InternalError(fmt.Sprintf("unexpected chunkerReadingState in getNumTuples: %v", s.state)) + colexecerror.InternalError(fmt.Sprintf("unexpected chunkerReadingState in getNumTuples: %v", s.state)) // This code is unreachable, but the compiler cannot infer that. return 0 } @@ -482,14 +483,14 @@ func (s *chunker) getPartitionsCol() []bool { case chunkerDone: return nil default: - vecerror.InternalError(fmt.Sprintf("unexpected chunkerReadingState in getPartitionsCol: %v", s.state)) + colexecerror.InternalError(fmt.Sprintf("unexpected chunkerReadingState in getPartitionsCol: %v", s.state)) // This code is unreachable, but the compiler cannot infer that. return nil } } func (s *chunker) getWindowedBatch(startIdx, endIdx int) coldata.Batch { - vecerror.InternalError("getWindowedBatch is not implemented on chunker spooler") + colexecerror.InternalError("getWindowedBatch is not implemented on chunker spooler") // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/sort_chunks_test.go b/pkg/sql/colexec/sort_chunks_test.go index 6c8bc3943004..8b79efa1162d 100644 --- a/pkg/sql/colexec/sort_chunks_test.go +++ b/pkg/sql/colexec/sort_chunks_test.go @@ -17,7 +17,8 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -183,7 +184,7 @@ func TestSortChunks(t *testing.T) { defer leaktest.AfterTest(t)() for _, tc := range sortChunksTestCases { - runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, func(input []colbase.Operator) (colbase.Operator, error) { + runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { return NewSortChunks(testAllocator, input[0], tc.typs, tc.ordCols, tc.matchLen) }) } @@ -224,7 +225,7 @@ func TestSortChunksRandomized(t *testing.T) { copy(expected, tups) sort.Slice(expected, less(expected, ordCols)) - runTests(t, []tuples{sortedTups}, expected, orderedVerifier, func(input []colbase.Operator) (colbase.Operator, error) { + runTests(t, []tuples{sortedTups}, expected, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { return NewSortChunks(testAllocator, input[0], typs[:nCols], ordCols, matchLen) }) } @@ -236,9 +237,9 @@ func BenchmarkSortChunks(b *testing.B) { rng, _ := randutil.NewPseudoRand() ctx := context.Background() - sorterConstructors := []func(*colbase.Allocator, colbase.Operator, []types.T, []execinfrapb.Ordering_Column, int) (colbase.Operator, error){ + sorterConstructors := []func(*colmem.Allocator, colexecbase.Operator, []types.T, []execinfrapb.Ordering_Column, int) (colexecbase.Operator, error){ NewSortChunks, - func(allocator *colbase.Allocator, input colbase.Operator, inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column, _ int) (colbase.Operator, error) { + func(allocator *colmem.Allocator, input colexecbase.Operator, inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column, _ int) (colexecbase.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 7cf3ef7a1dcb..75115599eccb 100644 --- a/pkg/sql/colexec/sort_test.go +++ b/pkg/sql/colexec/sort_test.go @@ -19,8 +19,8 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -138,7 +138,7 @@ func init() { func TestSort(t *testing.T) { defer leaktest.AfterTest(t)() for _, tc := range sortAllTestCases { - runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, func(input []colbase.Operator) (colbase.Operator, error) { + runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { return NewSorter(testAllocator, input[0], tc.typs, tc.ordCols) }) } @@ -164,7 +164,7 @@ func TestSortRandomized(t *testing.T) { if topK { expected = expected[:k] } - runTests(t, []tuples{tups}, expected, orderedVerifier, func(input []colbase.Operator) (colbase.Operator, error) { + runTests(t, []tuples{tups}, expected, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { if topK { return NewTopKSorter(testAllocator, input[0], typs[:nCols], ordCols, k), nil } @@ -250,7 +250,7 @@ func TestAllSpooler(t *testing.T) { }, } for _, tc := range tcs { - runTestsWithFn(t, []tuples{tc.tuples}, nil /* typs */, func(t *testing.T, input []colbase.Operator) { + runTestsWithFn(t, []tuples{tc.tuples}, nil /* typs */, func(t *testing.T, input []colexecbase.Operator) { allSpooler := newAllSpooler(testAllocator, input[0], tc.typ) allSpooler.init() allSpooler.spool(context.Background()) @@ -305,7 +305,7 @@ func BenchmarkSort(b *testing.B) { b.ResetTimer() for n := 0; n < b.N; n++ { source := newFiniteBatchSource(batch, typs, nBatches) - var sorter colbase.Operator + var sorter colexecbase.Operator if topK { sorter = NewTopKSorter(testAllocator, source, typs, ordCols, k) } else { @@ -398,7 +398,7 @@ func generateColumnOrdering( rng *rand.Rand, nCols int, nOrderingCols int, ) []execinfrapb.Ordering_Column { if nOrderingCols > nCols { - vecerror.InternalError("nOrderingCols > nCols in generateColumnOrdering") + colexecerror.InternalError("nOrderingCols > nCols in generateColumnOrdering") } orderingCols := make([]execinfrapb.Ordering_Column, nOrderingCols) for i, col := range rng.Perm(nCols)[:nOrderingCols] { diff --git a/pkg/sql/colexec/sort_tmpl.go b/pkg/sql/colexec/sort_tmpl.go index feb70c189cd3..a01021173e65 100644 --- a/pkg/sql/colexec/sort_tmpl.go +++ b/pkg/sql/colexec/sort_tmpl.go @@ -29,17 +29,18 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Declarations to make the template compile properly. @@ -78,7 +79,7 @@ const _ISNULL = false // _ASSIGN_LT is the template equality function for assigning the first input // to the result of the second input < the third input. func _ASSIGN_LT(_, _, _ string) bool { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} @@ -116,15 +117,15 @@ func newSingleSorter( return &sort_TYPE_DIR_HANDLES_NULLSOp{} // {{end}} default: - vecerror.InternalError("nulls switch failed") + colexecerror.InternalError("nulls switch failed") } // {{end}} default: - vecerror.InternalError("nulls switch failed") + colexecerror.InternalError("nulls switch failed") } // {{end}} default: - vecerror.InternalError("nulls switch failed") + colexecerror.InternalError("nulls switch failed") } // This code is unreachable, but the compiler cannot infer that. return nil @@ -154,7 +155,7 @@ func (s *sort_TYPE_DIR_HANDLES_NULLSOp) sort(ctx context.Context) { func (s *sort_TYPE_DIR_HANDLES_NULLSOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - vecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { diff --git a/pkg/sql/colexec/sorttopk.go b/pkg/sql/colexec/sorttopk.go index bb9e1d881a15..3a4205fa18e1 100644 --- a/pkg/sql/colexec/sorttopk.go +++ b/pkg/sql/colexec/sorttopk.go @@ -16,9 +16,10 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -32,12 +33,12 @@ const ( // columns given in orderingCols and returns the first K rows. The inputTypes // must correspond 1-1 with the columns in the input operator. func NewTopKSorter( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column, k uint16, -) colbase.Operator { +) colexecbase.Operator { return &topKSorter{ allocator: allocator, OneInputNode: NewOneInputNode(input), @@ -64,7 +65,7 @@ const ( type topKSorter struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator orderingCols []execinfrapb.Ordering_Column inputTypes []types.T k uint16 // TODO(solon): support larger k values @@ -117,7 +118,7 @@ func (t *topKSorter) Next(ctx context.Context) coldata.Batch { case topKSortEmitting: return t.emit() } - vecerror.InternalError(fmt.Sprintf("invalid sort state %v", t.state)) + colexecerror.InternalError(fmt.Sprintf("invalid sort state %v", t.state)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -250,7 +251,7 @@ func (t *topKSorter) compareRow(vecIdx1, vecIdx2 int, rowIdx1, rowIdx2 int) int case execinfrapb.Ordering_Column_DESC: return -res default: - vecerror.InternalError(fmt.Sprintf("unexpected direction value %d", d)) + colexecerror.InternalError(fmt.Sprintf("unexpected direction value %d", d)) } } } @@ -263,7 +264,7 @@ func (t *topKSorter) updateComparators(vecIdx int, batch coldata.Batch) { } } -func (t *topKSorter) ExportBuffered(colbase.Operator) coldata.Batch { +func (t *topKSorter) ExportBuffered(colexecbase.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 08ecd8ed5802..a0c5bcdeecee 100644 --- a/pkg/sql/colexec/sorttopk_test.go +++ b/pkg/sql/colexec/sorttopk_test.go @@ -13,7 +13,7 @@ package colexec import ( "testing" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -66,7 +66,7 @@ func TestTopKSorter(t *testing.T) { for _, tc := range topKSortTestCases { t.Run(tc.description, func(t *testing.T) { - runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, func(input []colbase.Operator) (colbase.Operator, error) { + runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { return NewTopKSorter(testAllocator, input[0], tc.typs, tc.ordCols, tc.k), nil }) }) diff --git a/pkg/sql/colexec/spilling_queue.go b/pkg/sql/colexec/spilling_queue.go index db5b8227ba47..ed5cd78f14a7 100644 --- a/pkg/sql/colexec/spilling_queue.go +++ b/pkg/sql/colexec/spilling_queue.go @@ -15,9 +15,9 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -40,7 +40,7 @@ import ( // batch is not kept around and thus its referenced memory will be GCed as soon // as the batch is updated. type spillingQueue struct { - unlimitedAllocator *colbase.Allocator + unlimitedAllocator *colmem.Allocator maxMemoryLimit int64 typs []types.T @@ -70,7 +70,7 @@ type spillingQueue struct { // If fdSemaphore is nil, no Acquire or Release calls will happen. The caller // may want to do this if requesting FDs up front. func newSpillingQueue( - unlimitedAllocator *colbase.Allocator, + unlimitedAllocator *colmem.Allocator, typs []types.T, memoryLimit int64, cfg colcontainer.DiskQueueCfg, @@ -84,7 +84,7 @@ func newSpillingQueue( if memoryLimit < 0 { memoryLimit = 0 } - itemsLen := memoryLimit / int64(colbase.EstimateBatchSizeBytesFromSQLTypes(typs, batchSize)) + itemsLen := memoryLimit / int64(colmem.EstimateBatchSizeBytesFromSQLTypes(typs, batchSize)) if itemsLen == 0 { // Make items at least of length 1. Even though batches will spill to disk // directly (this can only happen with a very low memory limit), it's nice @@ -109,7 +109,7 @@ func newSpillingQueue( // allocator must be passed in. The queue will use this allocator to check // whether memory usage exceeds the given memory limit and use disk if so. func newRewindableSpillingQueue( - unlimitedAllocator *colbase.Allocator, + unlimitedAllocator *colmem.Allocator, typs []types.T, memoryLimit int64, cfg colcontainer.DiskQueueCfg, @@ -168,7 +168,7 @@ func (q *spillingQueue) dequeue(ctx context.Context) (coldata.Batch, error) { // No more in-memory items. Fill the circular buffer as much as possible. // Note that there must be at least one element on disk. if !q.rewindable && q.curHeadIdx != q.curTailIdx { - vecerror.InternalError(fmt.Sprintf("assertion failed in spillingQueue: curHeadIdx != curTailIdx, %d != %d", q.curHeadIdx, q.curTailIdx)) + colexecerror.InternalError(fmt.Sprintf("assertion failed in spillingQueue: curHeadIdx != curTailIdx, %d != %d", q.curHeadIdx, q.curTailIdx)) } // NOTE: Only one item is dequeued from disk since a deserialized batch is // only valid until the next call to Dequeue. In practice we could Dequeue @@ -184,7 +184,7 @@ func (q *spillingQueue) dequeue(ctx context.Context) (coldata.Batch, error) { if !ok { // There was no batch to dequeue from disk. This should not really // happen, as it should have been caught by the q.empty() check above. - vecerror.InternalError("disk queue was not empty but failed to dequeue element in spillingQueue") + colexecerror.InternalError("disk queue was not empty but failed to dequeue element in spillingQueue") } // Account for this batch's memory. q.unlimitedAllocator.RetainBatch(q.dequeueScratch) @@ -297,7 +297,7 @@ func (q *spillingQueue) rewind() error { func (q *spillingQueue) reset(ctx context.Context) { if err := q.close(ctx); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } q.diskQueue = nil q.closed = false diff --git a/pkg/sql/colexec/spilling_queue_test.go b/pkg/sql/colexec/spilling_queue_test.go index 39d76e2128fc..f4913131e63b 100644 --- a/pkg/sql/colexec/spilling_queue_test.go +++ b/pkg/sql/colexec/spilling_queue_test.go @@ -16,8 +16,9 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "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/types" "github.com/cockroachdb/cockroach/pkg/testutils/colcontainerutils" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" @@ -58,12 +59,12 @@ func TestSpillingQueue(t *testing.T) { prefix, humanizeutil.IBytes(memoryLimit), diskQueueCacheMode, alwaysCompress, numBatches), func(t *testing.T) { // Create random input. batches := make([]coldata.Batch, 0, numBatches) - op := colbase.NewRandomDataOp(testAllocator, rng, colbase.RandomDataOpArgs{ + op := coldatatestutils.NewRandomDataOp(testAllocator, rng, coldatatestutils.RandomDataOpArgs{ NumBatches: cap(batches), BatchSize: 1 + rng.Intn(coldata.BatchSize()), Nulls: true, BatchAccumulator: func(b coldata.Batch, typs []types.T) { - batches = append(batches, colbase.CopyBatch(testAllocator, b, typs)) + batches = append(batches, coldatatestutils.CopyBatch(b, typs)) }, }) typs := op.Typs() @@ -77,13 +78,13 @@ func TestSpillingQueue(t *testing.T) { if rewindable { q = newRewindableSpillingQueue( testAllocator, typs, memoryLimit, queueCfg, - colbase.NewTestingSemaphore(2), coldata.BatchSize(), + colexecbase.NewTestingSemaphore(2), coldata.BatchSize(), testDiskAcc, ) } else { q = newSpillingQueue( testAllocator, typs, memoryLimit, queueCfg, - colbase.NewTestingSemaphore(2), coldata.BatchSize(), + colexecbase.NewTestingSemaphore(2), coldata.BatchSize(), testDiskAcc, ) } diff --git a/pkg/sql/colexec/stats.go b/pkg/sql/colexec/stats.go index acb2bbf39a69..2707d0fcc154 100644 --- a/pkg/sql/colexec/stats.go +++ b/pkg/sql/colexec/stats.go @@ -14,9 +14,9 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execpb" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) @@ -27,7 +27,7 @@ import ( // corresponding VectorizedStatsCollectors are also "connected" by sharing a // StopWatch. type VectorizedStatsCollector struct { - colbase.Operator + colexecbase.Operator NonExplainable execpb.VectorizedStats @@ -45,14 +45,14 @@ type VectorizedStatsCollector struct { diskMonitors []*mon.BytesMonitor } -var _ colbase.Operator = &VectorizedStatsCollector{} +var _ colexecbase.Operator = &VectorizedStatsCollector{} // NewVectorizedStatsCollector creates a new VectorizedStatsCollector which // wraps op that corresponds to a processor with ProcessorID id. isStall // indicates whether stall or execution time is being measured. inputWatch must // be non-nil. func NewVectorizedStatsCollector( - op colbase.Operator, + op colexecbase.Operator, id int32, isStall bool, inputWatch *timeutil.StopWatch, @@ -60,7 +60,7 @@ func NewVectorizedStatsCollector( diskMonitors []*mon.BytesMonitor, ) *VectorizedStatsCollector { if inputWatch == nil { - vecerror.InternalError("input watch for VectorizedStatsCollector is nil") + colexecerror.InternalError("input watch for VectorizedStatsCollector is nil") } return &VectorizedStatsCollector{ Operator: op, diff --git a/pkg/sql/colexec/stats_test.go b/pkg/sql/colexec/stats_test.go index dc37712b5139..07421cbee739 100644 --- a/pkg/sql/colexec/stats_test.go +++ b/pkg/sql/colexec/stats_test.go @@ -16,7 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -92,7 +92,7 @@ func TestVectorizedStatsCollector(t *testing.T) { mergeJoiner, err := newMergeJoinOp( testAllocator, defaultMemoryLimit, queueCfg, - colbase.NewTestingSemaphore(4), sqlbase.InnerJoin, leftInput, rightInput, + colexecbase.NewTestingSemaphore(4), sqlbase.InnerJoin, leftInput, rightInput, []types.T{*types.Int}, []types.T{*types.Int}, []execinfrapb.Ordering_Column{{ColIdx: 0}}, []execinfrapb.Ordering_Column{{ColIdx: 0}}, @@ -133,7 +133,7 @@ func TestVectorizedStatsCollector(t *testing.T) { } } -func makeFiniteChunksSourceWithBatchSize(nBatches int, batchSize int) colbase.Operator { +func makeFiniteChunksSourceWithBatchSize(nBatches int, batchSize int) colexecbase.Operator { typs := []types.T{*types.Int} batch := testAllocator.NewMemBatchWithSize(typs, batchSize) vec := batch.ColVec(0).Int64() @@ -152,7 +152,7 @@ type timeAdvancingOperator struct { timeSource *timeutil.TestTimeSource } -var _ colbase.Operator = &timeAdvancingOperator{} +var _ colexecbase.Operator = &timeAdvancingOperator{} func (o *timeAdvancingOperator) Init() { o.input.Init() diff --git a/pkg/sql/colexec/substring_tmpl.go b/pkg/sql/colexec/substring_tmpl.go index 052c4a7af190..bc45fe2fb82b 100644 --- a/pkg/sql/colexec/substring_tmpl.go +++ b/pkg/sql/colexec/substring_tmpl.go @@ -24,9 +24,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" ) @@ -40,12 +41,12 @@ var _ coltypes.T // */}} func newSubstringOperator( - allocator *colbase.Allocator, + allocator *colmem.Allocator, typs []types.T, argumentCols []int, outputIdx int, - input colbase.Operator, -) colbase.Operator { + input colexecbase.Operator, +) colexecbase.Operator { startType := &typs[argumentCols[1]] lengthType := &typs[argumentCols[2]] base := substringFunctionBase{ @@ -63,13 +64,13 @@ func newSubstringOperator( return &substring_StartType_LengthTypeOperator{base} // {{end}} default: - vecerror.InternalError(errors.Errorf("unsupported length argument type %s", lengthType)) + colexecerror.InternalError(errors.Errorf("unsupported length argument type %s", lengthType)) // This code is unreachable, but the compiler cannot infer that. return nil } // {{end}} default: - vecerror.InternalError(errors.Errorf("unsupported start argument type %s", startType)) + colexecerror.InternalError(errors.Errorf("unsupported start argument type %s", startType)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -77,7 +78,7 @@ func newSubstringOperator( type substringFunctionBase struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator argumentCols []int outputIdx int } @@ -93,7 +94,7 @@ type substring_StartType_LengthTypeOperator struct { substringFunctionBase } -var _ colbase.Operator = &substring_StartType_LengthTypeOperator{} +var _ colexecbase.Operator = &substring_StartType_LengthTypeOperator{} func (s *substring_StartType_LengthTypeOperator) Next(ctx context.Context) coldata.Batch { batch := s.input.Next(ctx) @@ -136,7 +137,7 @@ func (s *substring_StartType_LengthTypeOperator) Next(ctx context.Context) colda start := int(startVec[rowIdx]) - 1 length := int(lengthVec[rowIdx]) if length < 0 { - vecerror.ExpectedError(errors.Errorf("negative substring length %d not allowed", length)) + colexecerror.ExpectedError(errors.Errorf("negative substring length %d not allowed", length)) } end := start + length diff --git a/pkg/sql/colexec/sum_agg_tmpl.go b/pkg/sql/colexec/sum_agg_tmpl.go index 504c15f316a4..f7f25b2e33a3 100644 --- a/pkg/sql/colexec/sum_agg_tmpl.go +++ b/pkg/sql/colexec/sum_agg_tmpl.go @@ -23,8 +23,8 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" @@ -49,7 +49,7 @@ var _ coltypes.T // _ASSIGN_ADD is the template addition function for assigning the first input // to the result of the second input + the third input. func _ASSIGN_ADD(_, _, _ string) { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} diff --git a/pkg/sql/colexec/types_integration_test.go b/pkg/sql/colexec/types_integration_test.go index af177c2ab982..d78b5562b8c3 100644 --- a/pkg/sql/colexec/types_integration_test.go +++ b/pkg/sql/colexec/types_integration_test.go @@ -18,9 +18,10 @@ 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/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -58,7 +59,7 @@ func TestSupportedSQLTypesIntegration(t *testing.T) { var da sqlbase.DatumAlloc rng, _ := randutil.NewPseudoRand() - for _, typ := range colbase.AllSupportedSQLTypes { + for _, typ := range typeconv.AllSupportedSQLTypes { for _, numRows := range []int{ // A few interesting sizes. 1, @@ -126,11 +127,11 @@ type arrowTestOperator struct { r *colserde.RecordBatchSerializer } -var _ colbase.Operator = &arrowTestOperator{} +var _ colexecbase.Operator = &arrowTestOperator{} func newArrowTestOperator( - input colbase.Operator, c *colserde.ArrowBatchConverter, r *colserde.RecordBatchSerializer, -) colbase.Operator { + input colexecbase.Operator, c *colserde.ArrowBatchConverter, r *colserde.RecordBatchSerializer, +) colexecbase.Operator { return &arrowTestOperator{ OneInputNode: NewOneInputNode(input), c: c, @@ -148,19 +149,19 @@ func (a *arrowTestOperator) Next(ctx context.Context) coldata.Batch { var buf bytes.Buffer arrowDataIn, err := a.c.BatchToArrow(batchIn) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } _, _, err = a.r.Serialize(&buf, arrowDataIn) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } var arrowDataOut []*array.Data if err := a.r.Deserialize(&arrowDataOut, buf.Bytes()); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } batchOut := testAllocator.NewMemBatchWithSize(nil, 0) if err := a.c.ArrowToBatch(arrowDataOut, batchOut); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return batchOut } diff --git a/pkg/sql/colexec/unordered_distinct.go b/pkg/sql/colexec/unordered_distinct.go index 3352543d3c17..ef539199a3ec 100644 --- a/pkg/sql/colexec/unordered_distinct.go +++ b/pkg/sql/colexec/unordered_distinct.go @@ -14,8 +14,9 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -24,12 +25,12 @@ import ( // numHashBuckets determines the number of buckets that the hash table is // created with. func NewUnorderedDistinct( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, distinctCols []uint32, typs []types.T, numHashBuckets uint64, -) colbase.Operator { +) colexecbase.Operator { ht := newHashTable( allocator, numHashBuckets, @@ -57,7 +58,7 @@ func NewUnorderedDistinct( type unorderedDistinct struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator ht *hashTable buildFinished bool @@ -67,7 +68,7 @@ type unorderedDistinct struct { outputBatchStart int } -var _ colbase.Operator = &unorderedDistinct{} +var _ colexecbase.Operator = &unorderedDistinct{} func (op *unorderedDistinct) Init() { op.input.Init() diff --git a/pkg/sql/colexec/utils.go b/pkg/sql/colexec/utils.go index ea2d6d2316fa..0b1015f10fb3 100644 --- a/pkg/sql/colexec/utils.go +++ b/pkg/sql/colexec/utils.go @@ -17,10 +17,11 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" @@ -85,7 +86,7 @@ func makeWindowIntoBatch( } func newPartitionerToOperator( - allocator *colbase.Allocator, + allocator *colmem.Allocator, types []types.T, partitioner colcontainer.PartitionedQueue, partitionIdx int, @@ -101,7 +102,7 @@ func newPartitionerToOperator( // partition on every call to Next. It is a converter from filled in // PartitionedQueue to Operator. type partitionerToOperator struct { - colbase.ZeroInputNode + colexecbase.ZeroInputNode NonExplainable partitioner colcontainer.PartitionedQueue @@ -109,19 +110,19 @@ type partitionerToOperator struct { batch coldata.Batch } -var _ colbase.Operator = &partitionerToOperator{} +var _ colexecbase.Operator = &partitionerToOperator{} func (p *partitionerToOperator) Init() {} func (p *partitionerToOperator) Next(ctx context.Context) coldata.Batch { if err := p.partitioner.Dequeue(ctx, p.partitionIdx, p.batch); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return p.batch } func newAppendOnlyBufferedBatch( - allocator *colbase.Allocator, typs []types.T, initialSize int, + allocator *colmem.Allocator, typs []types.T, initialSize int, ) *appendOnlyBufferedBatch { batch := allocator.NewMemBatchWithSize(typs, initialSize) return &appendOnlyBufferedBatch{ @@ -169,11 +170,11 @@ func (b *appendOnlyBufferedBatch) ColVecs() []coldata.Vec { } func (b *appendOnlyBufferedBatch) AppendCol(coldata.Vec) { - vecerror.InternalError("AppendCol is prohibited on appendOnlyBufferedBatch") + colexecerror.InternalError("AppendCol is prohibited on appendOnlyBufferedBatch") } func (b *appendOnlyBufferedBatch) ReplaceCol(coldata.Vec, int) { - vecerror.InternalError("ReplaceCol is prohibited on appendOnlyBufferedBatch") + colexecerror.InternalError("ReplaceCol is prohibited on appendOnlyBufferedBatch") } // append is a helper method that appends all tuples with indices in range @@ -243,7 +244,7 @@ func getDatumToPhysicalFn(ct *types.T) func(tree.Datum) (interface{}, error) { return int64(*d), nil } } - vecerror.InternalError(fmt.Sprintf("unhandled INT width %d", ct.Width())) + colexecerror.InternalError(fmt.Sprintf("unhandled INT width %d", ct.Width())) case types.DateFamily: return func(datum tree.Datum) (interface{}, error) { d, ok := datum.(*tree.DDate) diff --git a/pkg/sql/colexec/utils_test.go b/pkg/sql/colexec/utils_test.go index 9b335140fd14..eff6c37a1141 100644 --- a/pkg/sql/colexec/utils_test.go +++ b/pkg/sql/colexec/utils_test.go @@ -23,10 +23,11 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -125,7 +126,7 @@ func (t tuple) less(other tuple) bool { case "string": return lhsVal.String() < rhsVal.String() default: - vecerror.InternalError(fmt.Sprintf("Unhandled comparison type: %s", typ)) + colexecerror.InternalError(fmt.Sprintf("Unhandled comparison type: %s", typ)) } } return false @@ -206,7 +207,7 @@ func maybeHasNulls(b coldata.Batch) bool { return false } -type testRunner func(*testing.T, []tuples, [][]types.T, tuples, interface{}, func([]colbase.Operator) (colbase.Operator, error)) +type testRunner func(*testing.T, []tuples, [][]types.T, tuples, interface{}, func([]colexecbase.Operator) (colexecbase.Operator, error)) // variableOutputBatchSizeInitializer is implemented by operators that can be // initialized with variable output size batches. This allows runTests to @@ -226,7 +227,7 @@ func runTests( tups []tuples, expected tuples, verifier interface{}, - constructor func(inputs []colbase.Operator) (colbase.Operator, error), + constructor func(inputs []colexecbase.Operator) (colexecbase.Operator, error), ) { runTestsWithTyps(t, tups, nil /* typs */, expected, verifier, constructor) } @@ -242,7 +243,7 @@ func runTestsWithTyps( typs [][]types.T, expected tuples, verifier interface{}, - constructor func(inputs []colbase.Operator) (colbase.Operator, error), + constructor func(inputs []colexecbase.Operator) (colexecbase.Operator, error), ) { runTestsWithoutAllNullsInjection(t, tups, typs, expected, verifier, constructor) @@ -262,8 +263,8 @@ func runTestsWithTyps( } } } - opConstructor := func(injectAllNulls bool) colbase.Operator { - inputSources := make([]colbase.Operator, len(tups)) + opConstructor := func(injectAllNulls bool) colexecbase.Operator { + inputSources := make([]colexecbase.Operator, len(tups)) var inputTypes []types.T for i, tup := range tups { if typs != nil { @@ -335,7 +336,7 @@ func runTestsWithoutAllNullsInjection( typs [][]types.T, expected tuples, verifier interface{}, - constructor func(inputs []colbase.Operator) (colbase.Operator, error), + constructor func(inputs []colexecbase.Operator) (colexecbase.Operator, error), ) { skipVerifySelAndNullsResets := true var verifyFn verifierFn @@ -351,12 +352,12 @@ func runTestsWithoutAllNullsInjection( case unorderedVerifier: verifyFn = (*opTestOutput).VerifyAnyOrder default: - vecerror.InternalError(fmt.Sprintf("unexpected verifierType %d", v)) + colexecerror.InternalError(fmt.Sprintf("unexpected verifierType %d", v)) } case verifierFn: verifyFn = v } - runTestsWithFn(t, tups, typs, func(t *testing.T, inputs []colbase.Operator) { + runTestsWithFn(t, tups, typs, func(t *testing.T, inputs []colexecbase.Operator) { op, err := constructor(inputs) if err != nil { t.Fatal(err) @@ -384,7 +385,7 @@ func runTestsWithoutAllNullsInjection( inputTypes []types.T ) for round := 0; round < 2; round++ { - inputSources := make([]colbase.Operator, len(tups)) + inputSources := make([]colexecbase.Operator, len(tups)) for i, tup := range tups { if typs != nil { inputTypes = typs[i] @@ -454,7 +455,7 @@ func runTestsWithoutAllNullsInjection( t.Run("randomNullsInjection", func(t *testing.T) { // This test randomly injects nulls in the input tuples and ensures that // the operator doesn't panic. - inputSources := make([]colbase.Operator, len(tups)) + inputSources := make([]colexecbase.Operator, len(tups)) var inputTypes []types.T for i, tup := range tups { if typs != nil { @@ -490,7 +491,10 @@ func runTestsWithoutAllNullsInjection( // - test is a function that takes a list of input Operators and performs // testing with t. func runTestsWithFn( - t *testing.T, tups []tuples, typs [][]types.T, test func(t *testing.T, inputs []colbase.Operator), + t *testing.T, + tups []tuples, + typs [][]types.T, + test func(t *testing.T, inputs []colexecbase.Operator), ) { // Run tests over batchSizes of 1, (sometimes) a batch size that is small but // greater than 1, and a full coldata.BatchSize(). @@ -505,7 +509,7 @@ func runTestsWithFn( for _, batchSize := range batchSizes { for _, useSel := range []bool{false, true} { t.Run(fmt.Sprintf("batchSize=%d/sel=%t", batchSize, useSel), func(t *testing.T) { - inputSources := make([]colbase.Operator, len(tups)) + inputSources := make([]colexecbase.Operator, len(tups)) var inputTypes []types.T if useSel { for i, tup := range tups { @@ -534,11 +538,11 @@ func runTestsWithFn( // function that takes a list of input Operators, which will give back the // tuples provided in batches. func runTestsWithFixedSel( - t *testing.T, tups []tuples, sel []int, test func(t *testing.T, inputs []colbase.Operator), + t *testing.T, tups []tuples, sel []int, test func(t *testing.T, inputs []colexecbase.Operator), ) { for _, batchSize := range []int{1, 2, 3, 16, 1024} { t.Run(fmt.Sprintf("batchSize=%d/fixedSel", batchSize), func(t *testing.T) { - inputSources := make([]colbase.Operator, len(tups)) + inputSources := make([]colexecbase.Operator, len(tups)) for i, tup := range tups { inputSources[i] = newOpFixedSelTestInput(sel, batchSize, tup) } @@ -569,7 +573,7 @@ func setColVal(vec coldata.Vec, idx int, val interface{}) { floatVal := val.(float64) decimalVal, _, err := apd.NewFromString(fmt.Sprintf("%f", floatVal)) if err != nil { - vecerror.InternalError( + colexecerror.InternalError( fmt.Sprintf("unable to set decimal %f: %v", floatVal, err)) } // .Set is used here instead of assignment to ensure the pointer address @@ -597,7 +601,7 @@ func setColVal(vec coldata.Vec, idx int, val interface{}) { // t.Fatal(err) // } type opTestInput struct { - colbase.ZeroInputNode + colexecbase.ZeroInputNode typs []types.T @@ -617,7 +621,7 @@ type opTestInput struct { injectRandomNulls bool } -var _ colbase.Operator = &opTestInput{} +var _ colexecbase.Operator = &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 @@ -645,7 +649,7 @@ func newOpTestSelInput(rng *rand.Rand, batchSize int, tuples tuples, typs []type func (s *opTestInput) Init() { if s.typs == nil { if len(s.tuples) == 0 { - vecerror.InternalError("empty tuple source with no specified types") + colexecerror.InternalError("empty tuple source with no specified types") } // The type schema was not provided, so we need to determine it based on @@ -659,7 +663,7 @@ func (s *opTestInput) Init() { if tup[i] != nil { t, err := typeconv.UnsafeToSQLType(coltypes.FromGoType(tup[i])) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } s.typs[i] = *t break @@ -690,7 +694,7 @@ func (s *opTestInput) Next(context.Context) coldata.Batch { tupleLen := len(tups[0]) for i := range tups { if len(tups[i]) != tupleLen { - vecerror.InternalError(fmt.Sprintf("mismatched tuple lens: found %+v expected %d vals", + colexecerror.InternalError(fmt.Sprintf("mismatched tuple lens: found %+v expected %d vals", tups[i], tupleLen)) } } @@ -761,7 +765,7 @@ func (s *opTestInput) Next(context.Context) coldata.Batch { d := apd.Decimal{} _, err := d.SetFloat64(rng.Float64()) if err != nil { - vecerror.InternalError(fmt.Sprintf("%v", err)) + colexecerror.InternalError(fmt.Sprintf("%v", err)) } col.Index(outputIdx).Set(reflect.ValueOf(d)) } else if typ == coltypes.Bytes { @@ -771,7 +775,7 @@ func (s *opTestInput) Next(context.Context) coldata.Batch { } else if val, ok := quick.Value(reflect.TypeOf(vec.Col()).Elem(), rng); ok { setColVal(vec, outputIdx, val.Interface()) } else { - vecerror.InternalError(fmt.Sprintf("could not generate a random value of type %T\n.", vec.Type())) + colexecerror.InternalError(fmt.Sprintf("could not generate a random value of type %T\n.", vec.Type())) } } } else { @@ -785,7 +789,7 @@ func (s *opTestInput) Next(context.Context) coldata.Batch { } type opFixedSelTestInput struct { - colbase.ZeroInputNode + colexecbase.ZeroInputNode typs []types.T @@ -799,7 +803,7 @@ type opFixedSelTestInput struct { idx int } -var _ colbase.Operator = &opFixedSelTestInput{} +var _ colexecbase.Operator = &opFixedSelTestInput{} // newOpFixedSelTestInput returns a new opFixedSelTestInput with the given // input tuples and selection vector. The input tuples are translated into @@ -815,7 +819,7 @@ func newOpFixedSelTestInput(sel []int, batchSize int, tuples tuples) *opFixedSel func (s *opFixedSelTestInput) Init() { if len(s.tuples) == 0 { - vecerror.InternalError("empty tuple source") + colexecerror.InternalError("empty tuple source") } s.typs = make([]types.T, len(s.tuples[0])) @@ -827,7 +831,7 @@ func (s *opFixedSelTestInput) Init() { if tup[i] != nil { t, err := typeconv.UnsafeToSQLType(coltypes.FromGoType(tup[i])) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } s.typs[i] = *t break @@ -839,7 +843,7 @@ func (s *opFixedSelTestInput) Init() { tupleLen := len(s.tuples[0]) for _, i := range s.sel { if len(s.tuples[i]) != tupleLen { - vecerror.InternalError(fmt.Sprintf("mismatched tuple lens: found %+v expected %d vals", + colexecerror.InternalError(fmt.Sprintf("mismatched tuple lens: found %+v expected %d vals", s.tuples[i], tupleLen)) } } @@ -921,7 +925,7 @@ 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 colbase.Operator, expected tuples) *opTestOutput { +func newOpTestOutput(input colexecbase.Operator, expected tuples) *opTestOutput { input.Init() return &opTestOutput{ @@ -1104,20 +1108,20 @@ func assertTuplesOrderedEqual(expected tuples, actual tuples) error { // finiteBatchSource is an Operator that returns the same batch a specified // number of times. type finiteBatchSource struct { - colbase.ZeroInputNode + colexecbase.ZeroInputNode - repeatableBatch *colbase.RepeatableBatchSource + repeatableBatch *colexecbase.RepeatableBatchSource usableCount int } -var _ colbase.Operator = &finiteBatchSource{} +var _ colexecbase.Operator = &finiteBatchSource{} // newFiniteBatchSource returns a new Operator initialized to return its input // batch a specified number of times. func newFiniteBatchSource(batch coldata.Batch, typs []types.T, usableCount int) *finiteBatchSource { return &finiteBatchSource{ - repeatableBatch: colbase.NewRepeatableBatchSource(testAllocator, batch, typs), + repeatableBatch: colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs), usableCount: usableCount, } } @@ -1143,21 +1147,21 @@ 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 { - colbase.ZeroInputNode - repeatableBatch *colbase.RepeatableBatchSource + colexecbase.ZeroInputNode + repeatableBatch *colexecbase.RepeatableBatchSource usableCount int matchLen int adjustment []int64 } -var _ colbase.Operator = &finiteChunksSource{} +var _ colexecbase.Operator = &finiteChunksSource{} func newFiniteChunksSource( batch coldata.Batch, typs []types.T, usableCount int, matchLen int, ) *finiteChunksSource { return &finiteChunksSource{ - repeatableBatch: colbase.NewRepeatableBatchSource(testAllocator, batch, typs), + repeatableBatch: colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs), usableCount: usableCount, matchLen: matchLen, } @@ -1211,7 +1215,7 @@ func TestOpTestInputOutput(t *testing.T) { {1, 5, 0}, }, } - runTestsWithFn(t, inputs, nil /* typs */, func(t *testing.T, sources []colbase.Operator) { + runTestsWithFn(t, inputs, nil /* typs */, func(t *testing.T, sources []colexecbase.Operator) { out := newOpTestOutput(sources[0], inputs[0]) if err := out.Verify(); err != nil { @@ -1229,7 +1233,7 @@ func TestRepeatableBatchSource(t *testing.T) { batchLen = coldata.BatchSize() } batch.SetLength(batchLen) - input := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) b := input.Next(context.Background()) b.SetLength(0) @@ -1253,12 +1257,12 @@ func TestRepeatableBatchSourceWithFixedSel(t *testing.T) { if batchSize > coldata.BatchSize() { batchSize = coldata.BatchSize() } - sel := colbase.RandomSel(rng, batchSize, 0 /* probOfOmitting */) + sel := coldatatestutils.RandomSel(rng, batchSize, 0 /* probOfOmitting */) batchLen := len(sel) batch.SetLength(batchLen) batch.SetSelection(true) copy(batch.Selection(), sel) - input := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) b := input.Next(context.Background()) b.SetLength(0) @@ -1277,7 +1281,7 @@ func TestRepeatableBatchSourceWithFixedSel(t *testing.T) { } } - newSel := colbase.RandomSel(rng, 10 /* batchSize */, 0.2 /* probOfOmitting */) + newSel := coldatatestutils.RandomSel(rng, 10 /* batchSize */, 0.2 /* probOfOmitting */) newBatchLen := len(sel) b.SetLength(newBatchLen) b.SetSelection(true) @@ -1300,7 +1304,7 @@ func TestRepeatableBatchSourceWithFixedSel(t *testing.T) { // chunkingBatchSource is a batch source that takes unlimited-size columns and // chunks them into BatchSize()-sized chunks when Nexted. type chunkingBatchSource struct { - colbase.ZeroInputNode + colexecbase.ZeroInputNode typs []types.T cols []coldata.Vec len int @@ -1309,7 +1313,7 @@ type chunkingBatchSource struct { batch coldata.Batch } -var _ colbase.Operator = &chunkingBatchSource{} +var _ colexecbase.Operator = &chunkingBatchSource{} // newChunkingBatchSource returns a new chunkingBatchSource with the given // column types, columns, and length. @@ -1495,11 +1499,11 @@ func (p *mockTypeContext) IndexedVarNodeFormatter(idx int) tree.NodeFormatter { func createTestProjectingOperator( ctx context.Context, flowCtx *execinfra.FlowCtx, - input colbase.Operator, + input colexecbase.Operator, inputTypes []types.T, projectingExpr string, canFallbackToRowexec bool, -) (colbase.Operator, error) { +) (colexecbase.Operator, error) { expr, err := parser.ParseExpr(projectingExpr) if err != nil { return nil, err @@ -1525,7 +1529,7 @@ func createTestProjectingOperator( } args := NewColOperatorArgs{ Spec: spec, - Inputs: []colbase.Operator{input}, + Inputs: []colexecbase.Operator{input}, StreamingMemAccount: testMemAcc, } if canFallbackToRowexec { diff --git a/pkg/sql/colexec/values_differ_tmpl.go b/pkg/sql/colexec/values_differ_tmpl.go index e1cdb466a898..7bf243586788 100644 --- a/pkg/sql/colexec/values_differ_tmpl.go +++ b/pkg/sql/colexec/values_differ_tmpl.go @@ -26,15 +26,16 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Declarations to make the template compile properly. @@ -60,7 +61,7 @@ const _TYPES_T = coltypes.Unhandled // _ASSIGN_NE is the template equality function for assigning the first input // to the result of the second input != the third input. func _ASSIGN_NE(_, _, _ string) bool { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} @@ -92,7 +93,7 @@ func valuesDiffer( return unique // {{end}} default: - vecerror.InternalError(fmt.Sprintf("unsupported valuesDiffer type %s", t)) + colexecerror.InternalError(fmt.Sprintf("unsupported valuesDiffer type %s", t)) // This code is unreachable, but the compiler cannot infer that. return false } diff --git a/pkg/sql/colexec/vec_comparators_tmpl.go b/pkg/sql/colexec/vec_comparators_tmpl.go index 80dd4d79d031..7e4f5a8f8e14 100644 --- a/pkg/sql/colexec/vec_comparators_tmpl.go +++ b/pkg/sql/colexec/vec_comparators_tmpl.go @@ -28,16 +28,17 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Declarations to make the template compile properly. @@ -66,7 +67,7 @@ var _ coltypes.T // _COMPARE is the template equality function for assigning the first input // to the result of comparing second and third inputs. func _COMPARE(_, _, _ string) bool { - vecerror.InternalError("") + colexecerror.InternalError("") } // */}} @@ -150,7 +151,7 @@ func GetVecComparator(t *types.T, numVecs int) vecComparator { } // {{end}} } - vecerror.InternalError(fmt.Sprintf("unhandled type %v", t)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %v", t)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/vec_elem_to_datum.go b/pkg/sql/colexec/vec_elem_to_datum.go index d903eaccf5b5..48fc2096254f 100644 --- a/pkg/sql/colexec/vec_elem_to_datum.go +++ b/pkg/sql/colexec/vec_elem_to_datum.go @@ -15,7 +15,7 @@ import ( "math/big" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -81,7 +81,7 @@ func PhysicalTypeColElemToDatum( // a copy. id, err := uuid.FromBytes(col.Bytes().Get(rowIdx)) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return da.NewDUuid(tree.DUuid{UUID: id}) case types.TimestampFamily: @@ -91,7 +91,7 @@ func PhysicalTypeColElemToDatum( case types.IntervalFamily: return da.NewDInterval(tree.DInterval{Duration: col.Interval()[rowIdx]}) default: - vecerror.InternalError(fmt.Sprintf("Unsupported column type %s", ct.String())) + colexecerror.InternalError(fmt.Sprintf("Unsupported column type %s", ct.String())) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/window_functions_test.go b/pkg/sql/colexec/window_functions_test.go index 829805b151e0..87d4b5e36374 100644 --- a/pkg/sql/colexec/window_functions_test.go +++ b/pkg/sql/colexec/window_functions_test.go @@ -16,7 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -272,7 +272,7 @@ func TestWindowFunctions(t *testing.T) { } { t.Run(fmt.Sprintf("spillForced=%t/%s", spillForced, tc.windowerSpec.WindowFns[0].Func.String()), func(t *testing.T) { var semsToCheck []semaphore.Semaphore - runTests(t, []tuples{tc.tuples}, tc.expected, unorderedVerifier, func(inputs []colbase.Operator) (colbase.Operator, error) { + runTests(t, []tuples{tc.tuples}, tc.expected, unorderedVerifier, func(inputs []colexecbase.Operator) (colexecbase.Operator, error) { tc.init() ct := make([]types.T, len(tc.tuples[0])) for i := range ct { @@ -284,7 +284,7 @@ func TestWindowFunctions(t *testing.T) { Windower: &tc.windowerSpec, }, } - sem := colbase.NewTestingSemaphore(maxNumberFDs) + sem := colexecbase.NewTestingSemaphore(maxNumberFDs) args := NewColOperatorArgs{ Spec: spec, Inputs: inputs, diff --git a/pkg/sql/colexec/window_functions_util.go b/pkg/sql/colexec/window_functions_util.go index a484a892f13e..9e43dd39e42f 100644 --- a/pkg/sql/colexec/window_functions_util.go +++ b/pkg/sql/colexec/window_functions_util.go @@ -13,7 +13,7 @@ package colexec import ( "fmt" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" ) @@ -47,7 +47,7 @@ func windowFnNeedsPeersInfo(windowFn execinfrapb.WindowerSpec_WindowFunc) bool { execinfrapb.WindowerSpec_CUME_DIST: return true default: - vecerror.InternalError(fmt.Sprintf("window function %s is not supported", windowFn.String())) + colexecerror.InternalError(fmt.Sprintf("window function %s is not supported", windowFn.String())) // This code is unreachable, but the compiler cannot infer that. return false } diff --git a/pkg/sql/colexec/window_peer_grouper_tmpl.go b/pkg/sql/colexec/window_peer_grouper_tmpl.go index b393109c9b93..34305ec69e70 100644 --- a/pkg/sql/colexec/window_peer_grouper_tmpl.go +++ b/pkg/sql/colexec/window_peer_grouper_tmpl.go @@ -23,7 +23,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -37,13 +38,13 @@ import ( // 'true' indicates the start of a new partition. // NOTE: the input *must* already be ordered on ordCols. func NewWindowPeerGrouper( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, typs []types.T, orderingCols []execinfrapb.Ordering_Column, partitionColIdx int, outputColIdx int, -) (op colbase.Operator, err error) { +) (op colexecbase.Operator, err error) { allPeers := len(orderingCols) == 0 var distinctCol []bool if !allPeers { @@ -89,7 +90,7 @@ func NewWindowPeerGrouper( type windowPeerGrouperInitFields struct { OneInputNode - allocator *colbase.Allocator + allocator *colmem.Allocator partitionColIdx int // distinctCol is the output column of the chain of ordered distinct // operators in which 'true' will indicate that a new peer group begins with @@ -107,7 +108,7 @@ type _PEER_GROUPER_STRINGOp struct { // {{end}} } -var _ colbase.Operator = &_PEER_GROUPER_STRINGOp{} +var _ colexecbase.Operator = &_PEER_GROUPER_STRINGOp{} func (p *_PEER_GROUPER_STRINGOp) Init() { p.input.Init() diff --git a/pkg/sql/colbase/vecerror/error.go b/pkg/sql/colexecbase/colexecerror/error.go similarity index 99% rename from pkg/sql/colbase/vecerror/error.go rename to pkg/sql/colexecbase/colexecerror/error.go index dac97066400d..bcf71d063b84 100644 --- a/pkg/sql/colbase/vecerror/error.go +++ b/pkg/sql/colexecbase/colexecerror/error.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package vecerror +package colexecerror import ( "bufio" diff --git a/pkg/sql/colbase/dep_test.go b/pkg/sql/colexecbase/dep_test.go similarity index 91% rename from pkg/sql/colbase/dep_test.go rename to pkg/sql/colexecbase/dep_test.go index dbf1815d5819..f9e2b8431674 100644 --- a/pkg/sql/colbase/dep_test.go +++ b/pkg/sql/colexecbase/dep_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package colbase +package colexecbase import ( "testing" @@ -21,7 +21,7 @@ func TestNoLinkForbidden(t *testing.T) { defer leaktest.AfterTest(t)() buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/colbase", true, + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase", true, []string{ "github.com/cockroachdb/cockroach/pkg/sql/colcontainer", "github.com/cockroachdb/cockroach/pkg/sql/colexec", diff --git a/pkg/sql/colbase/operator.go b/pkg/sql/colexecbase/operator.go similarity index 93% rename from pkg/sql/colbase/operator.go rename to pkg/sql/colexecbase/operator.go index bbbf2d2cbd44..9d6ea1fd0e90 100644 --- a/pkg/sql/colbase/operator.go +++ b/pkg/sql/colexecbase/operator.go @@ -8,14 +8,14 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package colbase +package colexecbase import ( "context" "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" ) @@ -56,7 +56,7 @@ func (ZeroInputNode) ChildCount(verbose bool) int { // Child implements the execinfra.OpNode interface. func (ZeroInputNode) Child(nth int, verbose bool) execinfra.OpNode { - vecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colbase/testutils.go b/pkg/sql/colexecbase/testutils.go similarity index 92% rename from pkg/sql/colbase/testutils.go rename to pkg/sql/colexecbase/testutils.go index 1a27adf71664..7a80f96c9901 100644 --- a/pkg/sql/colbase/testutils.go +++ b/pkg/sql/colexecbase/testutils.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package colbase +package colexecbase import ( "context" @@ -16,8 +16,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" ) @@ -79,7 +80,7 @@ var _ Operator = &RepeatableBatchSource{} // copies them into a separate output batch. The output batch is allowed to be // modified whereas the input batch is *not*. func NewRepeatableBatchSource( - allocator *Allocator, batch coldata.Batch, typs []types.T, + allocator *colmem.Allocator, batch coldata.Batch, typs []types.T, ) *RepeatableBatchSource { sel := batch.Selection() batchLen := batch.Length() @@ -96,7 +97,7 @@ func NewRepeatableBatchSource( output := allocator.NewMemBatchWithSize(typs, numToCopy) physTypes, err := typeconv.FromColumnTypes(typs) if err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } src := &RepeatableBatchSource{ colVecs: batch.ColVecs(), @@ -200,10 +201,10 @@ func (s *TestingSemaphore) TryAcquire(n int) bool { // Release implements the semaphore.Semaphore interface. func (s *TestingSemaphore) Release(n int) int { if n < 0 { - vecerror.InternalError("releasing a negative amount") + colexecerror.InternalError("releasing a negative amount") } if s.count-n < 0 { - vecerror.InternalError(fmt.Sprintf("testing semaphore too many resources released, releasing %d, have %d", n, s.count)) + colexecerror.InternalError(fmt.Sprintf("testing semaphore too many resources released, releasing %d, have %d", n, s.count)) } pre := s.count s.count -= n diff --git a/pkg/sql/colflow/colrpc/colrpc_test.go b/pkg/sql/colflow/colrpc/colrpc_test.go index 23fb54e9444c..bf324667f620 100644 --- a/pkg/sql/colflow/colrpc/colrpc_test.go +++ b/pkg/sql/colflow/colrpc/colrpc_test.go @@ -22,11 +22,13 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -188,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 = colbase.NewBatchBuffer() + inputBuffer = colexecbase.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 @@ -200,7 +202,7 @@ func TestOutboxInbox(t *testing.T) { // Test random selection as the Outbox should be deselecting before sending // over data. Nulls and types are not worth testing as those are tested in // colserde. - args := colbase.RandomDataOpArgs{ + args := coldatatestutils.RandomDataOpArgs{ DeterministicTyps: typs, NumBatches: 64, Selection: true, @@ -216,19 +218,19 @@ func TestOutboxInbox(t *testing.T) { } inputMemAcc := testMemMonitor.MakeBoundAccount() defer inputMemAcc.Close(ctx) - input := colbase.NewRandomDataOp( - colbase.NewAllocator(ctx, &inputMemAcc), rng, args, + input := coldatatestutils.NewRandomDataOp( + colmem.NewAllocator(ctx, &inputMemAcc), rng, args, ) outboxMemAcc := testMemMonitor.MakeBoundAccount() defer outboxMemAcc.Close(ctx) - outbox, err := NewOutbox(colbase.NewAllocator(ctx, &outboxMemAcc), input, typs, nil /* metadataSource */, nil /* toClose */) + outbox, err := NewOutbox(colmem.NewAllocator(ctx, &outboxMemAcc), input, typs, nil /* metadataSource */, nil /* toClose */) require.NoError(t, err) inboxMemAcc := testMemMonitor.MakeBoundAccount() defer inboxMemAcc.Close(ctx) inbox, err := NewInbox( - colbase.NewAllocator(ctx, &inboxMemAcc), typs, execinfrapb.StreamID(0), + colmem.NewAllocator(ctx, &inboxMemAcc), typs, execinfrapb.StreamID(0), ) require.NoError(t, err) @@ -267,14 +269,14 @@ func TestOutboxInbox(t *testing.T) { deselectorMemAcc := testMemMonitor.MakeBoundAccount() defer deselectorMemAcc.Close(ctx) inputBatches := colexec.NewDeselectorOp( - colbase.NewAllocator(ctx, &deselectorMemAcc), inputBuffer, typs, + colmem.NewAllocator(ctx, &deselectorMemAcc), inputBuffer, typs, ) inputBatches.Init() - outputBatches := colbase.NewBatchBuffer() + outputBatches := colexecbase.NewBatchBuffer() var readerErr error for { var outputBatch coldata.Batch - if err := vecerror.CatchVectorizedRuntimeError(func() { + if err := colexecerror.CatchVectorizedRuntimeError(func() { outputBatch = inbox.Next(readerCtx) }); err != nil { readerErr = err @@ -445,10 +447,10 @@ func TestOutboxInboxMetadataPropagation(t *testing.T) { serverStreamNotification = <-mockServer.InboundStreams serverStream = serverStreamNotification.Stream typs = []types.T{*types.Int} - input = colbase.NewRandomDataOp( + input = coldatatestutils.NewRandomDataOp( testAllocator, rng, - colbase.RandomDataOpArgs{ + coldatatestutils.RandomDataOpArgs{ DeterministicTyps: typs, NumBatches: tc.numBatches, Selection: true, @@ -460,7 +462,7 @@ func TestOutboxInboxMetadataPropagation(t *testing.T) { outboxMemAcc := testMemMonitor.MakeBoundAccount() defer outboxMemAcc.Close(ctx) - outbox, err := NewOutbox(colbase.NewAllocator(ctx, &outboxMemAcc), input, typs, []execinfrapb.MetadataSource{ + outbox, err := NewOutbox(colmem.NewAllocator(ctx, &outboxMemAcc), input, typs, []execinfrapb.MetadataSource{ execinfrapb.CallbackMetadataSource{ DrainMetaCb: func(context.Context) []execinfrapb.ProducerMetadata { return []execinfrapb.ProducerMetadata{{Err: errors.New(expectedMeta)}} @@ -472,7 +474,7 @@ func TestOutboxInboxMetadataPropagation(t *testing.T) { inboxMemAcc := testMemMonitor.MakeBoundAccount() defer inboxMemAcc.Close(ctx) inbox, err := NewInbox( - colbase.NewAllocator(ctx, &inboxMemAcc), + colmem.NewAllocator(ctx, &inboxMemAcc), typs, execinfrapb.StreamID(0), ) require.NoError(t, err) @@ -530,17 +532,17 @@ func BenchmarkOutboxInbox(b *testing.B) { batch := testAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) - input := colbase.NewRepeatableBatchSource(testAllocator, batch, typs) + input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) outboxMemAcc := testMemMonitor.MakeBoundAccount() defer outboxMemAcc.Close(ctx) - outbox, err := NewOutbox(colbase.NewAllocator(ctx, &outboxMemAcc), input, typs, nil /* metadataSources */, nil /* toClose */) + outbox, err := NewOutbox(colmem.NewAllocator(ctx, &outboxMemAcc), input, typs, nil /* metadataSources */, nil /* toClose */) require.NoError(b, err) inboxMemAcc := testMemMonitor.MakeBoundAccount() defer inboxMemAcc.Close(ctx) inbox, err := NewInbox( - colbase.NewAllocator(ctx, &inboxMemAcc), typs, execinfrapb.StreamID(0), + colmem.NewAllocator(ctx, &inboxMemAcc), typs, execinfrapb.StreamID(0), ) require.NoError(b, err) @@ -588,7 +590,7 @@ func TestOutboxStreamIDPropagation(t *testing.T) { var inTags *logtags.Buffer nextDone := make(chan struct{}) - input := &colbase.CallbackOperator{NextCb: func(ctx context.Context) coldata.Batch { + input := &colexecbase.CallbackOperator{NextCb: func(ctx context.Context) coldata.Batch { b := testAllocator.NewMemBatchWithSize(typs, 0) b.SetLength(0) inTags = logtags.FromContext(ctx) @@ -598,7 +600,7 @@ func TestOutboxStreamIDPropagation(t *testing.T) { outboxMemAcc := testMemMonitor.MakeBoundAccount() defer outboxMemAcc.Close(ctx) - outbox, err := NewOutbox(colbase.NewAllocator(ctx, &outboxMemAcc), input, typs, nil /* metadataSources */, nil /* toClose */) + outbox, err := NewOutbox(colmem.NewAllocator(ctx, &outboxMemAcc), input, typs, nil /* metadataSources */, nil /* toClose */) require.NoError(t, err) outboxDone := make(chan struct{}) diff --git a/pkg/sql/colflow/colrpc/inbox.go b/pkg/sql/colflow/colrpc/inbox.go index 71b487f6e3aa..b73f0c754f2b 100644 --- a/pkg/sql/colflow/colrpc/inbox.go +++ b/pkg/sql/colflow/colrpc/inbox.go @@ -19,8 +19,9 @@ 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/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -45,7 +46,7 @@ type flowStreamServer interface { // RunWithStream (or more specifically, the RPC handler) will unblock Next by // closing the stream. type Inbox struct { - colbase.ZeroInputNode + colexecbase.ZeroInputNode typs []types.T converter *colserde.ArrowBatchConverter @@ -113,11 +114,11 @@ type Inbox struct { } } -var _ colbase.Operator = &Inbox{} +var _ colexecbase.Operator = &Inbox{} // NewInbox creates a new Inbox. func NewInbox( - allocator *colbase.Allocator, typs []types.T, streamID execinfrapb.StreamID, + allocator *colmem.Allocator, typs []types.T, streamID execinfrapb.StreamID, ) (*Inbox, error) { c, err := colserde.NewArrowBatchConverter(typs) if err != nil { @@ -271,7 +272,7 @@ func (i *Inbox) Next(ctx context.Context) coldata.Batch { i.streamMu.Unlock() } i.closeLocked() - vecerror.InternalError(err) + colexecerror.InternalError(err) } }() @@ -282,7 +283,7 @@ func (i *Inbox) Next(ctx context.Context) coldata.Batch { if err := i.maybeInitLocked(ctx); err != nil { // An error occurred while initializing the Inbox and is likely caused by // the connection issues. It is expected that such an error can occur. - vecerror.ExpectedError(err) + colexecerror.ExpectedError(err) } for { @@ -306,7 +307,7 @@ func (i *Inbox) Next(ctx context.Context) coldata.Batch { return coldata.ZeroBatch } i.errCh <- err - vecerror.InternalError(err) + colexecerror.InternalError(err) } if len(m.Data.Metadata) != 0 { for _, rpm := range m.Data.Metadata { @@ -325,10 +326,10 @@ func (i *Inbox) Next(ctx context.Context) coldata.Batch { } i.scratch.data = i.scratch.data[:0] if err := i.serializer.Deserialize(&i.scratch.data, m.Data.RawBytes); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } if err := i.converter.ArrowToBatch(i.scratch.data, i.scratch.b); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return i.scratch.b } diff --git a/pkg/sql/colflow/colrpc/inbox_test.go b/pkg/sql/colflow/colrpc/inbox_test.go index 6ac485758ebf..4383fc8c5932 100644 --- a/pkg/sql/colflow/colrpc/inbox_test.go +++ b/pkg/sql/colflow/colrpc/inbox_test.go @@ -22,9 +22,10 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/col/colserde" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -66,7 +67,7 @@ func TestInboxCancellation(t *testing.T) { // Cancel the context. cancelFn() // Next should not block if the context is canceled. - err = vecerror.CatchVectorizedRuntimeError(func() { inbox.Next(ctx) }) + err = colexecerror.CatchVectorizedRuntimeError(func() { inbox.Next(ctx) }) require.True(t, testutils.IsError(err, "context canceled"), err) // Now, the remote stream arrives. err = inbox.RunWithStream(context.Background(), mockFlowStreamServer{}) @@ -159,7 +160,7 @@ func TestInboxTimeout(t *testing.T) { rpcLayer = makeMockFlowStreamRPCLayer() ) go func() { - readerErrCh <- vecerror.CatchVectorizedRuntimeError(func() { inbox.Next(ctx) }) + readerErrCh <- colexecerror.CatchVectorizedRuntimeError(func() { inbox.Next(ctx) }) }() // Timeout the inbox. @@ -200,7 +201,7 @@ func TestInboxShutdown(t *testing.T) { nextSleep = time.Millisecond * time.Duration(rng.Intn(10)) runWithStreamSleep = time.Millisecond * time.Duration(rng.Intn(10)) typs = []types.T{*types.Int} - batch = colbase.RandomBatch(testAllocator, rng, typs, coldata.BatchSize(), 0 /* length */, rng.Float64()) + batch = coldatatestutils.RandomBatch(testAllocator, rng, typs, coldata.BatchSize(), 0 /* length */, rng.Float64()) ) for _, runDrainMetaGoroutine := range []bool{false, true} { @@ -223,7 +224,7 @@ func TestInboxShutdown(t *testing.T) { inboxMemAccount := testMemMonitor.MakeBoundAccount() defer inboxMemAccount.Close(inboxCtx) inbox, err := NewInbox( - colbase.NewAllocator(inboxCtx, &inboxMemAccount), + colmem.NewAllocator(inboxCtx, &inboxMemAccount), typs, execinfrapb.StreamID(0), ) require.NoError(t, err) @@ -338,7 +339,7 @@ func TestInboxShutdown(t *testing.T) { err error ) for !done && err == nil { - err = vecerror.CatchVectorizedRuntimeError(func() { b := inbox.Next(inboxCtx); done = b.Length() == 0 }) + err = colexecerror.CatchVectorizedRuntimeError(func() { b := inbox.Next(inboxCtx); done = b.Length() == 0 }) } errCh <- err }() diff --git a/pkg/sql/colflow/colrpc/main_test.go b/pkg/sql/colflow/colrpc/main_test.go index 61c4d46d30b0..00de890ce0ab 100644 --- a/pkg/sql/colflow/colrpc/main_test.go +++ b/pkg/sql/colflow/colrpc/main_test.go @@ -16,7 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -24,7 +24,7 @@ import ( var ( // testAllocator is an Allocator with an unlimited budget for use in tests. - testAllocator *colbase.Allocator + testAllocator *colmem.Allocator // testMemMonitor and testMemAcc are a test monitor with an unlimited budget // and a memory account bound to it for use in tests. @@ -40,7 +40,7 @@ func TestMain(m *testing.M) { defer testMemMonitor.Stop(ctx) memAcc := testMemMonitor.MakeBoundAccount() testMemAcc = &memAcc - testAllocator = colbase.NewAllocator(ctx, testMemAcc) + testAllocator = colmem.NewAllocator(ctx, testMemAcc) defer testMemAcc.Close(ctx) return m.Run() }()) diff --git a/pkg/sql/colflow/colrpc/outbox.go b/pkg/sql/colflow/colrpc/outbox.go index 477c9798dca4..e47a1671e8b0 100644 --- a/pkg/sql/colflow/colrpc/outbox.go +++ b/pkg/sql/colflow/colrpc/outbox.go @@ -20,9 +20,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/colserde" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -75,8 +76,8 @@ type Outbox struct { // NewOutbox creates a new Outbox. func NewOutbox( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, typs []types.T, metadataSources []execinfrapb.MetadataSource, toClose []colexec.IdempotentCloser, @@ -247,7 +248,7 @@ func (o *Outbox) sendBatches( return true, nil } - if err := vecerror.CatchVectorizedRuntimeError(nextBatch); err != nil { + if err := colexecerror.CatchVectorizedRuntimeError(nextBatch); err != nil { if log.V(1) { log.Warningf(ctx, "Outbox Next error: %+v", err) } diff --git a/pkg/sql/colflow/colrpc/outbox_test.go b/pkg/sql/colflow/colrpc/outbox_test.go index 44988aa34618..7b28c1600184 100644 --- a/pkg/sql/colflow/colrpc/outbox_test.go +++ b/pkg/sql/colflow/colrpc/outbox_test.go @@ -16,7 +16,8 @@ import ( "sync/atomic" "testing" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -30,7 +31,7 @@ func TestOutboxCatchesPanics(t *testing.T) { ctx := context.Background() var ( - input = colbase.NewBatchBuffer() + input = colexecbase.NewBatchBuffer() typs = []types.T{*types.Int} rpcLayer = makeMockFlowStreamRPCLayer() ) @@ -53,7 +54,7 @@ func TestOutboxCatchesPanics(t *testing.T) { inboxMemAccount := testMemMonitor.MakeBoundAccount() defer inboxMemAccount.Close(ctx) inbox, err := NewInbox( - colbase.NewAllocator(ctx, &inboxMemAccount), typs, execinfrapb.StreamID(0), + colmem.NewAllocator(ctx, &inboxMemAccount), typs, execinfrapb.StreamID(0), ) require.NoError(t, err) @@ -80,13 +81,13 @@ func TestOutboxDrainsMetadataSources(t *testing.T) { ctx := context.Background() var ( - input = colbase.NewBatchBuffer() + input = colexecbase.NewBatchBuffer() typs = []types.T{*types.Int} ) // Define common function that returns both an Outbox and a pointer to a // uint32 that is set atomically when the outbox drains a metadata source. - newOutboxWithMetaSources := func(allocator *colbase.Allocator) (*Outbox, *uint32, error) { + newOutboxWithMetaSources := func(allocator *colmem.Allocator) (*Outbox, *uint32, error) { var sourceDrained uint32 outbox, err := NewOutbox(allocator, input, typs, []execinfrapb.MetadataSource{ execinfrapb.CallbackMetadataSource{ @@ -107,7 +108,7 @@ func TestOutboxDrainsMetadataSources(t *testing.T) { outboxMemAccount := testMemMonitor.MakeBoundAccount() defer outboxMemAccount.Close(ctx) outbox, sourceDrained, err := newOutboxWithMetaSources( - colbase.NewAllocator(ctx, &outboxMemAccount), + colmem.NewAllocator(ctx, &outboxMemAccount), ) require.NoError(t, err) diff --git a/pkg/sql/colflow/main_test.go b/pkg/sql/colflow/main_test.go index 936fe6bd6ce1..0a67f74d3044 100644 --- a/pkg/sql/colflow/main_test.go +++ b/pkg/sql/colflow/main_test.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/securitytest" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" @@ -31,7 +31,7 @@ import ( var ( // testAllocator is an Allocator with an unlimited budget for use in tests. - testAllocator *colbase.Allocator + testAllocator *colmem.Allocator // testMemMonitor and testMemAcc are a test monitor with an unlimited budget // and a memory account bound to it for use in tests. @@ -55,7 +55,7 @@ func TestMain(m *testing.M) { defer testMemMonitor.Stop(ctx) memAcc := testMemMonitor.MakeBoundAccount() testMemAcc = &memAcc - testAllocator = colbase.NewAllocator(ctx, testMemAcc) + testAllocator = colmem.NewAllocator(ctx, testMemAcc) defer testMemAcc.Close(ctx) testDiskMonitor = execinfra.NewTestDiskMonitor(ctx, cluster.MakeTestingClusterSettings()) diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index 611e8cbea011..73d489e0d04b 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -21,14 +21,15 @@ import ( "sync/atomic" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/settings" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colflow/colrpc" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" @@ -206,7 +207,7 @@ func (f *vectorizedFlow) Setup( } log.VEventf(ctx, 1, "flow %s spilled to disk, stack trace: %s", f.ID, util.GetSmallTrace(2)) if err := f.Cfg.TempFS.CreateDir(f.tempStorage.path); err != nil { - vecerror.InternalError(errors.Errorf("unable to create temporary storage directory: %v", err)) + colexecerror.InternalError(errors.Errorf("unable to create temporary storage directory: %v", err)) } f.tempStorage.createdStateMu.created = true }, @@ -347,8 +348,8 @@ func (f *vectorizedFlow) Cleanup(ctx context.Context) { // corresponding to operators in inputs (the latter must have already been // wrapped). func wrapWithVectorizedStatsCollector( - op colbase.Operator, - inputs []colbase.Operator, + op colexecbase.Operator, + inputs []colexecbase.Operator, pspec *execinfrapb.ProcessorSpec, monitors []*mon.BytesMonitor, ) (*colexec.VectorizedStatsCollector, error) { @@ -440,7 +441,7 @@ type flowCreatorHelper interface { // as the metadataSources and closers in this DAG that need to be drained and // closed. type opDAGWithMetaSources struct { - rootOperator colbase.Operator + rootOperator colexecbase.Operator metadataSources []execinfrapb.MetadataSource toClose []colexec.IdempotentCloser } @@ -449,20 +450,20 @@ type opDAGWithMetaSources struct { // several components in a remote flow. Mostly for testing purposes. type remoteComponentCreator interface { newOutbox( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, typs []types.T, metadataSources []execinfrapb.MetadataSource, toClose []colexec.IdempotentCloser, ) (*colrpc.Outbox, error) - newInbox(allocator *colbase.Allocator, typs []types.T, streamID execinfrapb.StreamID) (*colrpc.Inbox, error) + newInbox(allocator *colmem.Allocator, typs []types.T, streamID execinfrapb.StreamID) (*colrpc.Inbox, error) } type vectorizedRemoteComponentCreator struct{} func (vectorizedRemoteComponentCreator) newOutbox( - allocator *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, typs []types.T, metadataSources []execinfrapb.MetadataSource, toClose []colexec.IdempotentCloser, @@ -471,7 +472,7 @@ func (vectorizedRemoteComponentCreator) newOutbox( } func (vectorizedRemoteComponentCreator) newInbox( - allocator *colbase.Allocator, typs []types.T, streamID execinfrapb.StreamID, + allocator *colmem.Allocator, typs []types.T, streamID execinfrapb.StreamID, ) (*colrpc.Inbox, error) { return colrpc.NewInbox(allocator, typs, streamID) } @@ -595,14 +596,14 @@ func (s *vectorizedFlowCreator) newStreamingMemAccount( func (s *vectorizedFlowCreator) setupRemoteOutputStream( ctx context.Context, flowCtx *execinfra.FlowCtx, - op colbase.Operator, + op colexecbase.Operator, outputTyps []types.T, stream *execinfrapb.StreamEndpointSpec, metadataSourcesQueue []execinfrapb.MetadataSource, toClose []colexec.IdempotentCloser, ) (execinfra.OpNode, error) { outbox, err := s.remoteComponentCreator.newOutbox( - colbase.NewAllocator(ctx, s.newStreamingMemAccount(flowCtx)), + colmem.NewAllocator(ctx, s.newStreamingMemAccount(flowCtx)), op, outputTyps, metadataSourcesQueue, toClose, ) if err != nil { @@ -639,7 +640,7 @@ func (s *vectorizedFlowCreator) setupRemoteOutputStream( func (s *vectorizedFlowCreator) setupRouter( ctx context.Context, flowCtx *execinfra.FlowCtx, - input colbase.Operator, + input colexecbase.Operator, outputTyps []types.T, output *execinfrapb.OutputRouterSpec, metadataSourcesQueue []execinfrapb.MetadataSource, @@ -657,10 +658,10 @@ func (s *vectorizedFlowCreator) setupRouter( mmName := "hash-router-[" + strings.Join(streamIDs, ",") + "]" hashRouterMemMonitor := s.createBufferingUnlimitedMemMonitor(ctx, flowCtx, mmName) - allocators := make([]*colbase.Allocator, len(output.Streams)) + allocators := make([]*colmem.Allocator, len(output.Streams)) for i := range allocators { acc := hashRouterMemMonitor.MakeBoundAccount() - allocators[i] = colbase.NewAllocator(ctx, &acc) + allocators[i] = colmem.NewAllocator(ctx, &acc) s.accounts = append(s.accounts, &acc) } limit := execinfra.GetWorkMemLimit(flowCtx.Cfg) @@ -735,8 +736,8 @@ func (s *vectorizedFlowCreator) setupInput( flowCtx *execinfra.FlowCtx, input execinfrapb.InputSyncSpec, opt flowinfra.FuseOpt, -) (op colbase.Operator, _ []execinfrapb.MetadataSource, _ error) { - inputStreamOps := make([]colbase.Operator, 0, len(input.Streams)) +) (op colexecbase.Operator, _ []execinfrapb.MetadataSource, _ error) { + inputStreamOps := make([]colexecbase.Operator, 0, len(input.Streams)) metaSources := make([]execinfrapb.MetadataSource, 0, len(input.Streams)) for _, inputStream := range input.Streams { switch inputStream.Type { @@ -754,7 +755,7 @@ func (s *vectorizedFlowCreator) setupInput( return nil, nil, err } inbox, err := s.remoteComponentCreator.newInbox( - colbase.NewAllocator(ctx, s.newStreamingMemAccount(flowCtx)), + colmem.NewAllocator(ctx, s.newStreamingMemAccount(flowCtx)), input.ColumnTypes, inputStream.StreamID, ) if err != nil { @@ -793,7 +794,7 @@ func (s *vectorizedFlowCreator) setupInput( if input.Type == execinfrapb.InputSyncSpec_ORDERED { var err error op, err = colexec.NewOrderedSynchronizer( - colbase.NewAllocator(ctx, s.newStreamingMemAccount(flowCtx)), + colmem.NewAllocator(ctx, s.newStreamingMemAccount(flowCtx)), inputStreamOps, input.ColumnTypes, execinfrapb.ConvertToColumnOrdering(input.Ordering), ) if err != nil { @@ -834,7 +835,7 @@ func (s *vectorizedFlowCreator) setupOutput( ctx context.Context, flowCtx *execinfra.FlowCtx, pspec *execinfrapb.ProcessorSpec, - op colbase.Operator, + op colexecbase.Operator, opOutputTypes []types.T, metadataSourcesQueue []execinfrapb.MetadataSource, toClose []colexec.IdempotentCloser, @@ -968,7 +969,7 @@ func (s *vectorizedFlowCreator) setupFlow( queue = append(queue, i) } - inputs := make([]colbase.Operator, 0, 2) + inputs := make([]colexecbase.Operator, 0, 2) for len(queue) > 0 { pspec := &processorSpecs[queue[0]] queue = queue[1:] @@ -1098,7 +1099,7 @@ func (s *vectorizedFlowCreator) setupFlow( } if len(s.vectorizedStatsCollectorsQueue) > 0 { - vecerror.InternalError("not all vectorized stats collectors have been processed") + colexecerror.InternalError("not all vectorized stats collectors have been processed") } return s.leaves, nil } @@ -1262,7 +1263,7 @@ func SupportsVectorized( mon.Stop(ctx) } }() - if vecErr := vecerror.CatchVectorizedRuntimeError(func() { + if vecErr := colexecerror.CatchVectorizedRuntimeError(func() { leaves, err = creator.setupFlow(ctx, flowCtx, processorSpecs, fuseOpt) }); vecErr != nil { return leaves, vecErr diff --git a/pkg/sql/colflow/vectorized_flow_shutdown_test.go b/pkg/sql/colflow/vectorized_flow_shutdown_test.go index e9cfc43f7e03..248e7908ef7d 100644 --- a/pkg/sql/colflow/vectorized_flow_shutdown_test.go +++ b/pkg/sql/colflow/vectorized_flow_shutdown_test.go @@ -20,10 +20,12 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colflow/colrpc" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -144,10 +146,10 @@ func TestVectorizedFlowShutdown(t *testing.T) { err error wg sync.WaitGroup typs = []types.T{*types.Int} - hashRouterInput = colbase.NewRandomDataOp( + hashRouterInput = coldatatestutils.NewRandomDataOp( testAllocator, rng, - colbase.RandomDataOpArgs{ + coldatatestutils.RandomDataOpArgs{ DeterministicTyps: typs, // Set a high number of batches to ensure that the HashRouter is // very far from being finished when the flow is shut down. @@ -159,37 +161,37 @@ func TestVectorizedFlowShutdown(t *testing.T) { numInboxes = numHashRouterOutputs + 3 inboxes = make([]*colrpc.Inbox, 0, numInboxes+1) handleStreamErrCh = make([]chan error, numInboxes+1) - synchronizerInputs = make([]colbase.Operator, 0, numInboxes) + synchronizerInputs = make([]colexecbase.Operator, 0, numInboxes) materializerMetadataSources = make([]execinfrapb.MetadataSource, 0, numInboxes+1) streamID = 0 addAnotherRemote = rng.Float64() < 0.5 ) // Create an allocator for each output. - allocators := make([]*colbase.Allocator, numHashRouterOutputs) + allocators := make([]*colmem.Allocator, numHashRouterOutputs) diskAccounts := make([]*mon.BoundAccount, numHashRouterOutputs) for i := range allocators { acc := testMemMonitor.MakeBoundAccount() defer acc.Close(ctxRemote) - allocators[i] = colbase.NewAllocator(ctxRemote, &acc) + allocators[i] = colmem.NewAllocator(ctxRemote, &acc) diskAcc := testDiskMonitor.MakeBoundAccount() diskAccounts[i] = &diskAcc defer diskAcc.Close(ctxRemote) } hashRouter, hashRouterOutputs := colexec.NewHashRouter( allocators, hashRouterInput, typs, []uint32{0}, 64<<20, /* 64 MiB */ - queueCfg, &colbase.TestingSemaphore{}, diskAccounts, + queueCfg, &colexecbase.TestingSemaphore{}, diskAccounts, ) for i := 0; i < numInboxes; i++ { inboxMemAccount := testMemMonitor.MakeBoundAccount() defer inboxMemAccount.Close(ctxLocal) inbox, err := colrpc.NewInbox( - colbase.NewAllocator(ctxLocal, &inboxMemAccount), typs, execinfrapb.StreamID(streamID), + colmem.NewAllocator(ctxLocal, &inboxMemAccount), typs, execinfrapb.StreamID(streamID), ) require.NoError(t, err) inboxes = append(inboxes, inbox) materializerMetadataSources = append(materializerMetadataSources, inbox) - synchronizerInputs = append(synchronizerInputs, colbase.Operator(inbox)) + synchronizerInputs = append(synchronizerInputs, colexecbase.Operator(inbox)) } synchronizer := colexec.NewParallelUnorderedSynchronizer(synchronizerInputs, typs, &wg) flowID := execinfrapb.FlowID{UUID: uuid.MakeV4()} @@ -204,7 +206,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { ctx context.Context, cancelFn context.CancelFunc, outboxMemAcc *mon.BoundAccount, - outboxInput colbase.Operator, + outboxInput colexecbase.Operator, inbox *colrpc.Inbox, id int, outboxMetadataSources []execinfrapb.MetadataSource, @@ -212,7 +214,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { idToClosed.Lock() idToClosed.mapping[id] = false idToClosed.Unlock() - outbox, err := colrpc.NewOutbox(colbase.NewAllocator(ctx, outboxMemAcc), outboxInput, typs, append(outboxMetadataSources, + outbox, err := colrpc.NewOutbox(colmem.NewAllocator(ctx, outboxMemAcc), outboxInput, typs, append(outboxMetadataSources, execinfrapb.CallbackMetadataSource{ DrainMetaCb: func(ctx context.Context) []execinfrapb.ProducerMetadata { return []execinfrapb.ProducerMetadata{{Err: errors.Errorf("%d", id)}} @@ -262,22 +264,22 @@ func TestVectorizedFlowShutdown(t *testing.T) { } else { sourceMemAccount := testMemMonitor.MakeBoundAccount() defer sourceMemAccount.Close(ctxRemote) - remoteAllocator := colbase.NewAllocator(ctxRemote, &sourceMemAccount) + remoteAllocator := colmem.NewAllocator(ctxRemote, &sourceMemAccount) batch := remoteAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) - runOutboxInbox(ctxRemote, cancelRemote, &outboxMemAccount, colbase.NewRepeatableBatchSource(remoteAllocator, batch, typs), inboxes[i], streamID, outboxMetadataSources) + runOutboxInbox(ctxRemote, cancelRemote, &outboxMemAccount, colexecbase.NewRepeatableBatchSource(remoteAllocator, batch, typs), inboxes[i], streamID, outboxMetadataSources) } streamID++ } - var materializerInput colbase.Operator + var materializerInput colexecbase.Operator ctxAnotherRemote, cancelAnotherRemote := context.WithCancel(context.Background()) if addAnotherRemote { // Add another "remote" node to the flow. inboxMemAccount := testMemMonitor.MakeBoundAccount() defer inboxMemAccount.Close(ctxAnotherRemote) inbox, err := colrpc.NewInbox( - colbase.NewAllocator(ctxAnotherRemote, &inboxMemAccount), + colmem.NewAllocator(ctxAnotherRemote, &inboxMemAccount), typs, execinfrapb.StreamID(streamID), ) require.NoError(t, err) diff --git a/pkg/sql/colflow/vectorized_flow_space_test.go b/pkg/sql/colflow/vectorized_flow_space_test.go index c04fbe2cb46e..b511ff3e1fd4 100644 --- a/pkg/sql/colflow/vectorized_flow_space_test.go +++ b/pkg/sql/colflow/vectorized_flow_space_test.go @@ -17,9 +17,9 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -82,7 +82,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 := []colbase.Operator{colexec.NewZeroOp(nil)} + inputs := []colexecbase.Operator{colexec.NewZeroOp(nil)} if len(tc.spec.Input) > 1 { inputs = append(inputs, colexec.NewZeroOp(nil)) } @@ -200,9 +200,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 := []colbase.Operator{colbase.NewRepeatableBatchSource(testAllocator, batch, typs)} + inputs := []colexecbase.Operator{colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs)} if len(tc.spec.Input) > 1 { - inputs = append(inputs, colbase.NewRepeatableBatchSource(testAllocator, batch, typs)) + inputs = append(inputs, colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs)) } memMon := mon.MakeMonitor("MemoryMonitor", mon.MemoryResource, nil, nil, 0, math.MaxInt64, st) flowCtx.Cfg.TestingKnobs = execinfra.TestingKnobs{} @@ -226,7 +226,7 @@ func TestVectorizeAllocatorSpaceError(t *testing.T) { Spec: tc.spec, Inputs: inputs, StreamingMemAccount: &acc, - FDSemaphore: colbase.NewTestingSemaphore(256), + FDSemaphore: colexecbase.NewTestingSemaphore(256), } // The disk spilling infrastructure relies on different memory // accounts, so if the spilling is supported, we do *not* want to use @@ -234,7 +234,7 @@ func TestVectorizeAllocatorSpaceError(t *testing.T) { args.TestingKnobs.UseStreamingMemAccountForBuffering = !tc.spillingSupported result, err := colexec.NewColOperator(ctx, flowCtx, args) require.NoError(t, err) - err = vecerror.CatchVectorizedRuntimeError(func() { + err = colexecerror.CatchVectorizedRuntimeError(func() { result.Op.Init() result.Op.Next(ctx) result.Op.Next(ctx) diff --git a/pkg/sql/colflow/vectorized_flow_test.go b/pkg/sql/colflow/vectorized_flow_test.go index 2d354290dd97..5cd8e5175e30 100644 --- a/pkg/sql/colflow/vectorized_flow_test.go +++ b/pkg/sql/colflow/vectorized_flow_test.go @@ -19,10 +19,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colflow/colrpc" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" @@ -35,13 +36,13 @@ import ( ) type callbackRemoteComponentCreator struct { - newOutboxFn func(*colbase.Allocator, colbase.Operator, []types.T, []execinfrapb.MetadataSource) (*colrpc.Outbox, error) - newInboxFn func(allocator *colbase.Allocator, typs []types.T, streamID execinfrapb.StreamID) (*colrpc.Inbox, error) + newOutboxFn func(*colmem.Allocator, colexecbase.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 *colbase.Allocator, - input colbase.Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, typs []types.T, metadataSources []execinfrapb.MetadataSource, toClose []colexec.IdempotentCloser, @@ -50,7 +51,7 @@ func (c callbackRemoteComponentCreator) newOutbox( } func (c callbackRemoteComponentCreator) newInbox( - allocator *colbase.Allocator, typs []types.T, streamID execinfrapb.StreamID, + allocator *colmem.Allocator, typs []types.T, streamID execinfrapb.StreamID, ) (*colrpc.Inbox, error) { return c.newInboxFn(allocator, typs, streamID) } @@ -186,8 +187,8 @@ func TestDrainOnlyInputDAG(t *testing.T) { outboxCreated := false componentCreator := callbackRemoteComponentCreator{ newOutboxFn: func( - allocator *colbase.Allocator, - op colbase.Operator, + allocator *colmem.Allocator, + op colexecbase.Operator, typs []types.T, sources []execinfrapb.MetadataSource, ) (*colrpc.Outbox, error) { @@ -201,7 +202,7 @@ func TestDrainOnlyInputDAG(t *testing.T) { require.Len(t, inboxToNumInputTypes[sources[0].(*colrpc.Inbox)], numInputTypesToOutbox) return colrpc.NewOutbox(allocator, op, typs, sources, nil /* toClose */) }, - newInboxFn: func(allocator *colbase.Allocator, typs []types.T, streamID execinfrapb.StreamID) (*colrpc.Inbox, error) { + newInboxFn: func(allocator *colmem.Allocator, typs []types.T, streamID execinfrapb.StreamID) (*colrpc.Inbox, error) { inbox, err := colrpc.NewInbox(allocator, typs, streamID) inboxToNumInputTypes[inbox] = typs return inbox, err @@ -254,7 +255,7 @@ func TestVectorizedFlowTempDirectory(t *testing.T) { Cfg: &execinfra.ServerConfig{ TempFS: ngn, TempStoragePath: tempPath, - VecFDSemaphore: &colbase.TestingSemaphore{}, + VecFDSemaphore: &colexecbase.TestingSemaphore{}, Metrics: &execinfra.DistSQLMetrics{}, }, EvalCtx: &evalCtx, diff --git a/pkg/sql/colflow/vectorized_panic_propagation_test.go b/pkg/sql/colflow/vectorized_panic_propagation_test.go index 96cd2f486b61..88c8282c9f2d 100644 --- a/pkg/sql/colflow/vectorized_panic_propagation_test.go +++ b/pkg/sql/colflow/vectorized_panic_propagation_test.go @@ -16,9 +16,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -122,7 +122,7 @@ func TestNonVectorizedPanicPropagation(t *testing.T) { } // testVectorizedInternalPanicEmitter is an colexec.Operator that panics with -// vecerror.InternalError on every odd-numbered invocation of Next() +// colexecerror.InternalError on every odd-numbered invocation of Next() // 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 { @@ -130,9 +130,9 @@ type testVectorizedInternalPanicEmitter struct { emitBatch bool } -var _ colbase.Operator = &testVectorizedInternalPanicEmitter{} +var _ colexecbase.Operator = &testVectorizedInternalPanicEmitter{} -func newTestVectorizedInternalPanicEmitter(input colbase.Operator) colbase.Operator { +func newTestVectorizedInternalPanicEmitter(input colexecbase.Operator) colexecbase.Operator { return &testVectorizedInternalPanicEmitter{ OneInputNode: colexec.NewOneInputNode(input), } @@ -147,7 +147,7 @@ func (e *testVectorizedInternalPanicEmitter) Init() { func (e *testVectorizedInternalPanicEmitter) Next(ctx context.Context) coldata.Batch { if !e.emitBatch { e.emitBatch = true - vecerror.InternalError("") + colexecerror.InternalError("") } e.emitBatch = false @@ -163,9 +163,9 @@ type testNonVectorizedPanicEmitter struct { emitBatch bool } -var _ colbase.Operator = &testVectorizedInternalPanicEmitter{} +var _ colexecbase.Operator = &testVectorizedInternalPanicEmitter{} -func newTestNonVectorizedPanicEmitter(input colbase.Operator) colbase.Operator { +func newTestNonVectorizedPanicEmitter(input colexecbase.Operator) colexecbase.Operator { return &testNonVectorizedPanicEmitter{ OneInputNode: colexec.NewOneInputNode(input), } diff --git a/pkg/sql/colbase/allocator.go b/pkg/sql/colmem/allocator.go similarity index 95% rename from pkg/sql/colbase/allocator.go rename to pkg/sql/colmem/allocator.go index ee3d075053c7..c19c302410ef 100644 --- a/pkg/sql/colbase/allocator.go +++ b/pkg/sql/colmem/allocator.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package colbase +package colmem import ( "context" @@ -18,8 +18,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -92,7 +92,7 @@ func (a *Allocator) NewMemBatch(typs []types.T) coldata.Batch { func (a *Allocator) NewMemBatchWithSize(typs []types.T, size int) coldata.Batch { estimatedMemoryUsage := selVectorSize(size) + int64(EstimateBatchSizeBytesFromSQLTypes(typs, size)) if err := a.acc.Grow(a.ctx, estimatedMemoryUsage); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return coldata.NewMemBatchWithSize(typs, size) } @@ -103,7 +103,7 @@ func (a *Allocator) NewMemBatchWithSize(typs []types.T, size int) coldata.Batch func (a *Allocator) NewMemBatchNoCols(types []types.T, size int) coldata.Batch { estimatedMemoryUsage := selVectorSize(size) if err := a.acc.Grow(a.ctx, estimatedMemoryUsage); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return coldata.NewMemBatchNoCols(types, size) } @@ -126,7 +126,7 @@ func (a *Allocator) RetainBatch(b coldata.Batch) { usesSel := b.Selection() != nil b.SetSelection(true) if err := a.acc.Grow(a.ctx, selVectorSize(cap(b.Selection()))+getVecsMemoryFootprint(b.ColVecs())); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } b.SetSelection(usesSel) } @@ -157,7 +157,7 @@ func (a *Allocator) ReleaseBatch(b coldata.Batch) { func (a *Allocator) NewMemColumn(t *types.T, n int) coldata.Vec { estimatedMemoryUsage := int64(EstimateBatchSizeBytes([]coltypes.T{typeconv.FromColumnType(t)}, n)) if err := a.acc.Grow(a.ctx, estimatedMemoryUsage); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } return coldata.NewMemColumn(t, n) } @@ -175,7 +175,7 @@ func (a *Allocator) NewMemColumn(t *types.T, n int) coldata.Vec { // NOTE: b must be non-zero length batch. func (a *Allocator) MaybeAppendColumn(b coldata.Batch, t *types.T, colIdx int) { if b.Length() == 0 { - vecerror.InternalError("trying to add a column to zero length batch") + colexecerror.InternalError("trying to add a column to zero length batch") } width := b.Width() desiredPhysType := typeconv.FromColumnType(t) @@ -186,7 +186,7 @@ func (a *Allocator) MaybeAppendColumn(b coldata.Batch, t *types.T, colIdx int) { return default: // We have a vector with an unexpected type, so we panic. - vecerror.InternalError(errors.Errorf( + colexecerror.InternalError(errors.Errorf( "trying to add a column of %s type at index %d but %s vector already present", t, colIdx, presentPhysType, )) @@ -194,14 +194,14 @@ func (a *Allocator) MaybeAppendColumn(b coldata.Batch, t *types.T, colIdx int) { } else if colIdx > width { // We have a batch of unexpected width which indicates an error in the // planning stage. - vecerror.InternalError(errors.Errorf( + colexecerror.InternalError(errors.Errorf( "trying to add a column of %s type at index %d but batch has width %d", t, colIdx, width, )) } estimatedMemoryUsage := int64(EstimateBatchSizeBytes([]coltypes.T{desiredPhysType}, coldata.BatchSize())) if err := a.acc.Grow(a.ctx, estimatedMemoryUsage); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } b.AppendCol(a.NewMemColumn(t, coldata.BatchSize())) } @@ -221,7 +221,7 @@ func (a *Allocator) PerformOperation(destVecs []coldata.Vec, operation func()) { delta := after - before if delta >= 0 { if err := a.acc.Grow(a.ctx, delta); err != nil { - vecerror.InternalError(err) + colexecerror.InternalError(err) } } else { a.ReleaseMemory(-delta) @@ -303,7 +303,7 @@ func EstimateBatchSizeBytes(vecTypes []coltypes.T, batchLength int) int { case coltypes.Unhandled: // Placeholder coldata.Vecs of unknown types are allowed. default: - vecerror.InternalError(fmt.Sprintf("unhandled type %s", t)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", t)) } } return acc * batchLength diff --git a/pkg/sql/distsql/columnar_operators_test.go b/pkg/sql/distsql/columnar_operators_test.go index f5bc826e82fb..e4d0bdde0c49 100644 --- a/pkg/sql/distsql/columnar_operators_test.go +++ b/pkg/sql/distsql/columnar_operators_test.go @@ -20,8 +20,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" diff --git a/pkg/sql/distsql/columnar_utils_test.go b/pkg/sql/distsql/columnar_utils_test.go index 7fab2648f5a5..5bb0c12c47bc 100644 --- a/pkg/sql/distsql/columnar_utils_test.go +++ b/pkg/sql/distsql/columnar_utils_test.go @@ -18,9 +18,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" @@ -98,8 +99,8 @@ func verifyColOperator(args verifyColOperatorArgs) error { acc := evalCtx.Mon.MakeBoundAccount() defer acc.Close(ctx) - testAllocator := colbase.NewAllocator(ctx, &acc) - columnarizers := make([]colbase.Operator, len(args.inputs)) + testAllocator := colmem.NewAllocator(ctx, &acc) + columnarizers := make([]colexecbase.Operator, len(args.inputs)) for i, input := range inputsColOp { c, err := colexec.NewColumnarizer(ctx, testAllocator, flowCtx, int32(i)+1, input) if err != nil { @@ -114,7 +115,7 @@ func verifyColOperator(args verifyColOperatorArgs) error { StreamingMemAccount: &acc, ProcessorConstructor: rowexec.NewProcessor, DiskQueueCfg: colcontainer.DiskQueueCfg{FS: tempFS}, - FDSemaphore: colbase.NewTestingSemaphore(256), + FDSemaphore: colexecbase.NewTestingSemaphore(256), } var spilled bool if args.forceDiskSpill { diff --git a/pkg/sql/distsql/vectorized_panic_propagation_test.go b/pkg/sql/distsql/vectorized_panic_propagation_test.go index fa598dbc6fac..fc0e73fb4dcb 100644 --- a/pkg/sql/distsql/vectorized_panic_propagation_test.go +++ b/pkg/sql/distsql/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/colbase" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colflow" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -54,7 +54,7 @@ func TestNonVectorizedPanicDoesntHangServer(t *testing.T) { mat, err := colexec.NewMaterializer( &flowCtx, 0, /* processorID */ - &colbase.CallbackOperator{ + &colexecbase.CallbackOperator{ NextCb: func(ctx context.Context) coldata.Batch { panic("") }, diff --git a/pkg/sql/explain_vec.go b/pkg/sql/explain_vec.go index fd437784695f..2ad221887d6c 100644 --- a/pkg/sql/explain_vec.go +++ b/pkg/sql/explain_vec.go @@ -16,8 +16,8 @@ import ( "sort" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/vecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colflow" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -108,7 +108,7 @@ func (n *explainVecNode) startExec(params runParams) error { // It is possible that when iterating over execinfra.OpNodes we will hit // a panic (an input that doesn't implement OpNode interface), so we're // catching such errors. - if err := vecerror.CatchVectorizedRuntimeError(func() { + if err := colexecerror.CatchVectorizedRuntimeError(func() { for _, op := range opChains { formatOpChain(op, node, verbose) } diff --git a/pkg/sql/sem/tree/eval_test.go b/pkg/sql/sem/tree/eval_test.go index c5943547b9ca..04bd624b2bb1 100644 --- a/pkg/sql/sem/tree/eval_test.go +++ b/pkg/sql/sem/tree/eval_test.go @@ -22,8 +22,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colbase" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder" @@ -210,8 +210,8 @@ func TestEval(t *testing.T) { RenderExprs: []execinfrapb.Expression{{Expr: d.Input}}, }, }, - Inputs: []colbase.Operator{ - &colbase.CallbackOperator{ + Inputs: []colexecbase.Operator{ + &colexecbase.CallbackOperator{ NextCb: func(_ context.Context) coldata.Batch { if batchesReturned > 0 { return coldata.ZeroBatch diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 615e8a42898d..6f6bd369b527 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -920,7 +920,7 @@ func TestLint(t *testing.T) { ":!*.pb.gw.go", ":!sql/pgwire/pgerror/severity.go", ":!sql/pgwire/pgerror/with_candidate_code.go", - ":!sql/colbase/vecerror/error.go", + ":!sql/colexecbase/colexecerror/error.go", ":!util/protoutil/jsonpb_marshal.go", ":!util/protoutil/marshal.go", ":!util/protoutil/marshaler.go", @@ -1421,10 +1421,9 @@ func TestLint(t *testing.T) { // NOTE: if you're adding a new package to the list here because it // uses "panic-catch" error propagation mechanism of the vectorized // engine, don't forget to "register" the newly added package in - // sql/colbase/vecerror/error.go file. + // sql/colexecbase/colexecerror/error.go file. "sql/col*", - ":!sql/colbase/typeconv/typeconv.go", - ":!sql/colbase/vecerror/error.go", + ":!sql/colexecbase/colexecerror/error.go", ":!sql/colexec/execpb/stats.pb.go", ":!sql/colflow/vectorized_panic_propagation_test.go", ) @@ -1437,7 +1436,7 @@ func TestLint(t *testing.T) { } if err := stream.ForEach(filter, func(s string) { - t.Errorf("\n%s <- forbidden; use either vecerror.InternalError() or vecerror.ExpectedError() instead", s) + t.Errorf("\n%s <- forbidden; use either colexecerror.InternalError() or colexecerror.ExpectedError() instead", s) }); err != nil { t.Error(err) } @@ -1464,6 +1463,8 @@ func TestLint(t *testing.T) { // TODO(yuzefovich): prohibit call to coldata.NewMemBatchNoCols. fmt.Sprintf(`(coldata\.NewMem(Batch|BatchWithSize|Column)|\.AppendCol)\(`), "--", + // TODO(yuzefovich): prohibit calling coldata.* methods from other + // sql/col* packages. "sql/colexec", "sql/colflow", ":!sql/colexec/simple_project.go", @@ -1477,7 +1478,7 @@ func TestLint(t *testing.T) { } if err := stream.ForEach(filter, func(s string) { - t.Errorf("\n%s <- forbidden; use colbase.Allocator object instead", s) + t.Errorf("\n%s <- forbidden; use colmem.Allocator object instead", s) }); err != nil { t.Error(err) } @@ -1500,8 +1501,9 @@ func TestLint(t *testing.T) { // vectorTypeEnforcer and batchSchemaPrefixEnforcer. fmt.Sprintf(`(MaybeAppendColumn)\(`), "--", - "sql/colexec", + "sql/col*", ":!sql/colexec/operator.go", + ":!sql/colmem/allocator.go", ) if err != nil { t.Fatal(err) diff --git a/pkg/workload/dep_test.go b/pkg/workload/dep_test.go index 664602182d3a..1287ce849799 100644 --- a/pkg/workload/dep_test.go +++ b/pkg/workload/dep_test.go @@ -26,8 +26,8 @@ func TestDepWhitelist(t *testing.T) { []string{ `github.com/cockroachdb/cockroach/pkg/col/coldata`, `github.com/cockroachdb/cockroach/pkg/col/coltypes`, + `github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv`, `github.com/cockroachdb/cockroach/pkg/geo/geopb`, - `github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv`, `github.com/cockroachdb/cockroach/pkg/sql/lex`, `github.com/cockroachdb/cockroach/pkg/sql/oidext`, `github.com/cockroachdb/cockroach/pkg/sql/types`, diff --git a/pkg/workload/workload.go b/pkg/workload/workload.go index d71a53fe6d0d..3aa257038f21 100644 --- a/pkg/workload/workload.go +++ b/pkg/workload/workload.go @@ -26,7 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colbase/typeconv" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/bufalloc" "github.com/cockroachdb/cockroach/pkg/workload/histogram"