diff --git a/pkg/base/store_spec.go b/pkg/base/store_spec.go index be35f2e7b39a..dd9235b9acbd 100644 --- a/pkg/base/store_spec.go +++ b/pkg/base/store_spec.go @@ -26,8 +26,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/netutil" + "github.com/cockroachdb/errors" humanize "github.com/dustin/go-humanize" - "github.com/pkg/errors" "github.com/spf13/pflag" ) @@ -369,12 +369,23 @@ func PreventedStartupFile(dir string) string { return filepath.Join(dir, "_CRITICAL_ALERT.txt") } -// GetPreventedStartupMessage attempts to read the PreventedStartupFile for each -// store directory and returns their concatenated contents. These files -// typically request operator intervention after a corruption event by -// preventing the affected node(s) from starting back up. -func (ssl StoreSpecList) GetPreventedStartupMessage() (string, error) { - var buf strings.Builder +// PriorCriticalAlertError attempts to read the +// PreventedStartupFile for each store directory and returns their +// contents as a structured error. +// +// These files typically request operator intervention after a +// corruption event by preventing the affected node(s) from starting +// back up. +func (ssl StoreSpecList) PriorCriticalAlertError() (err error) { + addError := func(newErr error) { + if err == nil { + err = errors.New("startup forbidden by prior critical alert") + } + // We use WithDetailf here instead of errors.CombineErrors + // because we want the details to be printed to the screen + // (combined errors only show up via %+v). + err = errors.WithDetailf(err, "%v", newErr) + } for _, ss := range ssl.Specs { path := ss.PreventedStartupFile() if path == "" { @@ -383,15 +394,13 @@ func (ssl StoreSpecList) GetPreventedStartupMessage() (string, error) { b, err := ioutil.ReadFile(path) if err != nil { if !os.IsNotExist(err) { - return "", err + addError(errors.Wrapf(err, "%s", path)) } continue } - fmt.Fprintf(&buf, "From %s:\n\n", path) - _, _ = buf.Write(b) - fmt.Fprintln(&buf) + addError(errors.Newf("From %s:\n\n%s\n", path, b)) } - return buf.String(), nil + return err } // PreventedStartupFile returns the path to a file which, if it exists, should diff --git a/pkg/base/store_spec_test.go b/pkg/base/store_spec_test.go index b550db9e3889..7530440a6982 100644 --- a/pkg/base/store_spec_test.go +++ b/pkg/base/store_spec_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) @@ -245,13 +246,13 @@ func TestStoreSpecListPreventedStartupMessage(t *testing.T) { }, } - msg, err := ssl.GetPreventedStartupMessage() + err := ssl.PriorCriticalAlertError() require.NoError(t, err) - require.Empty(t, msg) require.NoError(t, ioutil.WriteFile(ssl.Specs[2].PreventedStartupFile(), []byte("boom"), 0644)) - msg, err = ssl.GetPreventedStartupMessage() - require.NoError(t, err) - require.Contains(t, msg, "boom") + err = ssl.PriorCriticalAlertError() + require.Error(t, err) + require.Contains(t, err.Error(), "startup forbidden by prior critical alert") + require.Contains(t, errors.FlattenDetails(err), "boom") } diff --git a/pkg/ccl/backupccl/backup_job.go b/pkg/ccl/backupccl/backup_job.go index f756dd9cbd6a..08a2869f1c27 100644 --- a/pkg/ccl/backupccl/backup_job.go +++ b/pkg/ccl/backupccl/backup_job.go @@ -541,6 +541,14 @@ func (b *backupResumer) Resume( } b.deleteCheckpoint(ctx, p.ExecCfg()) + if ptsID != nil && !b.testingKnobs.ignoreProtectedTimestamps { + if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + return b.releaseProtectedTimestamp(ctx, txn, p.ExecCfg().ProtectedTimestampProvider) + }); err != nil { + log.Errorf(ctx, "failed to release protected timestamp: %v", err) + } + } + resultsCh <- tree.Datums{ tree.NewDInt(tree.DInt(*b.job.ID())), tree.NewDString(string(jobs.StatusSucceeded)), @@ -550,14 +558,6 @@ func (b *backupResumer) Resume( tree.NewDInt(tree.DInt(res.DataSize)), } - if ptsID != nil && !b.testingKnobs.ignoreProtectedTimestamps { - if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return b.releaseProtectedTimestamp(ctx, txn, p.ExecCfg().ProtectedTimestampProvider) - }); err != nil { - log.Errorf(ctx, "failed to release protected timestamp: %v", err) - } - } - // Collect telemetry. { telemetry.Count("backup.total.succeeded") diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 2901e1a7ab8c..e686265b1e45 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -3626,7 +3626,14 @@ func TestProtectedTimestampsDuringBackup(t *testing.T) { rowCount := runner.QueryStr(t, "SELECT * FROM foo") go func() { - runner.Exec(t, `BACKUP TABLE FOO TO 'nodelocal://1/foo'`) + // N.B. We use the conn rather than the runner here since the test may + // finish before the job finishes. The test will finish as soon as the + // timestamp is no longer protected. If the test starts tearing down the + // cluster before the backup job is done, the test may still fail when the + // backup fails. This test does not particularly care if the BACKUP + // completes with a success or failure, as long as the timestamp is released + // shortly after the BACKUP is unblocked. + _, _ = conn.Exec(`BACKUP TABLE FOO TO 'nodelocal://1/foo'`) // ignore error. }() var jobID string @@ -3669,11 +3676,15 @@ func TestProtectedTimestampsDuringBackup(t *testing.T) { // Wait for the ranges to learn about the removed record and ensure that we // can GC from the range soon. - gcRanRE := regexp.MustCompile("(?s)shouldQueue=true.*processing replica.*GC score after GC") + // This regex matches when all float priorities other than 0.00000. It does + // this by matching either a float >= 1 (e.g. 1230.012) or a float < 1 (e.g. + // 0.000123). + matchNonZero := "[1-9]\\d*\\.\\d+|0\\.\\d*[1-9]\\d*" + nonZeroProgressRE := regexp.MustCompile(fmt.Sprintf("priority=(%s)", matchNonZero)) testutils.SucceedsSoon(t, func() error { writeGarbage(3, 10) - if trace := gcTable(false /* skipShouldQueue */); !gcRanRE.MatchString(trace) { - return fmt.Errorf("expected %v in trace: %v", gcRanRE, trace) + if trace := gcTable(false /* skipShouldQueue */); !nonZeroProgressRE.MatchString(trace) { + return fmt.Errorf("expected %v in trace: %v", nonZeroProgressRE, trace) } return nil }) diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index b82f62743d05..6ba67611c2b4 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -432,7 +432,6 @@ func WriteTableDescs( databases []*sqlbase.DatabaseDescriptor, tables []*sqlbase.TableDescriptor, descCoverage tree.DescriptorCoverage, - user string, settings *cluster.Settings, extra []roachpb.KeyValue, ) error { @@ -924,7 +923,7 @@ func createImportingTables( if !details.PrepareCompleted { err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { // Write the new TableDescriptors which are set in the OFFLINE state. - if err := WriteTableDescs(ctx, txn, databases, tables, details.DescriptorCoverage, r.job.Payload().Username, r.settings, nil /* extra */); err != nil { + if err := WriteTableDescs(ctx, txn, databases, tables, details.DescriptorCoverage, r.settings, nil /* extra */); err != nil { return errors.Wrapf(err, "restoring %d TableDescriptors from %d databases", len(r.tables), len(databases)) } diff --git a/pkg/ccl/importccl/import_into_test.go b/pkg/ccl/importccl/import_into_test.go index 74f1b59fcbba..89f0cf69bb38 100644 --- a/pkg/ccl/importccl/import_into_test.go +++ b/pkg/ccl/importccl/import_into_test.go @@ -147,11 +147,15 @@ func TestProtectedTimestampsDuringImportInto(t *testing.T) { // Wait for the ranges to learn about the removed record and ensure that we // can GC from the range soon. - gcRanRE := regexp.MustCompile("(?s)shouldQueue=true.*processing replica.*GC score after GC") + // This regex matches when all float priorities other than 0.00000. It does + // this by matching either a float >= 1 (e.g. 1230.012) or a float < 1 (e.g. + // 0.000123). + matchNonZero := "[1-9]\\d*\\.\\d+|0\\.\\d*[1-9]\\d*" + nonZeroProgressRE := regexp.MustCompile(fmt.Sprintf("priority=(%s)", matchNonZero)) testutils.SucceedsSoon(t, func() error { writeGarbage(3, 10) - if trace := gcTable(false /* skipShouldQueue */); !gcRanRE.MatchString(trace) { - return fmt.Errorf("expected %v in trace: %v", gcRanRE, trace) + if trace := gcTable(false /* skipShouldQueue */); !nonZeroProgressRE.MatchString(trace) { + return fmt.Errorf("expected %v in trace: %v", nonZeroProgressRE, trace) } return nil }) diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_stmt.go index c389971710c5..75c1b4b46126 100644 --- a/pkg/ccl/importccl/import_stmt.go +++ b/pkg/ccl/importccl/import_stmt.go @@ -902,7 +902,7 @@ func prepareNewTableDescsForIngestion( // Write the new TableDescriptors and flip the namespace entries over to // them. After this call, any queries on a table will be served by the newly // imported data. - if err := backupccl.WriteTableDescs(ctx, txn, nil /* databases */, tableDescs, tree.RequestedDescriptors, p.User(), p.ExecCfg().Settings, seqValKVs); err != nil { + if err := backupccl.WriteTableDescs(ctx, txn, nil /* databases */, tableDescs, tree.RequestedDescriptors, p.ExecCfg().Settings, seqValKVs); err != nil { return nil, errors.Wrapf(err, "creating tables") } diff --git a/pkg/cli/sql_util.go b/pkg/cli/sql_util.go index 048f71419c9d..2158796ede24 100644 --- a/pkg/cli/sql_util.go +++ b/pkg/cli/sql_util.go @@ -30,7 +30,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -921,7 +920,7 @@ func formatVal(val driver.Value, showPrintableUnicode bool, showNewLinesAndTabs // that we can let the user see and control the result using // `bytea_output`. return lex.EncodeByteArrayToRawBytes(string(t), - sessiondata.BytesEncodeEscape, false /* skipHexPrefix */) + lex.BytesEncodeEscape, false /* skipHexPrefix */) case time.Time: return t.Format(tree.TimestampOutputFormat) diff --git a/pkg/cli/start.go b/pkg/cli/start.go index 0157186aeab2..4054767eed82 100644 --- a/pkg/cli/start.go +++ b/pkg/cli/start.go @@ -470,14 +470,6 @@ func runStart(cmd *cobra.Command, args []string, disableReplication bool) error // not be world-readable. disableOtherPermissionBits() - // TODO(knz): the following call is not in the right place. - // See: https://github.com/cockroachdb/cockroach/issues/44041 - if s, err := serverCfg.Stores.GetPreventedStartupMessage(); err != nil { - return err - } else if s != "" { - log.Fatal(context.Background(), s) - } - // Set up the signal handlers. This also ensures that any of these // signals received beyond this point do not interrupt the startup // sequence until the point signals are checked below. @@ -521,6 +513,12 @@ func runStart(cmd *cobra.Command, args []string, disableReplication bool) error return err } + // If any store has something to say against a server start-up + // (e.g. previously detected corruption), listen to them now. + if err := serverCfg.Stores.PriorCriticalAlertError(); err != nil { + return err + } + // We don't care about GRPCs fairly verbose logs in most client commands, // but when actually starting a server, we enable them. grpcutil.SetSeverity(log.Severity_WARNING) diff --git a/pkg/col/coldata/batch.go b/pkg/col/coldata/batch.go index b7eaa4ac2f10..ed1a92a57607 100644 --- a/pkg/col/coldata/batch.go +++ b/pkg/col/coldata/batch.go @@ -17,6 +17,8 @@ import ( "sync/atomic" "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" ) @@ -47,14 +49,14 @@ type Batch interface { // provided Vec. ReplaceCol(Vec, int) // Reset modifies the caller in-place to have the given length and columns - // with the given coltypes. If it's possible, Reset will reuse the existing + // with the given types. If it's possible, Reset will reuse the existing // columns and allocations, invalidating existing references to the Batch or // its Vecs. However, Reset does _not_ zero out the column data. // // NOTE: Reset can allocate a new Batch, so when calling from the vectorized // engine consider either allocating a new Batch explicitly via // colexec.Allocator or calling ResetInternalBatch. - Reset(types []coltypes.T, length int) + Reset(types []types.T, length int) // ResetInternalBatch resets a batch and its underlying Vecs for reuse. It's // important for callers to call ResetInternalBatch if they own internal // batches that they reuse as not doing this could result in correctness @@ -98,16 +100,16 @@ func ResetBatchSizeForTests() { // NewMemBatch allocates a new in-memory Batch. A coltypes.Unknown type // will create a placeholder Vec that may not be accessed. // TODO(jordan): pool these allocations. -func NewMemBatch(types []coltypes.T) Batch { +func NewMemBatch(types []types.T) Batch { return NewMemBatchWithSize(types, BatchSize()) } // NewMemBatchWithSize allocates a new in-memory Batch with the given column // size. Use for operators that have a precisely-sized output batch. -func NewMemBatchWithSize(types []coltypes.T, size int) Batch { +func NewMemBatchWithSize(types []types.T, size int) Batch { b := NewMemBatchNoCols(types, size).(*MemBatch) for i, t := range types { - b.b[i] = NewMemColumn(t, size) + b.b[i] = NewMemColumn(&t, size) } return b } @@ -115,7 +117,7 @@ func NewMemBatchWithSize(types []coltypes.T, size int) Batch { // NewMemBatchNoCols creates a "skeleton" of new in-memory Batch. It allocates // memory for the selection vector but does *not* allocate any memory for the // column vectors - those will have to be added separately. -func NewMemBatchNoCols(types []coltypes.T, size int) Batch { +func NewMemBatchNoCols(types []types.T, size int) Batch { if max := math.MaxUint16; size > max { panic(fmt.Sprintf(`batches cannot have length larger than %d; requested %d`, max, size)) } @@ -156,7 +158,7 @@ func (b *zeroBatch) ReplaceCol(Vec, int) { panic("no columns should be replaced in zero batch") } -func (b *zeroBatch) Reset([]coltypes.T, int) { +func (b *zeroBatch) Reset([]types.T, int) { panic("zero batch should not be reset") } @@ -226,7 +228,7 @@ func (m *MemBatch) ReplaceCol(col Vec, colIdx int) { } // Reset implements the Batch interface. -func (m *MemBatch) Reset(types []coltypes.T, length int) { +func (m *MemBatch) Reset(types []types.T, length int) { ResetNoTruncation(m, types, length) m.b = m.b[:len(types)] } @@ -236,14 +238,15 @@ func (m *MemBatch) Reset(types []coltypes.T, length int) { // the prefix of already present columns matches the desired type schema, // the batch will be reused (meaning this method does *not* truncate the // type schema). -func ResetNoTruncation(m *MemBatch, types []coltypes.T, length int) { +func ResetNoTruncation(m *MemBatch, types []types.T, length int) { // The columns are always sized the same as the selection vector, so use it as // a shortcut for the capacity (like a go slice, the batch's `Length` could be // shorter than the capacity). We could be more defensive and type switch // every column to verify its capacity, but that doesn't seem necessary yet. cannotReuse := m == nil || len(m.sel) < length || m.Width() < len(types) for i := 0; i < len(types) && !cannotReuse; i++ { - if m.ColVec(i).Type() != types[i] { + // TODO(yuzefovich): change this when coltypes.Datum is introduced. + if m.ColVec(i).Type() != typeconv.FromColumnType(&types[i]) { cannotReuse = true } } diff --git a/pkg/col/coldata/batch_test.go b/pkg/col/coldata/batch_test.go index d23e30b1be7b..0509d7a2d350 100644 --- a/pkg/col/coldata/batch_test.go +++ b/pkg/col/coldata/batch_test.go @@ -17,6 +17,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/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/stretchr/testify/assert" ) @@ -24,7 +26,7 @@ import ( func TestBatchReset(t *testing.T) { defer leaktest.AfterTest(t)() - resetAndCheck := func(b coldata.Batch, typs []coltypes.T, n int, shouldReuse bool) { + resetAndCheck := func(b coldata.Batch, typs []types.T, n int, shouldReuse bool) { t.Helper() // Use the data backing the ColVecs slice as a proxy for when things get // reallocated. @@ -47,7 +49,7 @@ func TestBatchReset(t *testing.T) { for i, vec := range b.ColVecs() { assert.False(t, vec.MaybeHasNulls()) assert.False(t, vec.Nulls().NullAt(0)) - assert.Equal(t, typs[i], vec.Type()) + assert.Equal(t, typeconv.FromColumnType(&typs[i]), vec.Type()) // Sanity check that we can actually use the column. This is mostly for // making sure a flat bytes column gets reset. vec.Nulls().SetNull(0) @@ -68,9 +70,9 @@ func TestBatchReset(t *testing.T) { } } - typsInt := []coltypes.T{coltypes.Int64} - typsBytes := []coltypes.T{coltypes.Bytes} - typsIntBytes := []coltypes.T{coltypes.Int64, coltypes.Bytes} + typsInt := []types.T{*types.Int} + typsBytes := []types.T{*types.Bytes} + typsIntBytes := []types.T{*types.Int, *types.Bytes} var b coldata.Batch // Simple case, reuse 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/nulls_test.go b/pkg/col/coldata/nulls_test.go index f5d8126c3a3e..6c0238863f6b 100644 --- a/pkg/col/coldata/nulls_test.go +++ b/pkg/col/coldata/nulls_test.go @@ -14,7 +14,7 @@ import ( "fmt" "testing" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/stretchr/testify/require" ) @@ -224,7 +224,7 @@ func TestSetAndUnsetNulls(t *testing.T) { func TestNullsSet(t *testing.T) { args := SliceArgs{ // Neither type nor the length here matter. - Src: NewMemColumn(coltypes.Bool, 0), + Src: NewMemColumn(types.Bool, 0), } for _, withSel := range []bool{false, true} { t.Run(fmt.Sprintf("WithSel=%t", withSel), func(t *testing.T) { diff --git a/pkg/col/coldata/random_testutils.go b/pkg/col/coldata/random_testutils.go deleted file mode 100644 index 17ddef4e8389..000000000000 --- a/pkg/col/coldata/random_testutils.go +++ /dev/null @@ -1,127 +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/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 coltypes.T, bytesFixedLength int, vec Vec, n int, nullProbability float64, -) { - switch 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 5b3f1a4546be..a933d2204d29 100644 --- a/pkg/col/coldata/vec.go +++ b/pkg/col/coldata/vec.go @@ -16,6 +16,8 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" ) @@ -147,10 +149,10 @@ type memColumn struct { } // NewMemColumn returns a new memColumn, initialized with a length. -func NewMemColumn(t coltypes.T, n int) Vec { +func NewMemColumn(t *types.T, n int) Vec { nulls := NewNulls(n) - switch t { + switch t := typeconv.FromColumnType(t); t { case coltypes.Bool: return &memColumn{t: t, col: make([]bool, n), nulls: nulls} case coltypes.Bytes: diff --git a/pkg/col/coldata/vec_test.go b/pkg/col/coldata/vec_test.go index 65ebbaea03cf..a57a1e2c3b07 100644 --- a/pkg/col/coldata/vec_test.go +++ b/pkg/col/coldata/vec_test.go @@ -8,13 +8,17 @@ // 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/col/coltypes/typeconv" + "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" @@ -25,10 +29,10 @@ func TestMemColumnWindow(t *testing.T) { rng, _ := randutil.NewPseudoRand() - c := NewMemColumn(coltypes.Int64, 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. @@ -39,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) @@ -113,12 +117,12 @@ func TestNullRanges(t *testing.T) { }, } - c := NewMemColumn(coltypes.Int64, 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) @@ -134,9 +138,9 @@ func TestNullRanges(t *testing.T) { func TestAppend(t *testing.T) { // TODO(asubiotto): Test nulls. - const typ = coltypes.Int64 + 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 @@ -144,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, @@ -182,7 +186,7 @@ func TestAppend(t *testing.T) { }, { name: "AppendWithSel", - args: SliceArgs{ + args: coldata.SliceArgs{ DestIdx: 5, SrcStartIdx: 10, SrcEndIdx: 20, @@ -192,88 +196,35 @@ 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, }, } for _, tc := range testCases { tc.args.Src = src - tc.args.ColType = typ + 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(coltypes.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(coltypes.Bytes, 3) - expected := NewMemColumn(coltypes.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. - const typ = coltypes.Int64 + 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) @@ -293,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, @@ -330,9 +281,9 @@ func TestCopy(t *testing.T) { for _, tc := range testCases { tc.args.Src = src - tc.args.ColType = typ + 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 @@ -350,10 +301,10 @@ func TestCopy(t *testing.T) { } func TestCopyNulls(t *testing.T) { - const typ = coltypes.Int64 + 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) @@ -364,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 @@ -374,9 +325,9 @@ func TestCopyNulls(t *testing.T) { src.Nulls().SetNull(i) } - copyArgs := CopySliceArgs{ - SliceArgs: SliceArgs{ - ColType: typ, + copyArgs := coldata.CopySliceArgs{ + SliceArgs: coldata.SliceArgs{ + ColType: typeconv.FromColumnType(typ), Src: src, DestIdx: 3, SrcStartIdx: 3, @@ -399,18 +350,18 @@ 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") } } func TestCopySelOnDestDoesNotUnsetOldNulls(t *testing.T) { - const typ = coltypes.Int64 + var typ = types.Int // 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 @@ -419,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 @@ -429,10 +380,10 @@ 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{ - ColType: typ, + SliceArgs: coldata.SliceArgs{ + ColType: typeconv.FromColumnType(typ), Src: src, SrcStartIdx: 1, SrcEndIdx: 3, @@ -456,39 +407,39 @@ 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, }, }, } - for _, typ := range []coltypes.T{coltypes.Bytes, coltypes.Decimal, coltypes.Int64} { + 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 = typ - bc.args.SrcEndIdx = BatchSize() - dest := NewMemColumn(typ, BatchSize()) - b.Run(fmt.Sprintf("%s/%s/NullProbability=%.1f", typ, bc.name, nullProbability), func(b *testing.B) { - b.SetBytes(8 * int64(BatchSize())) + bc.args.ColType = typeconv.FromColumnType(&typ) + 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(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() } }) } @@ -498,40 +449,40 @@ 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, }, }, }, } - for _, typ := range []coltypes.T{coltypes.Bytes, coltypes.Decimal, coltypes.Int64} { + 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 = typ - bc.args.SrcEndIdx = BatchSize() - dest := NewMemColumn(typ, BatchSize()) - b.Run(fmt.Sprintf("%s/%s/NullProbability=%.1f", typ, bc.name, nullProbability), func(b *testing.B) { - b.SetBytes(8 * int64(BatchSize())) + bc.args.ColType = typeconv.FromColumnType(&typ) + 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(coldata.BatchSize())) for i := 0; i < b.N; i++ { dest.Copy(bc.args) - if typ == coltypes.Bytes { + if typ.Identical(types.Bytes) { // We need to reset flat bytes so that we could copy into it // (otherwise it'll panic on the second copy due to maxSetIndex // being not zero). diff --git a/pkg/sql/colexec/random_testutils.go b/pkg/col/coldatatestutils/random_testutils.go similarity index 50% rename from pkg/sql/colexec/random_testutils.go rename to pkg/col/coldatatestutils/random_testutils.go index 95dcd49617e6..014f06782727 100644 --- a/pkg/sql/colexec/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,27 +8,146 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package colexec +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/colexec/execerror" + "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) + } + } } -// randomTypes returns an n-length slice of random coltypes.T. -func randomTypes(rng *rand.Rand, n int) []coltypes.T { - typs := make([]coltypes.T, n) +func randomType(rng *rand.Rand) *types.T { + return &typeconv.AllSupportedSQLTypes[rng.Intn(len(typeconv.AllSupportedSQLTypes))] +} + +// 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 } @@ -37,9 +156,9 @@ 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 []coltypes.T, + typs []types.T, capacity int, length int, nullProbability float64, @@ -49,21 +168,21 @@ 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 } -// randomSel creates a random selection vector up to a given batchSize in +// RandomSel creates a random selection vector up to a given batchSize in // length. probOfOmitting specifies the probability that a row should be omitted // from the batch (i.e. whether it should be selected out). So if probOfOmitting // is 0, then the selection vector will contain all rows, but if it is > 0, then // some rows might be omitted and the length of the selection vector might be // less than batchSize. -func randomSel(rng *rand.Rand, batchSize int, probOfOmitting float64) []int { +func RandomSel(rng *rand.Rand, batchSize int, probOfOmitting float64) []int { if probOfOmitting < 0 || probOfOmitting > 1 { - execerror.VectorizedInternalPanic(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++ { @@ -79,22 +198,22 @@ func randomSel(rng *rand.Rand, batchSize int, probOfOmitting float64) []int { // TODO(asubiotto): Remove this once this function is actually used. var _ = randomTypes -// randomBatchWithSel is equivalent to RandomBatch, but will also add a +// RandomBatchWithSel is equivalent to RandomBatch, but will also add a // selection vector to the batch where each row is selected with probability // selProbability. If selProbability is 1, all the rows will be selected, if // 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, +func RandomBatchWithSel( + allocator *colmem.Allocator, rng *rand.Rand, - typs []coltypes.T, + typs []types.T, n int, nullProbability float64, selProbability float64, ) coldata.Batch { batch := RandomBatch(allocator, rng, typs, n, 0 /* length */, nullProbability) if selProbability != 0 { - sel := randomSel(rng, n, 1-selProbability) + sel := RandomSel(rng, n, 1-selProbability) batch.SetSelection(true) copy(batch.Selection(), sel) batch.SetLength(len(sel)) @@ -110,14 +229,14 @@ 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. - DeterministicTyps []coltypes.T + DeterministicTyps []types.T // AvailableTyps is the pool of types from which the operator's schema will // be generated. - AvailableTyps []coltypes.T + AvailableTyps []types.T // MaxSchemaLength is the maximum length of the operator's schema, which will // be at least one type. MaxSchemaLength int @@ -132,16 +251,15 @@ type RandomDataOpArgs struct { Nulls bool // BatchAccumulator, if set, will be called before returning a coldata.Batch // from Next. - BatchAccumulator func(b coldata.Batch) + BatchAccumulator func(b coldata.Batch, typs []types.T) } // RandomDataOp is an operator that generates random data according to // RandomDataOpArgs. Call GetBuffer to get all data that was returned. type RandomDataOp struct { - ZeroInputNode - allocator *Allocator - batchAccumulator func(b coldata.Batch) - typs []coltypes.T + allocator *colmem.Allocator + batchAccumulator func(b coldata.Batch, typs []types.T) + typs []types.T rng *rand.Rand batchSize int numBatches int @@ -150,10 +268,14 @@ type RandomDataOp struct { nulls bool } +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 = coltypes.AllTypes + availableTyps = typeconv.AllSupportedSQLTypes maxSchemaLength = defaultMaxSchemaLength batchSize = coldata.BatchSize() numBatches = defaultNumBatches @@ -174,7 +296,7 @@ func NewRandomDataOp(allocator *Allocator, rng *rand.Rand, args RandomDataOpArgs typs := args.DeterministicTyps if typs == nil { // Generate at least one type. - typs = make([]coltypes.T, 1+rng.Intn(maxSchemaLength)) + typs = make([]types.T, 1+rng.Intn(maxSchemaLength)) for i := range typs { typs[i] = availableTyps[rng.Intn(len(availableTyps))] } @@ -191,16 +313,16 @@ func NewRandomDataOp(allocator *Allocator, rng *rand.Rand, args RandomDataOpArgs } } -// Init is part of the Operator interface. +// Init is part of the colexec.Operator interface. func (o *RandomDataOp) Init() {} -// Next is part of the Operator interface. +// Next is part of the colexec.Operator interface. func (o *RandomDataOp) Next(ctx context.Context) coldata.Batch { if o.numReturned == o.numBatches { // Done. b := coldata.ZeroBatch if o.batchAccumulator != nil { - o.batchAccumulator(b) + o.batchAccumulator(b, o.typs) } return b } @@ -217,7 +339,7 @@ func (o *RandomDataOp) Next(ctx context.Context) coldata.Batch { nullProbability = o.rng.Float64() } - b := randomBatchWithSel(o.allocator, o.rng, o.typs, o.batchSize, nullProbability, selProbability) + b := RandomBatchWithSel(o.allocator, o.rng, o.typs, o.batchSize, nullProbability, selProbability) if !o.selection { b.SetSelection(false) } @@ -227,13 +349,25 @@ func (o *RandomDataOp) Next(ctx context.Context) coldata.Batch { } o.numReturned++ if o.batchAccumulator != nil { - o.batchAccumulator(b) + o.batchAccumulator(b, o.typs) } return b } } +// ChildCount implements the execinfra.OpNode interface. +func (o *RandomDataOp) ChildCount(verbose bool) int { + return 0 +} + +// Child implements the execinfra.OpNode interface. +func (o *RandomDataOp) Child(nth int, verbose bool) execinfra.OpNode { + colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + // This code is unreachable, but the compiler cannot infer that. + return nil +} + // Typs returns the output types of the RandomDataOp. -func (o *RandomDataOp) Typs() []coltypes.T { +func (o *RandomDataOp) Typs() []types.T { return o.typs } diff --git a/pkg/col/coldatatestutils/utils.go b/pkg/col/coldatatestutils/utils.go new file mode 100644 index 000000000000..22af95c1d09e --- /dev/null +++ b/pkg/col/coldatatestutils/utils.go @@ -0,0 +1,36 @@ +// 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 coldatatestutils + +import ( + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "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()). +// 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()) + 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 bd656acdc40a..38c4a360aa4c 100644 --- a/pkg/col/colserde/arrowbatchconverter.go +++ b/pkg/col/colserde/arrowbatchconverter.go @@ -21,6 +21,8 @@ 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/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/errors" ) @@ -28,7 +30,7 @@ import ( // ArrowBatchConverter converts batches to arrow column data // ([]*array.Data) and back again. type ArrowBatchConverter struct { - typs []coltypes.T + typs []types.T // builders are the set of builders that need to be kept around in order to // construct arrow representations of certain types when they cannot be simply @@ -54,10 +56,10 @@ type ArrowBatchConverter struct { // NewArrowBatchConverter converts coldata.Batches to []*array.Data and back // again according to the schema specified by typs. Converting data that does // not conform to typs results in undefined behavior. -func NewArrowBatchConverter(typs []coltypes.T) (*ArrowBatchConverter, error) { +func NewArrowBatchConverter(typs []types.T) (*ArrowBatchConverter, error) { for _, t := range typs { - if _, supported := supportedTypes[t]; !supported { - return nil, errors.Errorf("arrowbatchconverter unsupported type %v", t.String()) + if converted := typeconv.FromColumnType(&t); converted == coltypes.Unhandled { + return nil, errors.Errorf("arrowbatchconverter unsupported type %s", &t) } } c := &ArrowBatchConverter{typs: typs} @@ -81,24 +83,6 @@ const ( sizeOfFloat64 = int(unsafe.Sizeof(float64(0))) ) -var supportedTypes = func() map[coltypes.T]struct{} { - typs := make(map[coltypes.T]struct{}) - for _, t := range []coltypes.T{ - coltypes.Bool, - coltypes.Bytes, - coltypes.Decimal, - coltypes.Float64, - coltypes.Int16, - coltypes.Int32, - coltypes.Int64, - coltypes.Timestamp, - coltypes.Interval, - } { - typs[t] = struct{}{} - } - return typs -}() - // BatchToArrow converts the first batch.Length elements of the batch into an // arrow []*array.Data. It is assumed that the batch is not larger than // coldata.BatchSize(). The returned []*array.Data may only be used until the @@ -119,9 +103,10 @@ func (c *ArrowBatchConverter) BatchToArrow(batch coldata.Batch) ([]*array.Data, arrowBitmap = n.NullBitmap() } - if typ == coltypes.Bool || typ == coltypes.Decimal || typ == coltypes.Timestamp || typ == coltypes.Interval { + physType := typeconv.FromColumnType(&typ) + if physType == coltypes.Bool || physType == coltypes.Decimal || physType == coltypes.Timestamp || physType == coltypes.Interval { var data *array.Data - switch typ { + switch physType { case coltypes.Bool: c.builders.boolBuilder.AppendValues(vec.Bool()[:n], nil /* valid */) data = c.builders.boolBuilder.NewBooleanArray().Data() @@ -162,7 +147,7 @@ func (c *ArrowBatchConverter) BatchToArrow(batch coldata.Batch) ([]*array.Data, } data = c.builders.binaryBuilder.NewBinaryArray().Data() default: - panic(fmt.Sprintf("unexpected type %s", typ)) + panic(fmt.Sprintf("unexpected type %s", &typ)) } if arrowBitmap != nil { // Overwrite empty null bitmap with the true bitmap. @@ -183,7 +168,7 @@ func (c *ArrowBatchConverter) BatchToArrow(batch coldata.Batch) ([]*array.Data, datumSize int ) - switch typ { + switch physType { case coltypes.Bytes: var int32Offsets []int32 values, int32Offsets = vec.Bytes().ToArrowSerializationFormat(n) @@ -210,7 +195,7 @@ func (c *ArrowBatchConverter) BatchToArrow(batch coldata.Batch) ([]*array.Data, dataHeader = (*reflect.SliceHeader)(unsafe.Pointer(&floats)) datumSize = sizeOfFloat64 default: - panic(fmt.Sprintf("unsupported type for conversion to arrow data %s", typ)) + panic(fmt.Sprintf("unsupported type for conversion to arrow data %s", &typ)) } // Cast values if not set (mostly for non-byte types). @@ -273,7 +258,7 @@ func (c *ArrowBatchConverter) ArrowToBatch(data []*array.Data, b coldata.Batch) d := data[i] var arr array.Interface - switch typ { + switch physType := typeconv.FromColumnType(&typ); physType { case coltypes.Bool: boolArr := array.NewBooleanData(d) vecArr := vec.Bool() @@ -358,7 +343,7 @@ func (c *ArrowBatchConverter) ArrowToBatch(data []*array.Data, b coldata.Batch) arr = bytesArr default: var col interface{} - switch typ { + switch physType { case coltypes.Int16: intArr := array.NewInt16Data(d) col = intArr.Int16Values() diff --git a/pkg/col/colserde/arrowbatchconverter_test.go b/pkg/col/colserde/arrowbatchconverter_test.go index cefeb081f910..137934c28b03 100644 --- a/pkg/col/colserde/arrowbatchconverter_test.go +++ b/pkg/col/colserde/arrowbatchconverter_test.go @@ -17,35 +17,37 @@ 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/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "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 *colexec.Allocator) ([]coltypes.T, coldata.Batch) { +func randomBatch(allocator *colmem.Allocator) ([]types.T, coldata.Batch) { const maxTyps = 16 rng, _ := randutil.NewPseudoRand() - typs := make([]coltypes.T, rng.Intn(maxTyps)+1) + typs := make([]types.T, rng.Intn(maxTyps)+1) for i := range typs { - typs[i] = coltypes.AllTypes[rng.Intn(len(coltypes.AllTypes))] + typs[i] = typeconv.AllSupportedSQLTypes[rng.Intn(len(typeconv.AllSupportedSQLTypes))] } capacity := rng.Intn(coldata.BatchSize()) + 1 length := rng.Intn(capacity) - b := colexec.RandomBatch(allocator, rng, typs, capacity, length, rng.Float64()) + b := coldatatestutils.RandomBatch(allocator, rng, typs, capacity, length, rng.Float64()) return typs, b } func TestArrowBatchConverterRejectsUnsupportedTypes(t *testing.T) { defer leaktest.AfterTest(t)() - unsupportedTypes := []coltypes.T{coltypes.Unhandled} + unsupportedTypes := []types.T{*types.INet} for _, typ := range unsupportedTypes { - _, err := colserde.NewArrowBatchConverter([]coltypes.T{typ}) + _, err := colserde.NewArrowBatchConverter([]types.T{typ}) require.Error(t, err) } } @@ -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 := colexec.CopyBatch(testAllocator, b) + 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 := colexec.CopyBatch(testAllocator, b) + expected := coldatatestutils.CopyBatch(b, typs) actual, err := roundTripBatch(b, c, r) require.NoError(t, err) @@ -124,12 +126,12 @@ func BenchmarkArrowBatchConverter(b *testing.B) { rng, _ := randutil.NewPseudoRand() - typs := []coltypes.T{ - coltypes.Bool, - coltypes.Bytes, - coltypes.Decimal, - coltypes.Int64, - coltypes.Timestamp, + typs := []types.T{ + *types.Bool, + *types.Bytes, + *types.Decimal, + *types.Int, + *types.Timestamp, } // numBytes corresponds 1:1 to typs and specifies how many bytes we are // converting on one iteration of the benchmark for the corresponding type in @@ -143,11 +145,11 @@ func BenchmarkArrowBatchConverter(b *testing.B) { } // Run a benchmark on every type we care about. for typIdx, typ := range typs { - batch := colexec.RandomBatch(testAllocator, rng, []coltypes.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()) } - if typ == coltypes.Bytes { + if typ.Identical(types.Bytes) { // This type has variable length elements, fit all of them to be fixedLen // bytes long so that we can compare results of one benchmark with // another. Since we can't overwrite elements in a Bytes, create a new @@ -165,7 +167,7 @@ func BenchmarkArrowBatchConverter(b *testing.B) { } } batch.ColVec(0).SetCol(newBytes) - } else if typ == coltypes.Decimal { + } else if typ.Identical(types.Decimal) { // Decimal is variable length type, so we want to calculate precisely the // total size of all decimals in the vector. decimals := batch.ColVec(0).Decimal() @@ -175,7 +177,7 @@ func BenchmarkArrowBatchConverter(b *testing.B) { numBytes[typIdx] += int64(len(marshaled)) } } - c, err := colserde.NewArrowBatchConverter([]coltypes.T{typ}) + c, err := colserde.NewArrowBatchConverter([]types.T{typ}) require.NoError(b, err) nullFractions := []float64{0, 0.25, 0.5} setNullFraction := func(batch coldata.Batch, nullFraction float64) { @@ -209,7 +211,7 @@ func BenchmarkArrowBatchConverter(b *testing.B) { data, err := c.BatchToArrow(batch) require.NoError(b, err) testPrefix := fmt.Sprintf("%s/nullFraction=%0.2f", typ.String(), nullFraction) - result := coldata.NewMemBatch([]coltypes.T{typ}) + result := coldata.NewMemBatch([]types.T{typ}) b.Run(testPrefix+"/ArrowToBatch", func(b *testing.B) { b.SetBytes(numBytes[typIdx]) for i := 0; i < b.N; i++ { diff --git a/pkg/col/colserde/file.go b/pkg/col/colserde/file.go index d6034c0bacbc..ae0fc33a92ae 100644 --- a/pkg/col/colserde/file.go +++ b/pkg/col/colserde/file.go @@ -20,8 +20,10 @@ 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/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" mmap "github.com/edsrzf/mmap-go" flatbuffers "github.com/google/flatbuffers/go" "github.com/pkg/errors" @@ -44,7 +46,7 @@ type FileSerializer struct { scratch [4]byte w *countingWriter - typs []coltypes.T + typs []types.T fb *flatbuffers.Builder a *ArrowBatchConverter rb *RecordBatchSerializer @@ -52,9 +54,9 @@ type FileSerializer struct { recordBatches []fileBlock } -// NewFileSerializer creates a FileSerializer for the given coltypes. The caller is +// NewFileSerializer creates a FileSerializer for the given types. The caller is // responsible for closing the given writer. -func NewFileSerializer(w io.Writer, typs []coltypes.T) (*FileSerializer, error) { +func NewFileSerializer(w io.Writer, typs []types.T) (*FileSerializer, error) { a, err := NewArrowBatchConverter(typs) if err != nil { return nil, err @@ -161,7 +163,7 @@ type FileDeserializer struct { idx int end int - typs []coltypes.T + typs []types.T a *ArrowBatchConverter rb *RecordBatchSerializer @@ -170,13 +172,13 @@ type FileDeserializer struct { // NewFileDeserializerFromBytes constructs a FileDeserializer for an in-memory // buffer. -func NewFileDeserializerFromBytes(buf []byte) (*FileDeserializer, error) { - return newFileDeserializer(buf, func() error { return nil }) +func NewFileDeserializerFromBytes(typs []types.T, buf []byte) (*FileDeserializer, error) { + return newFileDeserializer(typs, buf, func() error { return nil }) } // NewFileDeserializerFromPath constructs a FileDeserializer by reading it from // a file. -func NewFileDeserializerFromPath(path string) (*FileDeserializer, error) { +func NewFileDeserializerFromPath(typs []types.T, path string) (*FileDeserializer, error) { f, err := os.Open(path) if err != nil { return nil, pgerror.Wrapf(err, pgcode.Io, `opening %s`, path) @@ -190,17 +192,19 @@ func NewFileDeserializerFromPath(path string) (*FileDeserializer, error) { if err != nil { return nil, pgerror.Wrapf(err, pgcode.Io, `mmaping %s`, path) } - return newFileDeserializer(buf, buf.Unmap) + return newFileDeserializer(typs, buf, buf.Unmap) } -func newFileDeserializer(buf []byte, bufCloseFn func() error) (*FileDeserializer, error) { +func newFileDeserializer( + typs []types.T, buf []byte, bufCloseFn func() error, +) (*FileDeserializer, error) { d := &FileDeserializer{ buf: buf, bufCloseFn: bufCloseFn, end: len(buf), } - typs, err := d.init() - if err != nil { + var err error + if err = d.init(); err != nil { return nil, err } d.typs = typs @@ -221,8 +225,8 @@ func (d *FileDeserializer) Close() error { return d.bufCloseFn() } -// Typs returns the in-memory columnar types for the data stored in this file. -func (d *FileDeserializer) Typs() []coltypes.T { +// Typs returns the in-memory types for the data stored in this file. +func (d *FileDeserializer) Typs() []types.T { return d.typs } @@ -269,42 +273,34 @@ func (d *FileDeserializer) readBackward(n int) ([]byte, error) { // init verifies the file magic and headers. After init, the `idx` and `end` // fields are set to the range of record batches and dictionary batches // described by the arrow spec's streaming format. -func (d *FileDeserializer) init() ([]coltypes.T, error) { +func (d *FileDeserializer) init() error { // Check the header magic if magic, err := d.read(8); err != nil { - return nil, pgerror.Wrap(err, pgcode.DataException, `verifying arrow file header magic`) + return pgerror.Wrap(err, pgcode.DataException, `verifying arrow file header magic`) } else if !bytes.Equal([]byte(fileMagic), magic[:len(fileMagic)]) { - return nil, errors.New(`arrow file header magic mismatch`) + return errors.New(`arrow file header magic mismatch`) } if magic, err := d.readBackward(len(fileMagic)); err != nil { - return nil, pgerror.Wrap(err, pgcode.DataException, `verifying arrow file footer magic`) + return pgerror.Wrap(err, pgcode.DataException, `verifying arrow file footer magic`) } else if !bytes.Equal([]byte(fileMagic), magic) { - return nil, errors.New(`arrow file magic footer mismatch`) + return errors.New(`arrow file magic footer mismatch`) } footerSize, err := d.readBackward(4) if err != nil { - return nil, pgerror.Wrap(err, pgcode.DataException, `reading arrow file footer`) + return pgerror.Wrap(err, pgcode.DataException, `reading arrow file footer`) } footerBytes, err := d.readBackward(int(binary.LittleEndian.Uint32(footerSize))) if err != nil { - return nil, pgerror.Wrap(err, pgcode.DataException, `reading arrow file footer`) + return pgerror.Wrap(err, pgcode.DataException, `reading arrow file footer`) } footer := arrowserde.GetRootAsFooter(footerBytes, 0) if footer.Version() != arrowserde.MetadataVersionV1 { - return nil, errors.Errorf(`only arrow V1 is supported got %d`, footer.Version()) + return errors.Errorf(`only arrow V1 is supported got %d`, footer.Version()) } - var schema arrowserde.Schema - footer.Schema(&schema) - typs := make([]coltypes.T, schema.FieldsLength()) - var field arrowserde.Field - for i := range typs { - schema.Fields(&field, i) - if typs[i], err = typeFromField(&field); err != nil { - return nil, err - } - } + // TODO(yuzefovich): we used to populate types here from the schema. Do we + // actually need it? var block arrowserde.Block d.recordBatches = d.recordBatches[:0] @@ -317,7 +313,7 @@ func (d *FileDeserializer) init() ([]coltypes.T, error) { }) } - return typs, nil + return nil } type countingWriter struct { @@ -331,12 +327,12 @@ func (w *countingWriter) Write(buf []byte) (int, error) { return n, err } -func schema(fb *flatbuffers.Builder, typs []coltypes.T) flatbuffers.UOffsetT { +func schema(fb *flatbuffers.Builder, typs []types.T) flatbuffers.UOffsetT { fieldOffsets := make([]flatbuffers.UOffsetT, len(typs)) for idx, typ := range typs { var fbTyp byte var fbTypOffset flatbuffers.UOffsetT - switch typ { + switch typeconv.FromColumnType(&typ) { case coltypes.Bool: arrowserde.BoolStart(fb) fbTypOffset = arrowserde.BoolEnd(fb) @@ -384,7 +380,7 @@ func schema(fb *flatbuffers.Builder, typs []coltypes.T) flatbuffers.UOffsetT { fbTypOffset = arrowserde.BinaryEnd(fb) fbTyp = arrowserde.TypeInterval default: - panic(errors.Errorf(`don't know how to map %s`, typ)) + panic(errors.Errorf(`don't know how to map %s`, &typ)) } arrowserde.FieldStart(fb) arrowserde.FieldAddTypeType(fb, fbTyp) @@ -405,7 +401,7 @@ func schema(fb *flatbuffers.Builder, typs []coltypes.T) flatbuffers.UOffsetT { return arrowserde.SchemaEnd(fb) } -func schemaMessage(fb *flatbuffers.Builder, typs []coltypes.T) flatbuffers.UOffsetT { +func schemaMessage(fb *flatbuffers.Builder, typs []types.T) flatbuffers.UOffsetT { schemaOffset := schema(fb, typs) arrowserde.MessageStart(fb) arrowserde.MessageAddVersion(fb, arrowserde.MetadataVersionV1) @@ -415,7 +411,7 @@ func schemaMessage(fb *flatbuffers.Builder, typs []coltypes.T) flatbuffers.UOffs } func fileFooter( - fb *flatbuffers.Builder, typs []coltypes.T, recordBatches []fileBlock, + fb *flatbuffers.Builder, typs []types.T, recordBatches []fileBlock, ) flatbuffers.UOffsetT { schemaOffset := schema(fb, typs) arrowserde.FooterStartRecordBatchesVector(fb, len(recordBatches)) @@ -432,51 +428,3 @@ func fileFooter( arrowserde.FooterAddRecordBatches(fb, recordBatchesOffset) return arrowserde.FooterEnd(fb) } - -func typeFromField(field *arrowserde.Field) (coltypes.T, error) { - var typeTab flatbuffers.Table - field.Type(&typeTab) - typeType := field.TypeType() - switch typeType { - case arrowserde.TypeBool: - return coltypes.Bool, nil - case arrowserde.TypeBinary: - return coltypes.Bytes, nil - case arrowserde.TypeInt: - var intType arrowserde.Int - intType.Init(typeTab.Bytes, typeTab.Pos) - if intType.IsSigned() > 0 { - switch intType.BitWidth() { - case 16: - return coltypes.Int16, nil - case 32: - return coltypes.Int32, nil - case 64: - return coltypes.Int64, nil - default: - return coltypes.Unhandled, errors.Errorf(`unhandled bit width %d`, intType.BitWidth()) - } - } - case arrowserde.TypeFloatingPoint: - var floatType arrowserde.FloatingPoint - floatType.Init(typeTab.Bytes, typeTab.Pos) - switch floatType.Precision() { - case arrowserde.PrecisionDOUBLE: - return coltypes.Float64, nil - default: - return coltypes.Unhandled, errors.Errorf(`unhandled float precision %d`, floatType.Precision()) - } - case arrowserde.TypeDecimal: - return coltypes.Decimal, nil - case arrowserde.TypeTimestamp: - return coltypes.Timestamp, nil - case arrowserde.TypeInterval: - return coltypes.Interval, nil - } - // It'd be nice if this error could include more details, but flatbuffers - // doesn't make a String method or anything like that. - if typeName, ok := arrowserde.EnumNamesType[typeType]; ok { - return coltypes.Unhandled, errors.Errorf(`unknown type: %s`, typeName) - } - return coltypes.Unhandled, errors.Errorf(`unknown type: %d`, typeType) -} diff --git a/pkg/col/colserde/file_test.go b/pkg/col/colserde/file_test.go index e8ba7f818959..1e150d9751a7 100644 --- a/pkg/col/colserde/file_test.go +++ b/pkg/col/colserde/file_test.go @@ -17,9 +17,10 @@ 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/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/stretchr/testify/require" @@ -32,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 := colexec.CopyBatch(testAllocator, b) + original := coldatatestutils.CopyBatch(b, typs) var buf bytes.Buffer s, err := colserde.NewFileSerializer(&buf, typs) @@ -46,7 +47,7 @@ func TestFileRoundtrip(t *testing.T) { for i := 0; i < 2; i++ { func() { roundtrip := coldata.NewMemBatchWithSize(nil, 0) - d, err := colserde.NewFileDeserializerFromBytes(buf.Bytes()) + d, err := colserde.NewFileDeserializerFromBytes(typs, buf.Bytes()) require.NoError(t, err) defer func() { require.NoError(t, d.Close()) }() require.Equal(t, typs, d.Typs()) @@ -65,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 := colexec.CopyBatch(testAllocator, b) + original := coldatatestutils.CopyBatch(b, typs) f, err := os.Create(path) require.NoError(t, err) @@ -82,7 +83,7 @@ func TestFileRoundtrip(t *testing.T) { for i := 0; i < 2; i++ { func() { roundtrip := coldata.NewMemBatchWithSize(nil, 0) - d, err := colserde.NewFileDeserializerFromPath(path) + d, err := colserde.NewFileDeserializerFromPath(typs, path) require.NoError(t, err) defer func() { require.NoError(t, d.Close()) }() require.Equal(t, typs, d.Typs()) @@ -99,7 +100,7 @@ func TestFileIndexing(t *testing.T) { defer leaktest.AfterTest(t)() const numInts = 10 - typs := []coltypes.T{coltypes.Int64} + typs := []types.T{*types.Int} var buf bytes.Buffer s, err := colserde.NewFileSerializer(&buf, typs) @@ -113,7 +114,7 @@ func TestFileIndexing(t *testing.T) { } require.NoError(t, s.Finish()) - d, err := colserde.NewFileDeserializerFromBytes(buf.Bytes()) + d, err := colserde.NewFileDeserializerFromBytes(typs, buf.Bytes()) require.NoError(t, err) defer func() { require.NoError(t, d.Close()) }() require.Equal(t, typs, d.Typs()) diff --git a/pkg/col/colserde/main_test.go b/pkg/col/colserde/main_test.go index aa71c99976e2..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/colexec" + "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 *colexec.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 = colexec.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 57b0155b0ec2..c6348e74c84f 100644 --- a/pkg/col/colserde/record_batch.go +++ b/pkg/col/colserde/record_batch.go @@ -18,6 +18,8 @@ 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/col/coltypes/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/types" flatbuffers "github.com/google/flatbuffers/go" "github.com/pkg/errors" ) @@ -68,7 +70,7 @@ type RecordBatchSerializer struct { // NewRecordBatchSerializer creates a new RecordBatchSerializer according to // typs. Note that Serializing or Deserializing data that does not follow the // passed in schema results in undefined behavior. -func NewRecordBatchSerializer(typs []coltypes.T) (*RecordBatchSerializer, error) { +func NewRecordBatchSerializer(typs []types.T) (*RecordBatchSerializer, error) { if len(typs) == 0 { return nil, errors.Errorf("zero length schema unsupported") } @@ -77,7 +79,7 @@ func NewRecordBatchSerializer(typs []coltypes.T) (*RecordBatchSerializer, error) builder: flatbuffers.NewBuilder(flatbufferBuilderInitialCapacity), } for i := range typs { - s.numBuffers[i] = numBuffersForType(typs[i]) + s.numBuffers[i] = numBuffersForType(typeconv.FromColumnType(&typs[i])) } // s.scratch.padding is used to align metadata to an 8 byte boundary, so // doesn't need to be larger than 7 bytes. diff --git a/pkg/col/colserde/record_batch_test.go b/pkg/col/colserde/record_batch_test.go index 43aae92db4f7..a0019e444247 100644 --- a/pkg/col/colserde/record_batch_test.go +++ b/pkg/col/colserde/record_batch_test.go @@ -27,6 +27,8 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/col/colserde" "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "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" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -36,7 +38,7 @@ import ( // randomDataFromType creates an *array.Data of length n and type t, filling it // with random values and inserting nulls with probability nullProbability. -func randomDataFromType(rng *rand.Rand, t coltypes.T, n int, nullProbability float64) *array.Data { +func randomDataFromType(rng *rand.Rand, t *types.T, n int, nullProbability float64) *array.Data { if nullProbability < 0 || nullProbability > 1 { panic(fmt.Sprintf("expected a value between 0 and 1 for nullProbability but got %f", nullProbability)) } @@ -55,7 +57,7 @@ func randomDataFromType(rng *rand.Rand, t coltypes.T, n int, nullProbability flo } var builder array.Builder - switch t { + switch typeconv.FromColumnType(t) { case coltypes.Bool: builder = array.NewBooleanBuilder(memory.DefaultAllocator) data := make([]bool, n) @@ -186,14 +188,14 @@ func TestRecordBatchSerializer(t *testing.T) { defer leaktest.AfterTest(t)() t.Run("UnsupportedSchema", func(t *testing.T) { - _, err := colserde.NewRecordBatchSerializer([]coltypes.T{}) + _, err := colserde.NewRecordBatchSerializer([]types.T{}) require.True(t, testutils.IsError(err, "zero length"), err) }) // Serializing and Deserializing an invalid schema is undefined. t.Run("SerializeDifferentColumnLengths", func(t *testing.T) { - s, err := colserde.NewRecordBatchSerializer([]coltypes.T{coltypes.Int64, coltypes.Int64}) + s, err := colserde.NewRecordBatchSerializer([]types.T{*types.Int, *types.Int}) require.NoError(t, err) b := array.NewInt64Builder(memory.DefaultAllocator) b.AppendValues([]int64{1, 2}, nil /* valid */) @@ -216,7 +218,7 @@ func TestRecordBatchSerializerSerializeDeserializeRandom(t *testing.T) { ) var ( - typs = make([]coltypes.T, rng.Intn(maxTypes)+1) + typs = make([]types.T, rng.Intn(maxTypes)+1) data = make([]*array.Data, len(typs)) dataLen = rng.Intn(maxDataLen) + 1 nullProbability = rng.Float64() @@ -224,8 +226,8 @@ func TestRecordBatchSerializerSerializeDeserializeRandom(t *testing.T) { ) for i := range typs { - typs[i] = coltypes.AllTypes[rng.Intn(len(coltypes.AllTypes))] - data[i] = randomDataFromType(rng, typs[i], dataLen, nullProbability) + typs[i] = typeconv.AllSupportedSQLTypes[rng.Intn(len(typeconv.AllSupportedSQLTypes))] + data[i] = randomDataFromType(rng, &typs[i], dataLen, nullProbability) } s, err := colserde.NewRecordBatchSerializer(typs) @@ -272,7 +274,7 @@ func BenchmarkRecordBatchSerializerInt64(b *testing.B) { rng, _ := randutil.NewPseudoRand() var ( - typs = []coltypes.T{coltypes.Int64} + typs = []types.T{*types.Int} buf = bytes.Buffer{} deserializedData []*array.Data ) @@ -283,7 +285,7 @@ func BenchmarkRecordBatchSerializerInt64(b *testing.B) { for _, dataLen := range []int{1, 16, 256, 2048, 4096} { // Only calculate useful bytes. numBytes := int64(dataLen * 8) - data := []*array.Data{randomDataFromType(rng, typs[0], dataLen, 0 /* nullProbability */)} + data := []*array.Data{randomDataFromType(rng, &typs[0], dataLen, 0 /* nullProbability */)} b.Run(fmt.Sprintf("Serialize/dataLen=%d", dataLen), func(b *testing.B) { b.SetBytes(numBytes) for i := 0; i < b.N; i++ { diff --git a/pkg/col/coltypes/typeconv/typeconv.go b/pkg/col/coltypes/typeconv/typeconv.go new file mode 100644 index 000000000000..007ae42f1303 --- /dev/null +++ b/pkg/col/coltypes/typeconv/typeconv.go @@ -0,0 +1,115 @@ +// Copyright 2018 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 typeconv + +import ( + "fmt" + + "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "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 AllSupportedSQLTypes as +// well. +func FromColumnType(ct *types.T) coltypes.T { + switch ct.Family() { + case types.BoolFamily: + return coltypes.Bool + case types.BytesFamily, types.StringFamily, types.UuidFamily: + return coltypes.Bytes + case types.DateFamily, types.OidFamily: + return coltypes.Int64 + case types.DecimalFamily: + return coltypes.Decimal + case types.IntFamily: + switch ct.Width() { + case 16: + return coltypes.Int16 + case 32: + return coltypes.Int32 + case 0, 64: + return coltypes.Int64 + } + panic(fmt.Sprintf("integer with unknown width %d", ct.Width())) + case types.FloatFamily: + return coltypes.Float64 + case types.TimestampFamily: + return coltypes.Timestamp + case types.TimestampTZFamily: + return coltypes.Timestamp + case types.IntervalFamily: + return coltypes.Interval + } + return coltypes.Unhandled +} + +// FromColumnTypes calls FromColumnType on each element of cts, returning the +// resulting slice. +func FromColumnTypes(cts []types.T) ([]coltypes.T, error) { + typs := make([]coltypes.T, len(cts)) + for i := range typs { + typs[i] = FromColumnType(&cts[i]) + if typs[i] == coltypes.Unhandled { + return nil, errors.Errorf("unsupported type %s", cts[i].String()) + } + } + return typs, nil +} + +// UnsafeToSQLType converts the given coltype to the logical SQL type. Note +// that this conversion is lossful since multiple logical types can map to a +// single coltype, so use this method *only* when such behavior is acceptable. +func UnsafeToSQLType(t coltypes.T) (*types.T, error) { + switch t { + case coltypes.Bool: + return types.Bool, nil + case coltypes.Bytes: + return types.Bytes, nil + case coltypes.Decimal: + return types.Decimal, nil + case coltypes.Int16: + return types.Int2, nil + case coltypes.Int32: + return types.Int4, nil + case coltypes.Int64: + return types.Int, nil + case coltypes.Float64: + return types.Float, nil + case coltypes.Timestamp: + return types.Timestamp, nil + case coltypes.Interval: + return types.Interval, nil + default: + return nil, errors.Errorf("unsupported coltype %s", t) + } +} diff --git a/pkg/internal/sqlsmith/type.go b/pkg/internal/sqlsmith/type.go index fc64f251e6c3..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/colexec/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/colcontainer/diskqueue.go b/pkg/sql/colcontainer/diskqueue.go index a36ccfdaed88..5dfadfa15408 100644 --- a/pkg/sql/colcontainer/diskqueue.go +++ b/pkg/sql/colcontainer/diskqueue.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/colserde" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/storage/fs" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -157,7 +157,7 @@ type diskQueue struct { // dirName is the directory in cfg.Path that holds this queue's files. dirName string - typs []coltypes.T + typs []types.T cfg DiskQueueCfg files []file seqNo int @@ -327,14 +327,14 @@ func (cfg *DiskQueueCfg) SetDefaultBufferSizeBytesForCacheMode() { // NewDiskQueue creates a Queue that spills to disk. func NewDiskQueue( - ctx context.Context, typs []coltypes.T, cfg DiskQueueCfg, diskAcc *mon.BoundAccount, + ctx context.Context, typs []types.T, cfg DiskQueueCfg, diskAcc *mon.BoundAccount, ) (Queue, error) { return newDiskQueue(ctx, typs, cfg, diskAcc) } // NewRewindableDiskQueue creates a RewindableQueue that spills to disk. func NewRewindableDiskQueue( - ctx context.Context, typs []coltypes.T, cfg DiskQueueCfg, diskAcc *mon.BoundAccount, + ctx context.Context, typs []types.T, cfg DiskQueueCfg, diskAcc *mon.BoundAccount, ) (RewindableQueue, error) { d, err := newDiskQueue(ctx, typs, cfg, diskAcc) if err != nil { @@ -345,7 +345,7 @@ func NewRewindableDiskQueue( } func newDiskQueue( - ctx context.Context, typs []coltypes.T, cfg DiskQueueCfg, diskAcc *mon.BoundAccount, + ctx context.Context, typs []types.T, cfg DiskQueueCfg, diskAcc *mon.BoundAccount, ) (*diskQueue, error) { if err := cfg.EnsureDefaults(); err != nil { return nil, err @@ -645,7 +645,7 @@ func (d *diskQueue) maybeInitDeserializer(ctx context.Context) (bool, error) { decompressedBytes = d.scratchDecompressedReadBytes } - deserializer, err := colserde.NewFileDeserializerFromBytes(decompressedBytes) + deserializer, err := colserde.NewFileDeserializerFromBytes(d.typs, decompressedBytes) if err != nil { return false, err } @@ -723,7 +723,7 @@ func (d *diskQueue) Dequeue(ctx context.Context, b coldata.Batch) (bool, error) // TODO(asubiotto): This is a stop-gap solution. The issue is that // ownership semantics are a bit murky. Can we do better? Refer to the // issue. - vecs[i] = coldata.NewMemColumn(d.typs[i], coldata.BatchSize()) + vecs[i] = coldata.NewMemColumn(&d.typs[i], coldata.BatchSize()) } } if err := d.deserializerState.GetBatch(d.deserializerState.curBatch, b); err != nil { diff --git a/pkg/sql/colcontainer/diskqueue_test.go b/pkg/sql/colcontainer/diskqueue_test.go index 589dbb952e9e..a6921232b8e8 100644 --- a/pkg/sql/colcontainer/diskqueue_test.go +++ b/pkg/sql/colcontainer/diskqueue_test.go @@ -16,9 +16,10 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "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/types" "github.com/cockroachdb/cockroach/pkg/testutils/colcontainerutils" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -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 := colexec.NewRandomDataOp(testAllocator, rng, colexec.RandomDataOpArgs{ + op := coldatatestutils.NewRandomDataOp(testAllocator, rng, coldatatestutils.RandomDataOpArgs{ NumBatches: cap(batches), BatchSize: 1 + rng.Intn(coldata.BatchSize()), Nulls: true, - BatchAccumulator: func(b coldata.Batch) { - batches = append(batches, colexec.CopyBatch(testAllocator, b)) + BatchAccumulator: func(b coldata.Batch, typs []types.T) { + batches = append(batches, coldatatestutils.CopyBatch(b, typs)) }, }) typs := op.Typs() @@ -199,9 +200,9 @@ func BenchmarkDiskQueue(b *testing.B) { queueCfg.MaxFileSizeBytes = int(blockSize) rng, _ := randutil.NewPseudoRand() - typs := []coltypes.T{coltypes.Int64} - batch := colexec.RandomBatch(testAllocator, rng, typs, coldata.BatchSize(), 0, 0) - op := colexec.NewRepeatableBatchSource(testAllocator, batch) + typs := []types.T{*types.Int} + 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 9fa14ad5c565..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/colexec" + "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 *colexec.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 = colexec.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 9ac967110fea..b627df6bb0ff 100644 --- a/pkg/sql/colcontainer/partitionedqueue.go +++ b/pkg/sql/colcontainer/partitionedqueue.go @@ -15,8 +15,8 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" "github.com/marusama/semaphore" @@ -105,7 +105,7 @@ const ( // PartitionedDiskQueue is a PartitionedQueue whose partitions are on-disk. type PartitionedDiskQueue struct { - typs []coltypes.T + typs []types.T strategy PartitionerStrategy cfg DiskQueueCfg @@ -134,7 +134,7 @@ var _ PartitionedQueue = &PartitionedDiskQueue{} // Note that actual file descriptors open may be less than, but never more than // the number acquired through the semaphore. func NewPartitionedDiskQueue( - typs []coltypes.T, + typs []types.T, cfg DiskQueueCfg, fdSemaphore semaphore.Semaphore, partitionerStrategy PartitionerStrategy, @@ -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. - execerror.VectorizedInternalPanic("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 { - execerror.VectorizedInternalPanic(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 { - execerror.VectorizedInternalPanic(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: - execerror.VectorizedInternalPanic(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. - execerror.VectorizedInternalPanic("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 1cf09dfdef53..757acc94f8e0 100644 --- a/pkg/sql/colcontainer/partitionedqueue_test.go +++ b/pkg/sql/colcontainer/partitionedqueue_test.go @@ -16,9 +16,9 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" "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/types" "github.com/cockroachdb/cockroach/pkg/storage/fs" "github.com/cockroachdb/cockroach/pkg/testutils/colcontainerutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -93,9 +93,9 @@ func TestPartitionedDiskQueue(t *testing.T) { var ( ctx = context.Background() - typs = []coltypes.T{coltypes.Int64} + typs = []types.T{*types.Int} batch = testAllocator.NewMemBatch(typs) - sem = &colexec.TestingSemaphore{} + sem = &colexecbase.TestingSemaphore{} ) batch.SetLength(coldata.BatchSize()) @@ -144,7 +144,7 @@ func TestPartitionedDiskQueueSimulatedExternal(t *testing.T) { var ( ctx = context.Background() - typs = []coltypes.T{coltypes.Int64} + typs = []types.T{*types.Int} batch = testAllocator.NewMemBatch(typs) rng, _ = randutil.NewPseudoRand() // maxPartitions is in [1, 10]. The maximum partitions on a single level. @@ -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 := colexec.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 := colexec.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 b07c71efa50a..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/colexec/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" @@ -30,18 +30,15 @@ func TestDecodeTableValueToCol(t *testing.T) { var scratch []byte nCols := 1000 datums := make([]tree.Datum, nCols) - colTyps := make([]*types.T, nCols) - typs := make([]coltypes.T, nCols) + typs := make([]types.T, nCols) for i := 0; i < nCols; i++ { ct := sqlbase.RandType(rng) - et := typeconv.FromColumnType(ct) - if et == coltypes.Unhandled { + if typeconv.FromColumnType(ct) == coltypes.Unhandled { i-- continue } datum := sqlbase.RandDatum(rng, ct, false /* nullOk */) - colTyps[i] = ct - typs[i] = et + typs[i] = *ct datums[i] = datum var err error fmt.Println(datum) @@ -58,7 +55,7 @@ func TestDecodeTableValueToCol(t *testing.T) { t.Fatal(err) } buf, err = DecodeTableValueToCol(batch.ColVec(i), 0 /* rowIdx */, typ, - dataOffset, colTyps[i], buf[typeOffset:]) + dataOffset, &typs[i], buf[typeOffset:]) if err != nil { t.Fatal(err) } diff --git a/pkg/sql/colexec/aggregator.go b/pkg/sql/colexec/aggregator.go index b9fd275dd0c9..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/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/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,13 +105,13 @@ type aggregateFunc interface { type orderedAggregator struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator done bool aggCols [][]uint32 - aggTypes [][]coltypes.T + aggTypes [][]types.T - outputTypes []coltypes.T + outputTypes []types.T // scratch is the Batch to output and variables related to it. Aggregate // function operators write directly to this output batch. @@ -149,21 +150,21 @@ type orderedAggregator struct { seenNonEmptyBatch bool } -var _ 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 *Allocator, - input Operator, - colTypes []coltypes.T, + allocator *colmem.Allocator, + input colexecbase.Operator, + typs []types.T, aggFns []execinfrapb.AggregatorSpec_Func, groupCols []uint32, aggCols [][]uint32, isScalar bool, -) (Operator, error) { +) (colexecbase.Operator, error) { if len(aggFns) != len(aggCols) { return nil, errors.Errorf( @@ -173,9 +174,9 @@ func NewOrderedAggregator( ) } - aggTypes := extractAggTypes(aggCols, colTypes) + aggTypes := extractAggTypes(aggCols, typs) - op, groupCol, err := OrderedDistinctColsToOperators(input, groupCols, colTypes) + op, groupCol, err := OrderedDistinctColsToOperators(input, groupCols, typs) if err != nil { return nil, err } @@ -228,7 +229,7 @@ func NewOrderedAggregator( } func makeAggregateFuncs( - allocator *Allocator, aggTyps [][]coltypes.T, aggFns []execinfrapb.AggregatorSpec_Func, + allocator *colmem.Allocator, aggTyps [][]types.T, aggFns []execinfrapb.AggregatorSpec_Func, ) ([]aggregateFunc, error) { funcs := make([]aggregateFunc, len(aggFns)) @@ -236,19 +237,19 @@ func makeAggregateFuncs( var err error switch aggFns[i] { case execinfrapb.AggregatorSpec_ANY_NOT_NULL: - funcs[i], err = newAnyNotNullAgg(allocator, aggTyps[i][0]) + funcs[i], err = newAnyNotNullAgg(allocator, &aggTyps[i][0]) case execinfrapb.AggregatorSpec_AVG: - funcs[i], err = newAvgAgg(aggTyps[i][0]) + funcs[i], err = newAvgAgg(&aggTyps[i][0]) case execinfrapb.AggregatorSpec_SUM, execinfrapb.AggregatorSpec_SUM_INT: - funcs[i], err = newSumAgg(aggTyps[i][0]) + funcs[i], err = newSumAgg(&aggTyps[i][0]) case execinfrapb.AggregatorSpec_COUNT_ROWS: funcs[i] = newCountRowAgg() case execinfrapb.AggregatorSpec_COUNT: funcs[i] = newCountAgg() case execinfrapb.AggregatorSpec_MIN: - funcs[i], err = newMinAgg(allocator, aggTyps[i][0]) + funcs[i], err = newMinAgg(allocator, &aggTyps[i][0]) case execinfrapb.AggregatorSpec_MAX: - funcs[i], err = newMaxAgg(allocator, aggTyps[i][0]) + funcs[i], err = newMaxAgg(allocator, &aggTyps[i][0]) case execinfrapb.AggregatorSpec_BOOL_AND: funcs[i] = newBoolAndAgg() case execinfrapb.AggregatorSpec_BOOL_OR: @@ -266,9 +267,9 @@ func makeAggregateFuncs( } func makeAggregateFuncsOutputTypes( - aggTyps [][]coltypes.T, aggFns []execinfrapb.AggregatorSpec_Func, -) ([]coltypes.T, error) { - outTyps := make([]coltypes.T, len(aggFns)) + aggTyps [][]types.T, aggFns []execinfrapb.AggregatorSpec_Func, +) ([]types.T, error) { + outTyps := make([]types.T, len(aggFns)) for i := range aggFns { // Set the output type of the aggregate. @@ -276,7 +277,7 @@ func makeAggregateFuncsOutputTypes( case execinfrapb.AggregatorSpec_COUNT_ROWS, execinfrapb.AggregatorSpec_COUNT: // TODO(jordan): this is a somewhat of a hack. The aggregate functions // should come with their own output types, somehow. - outTyps[i] = coltypes.Int64 + outTyps[i] = *types.Int case execinfrapb.AggregatorSpec_ANY_NOT_NULL, execinfrapb.AggregatorSpec_AVG, @@ -345,7 +346,7 @@ func (a *orderedAggregator) Next(ctx context.Context) coldata.Batch { vec.Append( coldata.SliceArgs{ Src: vec, - ColType: a.outputTypes[i], + ColType: typeconv.FromColumnType(&a.outputTypes[i]), DestIdx: 0, SrcStartIdx: a.scratch.outputSize, SrcEndIdx: a.scratch.resumeIdx + 1, @@ -366,7 +367,7 @@ func (a *orderedAggregator) Next(ctx context.Context) coldata.Batch { coldata.CopySliceArgs{ SliceArgs: coldata.SliceArgs{ Src: a.scratch.ColVec(i), - ColType: a.outputTypes[i], + ColType: typeconv.FromColumnType(&a.outputTypes[i]), SrcStartIdx: 0, SrcEndIdx: a.scratch.Length(), }, @@ -421,7 +422,7 @@ func (a *orderedAggregator) Next(ctx context.Context) coldata.Batch { coldata.CopySliceArgs{ SliceArgs: coldata.SliceArgs{ Src: a.scratch.ColVec(i), - ColType: a.outputTypes[i], + ColType: typeconv.FromColumnType(&a.outputTypes[i]), SrcStartIdx: 0, SrcEndIdx: a.scratch.Length(), }, @@ -456,13 +457,13 @@ func (a *orderedAggregator) reset(ctx context.Context) { // extractAggTypes returns a nested array representing the input types // corresponding to each aggregation function. -func extractAggTypes(aggCols [][]uint32, colTypes []coltypes.T) [][]coltypes.T { - aggTyps := make([][]coltypes.T, len(aggCols)) +func extractAggTypes(aggCols [][]uint32, typs []types.T) [][]types.T { + aggTyps := make([][]types.T, len(aggCols)) for aggIdx := range aggCols { - aggTyps[aggIdx] = make([]coltypes.T, len(aggCols[aggIdx])) + aggTyps[aggIdx] = make([]types.T, len(aggCols[aggIdx])) for i, colIdx := range aggCols[aggIdx] { - aggTyps[aggIdx][i] = colTypes[colIdx] + aggTyps[aggIdx][i] = typs[colIdx] } } @@ -475,7 +476,7 @@ func extractAggTypes(aggCols [][]uint32, colTypes []coltypes.T) [][]coltypes.T { func isAggregateSupported( aggFn execinfrapb.AggregatorSpec_Func, inputTypes []types.T, ) (bool, error) { - aggTypes, err := typeconv.FromColumnTypes(inputTypes) + _, err := typeconv.FromColumnTypes(inputTypes) if err != nil { return false, err } @@ -496,14 +497,14 @@ func isAggregateSupported( } _, err = makeAggregateFuncs( nil, /* allocator */ - [][]coltypes.T{aggTypes}, + [][]types.T{inputTypes}, []execinfrapb.AggregatorSpec_Func{aggFn}, ) if err != nil { return false, err } outputTypes, err := makeAggregateFuncsOutputTypes( - [][]coltypes.T{aggTypes}, + [][]types.T{inputTypes}, []execinfrapb.AggregatorSpec_Func{aggFn}, ) if err != nil { @@ -519,7 +520,7 @@ func isAggregateSupported( // return INT8), so we explicitly check whether the type the columnar // aggregate returns and the type the planning code will expect it to return // are the same. If they are not, we fallback to row-by-row engine. - if typeconv.FromColumnType(retType) != outputTypes[0] { + if !retType.Identical(&outputTypes[0]) { // TODO(yuzefovich): support this case through vectorize. Probably it needs // to be done at the same time as #38845. return false, errors.Newf("aggregates with different input and output types are not supported") diff --git a/pkg/sql/colexec/aggregator_test.go b/pkg/sql/colexec/aggregator_test.go index 325dbd5ad44c..a6e755c32bcf 100644 --- a/pkg/sql/colexec/aggregator_test.go +++ b/pkg/sql/colexec/aggregator_test.go @@ -18,8 +18,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/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" "github.com/cockroachdb/cockroach/pkg/util/randutil" ) @@ -28,13 +31,13 @@ var ( defaultGroupCols = []uint32{0} defaultAggCols = [][]uint32{{1}} defaultAggFns = []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_SUM} - defaultColTyps = []coltypes.T{coltypes.Int64, coltypes.Int64} + defaultTyps = []types.T{*types.Int, *types.Int} ) type aggregatorTestCase struct { - // colTypes, aggFns, groupCols, and aggCols will be set to their default + // typs, aggFns, groupCols, and aggCols will be set to their default // values before running a test if nil. - colTypes []coltypes.T + typs []types.T aggFns []execinfrapb.AggregatorSpec_Func groupCols []uint32 aggCols [][]uint32 @@ -57,14 +60,14 @@ type aggregatorTestCase struct { // hash aggregators at the same time. type aggType struct { new func( - allocator *Allocator, - input Operator, - colTypes []coltypes.T, + allocator *colmem.Allocator, + input colexecbase.Operator, + typs []types.T, aggFns []execinfrapb.AggregatorSpec_Func, groupCols []uint32, aggCols [][]uint32, isScalar bool, - ) (Operator, error) + ) (colexecbase.Operator, error) name string } @@ -73,16 +76,16 @@ var aggTypes = []aggType{ // This is a wrapper around NewHashAggregator so its signature is compatible // with orderedAggregator. new: func( - allocator *Allocator, - input Operator, - colTypes []coltypes.T, + allocator *colmem.Allocator, + input colexecbase.Operator, + typs []types.T, aggFns []execinfrapb.AggregatorSpec_Func, groupCols []uint32, aggCols [][]uint32, _ bool, - ) (Operator, error) { + ) (colexecbase.Operator, error) { return NewHashAggregator( - allocator, input, colTypes, aggFns, groupCols, aggCols) + allocator, input, typs, aggFns, groupCols, aggCols) }, name: "hash", }, @@ -128,8 +131,8 @@ func (tc *aggregatorTestCase) init() error { if tc.aggCols == nil { tc.aggCols = defaultAggCols } - if tc.colTypes == nil { - tc.colTypes = defaultColTyps + if tc.typs == nil { + tc.typs = defaultTyps } if tc.batchSize == 0 { tc.batchSize = coldata.BatchSize() @@ -268,7 +271,7 @@ func TestAggregatorOneFunc(t *testing.T) { batchSize: 1, outputBatchSize: 1, name: "UnusedInputColumns", - colTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, + typs: []types.T{*types.Int, *types.Int, *types.Int}, groupCols: []uint32{1, 2}, aggCols: [][]uint32{{0}}, }, @@ -299,7 +302,7 @@ func TestAggregatorOneFunc(t *testing.T) { a, err := NewOrderedAggregator( testAllocator, tupleSource, - tc.colTypes, + tc.typs, tc.aggFns, tc.groupCols, tc.aggCols, @@ -328,11 +331,11 @@ func TestAggregatorOneFunc(t *testing.T) { } t.Run(agg.name, func(t *testing.T) { runTests(t, []tuples{tc.input}, tc.expected, unorderedVerifier, - func(input []Operator) (Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return agg.new( testAllocator, input[0], - tc.colTypes, + tc.typs, tc.aggFns, tc.groupCols, tc.aggCols, @@ -358,7 +361,7 @@ func TestAggregatorMultiFunc(t *testing.T) { {0, 1, 2}, {0, 1, 2}, }, - colTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, + typs: []types.T{*types.Int, *types.Int, *types.Int}, expected: tuples{ {4, 2}, }, @@ -375,7 +378,7 @@ func TestAggregatorMultiFunc(t *testing.T) { {0, 1, 0.5}, {1, 1, 1.2}, }, - colTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Decimal}, + typs: []types.T{*types.Int, *types.Int, *types.Decimal}, expected: tuples{ {3.4, 3}, {1.2, 1}, @@ -395,7 +398,7 @@ func TestAggregatorMultiFunc(t *testing.T) { {1, 6.21}, {1, 2.43}, }, - colTypes: []coltypes.T{coltypes.Int64, coltypes.Decimal}, + typs: []types.T{*types.Int, *types.Decimal}, expected: tuples{ {"1.5333333333333333333", 4.6}, {4.32, 8.64}, @@ -430,7 +433,7 @@ func TestAggregatorMultiFunc(t *testing.T) { {8, nil}, {8, nil}, }, - colTypes: []coltypes.T{coltypes.Int64, coltypes.Bool}, + typs: []types.T{*types.Int, *types.Bool}, expected: tuples{ {true, true}, {false, false}, @@ -462,7 +465,7 @@ func TestAggregatorMultiFunc(t *testing.T) { {2, 2.0, "2.0", 6.0, 6.0}, }, batchSize: 1, - colTypes: []coltypes.T{coltypes.Int64, coltypes.Decimal, coltypes.Bytes, coltypes.Decimal}, + typs: []types.T{*types.Int, *types.Decimal, *types.Bytes, *types.Decimal}, name: "MultiGroupColsWithPointerTypes", groupCols: []uint32{0, 1, 2}, aggCols: [][]uint32{ @@ -477,9 +480,9 @@ func TestAggregatorMultiFunc(t *testing.T) { if err := tc.init(); err != nil { t.Fatal(err) } - runTestsWithTyps(t, []tuples{tc.input}, [][]coltypes.T{tc.colTypes}, tc.expected, unorderedVerifier, - func(input []Operator) (Operator, error) { - return agg.new(testAllocator, input[0], tc.colTypes, tc.aggFns, tc.groupCols, tc.aggCols, false /* isScalar */) + runTestsWithTyps(t, []tuples{tc.input}, [][]types.T{tc.typs}, tc.expected, unorderedVerifier, + func(input []colexecbase.Operator) (colexecbase.Operator, error) { + return agg.new(testAllocator, input[0], tc.typs, tc.aggFns, tc.groupCols, tc.aggCols, false /* isScalar */) }) }) } @@ -502,8 +505,8 @@ func TestAggregatorAllFunctions(t *testing.T) { execinfrapb.AggregatorSpec_BOOL_AND, execinfrapb.AggregatorSpec_BOOL_OR, }, - aggCols: [][]uint32{{0}, {4}, {1}, {}, {1}, {2}, {2}, {2}, {3}, {3}}, - colTypes: []coltypes.T{coltypes.Int64, coltypes.Decimal, coltypes.Int64, coltypes.Bool, coltypes.Bytes}, + aggCols: [][]uint32{{0}, {4}, {1}, {}, {1}, {2}, {2}, {2}, {3}, {3}}, + typs: []types.T{*types.Int, *types.Decimal, *types.Int, *types.Bool, *types.Bytes}, input: tuples{ {0, 3.1, 2, true, "zero"}, {0, 1.1, 3, false, "zero"}, @@ -537,8 +540,8 @@ func TestAggregatorAllFunctions(t *testing.T) { execinfrapb.AggregatorSpec_BOOL_AND, execinfrapb.AggregatorSpec_BOOL_OR, }, - aggCols: [][]uint32{{0}, {1}, {}, {1}, {1}, {2}, {2}, {2}, {1}, {3}, {3}}, - colTypes: []coltypes.T{coltypes.Int64, coltypes.Decimal, coltypes.Int64, coltypes.Bool}, + aggCols: [][]uint32{{0}, {1}, {}, {1}, {1}, {2}, {2}, {2}, {1}, {3}, {3}}, + typs: []types.T{*types.Int, *types.Decimal, *types.Int, *types.Bool}, input: tuples{ {nil, 1.1, 4, true}, {0, nil, nil, nil}, @@ -570,8 +573,8 @@ func TestAggregatorAllFunctions(t *testing.T) { []tuples{tc.input}, tc.expected, verifier, - func(input []Operator) (Operator, error) { - return agg.new(testAllocator, input[0], tc.colTypes, tc.aggFns, tc.groupCols, tc.aggCols, false /* isScalar */) + func(input []colexecbase.Operator) (colexecbase.Operator, error) { + return agg.new(testAllocator, input[0], tc.typs, tc.aggFns, tc.groupCols, tc.aggCols, false /* isScalar */) }) }) } @@ -596,10 +599,10 @@ func TestAggregatorRandom(t *testing.T) { t.Run(fmt.Sprintf("%s/groupSize=%d/numInputBatches=%d/hasNulls=%t", agg.name, groupSize, numInputBatches, hasNulls), func(t *testing.T) { nTuples := coldata.BatchSize() * numInputBatches - typs := []coltypes.T{coltypes.Int64, coltypes.Float64} + typs := []types.T{*types.Int, *types.Float} cols := []coldata.Vec{ - testAllocator.NewMemColumn(typs[0], nTuples), - testAllocator.NewMemColumn(typs[1], nTuples), + testAllocator.NewMemColumn(&typs[0], nTuples), + testAllocator.NewMemColumn(&typs[1], nTuples), } groups, aggCol, aggColNulls := cols[0].Int64(), cols[1].Float64(), cols[1].Nulls() expectedTuples := tuples{} @@ -717,12 +720,12 @@ func BenchmarkAggregator(b *testing.B) { fName := execinfrapb.AggregatorSpec_Func_name[int32(aggFn)] b.Run(fName, func(b *testing.B) { for _, agg := range aggTypes { - for typIdx, typ := range []coltypes.T{coltypes.Int64, coltypes.Decimal, coltypes.Bytes} { + for typIdx, typ := range []types.T{*types.Int, *types.Decimal, *types.Bytes} { for _, groupSize := range []int{1, 2, coldata.BatchSize() / 2, coldata.BatchSize()} { for _, hasNulls := range []bool{false, true} { for _, numInputBatches := range []int{64} { if aggFn == execinfrapb.AggregatorSpec_BOOL_AND || aggFn == execinfrapb.AggregatorSpec_BOOL_OR { - typ = coltypes.Bool + typ = *types.Bool if typIdx > 0 { // We don't need to run the benchmark of bool_and and // bool_or multiple times, so we skip all runs except @@ -733,11 +736,11 @@ func BenchmarkAggregator(b *testing.B) { b.Run(fmt.Sprintf("%s/%s/groupSize=%d/hasNulls=%t/numInputBatches=%d", agg.name, typ.String(), groupSize, hasNulls, numInputBatches), func(b *testing.B) { - colTypes := []coltypes.T{coltypes.Int64, typ} + typs := []types.T{*types.Int, typ} nTuples := numInputBatches * coldata.BatchSize() cols := []coldata.Vec{ - testAllocator.NewMemColumn(coltypes.Int64, nTuples), - testAllocator.NewMemColumn(typ, nTuples), + testAllocator.NewMemColumn(types.Int, nTuples), + testAllocator.NewMemColumn(&typ, nTuples), } groups := cols[0].Int64() curGroup := -1 @@ -751,8 +754,8 @@ func BenchmarkAggregator(b *testing.B) { if hasNulls { nullProb = nullProbability } - coldata.RandomVec(rng, typ, bytesFixedLength, cols[1], nTuples, nullProb) - if typ == coltypes.Int64 && aggFn == execinfrapb.AggregatorSpec_SUM { + 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 // restrict the range of values. @@ -761,7 +764,7 @@ func BenchmarkAggregator(b *testing.B) { vals[i] = vals[i] % 1024 } } - source := newChunkingBatchSource(colTypes, cols, nTuples) + source := newChunkingBatchSource(typs, cols, nTuples) nCols := 1 if aggFn == execinfrapb.AggregatorSpec_COUNT_ROWS { @@ -770,7 +773,7 @@ func BenchmarkAggregator(b *testing.B) { a, err := agg.new( testAllocator, source, - colTypes, + typs, []execinfrapb.AggregatorSpec_Func{aggFn}, []uint32{0}, [][]uint32{[]uint32{1}[:nCols]}, @@ -817,7 +820,7 @@ func TestHashAggregator(t *testing.T) { {0, 3}, {0, 7}, }, - colTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, + typs: []types.T{*types.Int, *types.Int}, groupCols: []uint32{0}, aggCols: [][]uint32{{1}}, @@ -834,7 +837,7 @@ func TestHashAggregator(t *testing.T) { input: tuples{ {5}, }, - colTypes: []coltypes.T{coltypes.Int64}, + typs: []types.T{*types.Int}, groupCols: []uint32{0}, aggCols: [][]uint32{{0}}, @@ -853,7 +856,7 @@ func TestHashAggregator(t *testing.T) { {0, 5}, {hashTableNumBuckets, 7}, }, - colTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, + typs: []types.T{*types.Int, *types.Int}, groupCols: []uint32{0}, aggCols: [][]uint32{{1}}, @@ -871,7 +874,7 @@ func TestHashAggregator(t *testing.T) { {0, 1, 0.5}, {1, 1, 1.2}, }, - colTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Decimal}, + typs: []types.T{*types.Int, *types.Int, *types.Decimal}, convToDecimal: true, aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_SUM, execinfrapb.AggregatorSpec_SUM}, @@ -897,7 +900,7 @@ func TestHashAggregator(t *testing.T) { {0, 1, 6, 11}, {1, 2, 6, 13}, }, - colTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64, coltypes.Int64}, + typs: []types.T{*types.Int, *types.Int, *types.Int, *types.Int}, groupCols: []uint32{0, 1}, aggCols: [][]uint32{{3}}, @@ -917,8 +920,8 @@ 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 []Operator) (Operator, error) { - a, err := NewHashAggregator(testAllocator, sources[0], tc.colTypes, tc.aggFns, tc.groupCols, tc.aggCols) + 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 d07543850c2f..19eb9150eee1 100644 --- a/pkg/sql/colexec/and_or_projection_test.go +++ b/pkg/sql/colexec/and_or_projection_test.go @@ -16,8 +16,8 @@ import ( "testing" "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/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -194,10 +194,10 @@ func TestAndOrOps(t *testing.T) { runner( t, []tuples{tc.tuples}, - [][]coltypes.T{{coltypes.Bool, coltypes.Bool}}, + [][]types.T{{*types.Bool, *types.Bool}}, tc.expected, orderedVerifier, - func(input []Operator) (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 */ @@ -229,7 +229,7 @@ func benchmarkLogicalProjOp( } rng, _ := randutil.NewPseudoRand() - batch := testAllocator.NewMemBatch([]coltypes.T{coltypes.Bool, coltypes.Bool}) + batch := testAllocator.NewMemBatch([]types.T{*types.Bool, *types.Bool}) col1 := batch.ColVec(0).Bool() col2 := batch.ColVec(0).Bool() for i := 0; i < coldata.BatchSize(); i++ { @@ -256,9 +256,10 @@ func benchmarkLogicalProjOp( sel[i] = i } } - input := NewRepeatableBatchSource(testAllocator, batch) + typs := []types.T{*types.Bool, *types.Bool} + input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) logicalProjOp, err := createTestProjectingOperator( - ctx, flowCtx, input, []types.T{*types.Bool, *types.Bool}, + ctx, flowCtx, input, typs, fmt.Sprintf("@1 %s @2", operation), false, /* canFallbackToRowexec */ ) require.NoError(b, err) diff --git a/pkg/sql/colexec/and_or_projection_tmpl.go b/pkg/sql/colexec/and_or_projection_tmpl.go index 02026283fa83..fbaa2e181d6c 100644 --- a/pkg/sql/colexec/and_or_projection_tmpl.go +++ b/pkg/sql/colexec/and_or_projection_tmpl.go @@ -24,18 +24,20 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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 *Allocator - input Operator + allocator *colmem.Allocator + input colexecbase.Operator - leftProjOpChain Operator - rightProjOpChain Operator + leftProjOpChain colexecbase.Operator + rightProjOpChain colexecbase.Operator leftFeedOp *feedOperator rightFeedOp *feedOperator @@ -53,11 +55,11 @@ type _OP_LOWERProjOp struct { // the boolean columns at leftIdx and rightIdx, returning the result in // outputIdx. func New_OP_TITLEProjOp( - allocator *Allocator, - input, leftProjOpChain, rightProjOpChain Operator, + allocator *colmem.Allocator, + input, leftProjOpChain, rightProjOpChain colexecbase.Operator, leftFeedOp, rightFeedOp *feedOperator, leftIdx, rightIdx, outputIdx int, -) Operator { +) colexecbase.Operator { return &_OP_LOWERProjOp{ allocator: allocator, input: input, @@ -85,7 +87,7 @@ func (o *_OP_LOWERProjOp) Child(nth int, verbose bool) execinfra.OpNode { case 2: return o.rightProjOpChain default: - execerror.VectorizedInternalPanic(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 e59669cc8fe9..08905465ff9d 100644 --- a/pkg/sql/colexec/any_not_null_agg_tmpl.go +++ b/pkg/sql/colexec/any_not_null_agg_tmpl.go @@ -25,15 +25,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/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 *Allocator, t coltypes.T) (aggregateFunc, error) { - switch t { +func newAnyNotNullAgg(allocator *colmem.Allocator, t *types.T) (aggregateFunc, error) { + switch typeconv.FromColumnType(t) { // {{range .}} case _TYPES_T: return &anyNotNull_TYPEAgg{allocator: allocator}, nil @@ -43,6 +44,9 @@ func newAnyNotNullAgg(allocator *Allocator, t coltypes.T) (aggregateFunc, error) } } +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Declarations to make the template compile properly. @@ -72,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 *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 61c1d7c75e69..7d3fcff6bee0 100644 --- a/pkg/sql/colexec/avg_agg_tmpl.go +++ b/pkg/sql/colexec/avg_agg_tmpl.go @@ -23,8 +23,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/colexec/execerror" + "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" ) @@ -37,23 +39,26 @@ var _ apd.Decimal // Dummy import to pull in "tree" package. var _ tree.Datum +// Dummy import to pull in "coltypes" package. +var _ coltypes.T + // _ASSIGN_DIV_INT64 is the template division function for assigning the first // input to the result of the second input / the third input, where the third // input is an int64. func _ASSIGN_DIV_INT64(_, _, _ string) { - execerror.VectorizedInternalPanic("") + 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) { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} -func newAvgAgg(t coltypes.T) (aggregateFunc, error) { - switch t { +func newAvgAgg(t *types.T) (aggregateFunc, error) { + switch typeconv.FromColumnType(t) { // {{range .}} case _TYPES_T: return &avg_TYPEAgg{}, nil diff --git a/pkg/sql/colexec/bool_and_or_agg_tmpl.go b/pkg/sql/colexec/bool_and_or_agg_tmpl.go index 0ec17b55f588..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/colexec/execerror" - // */}} - // HACK: crlfmt removes the "*/}}" comment if it's the last line in the import - // block. This was picked because it sorts after "pkg/sql/colexec/execerror" 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) { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} diff --git a/pkg/sql/colexec/bool_vec_to_sel.go b/pkg/sql/colexec/bool_vec_to_sel.go index 92218258db9a..f6c003ba23bf 100644 --- a/pkg/sql/colexec/bool_vec_to_sel.go +++ b/pkg/sql/colexec/bool_vec_to_sel.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" ) // boolVecToSelOp transforms a boolean column into a selection vector by adding @@ -27,7 +28,7 @@ type boolVecToSelOp struct { outputCol []bool } -var _ 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 @@ -101,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 Operator, colIdx int) 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 efffeb52e0d5..2af97a7220c2 100644 --- a/pkg/sql/colexec/bool_vec_to_sel_test.go +++ b/pkg/sql/colexec/bool_vec_to_sel_test.go @@ -13,6 +13,7 @@ package colexec import ( "testing" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) @@ -30,7 +31,7 @@ func TestBoolVecToSelOp(t *testing.T) { }, } for _, tc := range tcs { - runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, func(input []Operator) (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 0cb986196671..b127eede0374 100644 --- a/pkg/sql/colexec/buffer.go +++ b/pkg/sql/colexec/buffer.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" ) // bufferOp is an operator that buffers a single batch at a time from an input, @@ -27,11 +28,11 @@ type bufferOp struct { batch coldata.Batch } -var _ Operator = &bufferOp{} +var _ colexecbase.Operator = &bufferOp{} // NewBufferOp returns a new bufferOp, initialized to buffer batches from the // supplied input. -func NewBufferOp(input Operator) Operator { +func NewBufferOp(input colexecbase.Operator) colexecbase.Operator { return &bufferOp{ OneInputNode: NewOneInputNode(input), } diff --git a/pkg/sql/colexec/buffer_test.go b/pkg/sql/colexec/buffer_test.go index bdf3ccc50ec1..9c604e298471 100644 --- a/pkg/sql/colexec/buffer_test.go +++ b/pkg/sql/colexec/buffer_test.go @@ -15,7 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/stretchr/testify/require" ) @@ -25,7 +25,7 @@ func TestBufferOp(t *testing.T) { ctx := context.Background() inputTuples := tuples{{int64(1)}, {int64(2)}, {int64(3)}} - input := newOpTestInput(coldata.BatchSize(), inputTuples, []coltypes.T{coltypes.Int64}) + input := newOpTestInput(coldata.BatchSize(), inputTuples, []types.T{*types.Int}) buffer := NewBufferOp(input).(*bufferOp) buffer.Init() diff --git a/pkg/sql/colexec/builtin_funcs.go b/pkg/sql/colexec/builtin_funcs.go index 745e3025040a..e2fbc12ae688 100644 --- a/pkg/sql/colexec/builtin_funcs.go +++ b/pkg/sql/colexec/builtin_funcs.go @@ -15,8 +15,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/typeconv" + "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" @@ -25,7 +27,7 @@ import ( type defaultBuiltinFuncOperator struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator evalCtx *tree.EvalContext funcExpr *tree.FuncExpr columnTypes []types.T @@ -39,7 +41,7 @@ type defaultBuiltinFuncOperator struct { da sqlbase.DatumAlloc } -var _ Operator = &defaultBuiltinFuncOperator{} +var _ colexecbase.Operator = &defaultBuiltinFuncOperator{} func (b *defaultBuiltinFuncOperator) Init() { b.input.Init() @@ -81,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 { - execerror.NonVectorizedPanic(err) + colexecerror.ExpectedError(err) } } @@ -91,7 +93,7 @@ func (b *defaultBuiltinFuncOperator) Next(ctx context.Context) coldata.Batch { } else { converted, err := b.converter(res) if err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } coldata.SetValueAt(output, converted, rowIdx, b.outputPhysType) } @@ -106,17 +108,17 @@ func (b *defaultBuiltinFuncOperator) Next(ctx context.Context) coldata.Batch { // NewBuiltinFunctionOperator returns an operator that applies builtin functions. func NewBuiltinFunctionOperator( - allocator *Allocator, + allocator *colmem.Allocator, evalCtx *tree.EvalContext, funcExpr *tree.FuncExpr, columnTypes []types.T, argumentCols []int, outputIdx int, - input Operator, -) (Operator, error) { + input colexecbase.Operator, +) (colexecbase.Operator, error) { switch funcExpr.ResolvedOverload().SpecializedVecBuiltin { case tree.SubstringStringIntInt: - input = newVectorTypeEnforcer(allocator, input, coltypes.Bytes, outputIdx) + input = newVectorTypeEnforcer(allocator, input, types.Bytes, outputIdx) return newSubstringOperator( allocator, columnTypes, argumentCols, outputIdx, input, ), nil @@ -129,7 +131,7 @@ func NewBuiltinFunctionOperator( outputType.String(), funcExpr.String(), ) } - input = newVectorTypeEnforcer(allocator, input, outputPhysType, outputIdx) + input = newVectorTypeEnforcer(allocator, input, outputType, outputIdx) return &defaultBuiltinFuncOperator{ OneInputNode: NewOneInputNode(input), allocator: allocator, @@ -139,7 +141,7 @@ func NewBuiltinFunctionOperator( columnTypes: columnTypes, outputType: outputType, outputPhysType: outputPhysType, - converter: typeconv.GetDatumToPhysicalFn(outputType), + converter: getDatumToPhysicalFn(outputType), row: make(tree.Datums, len(argumentCols)), argumentCols: argumentCols, }, nil diff --git a/pkg/sql/colexec/builtin_funcs_test.go b/pkg/sql/colexec/builtin_funcs_test.go index 4ffc2363095b..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/colexec/typeconv" + "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 []Operator) (Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return createTestProjectingOperator( ctx, flowCtx, input[0], tc.inputTypes, tc.expr, false, /* canFallbackToRowexec */ @@ -95,7 +95,7 @@ func benchmarkBuiltinFunctions(b *testing.B, useSelectionVector bool, hasNulls b }, } - batch := testAllocator.NewMemBatch([]coltypes.T{coltypes.Int64}) + batch := testAllocator.NewMemBatch([]types.T{*types.Int}) col := batch.ColVec(0).Int64() for i := 0; i < coldata.BatchSize(); i++ { @@ -124,9 +124,10 @@ func benchmarkBuiltinFunctions(b *testing.B, useSelectionVector bool, hasNulls b } } - source := NewRepeatableBatchSource(testAllocator, batch) + typs := []types.T{*types.Int} + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) op, err := createTestProjectingOperator( - ctx, flowCtx, source, []types.T{*types.Int}, + ctx, flowCtx, source, typs, "abs(@1)" /* projectingExpr */, false, /* canFallbackToRowexec */ ) require.NoError(b, err) @@ -156,7 +157,8 @@ func BenchmarkCompareSpecializedOperators(b *testing.B) { ctx := context.Background() tctx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) - batch := testAllocator.NewMemBatch([]coltypes.T{coltypes.Bytes, coltypes.Int64, coltypes.Int64}) + typs := []types.T{*types.String, *types.Int, *types.Int} + batch := testAllocator.NewMemBatch(typs) outputIdx := 3 bCol := batch.ColVec(0).Bytes() sCol := batch.ColVec(1).Int64() @@ -167,16 +169,15 @@ func BenchmarkCompareSpecializedOperators(b *testing.B) { eCol[i] = 4 } batch.SetLength(coldata.BatchSize()) - var source Operator - source = NewRepeatableBatchSource(testAllocator, batch) - source = newVectorTypeEnforcer(testAllocator, source, coltypes.Bytes, outputIdx) + var source colexecbase.Operator + source = colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + source = newVectorTypeEnforcer(testAllocator, source, types.Bytes, outputIdx) // Set up the default operator. expr, err := parser.ParseExpr("substring(@1, @2, @3)") if err != nil { b.Fatal(err) } - typs := []types.T{*types.String, *types.Int, *types.Int} inputCols := []int{0, 1, 2} p := &mockTypeContext{typs: typs} typedExpr, err := tree.TypeCheck(expr, &tree.SemaContext{IVarContainer: p}, types.Any) @@ -192,7 +193,7 @@ func BenchmarkCompareSpecializedOperators(b *testing.B) { columnTypes: typs, outputType: types.String, outputPhysType: coltypes.Bytes, - converter: typeconv.GetDatumToPhysicalFn(types.String), + converter: getDatumToPhysicalFn(types.String), row: make(tree.Datums, outputIdx), argumentCols: inputCols, } diff --git a/pkg/sql/colexec/cancel_checker.go b/pkg/sql/colexec/cancel_checker.go index c7eab848b8b1..420f807ed5d7 100644 --- a/pkg/sql/colexec/cancel_checker.go +++ b/pkg/sql/colexec/cancel_checker.go @@ -14,7 +14,8 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" ) @@ -34,10 +35,10 @@ func (c *CancelChecker) Init() { c.input.Init() } -var _ Operator = &CancelChecker{} +var _ colexecbase.Operator = &CancelChecker{} // NewCancelChecker creates a new CancelChecker. -func NewCancelChecker(op Operator) *CancelChecker { +func NewCancelChecker(op colexecbase.Operator) *CancelChecker { return &CancelChecker{OneInputNode: NewOneInputNode(op)} } @@ -71,7 +72,7 @@ func (c *CancelChecker) check(ctx context.Context) { func (c *CancelChecker) checkEveryCall(ctx context.Context) { select { case <-ctx.Done(): - execerror.NonVectorizedPanic(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 b1c795f91ed0..60b9a6dfa8b8 100644 --- a/pkg/sql/colexec/cancel_checker_test.go +++ b/pkg/sql/colexec/cancel_checker_test.go @@ -14,9 +14,10 @@ import ( "context" "testing" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" @@ -27,10 +28,11 @@ import ( func TestCancelChecker(t *testing.T) { defer leaktest.AfterTest(t)() ctx, cancel := context.WithCancel(context.Background()) - batch := testAllocator.NewMemBatch([]coltypes.T{coltypes.Int64}) - op := NewCancelChecker(NewNoop(NewRepeatableBatchSource(testAllocator, batch))) + typs := []types.T{*types.Int} + batch := testAllocator.NewMemBatch(typs) + op := NewCancelChecker(NewNoop(colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs))) cancel() - err := execerror.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 5b8cc8dcd078..0970b8b21c36 100644 --- a/pkg/sql/colexec/case.go +++ b/pkg/sql/colexec/case.go @@ -15,21 +15,24 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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 *Allocator + allocator *colmem.Allocator buffer *bufferOp - caseOps []Operator - elseOp Operator + caseOps []colexecbase.Operator + elseOp colexecbase.Operator thenIdxs []int outputIdx int - typ coltypes.T + typ *types.T // origSel is a buffer used to keep track of the original selection vector of // the input batch. We need to do this because we're going to destructively @@ -58,14 +61,14 @@ func (c *caseOp) Child(nth int, verbose bool) execinfra.OpNode { } else if nth == 1+len(c.caseOps) { return c.elseOp } - execerror.VectorizedInternalPanic(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 * sizeOfBatchSizeSelVector + return 2 * colmem.SizeOfBatchSizeSelVector } // NewCaseOp returns an operator that runs a case statement. @@ -79,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 *Allocator, - buffer Operator, - caseOps []Operator, - elseOp Operator, + allocator *colmem.Allocator, + buffer colexecbase.Operator, + caseOps []colexecbase.Operator, + elseOp colexecbase.Operator, thenIdxs []int, outputIdx int, - typ coltypes.T, -) Operator { + typ *types.T, +) colexecbase.Operator { return &caseOp{ allocator: allocator, buffer: buffer.(*bufferOp), @@ -165,7 +168,7 @@ func (c *caseOp) Next(ctx context.Context) coldata.Batch { outputCol.Copy( coldata.CopySliceArgs{ SliceArgs: coldata.SliceArgs{ - ColType: c.typ, + ColType: typeconv.FromColumnType(c.typ), Src: inputCol, Sel: toSubtract, SrcStartIdx: 0, @@ -232,7 +235,7 @@ func (c *caseOp) Next(ctx context.Context) coldata.Batch { outputCol.Copy( coldata.CopySliceArgs{ SliceArgs: coldata.SliceArgs{ - ColType: c.typ, + ColType: typeconv.FromColumnType(c.typ), Src: inputCol, Sel: batch.Selection(), SrcStartIdx: 0, diff --git a/pkg/sql/colexec/case_test.go b/pkg/sql/colexec/case_test.go index 46aff8385cd6..dd75897b60b9 100644 --- a/pkg/sql/colexec/case_test.go +++ b/pkg/sql/colexec/case_test.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "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" @@ -79,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 []Operator) (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 ab032dd0e7f1..3532bee96408 100644 --- a/pkg/sql/colexec/cast_test.go +++ b/pkg/sql/colexec/cast_test.go @@ -16,9 +16,9 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/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" @@ -113,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 []Operator) (Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return createTestCastOperator(ctx, flowCtx, input[0], c.fromTyp, c.toTyp) }) }) @@ -143,7 +143,6 @@ func BenchmarkCastOp(b *testing.B) { fmt.Sprintf("useSel=%t/hasNulls=%t/%s_to_%s", useSel, hasNulls, typePair[0].Name(), typePair[1].Name(), ), func(b *testing.B) { - fromType := typeconv.FromColumnType(&typePair[0]) nullProbability := nullProbability if !hasNulls { nullProbability = 0 @@ -152,11 +151,12 @@ func BenchmarkCastOp(b *testing.B) { if !useSel { selectivity = 1.0 } - batch := randomBatchWithSel( - testAllocator, rng, []coltypes.T{fromType}, + typs := []types.T{typePair[0]} + batch := coldatatestutils.RandomBatchWithSel( + testAllocator, rng, typs, coldata.BatchSize(), nullProbability, selectivity, ) - source := NewRepeatableBatchSource(testAllocator, batch) + 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())) @@ -172,8 +172,12 @@ func BenchmarkCastOp(b *testing.B) { } func createTestCastOperator( - ctx context.Context, flowCtx *execinfra.FlowCtx, input Operator, fromTyp *types.T, toTyp *types.T, -) (Operator, error) { + ctx context.Context, + flowCtx *execinfra.FlowCtx, + input colexecbase.Operator, + fromTyp *types.T, + toTyp *types.T, +) (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 c4349ac84b63..d56b4a151143 100644 --- a/pkg/sql/colexec/cast_tmpl.go +++ b/pkg/sql/colexec/cast_tmpl.go @@ -27,16 +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/colexec/execerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} - "github.com/cockroachdb/cockroach/pkg/sql/colexec/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" - semtypes "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/pkg/errors" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* type _ALLTYPES interface{} @@ -47,33 +50,34 @@ type _GOTYPE interface{} var _ apd.Decimal var _ = math.MaxInt8 var _ tree.Datum +var _ coltypes.T func _ASSIGN_CAST(to, from interface{}) { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // This will be replaced with execgen.UNSAFEGET func _FROM_TYPE_UNSAFEGET(to, from interface{}) interface{} { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // This will be replaced with execgen.SET. func _TO_TYPE_SET(to, from interface{}) { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // This will be replaced with execgen.SLICE. func _FROM_TYPE_SLICE(col, i, j interface{}) interface{} { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} -func cast(fromType, toType coltypes.T, inputVec, outputVec coldata.Vec, n int, sel []int) { - switch fromType { +func cast(fromType, toType *types.T, inputVec, outputVec coldata.Vec, n int, sel []int) { + switch typeconv.FromColumnType(fromType) { // {{ range $typ, $overloads := . }} case coltypes._ALLTYPES: - switch toType { + switch typeconv.FromColumnType(toType) { // {{ range $overloads }} // {{ if isCastFuncSet . }} case coltypes._TOTYPE: @@ -129,25 +133,24 @@ func cast(fromType, toType coltypes.T, inputVec, outputVec coldata.Vec, n int, s // {{end}} // {{end}} default: - execerror.VectorizedInternalPanic(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: - execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled FROM type: %s", fromType)) + colexecerror.InternalError(fmt.Sprintf("unhandled FROM type: %s", fromType)) } } func GetCastOperator( - allocator *Allocator, - input Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, colIdx int, resultIdx int, - fromType *semtypes.T, - toType *semtypes.T, -) (Operator, error) { - to := typeconv.FromColumnType(toType) - input = newVectorTypeEnforcer(allocator, input, to, resultIdx) - if fromType.Family() == semtypes.UnknownFamily { + fromType *types.T, + toType *types.T, +) (colexecbase.Operator, error) { + input = newVectorTypeEnforcer(allocator, input, toType, resultIdx) + if fromType.Family() == types.UnknownFamily { return &castOpNullAny{ OneInputNode: NewOneInputNode(input), allocator: allocator, @@ -155,10 +158,10 @@ func GetCastOperator( outputIdx: resultIdx, }, nil } - switch from := typeconv.FromColumnType(fromType); from { + switch typeconv.FromColumnType(fromType) { // {{ range $typ, $overloads := . }} case coltypes._ALLTYPES: - switch to { + switch typeconv.FromColumnType(toType) { // {{ range $overloads }} // {{ if isCastFuncSet . }} case coltypes._TOTYPE: @@ -167,28 +170,28 @@ func GetCastOperator( allocator: allocator, colIdx: colIdx, outputIdx: resultIdx, - fromType: from, - toType: to, + fromType: fromType, + toType: toType, }, nil // {{end}} // {{end}} default: - return nil, errors.Errorf("unhandled cast FROM -> TO type: %s -> %s", from, to) + return nil, errors.Errorf("unhandled cast FROM -> TO type: %s -> %s", fromType, toType) } // {{end}} default: - return nil, errors.Errorf("unhandled FROM type: %s", from) + return nil, errors.Errorf("unhandled FROM type: %s", fromType) } } type castOpNullAny struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator colIdx int outputIdx int } -var _ Operator = &castOpNullAny{} +var _ colexecbase.Operator = &castOpNullAny{} func (c *castOpNullAny) Init() { c.input.Init() @@ -210,7 +213,7 @@ func (c *castOpNullAny) Next(ctx context.Context) coldata.Batch { if vecNulls.NullAt(i) { projNulls.SetNull(i) } else { - execerror.VectorizedInternalPanic(errors.Errorf("unexpected non-null at index %d", i)) + colexecerror.InternalError(errors.Errorf("unexpected non-null at index %d", i)) } } } else { @@ -218,7 +221,7 @@ func (c *castOpNullAny) Next(ctx context.Context) coldata.Batch { if vecNulls.NullAt(i) { projNulls.SetNull(i) } else { - execerror.VectorizedInternalPanic(fmt.Errorf("unexpected non-null at index %d", i)) + colexecerror.InternalError(fmt.Errorf("unexpected non-null at index %d", i)) } } } @@ -227,14 +230,14 @@ func (c *castOpNullAny) Next(ctx context.Context) coldata.Batch { type castOp struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator colIdx int outputIdx int - fromType coltypes.T - toType coltypes.T + fromType *types.T + toType *types.T } -var _ 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 691a53559eef..16236074d33a 100644 --- a/pkg/sql/colexec/cfetcher.go +++ b/pkg/sql/colexec/cfetcher.go @@ -19,12 +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/colencoding" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/typeconv" + "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" @@ -234,7 +235,7 @@ type cFetcher struct { // adapter is a utility struct that helps with memory accounting. adapter struct { ctx context.Context - allocator *Allocator + allocator *colmem.Allocator batch coldata.Batch err error } @@ -244,7 +245,7 @@ type cFetcher struct { // non-primary index, tables.ValNeededForCol can only refer to columns in the // index. func (rf *cFetcher) Init( - allocator *Allocator, + allocator *colmem.Allocator, reverse bool, lockStr sqlbase.ScanLockingStrength, returnRangeInfo bool, @@ -293,10 +294,11 @@ func (rf *cFetcher) Init( allExtraValColOrdinals: oldTable.allExtraValColOrdinals[:0], } - typs := make([]coltypes.T, len(colDescriptors)) + typs := make([]types.T, len(colDescriptors)) for i := range typs { - typs[i] = typeconv.FromColumnType(&colDescriptors[i].Type) - if typs[i] == coltypes.Unhandled && tableArgs.ValNeededForCol.Contains(i) { + typs[i] = colDescriptors[i].Type + physType := typeconv.FromColumnType(&typs[i]) + if physType == coltypes.Unhandled && tableArgs.ValNeededForCol.Contains(i) { // Only return an error if the type is unhandled and needed. If not needed, // a placeholder Vec will be created. return errors.Errorf("unhandled type %+v", &colDescriptors[i].Type) @@ -599,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, execerror.NewStorageError(err) + return nil, colexecerror.NewStorageError(err) } if !moreKeys { rf.machine.state[0] = stateEmitLastBatch @@ -748,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, execerror.NewStorageError(err) + return nil, colexecerror.NewStorageError(err) } if debugState { log.Infof(ctx, "found kv %s, seeking to prefix %s", kv.Key, rf.machine.seekPrefix) @@ -780,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, execerror.NewStorageError(err) + return nil, colexecerror.NewStorageError(err) } if !moreKVs { // No more data. Finalize the row and exit. @@ -880,8 +882,8 @@ func (rf *cFetcher) pushState(state fetcherState) { // getDatumAt returns the converted datum object at the given (colIdx, rowIdx). // This function is meant for tracing and should not be used in hot paths. -func (rf *cFetcher) getDatumAt(colIdx int, rowIdx int, typ types.T) tree.Datum { - return PhysicalTypeColElemToDatum(rf.machine.colvecs[colIdx], rowIdx, rf.table.da, &typ) +func (rf *cFetcher) getDatumAt(colIdx int, rowIdx int, typ *types.T) tree.Datum { + return PhysicalTypeColElemToDatum(rf.machine.colvecs[colIdx], rowIdx, rf.table.da, typ) } // processValue processes the state machine's current value component, setting @@ -903,7 +905,7 @@ func (rf *cFetcher) processValue( for _, idx := range rf.table.allIndexColOrdinals { buf.WriteByte('/') if idx != -1 { - buf.WriteString(rf.getDatumAt(idx, rf.machine.rowIdx, rf.table.cols[idx].Type).String()) + buf.WriteString(rf.getDatumAt(idx, rf.machine.rowIdx, &rf.table.cols[idx].Type).String()) } else { buf.WriteByte('?') } @@ -997,7 +999,7 @@ func (rf *cFetcher) processValue( for j := range table.extraTypes { idx := table.allExtraValColOrdinals[j] buf.WriteByte('/') - buf.WriteString(rf.getDatumAt(idx, rf.machine.rowIdx, rf.table.cols[idx].Type).String()) + buf.WriteString(rf.getDatumAt(idx, rf.machine.rowIdx, &rf.table.cols[idx].Type).String()) } prettyValue = buf.String() } @@ -1076,7 +1078,7 @@ func (rf *cFetcher) processValueSingle( rf.machine.remainingValueColsByIdx.Remove(idx) if rf.traceKV { - prettyValue = rf.getDatumAt(idx, rf.machine.rowIdx, *typ).String() + prettyValue = rf.getDatumAt(idx, rf.machine.rowIdx, typ).String() } if row.DebugRowFetch { log.Infof(ctx, "Scan %s -> %v", rf.machine.nextKV.Key, "?") @@ -1166,7 +1168,7 @@ func (rf *cFetcher) processValueBytes( } rf.machine.remainingValueColsByIdx.Remove(idx) if rf.traceKV { - dVal := rf.getDatumAt(idx, rf.machine.rowIdx, *valTyp) + dVal := rf.getDatumAt(idx, rf.machine.rowIdx, valTyp) if _, err := fmt.Fprintf(rf.machine.prettyValueBuf, "/%v", dVal.String()); err != nil { return "", "", err } @@ -1201,7 +1203,7 @@ func (rf *cFetcher) fillNulls() error { var indexColValues []string for _, idx := range table.indexColOrdinals { if idx != -1 { - indexColValues = append(indexColValues, rf.getDatumAt(idx, rf.machine.rowIdx, rf.table.cols[idx].Type).String()) + indexColValues = append(indexColValues, rf.getDatumAt(idx, rf.machine.rowIdx, &rf.table.cols[idx].Type).String()) } else { indexColValues = append(indexColValues, "?") } diff --git a/pkg/sql/colexec/colbatch_scan.go b/pkg/sql/colexec/colbatch_scan.go index ace9fba46ace..c1accb8cb359 100644 --- a/pkg/sql/colexec/colbatch_scan.go +++ b/pkg/sql/colexec/colbatch_scan.go @@ -15,7 +15,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" @@ -35,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 { - ZeroInputNode + colexecbase.ZeroInputNode spans roachpb.Spans flowCtx *execinfra.FlowCtx rf *cFetcher @@ -48,7 +50,7 @@ type colBatchScan struct { init bool } -var _ Operator = &colBatchScan{} +var _ colexecbase.Operator = &colBatchScan{} func (s *colBatchScan) Init() { s.ctx = context.Background() @@ -60,17 +62,17 @@ func (s *colBatchScan) Init() { s.ctx, s.flowCtx.Txn, s.spans, limitBatches, s.limitHint, s.flowCtx.TraceKV, ); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } } func (s *colBatchScan) Next(ctx context.Context) coldata.Batch { bat, err := s.rf.NextBatch(ctx) if err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } if bat.Selection() != nil { - execerror.VectorizedInternalPanic("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 } @@ -98,7 +100,7 @@ func (s *colBatchScan) DrainMeta(ctx context.Context) []execinfrapb.ProducerMeta // newColBatchScan creates a new colBatchScan operator. func newColBatchScan( - allocator *Allocator, + allocator *colmem.Allocator, flowCtx *execinfra.FlowCtx, spec *execinfrapb.TableReaderSpec, post *execinfrapb.PostProcessSpec, @@ -148,7 +150,7 @@ func newColBatchScan( // initCRowFetcher initializes a row.cFetcher. See initRowFetcher. func initCRowFetcher( - allocator *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 29c3e19b1938..0d5863bb51a7 100644 --- a/pkg/sql/colexec/columnarizer.go +++ b/pkg/sql/colexec/columnarizer.go @@ -15,12 +15,14 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/typeconv" + "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" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) @@ -37,7 +39,7 @@ type Columnarizer struct { // which will simplify this model. mu syncutil.Mutex - allocator *Allocator + allocator *colmem.Allocator input execinfra.RowSource da sqlbase.DatumAlloc initStatus OperatorInitStatus @@ -46,15 +48,15 @@ type Columnarizer struct { batch coldata.Batch accumulatedMeta []execinfrapb.ProducerMetadata ctx context.Context - typs []coltypes.T + typs []types.T } -var _ Operator = &Columnarizer{} +var _ colexecbase.Operator = &Columnarizer{} // NewColumnarizer returns a new Columnarizer. func NewColumnarizer( ctx context.Context, - allocator *Allocator, + allocator *colmem.Allocator, flowCtx *execinfra.FlowCtx, processorID int32, input execinfra.RowSource, @@ -77,8 +79,8 @@ func NewColumnarizer( ); err != nil { return nil, err } - c.typs, err = typeconv.FromColumnTypes(c.OutputTypes()) - + c.typs = c.OutputTypes() + _, err = typeconv.FromColumnTypes(c.typs) return c, err } @@ -126,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } } c.batch.SetLength(nRows) @@ -138,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) { - execerror.VectorizedInternalPanic("Columnarizer should not be Run") + colexecerror.InternalError("Columnarizer should not be Run") } -var _ Operator = &Columnarizer{} +var _ colexecbase.Operator = &Columnarizer{} var _ execinfrapb.MetadataSource = &Columnarizer{} // DrainMeta is part of the MetadataSource interface. @@ -173,9 +175,9 @@ func (c *Columnarizer) Child(nth int, verbose bool) execinfra.OpNode { if n, ok := c.input.(execinfra.OpNode); ok { return n } - execerror.VectorizedInternalPanic("input to Columnarizer is not an execinfra.OpNode") + colexecerror.InternalError("input to Columnarizer is not an execinfra.OpNode") } - execerror.VectorizedInternalPanic(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 ee0ac8a3de91..aac5d231c223 100644 --- a/pkg/sql/colexec/const_test.go +++ b/pkg/sql/colexec/const_test.go @@ -14,8 +14,8 @@ import ( "context" "testing" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "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" @@ -48,8 +48,8 @@ func TestConst(t *testing.T) { }, } for _, tc := range tcs { - runTestsWithTyps(t, []tuples{tc.tuples}, [][]coltypes.T{{coltypes.Int64}}, tc.expected, orderedVerifier, - func(input []Operator) (Operator, error) { + runTestsWithTyps(t, []tuples{tc.tuples}, [][]types.T{{*types.Int}}, tc.expected, orderedVerifier, + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return createTestProjectingOperator( ctx, flowCtx, input[0], []types.T{*types.Int}, "9" /* projectingExpr */, false, /* canFallbackToRowexec */ @@ -84,8 +84,8 @@ func TestConstNull(t *testing.T) { }, } for _, tc := range tcs { - runTestsWithTyps(t, []tuples{tc.tuples}, [][]coltypes.T{{coltypes.Int64}}, tc.expected, orderedVerifier, - func(input []Operator) (Operator, error) { + runTestsWithTyps(t, []tuples{tc.tuples}, [][]types.T{{*types.Int}}, tc.expected, orderedVerifier, + 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 850f80df3f82..fe688f7da376 100644 --- a/pkg/sql/colexec/const_tmpl.go +++ b/pkg/sql/colexec/const_tmpl.go @@ -26,13 +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/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. @@ -60,10 +65,14 @@ type _GOTYPE interface{} // NewConstOp creates a new operator that produces a constant value constVal of // type t at index outputIdx. func NewConstOp( - allocator *Allocator, input Operator, t coltypes.T, constVal interface{}, outputIdx int, -) (Operator, error) { + allocator *colmem.Allocator, + input colexecbase.Operator, + t *types.T, + constVal interface{}, + outputIdx int, +) (colexecbase.Operator, error) { input = newVectorTypeEnforcer(allocator, input, t, outputIdx) - switch t { + switch typeconv.FromColumnType(t) { // {{range .}} case _TYPES_T: return &const_TYPEOp{ @@ -83,7 +92,7 @@ func NewConstOp( type const_TYPEOp struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator outputIdx int constVal _GOTYPE } @@ -122,7 +131,9 @@ 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 *Allocator, input Operator, outputIdx int, typ coltypes.T) Operator { +func NewConstNullOp( + allocator *colmem.Allocator, input colexecbase.Operator, outputIdx int, typ *types.T, +) colexecbase.Operator { input = newVectorTypeEnforcer(allocator, input, typ, outputIdx) return &constNullOp{ OneInputNode: NewOneInputNode(input), @@ -133,11 +144,11 @@ func NewConstNullOp(allocator *Allocator, input Operator, outputIdx int, typ col type constNullOp struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator outputIdx int } -var _ 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 081c135bc0ed..d6ecdae1b7cf 100644 --- a/pkg/sql/colexec/count.go +++ b/pkg/sql/colexec/count.go @@ -14,7 +14,9 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/types" ) // countOp is an operator that counts the number of input rows it receives, @@ -29,14 +31,14 @@ type countOp struct { count int64 } -var _ Operator = &countOp{} +var _ colexecbase.Operator = &countOp{} // NewCountOp returns a new count operator that counts the rows in its input. -func NewCountOp(allocator *Allocator, input Operator) Operator { +func NewCountOp(allocator *colmem.Allocator, input colexecbase.Operator) colexecbase.Operator { c := &countOp{ OneInputNode: NewOneInputNode(input), } - c.internalBatch = allocator.NewMemBatchWithSize([]coltypes.T{coltypes.Int64}, 1) + c.internalBatch = allocator.NewMemBatchWithSize([]types.T{*types.Int}, 1) return c } diff --git a/pkg/sql/colexec/count_test.go b/pkg/sql/colexec/count_test.go index a228de48750b..fd242ecd6b91 100644 --- a/pkg/sql/colexec/count_test.go +++ b/pkg/sql/colexec/count_test.go @@ -13,6 +13,7 @@ package colexec import ( "testing" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) @@ -34,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 []Operator) (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 37e004bf3776..100fd65f6bca 100644 --- a/pkg/sql/colexec/deselector.go +++ b/pkg/sql/colexec/deselector.go @@ -14,7 +14,10 @@ import ( "context" "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/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/types" ) // deselectorOp consumes the input operator, and if resulting batches have a @@ -24,21 +27,23 @@ import ( type deselectorOp struct { OneInputNode NonExplainable - allocator *Allocator - inputTypes []coltypes.T + allocator *colmem.Allocator + inputTypes []types.T output coldata.Batch } -var _ 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 *Allocator, input Operator, colTypes []coltypes.T) Operator { +func NewDeselectorOp( + allocator *colmem.Allocator, input colexecbase.Operator, typs []types.T, +) colexecbase.Operator { return &deselectorOp{ OneInputNode: NewOneInputNode(input), allocator: allocator, - inputTypes: colTypes, + inputTypes: typs, } } @@ -61,7 +66,7 @@ func (p *deselectorOp) Next(ctx context.Context) coldata.Batch { toCol.Copy( coldata.CopySliceArgs{ SliceArgs: coldata.SliceArgs{ - ColType: t, + ColType: typeconv.FromColumnType(&t), Src: fromCol, Sel: sel, SrcEndIdx: batch.Length(), diff --git a/pkg/sql/colexec/deselector_test.go b/pkg/sql/colexec/deselector_test.go index 85ece3576503..b2d481e7dccd 100644 --- a/pkg/sql/colexec/deselector_test.go +++ b/pkg/sql/colexec/deselector_test.go @@ -16,7 +16,9 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "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" ) @@ -24,37 +26,37 @@ import ( func TestDeselector(t *testing.T) { defer leaktest.AfterTest(t)() tcs := []struct { - colTypes []coltypes.T + typs []types.T tuples []tuple sel []int expected []tuple }{ { - colTypes: []coltypes.T{coltypes.Int64}, + typs: []types.T{*types.Int}, tuples: tuples{{0}, {1}, {2}}, sel: nil, expected: tuples{{0}, {1}, {2}}, }, { - colTypes: []coltypes.T{coltypes.Int64}, + typs: []types.T{*types.Int}, tuples: tuples{{0}, {1}, {2}}, sel: []int{}, expected: tuples{}, }, { - colTypes: []coltypes.T{coltypes.Int64}, + typs: []types.T{*types.Int}, tuples: tuples{{0}, {1}, {2}}, sel: []int{1}, expected: tuples{{1}}, }, { - colTypes: []coltypes.T{coltypes.Int64}, + typs: []types.T{*types.Int}, tuples: tuples{{0}, {1}, {2}}, sel: []int{0, 2}, expected: tuples{{0}, {2}}, }, { - colTypes: []coltypes.T{coltypes.Int64}, + typs: []types.T{*types.Int}, tuples: tuples{{0}, {1}, {2}}, sel: []int{0, 1, 2}, expected: tuples{{0}, {1}, {2}}, @@ -62,8 +64,8 @@ func TestDeselector(t *testing.T) { } for _, tc := range tcs { - runTestsWithFixedSel(t, []tuples{tc.tuples}, tc.sel, func(t *testing.T, input []Operator) { - op := NewDeselectorOp(testAllocator, input[0], tc.colTypes) + 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) if err := out.Verify(); err != nil { @@ -78,10 +80,10 @@ func BenchmarkDeselector(b *testing.B) { ctx := context.Background() nCols := 1 - inputTypes := make([]coltypes.T, nCols) + inputTypes := make([]types.T, nCols) for colIdx := 0; colIdx < nCols; colIdx++ { - inputTypes[colIdx] = coltypes.Int64 + inputTypes[colIdx] = *types.Int } batch := testAllocator.NewMemBatch(inputTypes) @@ -93,7 +95,7 @@ func BenchmarkDeselector(b *testing.B) { } } for _, probOfOmitting := range []float64{0.1, 0.9} { - sel := 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} { @@ -103,7 +105,7 @@ func BenchmarkDeselector(b *testing.B) { batch.SetSelection(true) copy(batch.Selection(), sel) batch.SetLength(batchLen) - input := NewRepeatableBatchSource(testAllocator, batch) + 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 1b5f2e9de324..e0751cc94b49 100644 --- a/pkg/sql/colexec/disk_spiller.go +++ b/pkg/sql/colexec/disk_spiller.go @@ -16,7 +16,8 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" ) @@ -25,7 +26,7 @@ import ( // in memory and knows how to export them once the memory limit has been // reached. type bufferingInMemoryOperator interface { - 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 @@ -35,7 +36,7 @@ type bufferingInMemoryOperator interface { // // Calling ExportBuffered may invalidate the contents of the last batch // returned by ExportBuffered. - ExportBuffered(input Operator) coldata.Batch + ExportBuffered(input colexecbase.Operator) coldata.Batch } // oneInputDiskSpiller is an Operator that manages the fallback from a one @@ -88,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 Operator, + input colexecbase.Operator, inMemoryOp bufferingInMemoryOperator, inMemoryMemMonitorName string, - diskBackedOpConstructor func(input Operator) Operator, + diskBackedOpConstructor func(input colexecbase.Operator) colexecbase.Operator, spillingCallbackFn func(), -) Operator { +) colexecbase.Operator { diskBackedOpInput := newBufferExportingOperator(inMemoryOp, input) return &diskSpillerBase{ - inputs: []Operator{input}, + inputs: []colexecbase.Operator{input}, inMemoryOp: inMemoryOp, inMemoryMemMonitorName: inMemoryMemMonitorName, diskBackedOp: diskBackedOpConstructor(diskBackedOpInput), @@ -155,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 Operator, + inputOne, inputTwo colexecbase.Operator, inMemoryOp bufferingInMemoryOperator, inMemoryMemMonitorName string, - diskBackedOpConstructor func(inputOne, inputTwo Operator) Operator, + diskBackedOpConstructor func(inputOne, inputTwo colexecbase.Operator) colexecbase.Operator, spillingCallbackFn func(), -) Operator { +) colexecbase.Operator { diskBackedOpInputOne := newBufferExportingOperator(inMemoryOp, inputOne) diskBackedOpInputTwo := newBufferExportingOperator(inMemoryOp, inputTwo) return &diskSpillerBase{ - inputs: []Operator{inputOne, inputTwo}, + inputs: []colexecbase.Operator{inputOne, inputTwo}, inMemoryOp: inMemoryOp, inMemoryOpInitStatus: OperatorNotInitialized, inMemoryMemMonitorName: inMemoryMemMonitorName, @@ -181,13 +182,13 @@ type diskSpillerBase struct { closerHelper - inputs []Operator + inputs []colexecbase.Operator spilled bool inMemoryOp bufferingInMemoryOperator inMemoryOpInitStatus OperatorInitStatus inMemoryMemMonitorName string - diskBackedOp Operator + diskBackedOp colexecbase.Operator distBackedOpInitStatus OperatorInitStatus spillingCallbackFn func() } @@ -212,7 +213,7 @@ func (d *diskSpillerBase) Next(ctx context.Context) coldata.Batch { return d.diskBackedOp.Next(ctx) } var batch coldata.Batch - if err := execerror.CatchVectorizedRuntimeError( + if err := colexecerror.CatchVectorizedRuntimeError( func() { batch = d.inMemoryOp.Next(ctx) }, @@ -229,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. - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return batch } @@ -294,7 +295,7 @@ func (d *diskSpillerBase) Child(nth int, verbose bool) execinfra.OpNode { case 0: return d.inMemoryOp default: - execerror.VectorizedInternalPanic(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 } @@ -308,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 { - ZeroInputNode + colexecbase.ZeroInputNode NonExplainable firstSource bufferingInMemoryOperator - secondSource Operator + secondSource colexecbase.Operator firstSourceDone bool } var _ resettableOperator = &bufferExportingOperator{} func newBufferExportingOperator( - firstSource bufferingInMemoryOperator, secondSource Operator, -) 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 b543f851003f..6f0fc6075e27 100644 --- a/pkg/sql/colexec/distinct_test.go +++ b/pkg/sql/colexec/distinct_test.go @@ -17,7 +17,9 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "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" ) @@ -27,14 +29,14 @@ func TestDistinct(t *testing.T) { rng, _ := randutil.NewPseudoRand() tcs := []struct { distinctCols []uint32 - colTypes []coltypes.T + typs []types.T tuples []tuple expected []tuple isOrderedOnDistinctCols bool }{ { distinctCols: []uint32{0, 1, 2}, - colTypes: []coltypes.T{coltypes.Float64, coltypes.Int64, coltypes.Bytes, coltypes.Int64}, + typs: []types.T{*types.Float, *types.Int, *types.Bytes, *types.Int}, tuples: tuples{ {nil, nil, nil, nil}, {nil, nil, nil, nil}, @@ -58,7 +60,7 @@ func TestDistinct(t *testing.T) { }, { distinctCols: []uint32{1, 0, 2}, - colTypes: []coltypes.T{coltypes.Float64, coltypes.Int64, coltypes.Bytes, coltypes.Int64}, + typs: []types.T{*types.Float, *types.Int, *types.Bytes, *types.Int}, tuples: tuples{ {nil, nil, nil, nil}, {nil, nil, nil, nil}, @@ -82,7 +84,7 @@ func TestDistinct(t *testing.T) { }, { distinctCols: []uint32{0, 1, 2}, - colTypes: []coltypes.T{coltypes.Float64, coltypes.Int64, coltypes.Bytes, coltypes.Int64}, + typs: []types.T{*types.Float, *types.Int, *types.Bytes, *types.Int}, tuples: tuples{ {1.0, 2, "30", 4}, {1.0, 2, "30", 4}, @@ -105,7 +107,7 @@ func TestDistinct(t *testing.T) { }, { distinctCols: []uint32{0}, - colTypes: []coltypes.T{coltypes.Int64, coltypes.Bytes}, + typs: []types.T{*types.Int, *types.Bytes}, tuples: tuples{ {1, "a"}, {2, "b"}, @@ -130,7 +132,7 @@ func TestDistinct(t *testing.T) { // This is to test hashTable deduplication with various batch size // boundaries and ensure it always emits the first tuple it encountered. distinctCols: []uint32{0}, - colTypes: []coltypes.T{coltypes.Int64, coltypes.Bytes}, + typs: []types.T{*types.Int, *types.Bytes}, tuples: tuples{ {1, "1"}, {1, "2"}, @@ -161,9 +163,9 @@ 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 []Operator) (Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return NewUnorderedDistinct( - testAllocator, input[0], tc.distinctCols, tc.colTypes, + testAllocator, input[0], tc.distinctCols, tc.typs, numOfBuckets), nil }) }) @@ -176,18 +178,18 @@ func TestDistinct(t *testing.T) { orderedCols[i] = tc.distinctCols[j] } runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, - func(input []Operator) (Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return newPartiallyOrderedDistinct( testAllocator, input[0], tc.distinctCols, - orderedCols, tc.colTypes, + orderedCols, tc.typs, ) }) }) } t.Run("ordered", func(t *testing.T) { runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, - func(input []Operator) (Operator, error) { - return NewOrderedDistinct(input[0], tc.distinctCols, tc.colTypes) + func(input []colexecbase.Operator) (colexecbase.Operator, error) { + return NewOrderedDistinct(input[0], tc.distinctCols, tc.typs) }) }) } @@ -198,14 +200,14 @@ func BenchmarkDistinct(b *testing.B) { rng, _ := randutil.NewPseudoRand() ctx := context.Background() - distinctConstructors := []func(*Allocator, Operator, []uint32, int, []coltypes.T) (Operator, error){ - func(allocator *Allocator, input Operator, distinctCols []uint32, numOrderedCols int, typs []coltypes.T) (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 *Allocator, input Operator, distinctCols []uint32, numOrderedCols int, typs []coltypes.T) (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 *Allocator, input Operator, distinctCols []uint32, numOrderedCols int, typs []coltypes.T) (Operator, error) { + func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []types.T) (colexecbase.Operator, error) { return NewOrderedDistinct(input, distinctCols, typs) }, } @@ -215,9 +217,9 @@ func BenchmarkDistinct(b *testing.B) { for _, newTupleProbability := range []float64{0.001, 0.01, 0.1} { for _, nBatches := range []int{1 << 2, 1 << 6} { for _, nCols := range []int{2, 4} { - typs := make([]coltypes.T, nCols) + typs := make([]types.T, nCols) for i := range typs { - typs[i] = coltypes.Int64 + typs[i] = *types.Int } batch := testAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) @@ -257,7 +259,7 @@ func BenchmarkDistinct(b *testing.B) { // Note that the source will be ordered on all nCols so that the // number of distinct tuples doesn't vary between different // distinct operator variations. - source := newFiniteChunksSource(batch, nBatches, nCols) + source := newFiniteChunksSource(batch, typs, nBatches, nCols) distinct, err := distinctConstructor(testAllocator, source, distinctCols, numOrderedCols, typs) if err != nil { b.Fatal(err) diff --git a/pkg/sql/colexec/distinct_tmpl.go b/pkg/sql/colexec/distinct_tmpl.go index 99264095bf49..ec296f4fcf56 100644 --- a/pkg/sql/colexec/distinct_tmpl.go +++ b/pkg/sql/colexec/distinct_tmpl.go @@ -28,11 +28,12 @@ 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/colexec/execerror" - // {{/* + "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" ) @@ -41,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 Operator, distinctCols []uint32, typs []coltypes.T, -) (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{ @@ -55,13 +56,13 @@ func OrderedDistinctColsToOperators( ok bool ) for i := range distinctCols { - input, err = newSingleOrderedDistinct(input, int(distinctCols[i]), distinctCol, typs[distinctCols[i]]) + input, err = newSingleOrderedDistinct(input, int(distinctCols[i]), distinctCol, &typs[distinctCols[i]]) if err != nil { return nil, nil, err } } if r, ok = input.(resettableOperator); !ok { - execerror.VectorizedInternalPanic("unexpectedly an ordered distinct is not a resetter") + colexecerror.InternalError("unexpectedly an ordered distinct is not a resetter") } distinctChain := &distinctChainOps{ resettableOperator: r, @@ -78,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 Operator, distinctCols []uint32, typs []coltypes.T, -) (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 @@ -90,6 +91,9 @@ func NewOrderedDistinct( }, nil } +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Declarations to make the template compile properly. @@ -129,15 +133,15 @@ 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 { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} func newSingleOrderedDistinct( - input Operator, distinctColIdx int, outputCol []bool, t coltypes.T, -) (Operator, error) { - switch t { + input colexecbase.Operator, distinctColIdx int, outputCol []bool, t *types.T, +) (colexecbase.Operator, error) { + switch typeconv.FromColumnType(t) { // {{range .}} case _TYPES_T: return &sortedDistinct_TYPEOp{ @@ -169,8 +173,8 @@ type partitioner interface { } // newPartitioner returns a new partitioner on type t. -func newPartitioner(t coltypes.T) (partitioner, error) { - switch t { +func newPartitioner(t *types.T) (partitioner, error) { + switch typeconv.FromColumnType(t) { // {{range .}} case _TYPES_T: return partitioner_TYPE{}, nil 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 93f00679f105..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/colexec/execerror" + "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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) }`, target, r, target, l, target, ) case coltypes.Float64: return fmt.Sprintf("%s = %s / float64(%s)", target, l, r) default: - execerror.VectorizedInternalPanic("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 e85634eb9a2e..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/colexec/execerror" + "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: - execerror.VectorizedInternalPanic("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 404b5698b49f..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/colexec/execerror" + "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 { - execerror.VectorizedInternalPanic(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 79506801a32e..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/colexec/execerror" + "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) { - execerror.NonVectorizedPanic(tree.ErrIntOutOfRange) + colexecerror.ExpectedError(tree.ErrIntOutOfRange) } %[1]s = int%[3]d(%[2]s) ` @@ -233,9 +233,9 @@ func floatToDecimal(to, from string) string { { var tmpDec apd.Decimal _, tmpErr := tmpDec.SetFloat64(float64(%[2]s)) - if tmpErr != nil { - execerror.NonVectorizedPanic(tmpErr) - } + if tmpErr != nil { + colexecerror.ExpectedError(tmpErr) + } %[1]s = tmpDec } ` @@ -322,7 +322,7 @@ func init() { } `)) if err := t.Execute(&buf, args); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return buf.String() } @@ -615,7 +615,7 @@ func (boolCustomizer) getCmpOpCompareFunc() compareFunc { `)) if err := t.Execute(&buf, args); err != nil { - execerror.VectorizedInternalPanic(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() { - execerror.NonVectorizedPanic(tree.ErrDivByZero) + colexecerror.ExpectedError(tree.ErrDivByZero) } if err != nil { - execerror.NonVectorizedPanic(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 { execerror.NonVectorizedPanic(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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return buf.String() } @@ -795,7 +795,7 @@ func (c intCustomizer) getCmpOpCompareFunc() compareFunc { `)) if err := t.Execute(&buf, args); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return buf.String() } @@ -821,7 +821,7 @@ func (c intCustomizer) getBinOpAssignFunc() assignFunc { { result := {{.Left}} + {{.Right}} if (result < {{.Left}}) != ({{.Right}} < 0) { - execerror.NonVectorizedPanic(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) { - execerror.NonVectorizedPanic(tree.ErrIntOutOfRange) + colexecerror.ExpectedError(tree.ErrIntOutOfRange) } {{.Target}} = result } @@ -857,7 +857,7 @@ func (c intCustomizer) getBinOpAssignFunc() assignFunc { upperBound = "math.MaxInt32" lowerBound = "math.MinInt32" default: - execerror.VectorizedInternalPanic(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 { - execerror.NonVectorizedPanic(tree.ErrIntOutOfRange) + colexecerror.ExpectedError(tree.ErrIntOutOfRange) } else if result/{{.Right}} != {{.Left}} { - execerror.NonVectorizedPanic(tree.ErrIntOutOfRange) + colexecerror.ExpectedError(tree.ErrIntOutOfRange) } } } @@ -886,23 +886,23 @@ func (c intCustomizer) getBinOpAssignFunc() assignFunc { t = template.Must(template.New("").Parse(` { if {{.Right}} == 0 { - execerror.NonVectorizedPanic(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 { - execerror.NonVectorizedPanic(err) + colexecerror.ExpectedError(err) } } `)) default: - execerror.VectorizedInternalPanic(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 { - execerror.VectorizedInternalPanic(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 { - execerror.NonVectorizedPanic(err) + colexecerror.ExpectedError(err) } {{.Target}} = tree.CompareDecimals(&{{.Left}}, tmpDec) } `)) if err := t.Execute(&buf, args); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return buf.String() } @@ -940,7 +940,7 @@ func (c decimalIntCustomizer) getCmpOpCompareFunc() compareFunc { } `)) if err := t.Execute(&buf, args); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return buf.String() } @@ -960,18 +960,18 @@ func (c decimalIntCustomizer) getBinOpAssignFunc() assignFunc { { {{ if .IsDivision }} if {{.Right}} == 0 { - execerror.NonVectorizedPanic(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 { - execerror.NonVectorizedPanic(err) + colexecerror.ExpectedError(err) } } `)) if err := t.Execute(&buf, args); err != nil { - execerror.VectorizedInternalPanic(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 { - execerror.NonVectorizedPanic(err) + colexecerror.ExpectedError(err) } {{.Target}} = tree.CompareDecimals(tmpDec, &{{.Right}}) } `)) if err := t.Execute(&buf, args); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return buf.String() } @@ -1010,7 +1010,7 @@ func (c intDecimalCustomizer) getCmpOpCompareFunc() compareFunc { `)) if err := t.Execute(&buf, args); err != nil { - execerror.VectorizedInternalPanic(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() { - execerror.NonVectorizedPanic(tree.ErrDivByZero) + colexecerror.ExpectedError(tree.ErrDivByZero) } {{ else }} _, err := tree.{{.Ctx}}.{{.Op}}(&{{.Target}}, tmpDec, &{{.Right}}) {{ end }} if err != nil { - execerror.NonVectorizedPanic(err) + colexecerror.ExpectedError(err) } } `)) if err := t.Execute(&buf, args); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return buf.String() } @@ -1077,7 +1077,7 @@ func (c timestampCustomizer) getCmpOpCompareFunc() compareFunc { }`)) if err := t.Execute(&buf, args); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return buf.String() } @@ -1102,7 +1102,7 @@ func (c timestampCustomizer) getBinOpAssignFunc() assignFunc { `, target, l, r) default: - execerror.VectorizedInternalPanic(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: - execerror.VectorizedInternalPanic(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: - execerror.VectorizedInternalPanic(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: - execerror.VectorizedInternalPanic(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 { - execerror.NonVectorizedPanic(tree.ErrDivByZero) + colexecerror.ExpectedError(tree.ErrDivByZero) } %[1]s = %[2]s.Div(int64(%[3]s))`, target, l, r) default: - execerror.VectorizedInternalPanic(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: - execerror.VectorizedInternalPanic(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 { - execerror.NonVectorizedPanic(tree.ErrDivByZero) + colexecerror.ExpectedError(tree.ErrDivByZero) } %[1]s = %[2]s.DivFloat(float64(%[3]s))`, target, l, r) default: - execerror.VectorizedInternalPanic(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: - execerror.VectorizedInternalPanic(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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } %[1]s = %[2]s.MulFloat(f)`, target, l, r) default: - execerror.VectorizedInternalPanic(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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } %[1]s = %[3]s.MulFloat(f)`, target, l, r) default: - execerror.VectorizedInternalPanic(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 496b0b4f1c88..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/colexec/execerror" + "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 f30565e1ebd3..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/colexec/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 d035be62f401..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/colexec/execerror" +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{} { - execerror.VectorizedInternalPanic(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{}) { - execerror.VectorizedInternalPanic(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) } // SET is a template function. func SET(target, i, new interface{}) { - execerror.VectorizedInternalPanic(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) } // SLICE is a template function. func SLICE(target, start, end interface{}) interface{} { - execerror.VectorizedInternalPanic(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) return nil } // COPYSLICE is a template function. func COPYSLICE(target, src, destIdx, srcStartIdx, srcEndIdx interface{}) { - execerror.VectorizedInternalPanic(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) } // APPENDSLICE is a template function. func APPENDSLICE(target, src, destIdx, srcStartIdx, srcEndIdx interface{}) { - execerror.VectorizedInternalPanic(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) } // APPENDVAL is a template function. func APPENDVAL(target, v interface{}) { - execerror.VectorizedInternalPanic(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) } // LEN is a template function. func LEN(target interface{}) interface{} { - execerror.VectorizedInternalPanic(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) return nil } // ZERO is a template function. func ZERO(target interface{}) { - execerror.VectorizedInternalPanic(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) } // RANGE is a template function. func RANGE(loopVariableIdent, target, start, end interface{}) bool { - execerror.VectorizedInternalPanic(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) return false } // WINDOW is a template function. func WINDOW(target, start, end interface{}) interface{} { - execerror.VectorizedInternalPanic(nonTemplatePanic) + colexecerror.InternalError(nonTemplatePanic) return nil } diff --git a/pkg/sql/colexec/execplan.go b/pkg/sql/colexec/execplan.go index 1d367240381b..9e11538d987b 100644 --- a/pkg/sql/colexec/execplan.go +++ b/pkg/sql/colexec/execplan.go @@ -17,9 +17,11 @@ import ( "reflect" "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/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/sem/tree" @@ -33,7 +35,7 @@ import ( "github.com/marusama/semaphore" ) -func checkNumIn(inputs []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)) } @@ -45,7 +47,7 @@ func checkNumIn(inputs []Operator, numIn int) error { func wrapRowSources( ctx context.Context, flowCtx *execinfra.FlowCtx, - inputs []Operator, + inputs []colexecbase.Operator, inputTypes [][]types.T, acc *mon.BoundAccount, newToWrap func([]execinfra.RowSource) (execinfra.RowSource, error), @@ -87,14 +89,14 @@ func wrapRowSources( return nil, err } - return NewColumnarizer(ctx, 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 []Operator + Inputs []colexecbase.Operator StreamingMemAccount *mon.BoundAccount ProcessorConstructor execinfra.ProcessorConstructor DiskQueueCfg colcontainer.DiskQueueCfg @@ -135,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 Operator + Op colexecbase.Operator ColumnTypes []types.T InternalMemUsage int MetadataSources []execinfrapb.MetadataSource @@ -322,20 +324,20 @@ func (r *NewColOperatorResult) createDiskBackedSort( ctx context.Context, flowCtx *execinfra.FlowCtx, args NewColOperatorArgs, - input Operator, - inputTypes []coltypes.T, + input colexecbase.Operator, + inputTypes []types.T, ordering execinfrapb.Ordering, matchLen uint32, maxNumberPartitions int, processorID int32, post *execinfrapb.PostProcessSpec, memMonitorNamePrefix string, -) (Operator, error) { +) (colexecbase.Operator, error) { streamingMemAccount := args.StreamingMemAccount useStreamingMemAccountForBuffering := args.TestingKnobs.UseStreamingMemAccountForBuffering var ( sorterMemMonitorName string - inMemorySorter Operator + inMemorySorter colexecbase.Operator err error ) if len(ordering.Columns) == int(matchLen) { @@ -355,7 +357,7 @@ func (r *NewColOperatorResult) createDiskBackedSort( ) } inMemorySorter, err = NewSortChunks( - 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 { @@ -373,7 +375,7 @@ func (r *NewColOperatorResult) createDiskBackedSort( } k := uint16(post.Limit + post.Offset) inMemorySorter = NewTopKSorter( - NewAllocator(ctx, topKSorterMemAccount), input, inputTypes, + colmem.NewAllocator(ctx, topKSorterMemAccount), input, inputTypes, ordering.Columns, k, ) } else { @@ -388,7 +390,7 @@ func (r *NewColOperatorResult) createDiskBackedSort( ) } inMemorySorter, err = NewSorter( - NewAllocator(ctx, sorterMemAccount), input, inputTypes, ordering.Columns, + colmem.NewAllocator(ctx, sorterMemAccount), input, inputTypes, ordering.Columns, ) } if err != nil { @@ -404,12 +406,12 @@ func (r *NewColOperatorResult) createDiskBackedSort( return newOneInputDiskSpiller( input, inMemorySorter.(bufferingInMemoryOperator), sorterMemMonitorName, - func(input Operator) 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 := NewAllocator( + unlimitedAllocator := colmem.NewAllocator( ctx, r.createBufferingUnlimitedMemAccount( ctx, flowCtx, monitorNamePrefix, )) @@ -457,7 +459,7 @@ func (r *NewColOperatorResult) createDiskBackedSort( func (r *NewColOperatorResult) createAndWrapRowSource( ctx context.Context, flowCtx *execinfra.FlowCtx, - inputs []Operator, + inputs []colexecbase.Operator, inputTypes [][]types.T, streamingMemAccount *mon.BoundAccount, spec *execinfrapb.ProcessorSpec, @@ -531,7 +533,7 @@ func NewColOperator( result.OpMonitors = result.OpMonitors[:0] } if panicErr != nil { - execerror.VectorizedInternalPanic(panicErr) + colexecerror.InternalError(panicErr) } }() spec := args.Spec @@ -595,13 +597,13 @@ func NewColOperator( return result, err } result.Op, result.IsStreaming = NewNoop(inputs[0]), true - result.ColumnTypes = spec.Input[0].ColumnTypes + result.ColumnTypes = append([]types.T{}, spec.Input[0].ColumnTypes...) case core.TableReader != nil: if err := checkNumIn(inputs, 0); err != nil { return result, err } var scanOp *colBatchScan - scanOp, err = newColBatchScan(NewAllocator(ctx, streamingMemAccount), flowCtx, core.TableReader, post) + scanOp, err = newColBatchScan(colmem.NewAllocator(ctx, streamingMemAccount), flowCtx, core.TableReader, post) if err != nil { return result, err } @@ -635,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(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(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 } @@ -680,9 +682,8 @@ func NewColOperator( } result.ColumnTypes[i] = *retType } - var typs []coltypes.T - typs, err = typeconv.FromColumnTypes(spec.Input[0].ColumnTypes) - if err != nil { + typs := append([]types.T{}, spec.Input[0].ColumnTypes...) + if _, err = typeconv.FromColumnTypes(typs); err != nil { return result, err } if needHash { @@ -697,12 +698,12 @@ func NewColOperator( hashAggregatorMemAccount = result.createBufferingUnlimitedMemAccount(ctx, flowCtx, "hash-aggregator") } result.Op, err = NewHashAggregator( - NewAllocator(ctx, hashAggregatorMemAccount), inputs[0], typs, aggFns, + colmem.NewAllocator(ctx, hashAggregatorMemAccount), inputs[0], typs, aggFns, aggSpec.GroupCols, aggCols, ) } else { result.Op, err = NewOrderedAggregator( - NewAllocator(ctx, streamingMemAccount), inputs[0], typs, aggFns, + colmem.NewAllocator(ctx, streamingMemAccount), inputs[0], typs, aggFns, aggSpec.GroupCols, aggCols, aggSpec.IsScalar(), ) result.IsStreaming = true @@ -712,14 +713,12 @@ func NewColOperator( if err := checkNumIn(inputs, 1); err != nil { return result, err } - result.ColumnTypes = spec.Input[0].ColumnTypes - var typs []coltypes.T - typs, err = typeconv.FromColumnTypes(result.ColumnTypes) - if err != nil { + result.ColumnTypes = append([]types.T{}, spec.Input[0].ColumnTypes...) + if _, err = typeconv.FromColumnTypes(result.ColumnTypes); err != nil { return result, err } if len(core.Distinct.OrderedColumns) == len(core.Distinct.DistinctColumns) { - result.Op, err = NewOrderedDistinct(inputs[0], core.Distinct.OrderedColumns, typs) + result.Op, err = NewOrderedDistinct(inputs[0], core.Distinct.OrderedColumns, result.ColumnTypes) result.IsStreaming = true } else { distinctMemAccount := streamingMemAccount @@ -738,8 +737,8 @@ func NewColOperator( // columns and we think that the probability of distinct tuples in the // input is about 0.01 or less. result.Op = NewUnorderedDistinct( - NewAllocator(ctx, distinctMemAccount), inputs[0], - core.Distinct.DistinctColumns, typs, hashTableNumBuckets, + colmem.NewAllocator(ctx, distinctMemAccount), inputs[0], + core.Distinct.DistinctColumns, result.ColumnTypes, hashTableNumBuckets, ) } @@ -749,23 +748,23 @@ func NewColOperator( } outputIdx := len(spec.Input[0].ColumnTypes) result.Op = NewOrdinalityOp( - NewAllocator(ctx, streamingMemAccount), inputs[0], outputIdx, + colmem.NewAllocator(ctx, streamingMemAccount), inputs[0], outputIdx, ) result.IsStreaming = true - result.ColumnTypes = append(spec.Input[0].ColumnTypes, *types.Int) + result.ColumnTypes = make([]types.T, outputIdx+1) + copy(result.ColumnTypes, spec.Input[0].ColumnTypes) + result.ColumnTypes[outputIdx] = *types.Int case core.HashJoiner != nil: if err := checkNumIn(inputs, 2); err != nil { return result, err } - leftLogTypes := spec.Input[0].ColumnTypes - leftPhysTypes, err := typeconv.FromColumnTypes(leftLogTypes) - if err != nil { + leftTypes := append([]types.T{}, spec.Input[0].ColumnTypes...) + if _, err := typeconv.FromColumnTypes(leftTypes); err != nil { return result, err } - rightLogTypes := spec.Input[1].ColumnTypes - rightPhysTypes, err := typeconv.FromColumnTypes(rightLogTypes) - if err != nil { + rightTypes := append([]types.T{}, spec.Input[1].ColumnTypes...) + if _, err := typeconv.FromColumnTypes(rightTypes); err != nil { return result, err } @@ -787,15 +786,15 @@ func NewColOperator( core.HashJoiner.Type, core.HashJoiner.LeftEqColumns, core.HashJoiner.RightEqColumns, - leftPhysTypes, - rightPhysTypes, + leftTypes, + rightTypes, rightEqColsAreKey, ) if err != nil { return result, err } inMemoryHashJoiner := newHashJoiner( - 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 @@ -807,9 +806,9 @@ func NewColOperator( result.Op = newTwoInputDiskSpiller( inputs[0], inputs[1], inMemoryHashJoiner.(bufferingInMemoryOperator), hashJoinerMemMonitorName, - func(inputOne, inputTwo Operator) Operator { + func(inputOne, inputTwo colexecbase.Operator) colexecbase.Operator { monitorNamePrefix := "external-hash-joiner" - unlimitedAllocator := NewAllocator( + unlimitedAllocator := colmem.NewAllocator( ctx, result.createBufferingUnlimitedMemAccount( ctx, flowCtx, monitorNamePrefix, )) @@ -825,7 +824,7 @@ func NewColOperator( execinfra.GetWorkMemLimit(flowCtx.Cfg), diskQueueCfg, args.FDSemaphore, - func(input Operator, inputTypes []coltypes.T, orderingCols []execinfrapb.Ordering_Column, maxNumberPartitions int) (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 @@ -849,7 +848,7 @@ func NewColOperator( args.TestingKnobs.SpillingCallbackFn, ) } - result.ColumnTypes = append(leftLogTypes, rightLogTypes...) + result.ColumnTypes = append(leftTypes, rightTypes...) if !core.HashJoiner.OnExpr.Empty() && core.HashJoiner.Type == sqlbase.JoinType_INNER { if err = result.planAndMaybeWrapOnExprAsFilter(ctx, flowCtx, core.HashJoiner.OnExpr, streamingMemAccount, processorConstructor); err != nil { @@ -865,14 +864,12 @@ func NewColOperator( // for both of the inputs. result.IsStreaming = core.MergeJoiner.LeftEqColumnsAreKey && core.MergeJoiner.RightEqColumnsAreKey - leftLogTypes := spec.Input[0].ColumnTypes - leftPhysTypes, err := typeconv.FromColumnTypes(leftLogTypes) - if err != nil { + leftTypes := append([]types.T{}, spec.Input[0].ColumnTypes...) + if _, err := typeconv.FromColumnTypes(leftTypes); err != nil { return result, err } - rightLogTypes := spec.Input[1].ColumnTypes - rightPhysTypes, err := typeconv.FromColumnTypes(rightLogTypes) - if err != nil { + rightTypes := append([]types.T{}, spec.Input[1].ColumnTypes...) + if _, err := typeconv.FromColumnTypes(rightTypes); err != nil { return result, err } @@ -892,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 := NewAllocator( + unlimitedAllocator := colmem.NewAllocator( ctx, result.createBufferingUnlimitedMemAccount( ctx, flowCtx, monitorName, )) @@ -900,7 +897,7 @@ func NewColOperator( mj, err := newMergeJoinOp( unlimitedAllocator, execinfra.GetWorkMemLimit(flowCtx.Cfg), args.DiskQueueCfg, args.FDSemaphore, - joinType, inputs[0], inputs[1], leftPhysTypes, rightPhysTypes, + joinType, inputs[0], inputs[1], leftTypes, rightTypes, core.MergeJoiner.LeftOrdering.Columns, core.MergeJoiner.RightOrdering.Columns, diskAccount, ) @@ -910,7 +907,7 @@ func NewColOperator( result.Op = mj result.ToClose = append(result.ToClose, mj.(IdempotentCloser)) - result.ColumnTypes = append(leftLogTypes, rightLogTypes...) + result.ColumnTypes = append(leftTypes, rightTypes...) if onExpr != nil { if err = result.planAndMaybeWrapOnExprAsFilter(ctx, flowCtx, *onExpr, streamingMemAccount, processorConstructor); err != nil { @@ -923,9 +920,8 @@ func NewColOperator( return result, err } input := inputs[0] - var inputTypes []coltypes.T - inputTypes, err = typeconv.FromColumnTypes(spec.Input[0].ColumnTypes) - if err != nil { + inputTypes := append([]types.T{}, spec.Input[0].ColumnTypes...) + if _, err = typeconv.FromColumnTypes(inputTypes); err != nil { return result, err } ordering := core.Sorter.OutputOrdering @@ -934,7 +930,7 @@ func NewColOperator( ctx, flowCtx, args, input, inputTypes, ordering, matchLen, 0, /* maxNumberPartitions */ spec.ProcessorID, post, "", /* memMonitorNamePrefix */ ) - result.ColumnTypes = spec.Input[0].ColumnTypes + result.ColumnTypes = inputTypes case core.Windower != nil: if err := checkNumIn(inputs, 1); err != nil { @@ -942,11 +938,10 @@ func NewColOperator( } memMonitorsPrefix := "window-" input := inputs[0] - result.ColumnTypes = spec.Input[0].ColumnTypes + result.ColumnTypes = append([]types.T{}, spec.Input[0].ColumnTypes...) for _, wf := range core.Windower.WindowFns { - var typs []coltypes.T - typs, err = typeconv.FromColumnTypes(result.ColumnTypes) - if err != nil { + typs := append([]types.T{}, result.ColumnTypes...) + if _, err = typeconv.FromColumnTypes(typs); err != nil { return result, err } tempColOffset, partitionColIdx := uint32(0), columnOmitted @@ -958,9 +953,9 @@ func NewColOperator( // which kind of partitioner to use should come from the optimizer. partitionColIdx = int(wf.OutputColIdx) input, err = NewWindowSortingPartitioner( - NewAllocator(ctx, streamingMemAccount), input, typs, + colmem.NewAllocator(ctx, streamingMemAccount), input, typs, core.Windower.PartitionBy, wf.Ordering.Columns, int(wf.OutputColIdx), - func(input Operator, inputTypes []coltypes.T, orderingCols []execinfrapb.Ordering_Column) (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 */ @@ -970,7 +965,7 @@ func NewColOperator( ) // Window partitioner will append a boolean column. tempColOffset++ - typs = append(typs, coltypes.Bool) + typs = append(typs, *types.Bool) } else { if len(wf.Ordering.Columns) > 0 { input, err = result.createDiskBackedSort( @@ -986,24 +981,24 @@ func NewColOperator( if windowFnNeedsPeersInfo(*wf.Func.WindowFunc) { peersColIdx = int(wf.OutputColIdx + tempColOffset) input, err = NewWindowPeerGrouper( - NewAllocator(ctx, streamingMemAccount), + colmem.NewAllocator(ctx, streamingMemAccount), input, typs, wf.Ordering.Columns, partitionColIdx, peersColIdx, ) // Window peer grouper will append a boolean column. tempColOffset++ - typs = append(typs, coltypes.Bool) + typs = append(typs, *types.Bool) } outputIdx := int(wf.OutputColIdx + tempColOffset) switch windowFn { case execinfrapb.WindowerSpec_ROW_NUMBER: result.Op = NewRowNumberOperator( - 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( - 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: @@ -1012,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 := NewAllocator( + unlimitedAllocator := colmem.NewAllocator( ctx, result.createBufferingUnlimitedMemAccount(ctx, flowCtx, memAccName), ) diskAcc := result.createDiskAccount(ctx, flowCtx, memAccName) @@ -1165,7 +1160,7 @@ func (r *NewColOperatorResult) wrapPostProcessSpec( Post: *post, } return r.createAndWrapRowSource( - ctx, flowCtx, []Operator{r.Op}, [][]types.T{r.ColumnTypes}, streamingMemAccount, noopSpec, processorConstructor, + ctx, flowCtx, []colexecbase.Operator{r.Op}, [][]types.T{r.ColumnTypes}, streamingMemAccount, noopSpec, processorConstructor, ) } @@ -1312,7 +1307,7 @@ func (r *NewColOperatorResult) createDiskAccount( } type postProcessResult struct { - Op Operator + Op colexecbase.Operator ColumnTypes []types.T InternalMemUsage int } @@ -1380,9 +1375,9 @@ func planSelectionOperators( evalCtx *tree.EvalContext, expr tree.TypedExpr, columnTypes []types.T, - input Operator, + input colexecbase.Operator, acc *mon.BoundAccount, -) (op Operator, resultIdx int, ct []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 @@ -1391,18 +1386,18 @@ 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 Operator + var leftOp, rightOp colexecbase.Operator var internalMemUsedLeft, internalMemUsedRight int - leftOp, _, ct, internalMemUsedLeft, err = planSelectionOperators( + leftOp, _, typs, internalMemUsedLeft, err = planSelectionOperators( ctx, evalCtx, t.TypedLeft(), columnTypes, input, acc, ) if err != nil { - return nil, resultIdx, ct, internalMemUsed, err + return nil, resultIdx, typs, internalMemUsed, err } - rightOp, resultIdx, ct, internalMemUsedRight, err = planSelectionOperators( - ctx, evalCtx, t.TypedRight(), ct, leftOp, acc, + rightOp, resultIdx, typs, internalMemUsedRight, err = planSelectionOperators( + ctx, evalCtx, t.TypedRight(), typs, leftOp, acc, ) - return rightOp, resultIdx, ct, internalMemUsedLeft + internalMemUsedRight, err + return rightOp, resultIdx, typs, internalMemUsedLeft + internalMemUsedRight, err case *tree.OrExpr: // OR expressions are handled by converting them to an equivalent CASE // statement. Since CASE statements don't have a selection form, plan a @@ -1421,19 +1416,19 @@ func planSelectionOperators( tree.DBoolFalse, types.Bool) if err != nil { - return nil, resultIdx, ct, internalMemUsed, err + return nil, resultIdx, typs, internalMemUsed, err } - op, resultIdx, ct, internalMemUsed, err = planProjectionOperators( + op, resultIdx, typs, internalMemUsed, err = planProjectionOperators( ctx, evalCtx, caseExpr, columnTypes, input, acc, ) op = NewBoolVecToSelOp(op, resultIdx) - return op, resultIdx, ct, internalMemUsed, err + return op, resultIdx, typs, internalMemUsed, err case *tree.CaseExpr: - op, resultIdx, ct, internalMemUsed, err = planProjectionOperators( + op, resultIdx, typs, internalMemUsed, err = planProjectionOperators( ctx, evalCtx, expr, columnTypes, input, acc, ) op = NewBoolVecToSelOp(op, resultIdx) - return op, resultIdx, ct, internalMemUsed, err + return op, resultIdx, typs, internalMemUsed, err case *tree.ComparisonExpr: cmpOp := t.Operator leftOp, leftIdx, ct, internalMemUsedLeft, err := planProjectionOperators( @@ -1496,14 +1491,14 @@ func planTypedMaybeNullProjectionOperators( expr tree.TypedExpr, exprTyp *types.T, columnTypes []types.T, - input Operator, + input colexecbase.Operator, acc *mon.BoundAccount, -) (op Operator, resultIdx int, ct []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(NewAllocator(ctx, acc), input, resultIdx, typeconv.FromColumnType(exprTyp)) - ct = append(columnTypes, *exprTyp) - return op, resultIdx, ct, internalMemUsed, nil + op = NewConstNullOp(colmem.NewAllocator(ctx, acc), input, resultIdx, exprTyp) + typs = append(columnTypes, *exprTyp) + return op, resultIdx, typs, internalMemUsed, nil } return planProjectionOperators(ctx, evalCtx, expr, columnTypes, input, acc) } @@ -1515,7 +1510,7 @@ func checkCastSupported(fromType, toType *types.T) error { // decimal of the same precision due to the fact that we're losing // precision information once we start operating on coltypes.T. For // such casts we will fallback to row-by-row engine. - if !fromType.Equal(*toType) { + if !fromType.Identical(toType) { return errors.New("decimal casts with rounding unsupported") } } @@ -1529,42 +1524,18 @@ func planCastOperator( ctx context.Context, acc *mon.BoundAccount, columnTypes []types.T, - input Operator, + input colexecbase.Operator, inputIdx int, fromType *types.T, toType *types.T, -) (op Operator, resultIdx int, ct []types.T, err error) { +) (op colexecbase.Operator, resultIdx int, typs []types.T, err error) { if err := checkCastSupported(fromType, toType); err != nil { - return op, resultIdx, ct, err + return op, resultIdx, typs, err } outputIdx := len(columnTypes) - op, err = GetCastOperator(NewAllocator(ctx, acc), input, inputIdx, outputIdx, fromType, toType) - ct = append(columnTypes, *toType) - return op, outputIdx, ct, err -} - -// toPhysTypesMaybeUnhandled converts logical types to their physical -// equivalents. If a logical type is not supported by the vectorized engine, it -// is converted into coltypes.Unhandled and no error occurs. -// -// It is the same as typeconv.FromColumnTypes but without an error, and this -// method was created for the sole purpose of supporting render expressions -// with projection operators that are planned after colBatchScan which can -// output batches with coltypes.Unhandled when those columns are not needed. -// For example, if we have a table with type schema (a INT, j JSON), but the -// query uses only column 'a', then colBatchScan will output batch with schema -// (coltypes.Int64, coltypes.Unhandled), and we want to be ok with that when -// enforcing the prefix of type schema of the batch. -// -// WARNING: use this with caution - only when the caller of this method is -// *not* responsible for making sure that we support all types in ct, when it -// should have been checked before. -func toPhysTypesMaybeUnhandled(ct []types.T) []coltypes.T { - physTypesMaybeUnhandled := make([]coltypes.T, len(ct)) - for i, logType := range ct { - physTypesMaybeUnhandled[i] = typeconv.FromColumnType(&logType) - } - return physTypesMaybeUnhandled + op, err = GetCastOperator(colmem.NewAllocator(ctx, acc), input, inputIdx, outputIdx, fromType, toType) + typs = append(columnTypes, *toType) + return op, outputIdx, typs, err } // planProjectionOperators plans a chain of operators to execute the provided @@ -1576,9 +1547,9 @@ func planProjectionOperators( evalCtx *tree.EvalContext, expr tree.TypedExpr, columnTypes []types.T, - input Operator, + input colexecbase.Operator, acc *mon.BoundAccount, -) (op Operator, resultIdx int, ct []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: @@ -1594,29 +1565,29 @@ func planProjectionOperators( // We can't use planProjectionOperators because it will reject planning a constNullOp without knowing // the post typechecking "type" of the NULL. if expr.ResolvedType() == types.Unknown { - op, resultIdx, ct, internalMemUsed, err = planTypedMaybeNullProjectionOperators(ctx, evalCtx, expr, t.Type, columnTypes, input, acc) + op, resultIdx, typs, internalMemUsed, err = planTypedMaybeNullProjectionOperators(ctx, evalCtx, expr, t.Type, columnTypes, input, acc) } else { - op, resultIdx, ct, internalMemUsed, err = planProjectionOperators(ctx, evalCtx, expr, columnTypes, input, acc) + op, resultIdx, typs, internalMemUsed, err = planProjectionOperators(ctx, evalCtx, expr, columnTypes, input, acc) } if err != nil { return nil, 0, nil, internalMemUsed, err } - op, resultIdx, ct, err = planCastOperator(ctx, acc, ct, op, resultIdx, expr.ResolvedType(), t.Type) - return op, resultIdx, ct, internalMemUsed, err + op, resultIdx, typs, err = planCastOperator(ctx, acc, typs, op, resultIdx, expr.ResolvedType(), t.Type) + return op, resultIdx, typs, internalMemUsed, err case *tree.FuncExpr: var ( inputCols []int projectionInternalMem int ) - ct = columnTypes + typs = columnTypes op = input for _, e := range t.Exprs { var err error // TODO(rohany): This could be done better, especially in the case of // constant arguments, because the vectorized engine right now // creates a new column full of the constant value. - op, resultIdx, ct, projectionInternalMem, err = planProjectionOperators( - ctx, evalCtx, e.(tree.TypedExpr), ct, op, acc, + op, resultIdx, typs, projectionInternalMem, err = planProjectionOperators( + ctx, evalCtx, e.(tree.TypedExpr), typs, op, acc, ) if err != nil { return nil, resultIdx, nil, internalMemUsed, err @@ -1625,56 +1596,55 @@ func planProjectionOperators( internalMemUsed += projectionInternalMem } funcOutputType := t.ResolvedType() - resultIdx = len(ct) - ct = append(ct, *funcOutputType) + resultIdx = len(typs) + typs = append(typs, *funcOutputType) op, err = NewBuiltinFunctionOperator( - NewAllocator(ctx, acc), evalCtx, t, ct, inputCols, resultIdx, op, + colmem.NewAllocator(ctx, acc), evalCtx, t, typs, inputCols, resultIdx, op, ) - return op, resultIdx, ct, internalMemUsed, err + return op, resultIdx, typs, internalMemUsed, err case tree.Datum: datumType := t.ResolvedType() - ct = columnTypes - resultIdx = len(ct) - ct = append(ct, *datumType) + typs = columnTypes + resultIdx = len(typs) + typs = append(typs, *datumType) if datumType.Family() == types.UnknownFamily { - return nil, resultIdx, ct, internalMemUsed, errors.New("cannot plan null type unknown") + return nil, resultIdx, typs, internalMemUsed, errors.New("cannot plan null type unknown") } - typ := typeconv.FromColumnType(datumType) - constVal, err := typeconv.GetDatumToPhysicalFn(datumType)(t) + constVal, err := getDatumToPhysicalFn(datumType)(t) if err != nil { - return nil, resultIdx, ct, internalMemUsed, err + return nil, resultIdx, typs, internalMemUsed, err } - op, err := NewConstOp(NewAllocator(ctx, acc), input, typ, constVal, resultIdx) + op, err := NewConstOp(colmem.NewAllocator(ctx, acc), input, datumType, constVal, resultIdx) if err != nil { - return nil, resultIdx, ct, internalMemUsed, err + return nil, resultIdx, typs, internalMemUsed, err } - return op, resultIdx, ct, internalMemUsed, nil + return op, resultIdx, typs, internalMemUsed, nil case *tree.CaseExpr: if t.Expr != nil { - return nil, resultIdx, ct, internalMemUsed, errors.New("CASE WHEN expressions unsupported") + return nil, resultIdx, typs, internalMemUsed, errors.New("CASE WHEN expressions unsupported") } - allocator := 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([]Operator, len(t.Whens)) - caseOutputType := typeconv.FromColumnType(t.ResolvedType()) - switch caseOutputType { + caseOps := make([]colexecbase.Operator, len(t.Whens)) + caseOutputType := t.ResolvedType() + switch typeconv.FromColumnType(caseOutputType) { case coltypes.Bytes: // Currently, there is a contradiction between the way CASE operator // works (which populates its output in arbitrary order) and the flat // bytes implementation of Bytes type (which prohibits sets in arbitrary // order), so we reject such scenario to fall back to row-by-row engine. - return nil, resultIdx, ct, internalMemUsed, errors.Newf( - "unsupported type %s in CASE operator", t.ResolvedType().String()) + return nil, resultIdx, typs, internalMemUsed, errors.Newf( + "unsupported type %s in CASE operator", caseOutputType) case coltypes.Unhandled: - return nil, resultIdx, ct, internalMemUsed, errors.Newf( - "unsupported type %s", t.ResolvedType().String()) + return nil, resultIdx, typs, internalMemUsed, errors.Newf( + "unsupported type %s", caseOutputType) } caseOutputIdx := len(columnTypes) - ct = append(columnTypes, *t.ResolvedType()) + typs = append(columnTypes, *caseOutputType) thenIdxs := make([]int, len(t.Whens)+1) for i, when := range t.Whens { // The case operator is assembled from n WHEN arms, n THEN arms, and an @@ -1692,74 +1662,73 @@ func planProjectionOperators( // result of the case projection. whenTyped := when.Cond.(tree.TypedExpr) whenResolvedType := whenTyped.ResolvedType() - whenColType := typeconv.FromColumnType(whenResolvedType) - if whenColType == coltypes.Unhandled { - return nil, resultIdx, ct, internalMemUsed, errors.Newf( - "unsupported type %s in CASE WHEN expression", whenResolvedType.String()) + if typeconv.FromColumnType(whenResolvedType) == coltypes.Unhandled { + return nil, resultIdx, typs, internalMemUsed, errors.Newf( + "unsupported type %s in CASE WHEN expression", whenResolvedType) } var whenInternalMemUsed, thenInternalMemUsed int - caseOps[i], resultIdx, ct, whenInternalMemUsed, err = planTypedMaybeNullProjectionOperators( - ctx, evalCtx, whenTyped, whenResolvedType, ct, buffer, acc, + caseOps[i], resultIdx, typs, whenInternalMemUsed, err = planTypedMaybeNullProjectionOperators( + ctx, evalCtx, whenTyped, whenResolvedType, typs, buffer, acc, ) if err != nil { - return nil, resultIdx, ct, internalMemUsed, err + return nil, resultIdx, typs, internalMemUsed, err } // Transform the booleans to a selection vector. caseOps[i] = NewBoolVecToSelOp(caseOps[i], resultIdx) // Run the "then" clause on those tuples that were selected. - caseOps[i], thenIdxs[i], ct, thenInternalMemUsed, err = planTypedMaybeNullProjectionOperators( - ctx, evalCtx, when.Val.(tree.TypedExpr), t.ResolvedType(), ct, caseOps[i], acc, + caseOps[i], thenIdxs[i], typs, thenInternalMemUsed, err = planTypedMaybeNullProjectionOperators( + ctx, evalCtx, when.Val.(tree.TypedExpr), t.ResolvedType(), typs, caseOps[i], acc, ) if err != nil { - return nil, resultIdx, ct, internalMemUsed, err + return nil, resultIdx, typs, internalMemUsed, err } internalMemUsed += whenInternalMemUsed + thenInternalMemUsed - if !ct[thenIdxs[i]].Equal(ct[caseOutputIdx]) { + if !typs[thenIdxs[i]].Identical(&typs[caseOutputIdx]) { // It is possible that the projection of this THEN arm has different // column type (for example, we expect INT2, but INT8 is given). In // such case, we need to plan a cast. - fromType, toType := &ct[thenIdxs[i]], &ct[caseOutputIdx] - caseOps[i], thenIdxs[i], ct, err = planCastOperator( - ctx, acc, ct, caseOps[i], thenIdxs[i], fromType, toType, + fromType, toType := &typs[thenIdxs[i]], &typs[caseOutputIdx] + caseOps[i], thenIdxs[i], typs, err = planCastOperator( + ctx, acc, typs, caseOps[i], thenIdxs[i], fromType, toType, ) if err != nil { - return nil, resultIdx, ct, internalMemUsed, err + return nil, resultIdx, typs, internalMemUsed, err } } } var elseInternalMemUsed int - var elseOp Operator + var elseOp colexecbase.Operator elseExpr := t.Else if elseExpr == nil { // If there's no ELSE arm, we write NULLs. elseExpr = tree.DNull } - elseOp, thenIdxs[len(t.Whens)], ct, elseInternalMemUsed, err = planTypedMaybeNullProjectionOperators( - ctx, evalCtx, elseExpr.(tree.TypedExpr), t.ResolvedType(), ct, buffer, acc, + elseOp, thenIdxs[len(t.Whens)], typs, elseInternalMemUsed, err = planTypedMaybeNullProjectionOperators( + ctx, evalCtx, elseExpr.(tree.TypedExpr), t.ResolvedType(), typs, buffer, acc, ) if err != nil { - return nil, resultIdx, ct, internalMemUsed, err + return nil, resultIdx, typs, internalMemUsed, err } internalMemUsed += elseInternalMemUsed - if !ct[thenIdxs[len(t.Whens)]].Equal(ct[caseOutputIdx]) { + if !typs[thenIdxs[len(t.Whens)]].Identical(&typs[caseOutputIdx]) { // It is possible that the projection of the ELSE arm has different // column type (for example, we expect INT2, but INT8 is given). In // such case, we need to plan a cast. elseIdx := thenIdxs[len(t.Whens)] - fromType, toType := &ct[elseIdx], &ct[caseOutputIdx] - elseOp, thenIdxs[len(t.Whens)], ct, err = planCastOperator( - ctx, acc, ct, elseOp, elseIdx, fromType, toType, + fromType, toType := &typs[elseIdx], &typs[caseOutputIdx] + elseOp, thenIdxs[len(t.Whens)], typs, err = planCastOperator( + ctx, acc, typs, elseOp, elseIdx, fromType, toType, ) if err != nil { - return nil, resultIdx, ct, internalMemUsed, err + return nil, resultIdx, typs, internalMemUsed, err } } - schemaEnforcer.typs = toPhysTypesMaybeUnhandled(ct) + schemaEnforcer.typs = typs op := NewCaseOp(allocator, buffer, caseOps, elseOp, thenIdxs, caseOutputIdx, caseOutputType) internalMemUsed += op.(InternalMemoryOperator).InternalMemoryUsage() - return op, caseOutputIdx, ct, internalMemUsed, err + return op, caseOutputIdx, typs, internalMemUsed, err case *tree.AndExpr, *tree.OrExpr: return planLogicalProjectionOp(ctx, evalCtx, expr, columnTypes, input, acc) default: @@ -1801,14 +1770,13 @@ func planProjectionExpr( outputType *types.T, left, right tree.TypedExpr, columnTypes []types.T, - input Operator, + input colexecbase.Operator, acc *mon.BoundAccount, -) (op Operator, resultIdx int, ct []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, ct, internalMemUsed, err + return nil, resultIdx, typs, internalMemUsed, err } resultIdx = -1 - outputPhysType := typeconv.FromColumnType(outputType) // actualOutputType tracks the logical type of the output column of the // projection operator. See the comment below for more details. actualOutputType := outputType @@ -1827,14 +1795,12 @@ func planProjectionExpr( // // We do, however, need to plan a cast to the expected logical type and // we will do that below. - leftPhysType := typeconv.FromColumnType(left.ResolvedType()) - rightPhysType := typeconv.FromColumnType(right.ResolvedType()) - if leftPhysType == coltypes.Int16 && rightPhysType == coltypes.Int16 { + leftType := left.ResolvedType() + rightType := right.ResolvedType() + if leftType.Identical(types.Int2) && rightType.Identical(types.Int2) { actualOutputType = types.Int2 - outputPhysType = coltypes.Int16 - } else if leftPhysType == coltypes.Int32 && rightPhysType == coltypes.Int32 { + } else if leftType.Identical(types.Int4) && rightType.Identical(types.Int4) { actualOutputType = types.Int4 - outputPhysType = coltypes.Int32 } } // There are 3 cases. Either the left is constant, the right is constant, @@ -1845,17 +1811,17 @@ func planProjectionExpr( // argument is on the right side. This doesn't happen for non-commutative // operators such as - and /, though, so we still need this case. var rightIdx int - input, rightIdx, ct, internalMemUsed, err = planProjectionOperators( + input, rightIdx, typs, internalMemUsed, err = planProjectionOperators( ctx, evalCtx, right, columnTypes, input, acc, ) if err != nil { - return nil, resultIdx, ct, internalMemUsed, err + return nil, resultIdx, typs, internalMemUsed, err } - resultIdx = len(ct) + resultIdx = len(typs) // The projection result will be outputted to a new column which is appended // to the input batch. op, err = GetProjectionLConstOperator( - NewAllocator(ctx, acc), left.ResolvedType(), &ct[rightIdx], outputPhysType, + colmem.NewAllocator(ctx, acc), left.ResolvedType(), &typs[rightIdx], actualOutputType, projOp, input, rightIdx, lConstArg, resultIdx, ) } else { @@ -1863,22 +1829,22 @@ func planProjectionExpr( leftIdx int internalMemUsedLeft int ) - input, leftIdx, ct, internalMemUsedLeft, err = planProjectionOperators( + input, leftIdx, typs, internalMemUsedLeft, err = planProjectionOperators( ctx, evalCtx, left, columnTypes, input, acc, ) if err != nil { - return nil, resultIdx, ct, internalMemUsed, err + return nil, resultIdx, typs, internalMemUsed, err } internalMemUsed += internalMemUsedLeft if rConstArg, rConst := right.(tree.Datum); rConst { // Case 2: The right is constant. // The projection result will be outputted to a new column which is appended // to the input batch. - resultIdx = len(ct) + resultIdx = len(typs) if projOp == tree.Like || projOp == tree.NotLike { negate := projOp == tree.NotLike op, err = GetLikeProjectionOperator( - 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 { @@ -1886,24 +1852,24 @@ func planProjectionExpr( datumTuple, ok := tree.AsDTuple(rConstArg) if !ok { err = errors.Errorf("IN operator supported only on constant expressions") - return nil, resultIdx, ct, internalMemUsed, err + return nil, resultIdx, typs, internalMemUsed, err } op, err = GetInProjectionOperator( - NewAllocator(ctx, acc), &ct[leftIdx], input, leftIdx, + colmem.NewAllocator(ctx, acc), &typs[leftIdx], input, leftIdx, resultIdx, datumTuple, negate, ) } else if projOp == tree.IsDistinctFrom || projOp == tree.IsNotDistinctFrom { if right != tree.DNull { err = errors.Errorf("IS DISTINCT FROM and IS NOT DISTINCT FROM are supported only with NULL argument") - return nil, resultIdx, ct, internalMemUsed, err + return nil, resultIdx, typs, internalMemUsed, err } // 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(NewAllocator(ctx, acc), input, leftIdx, resultIdx, negate) + op = newIsNullProjOp(colmem.NewAllocator(ctx, acc), input, leftIdx, resultIdx, negate) } else { op, err = GetProjectionRConstOperator( - NewAllocator(ctx, acc), &ct[leftIdx], right.ResolvedType(), outputPhysType, + colmem.NewAllocator(ctx, acc), &typs[leftIdx], right.ResolvedType(), actualOutputType, projOp, input, leftIdx, rConstArg, resultIdx, ) } @@ -1913,44 +1879,44 @@ func planProjectionExpr( rightIdx int internalMemUsedRight int ) - input, rightIdx, ct, internalMemUsedRight, err = planProjectionOperators( - ctx, evalCtx, right, ct, input, acc, + input, rightIdx, typs, internalMemUsedRight, err = planProjectionOperators( + ctx, evalCtx, right, typs, input, acc, ) if err != nil { return nil, resultIdx, nil, internalMemUsed, err } internalMemUsed += internalMemUsedRight - resultIdx = len(ct) + resultIdx = len(typs) op, err = GetProjectionOperator( - NewAllocator(ctx, acc), &ct[leftIdx], &ct[rightIdx], outputPhysType, + colmem.NewAllocator(ctx, acc), &typs[leftIdx], &typs[rightIdx], actualOutputType, projOp, input, leftIdx, rightIdx, resultIdx, ) } } if err != nil { - return op, resultIdx, ct, internalMemUsed, err + return op, resultIdx, typs, internalMemUsed, err } if sMem, ok := op.(InternalMemoryOperator); ok { internalMemUsed += sMem.InternalMemoryUsage() } - ct = append(ct, *actualOutputType) + typs = append(typs, *actualOutputType) if !outputType.Identical(actualOutputType) { // The projection operator outputs a column of a different type than // the expected logical type. In order to "synchronize" the reality and // the expectations, we plan a cast. // // For example, INT2 + INT2 will be typed as INT8 by the SQL type - // system, but we will plan a projection operator that outputs - // coltypes.Int16 = INT2, so in such scenario we will have + // system, but we will plan a projection operator that outputs INT2, so + // in such scenario we will have // actualOutputType = types.Int2 // outputType = types.Int8 // and will plan the corresponding cast. // // NOTE: this is *only* needed for integer types and should be removed // once #46940 is resolved. - op, resultIdx, ct, err = planCastOperator(ctx, acc, ct, op, resultIdx, actualOutputType, outputType) + op, resultIdx, typs, err = planCastOperator(ctx, acc, typs, op, resultIdx, actualOutputType, outputType) } - return op, resultIdx, ct, internalMemUsed, err + return op, resultIdx, typs, internalMemUsed, err } // planLogicalProjectionOp plans all the needed operators for a projection of @@ -1960,15 +1926,15 @@ func planLogicalProjectionOp( evalCtx *tree.EvalContext, expr tree.TypedExpr, columnTypes []types.T, - input Operator, + input colexecbase.Operator, acc *mon.BoundAccount, -) (op Operator, resultIdx int, ct []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) - ct = append(columnTypes, *types.Bool) + typs = append(columnTypes, *types.Bool) var ( typedLeft, typedRight tree.TypedExpr - leftProjOpChain, rightProjOpChain, outputOp Operator + leftProjOpChain, rightProjOpChain, outputOp colexecbase.Operator leftIdx, rightIdx int internalMemUsedLeft, internalMemUsedRight int leftFeedOp, rightFeedOp feedOperator @@ -1981,22 +1947,22 @@ func planLogicalProjectionOp( typedLeft = t.TypedLeft() typedRight = t.TypedRight() default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unexpected logical expression type %s", t.String())) + colexecerror.InternalError(fmt.Sprintf("unexpected logical expression type %s", t.String())) } - leftProjOpChain, leftIdx, ct, internalMemUsedLeft, err = planTypedMaybeNullProjectionOperators( - ctx, evalCtx, typedLeft, types.Bool, ct, &leftFeedOp, acc, + leftProjOpChain, leftIdx, typs, internalMemUsedLeft, err = planTypedMaybeNullProjectionOperators( + ctx, evalCtx, typedLeft, types.Bool, typs, &leftFeedOp, acc, ) if err != nil { - return nil, resultIdx, ct, internalMemUsed, err + return nil, resultIdx, typs, internalMemUsed, err } - rightProjOpChain, rightIdx, ct, internalMemUsedRight, err = planTypedMaybeNullProjectionOperators( - ctx, evalCtx, typedRight, types.Bool, ct, &rightFeedOp, acc, + rightProjOpChain, rightIdx, typs, internalMemUsedRight, err = planTypedMaybeNullProjectionOperators( + ctx, evalCtx, typedRight, types.Bool, typs, &rightFeedOp, acc, ) if err != nil { - return nil, resultIdx, ct, internalMemUsed, err + return nil, resultIdx, typs, internalMemUsed, err } - allocator := NewAllocator(ctx, acc) - input = newBatchSchemaPrefixEnforcer(allocator, input, toPhysTypesMaybeUnhandled(ct)) + allocator := colmem.NewAllocator(ctx, acc) + input = newBatchSchemaPrefixEnforcer(allocator, input, typs) switch expr.(type) { case *tree.AndExpr: outputOp = NewAndProjOp( @@ -2013,5 +1979,5 @@ func planLogicalProjectionOp( leftIdx, rightIdx, resultIdx, ) } - return outputOp, resultIdx, ct, internalMemUsedLeft + internalMemUsedRight, nil + return outputOp, resultIdx, typs, internalMemUsedLeft + internalMemUsedRight, nil } diff --git a/pkg/sql/colexec/external_hash_joiner.go b/pkg/sql/colexec/external_hash_joiner.go index 5d70ec1fe80b..49149288bd6e 100644 --- a/pkg/sql/colexec/external_hash_joiner.go +++ b/pkg/sql/colexec/external_hash_joiner.go @@ -16,10 +16,13 @@ import ( "math" "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/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -165,7 +168,7 @@ type externalHashJoiner struct { mu syncutil.Mutex state externalHashJoinerState - unlimitedAllocator *Allocator + unlimitedAllocator *colmem.Allocator spec hashJoinerSpec diskQueueCfg colcontainer.DiskQueueCfg @@ -270,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 *Allocator, + unlimitedAllocator *colmem.Allocator, spec hashJoinerSpec, - leftInput, rightInput Operator, + leftInput, rightInput colexecbase.Operator, memoryLimit int64, diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, - createReusableDiskBackedSorter func(input Operator, inputTypes []coltypes.T, orderingCols []execinfrapb.Ordering_Column, maxNumberPartitions int) (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, -) Operator { +) colexecbase.Operator { if diskQueueCfg.CacheMode != colcontainer.DiskQueueCacheModeClearAndReuseCache { - execerror.VectorizedInternalPanic(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 { @@ -343,14 +346,14 @@ func newExternalHashJoiner( leftJoinerInput, spec.left.sourceTypes, leftOrdering, externalSorterMaxNumberPartitions, ) if err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } rightOrdering := makeOrderingCols(spec.right.eqCols) rightPartitionSorter, err := createReusableDiskBackedSorter( rightJoinerInput, spec.right.sourceTypes, rightOrdering, externalSorterMaxNumberPartitions, ) if err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } diskBackedSortMerge, err := newMergeJoinOp( unlimitedAllocator, memoryLimit, diskQueueCfg, @@ -359,7 +362,7 @@ func newExternalHashJoiner( diskAcc, ) if err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } ehj := &externalHashJoiner{ twoInputNode: newTwoInputNode(leftInput, rightInput), @@ -399,7 +402,7 @@ func newExternalHashJoiner( ehj.recursiveScratch.leftBatch = unlimitedAllocator.NewMemBatchNoCols(spec.left.sourceTypes, 0 /* size */) sameSourcesSchema := len(spec.left.sourceTypes) == len(spec.right.sourceTypes) for i, leftType := range spec.left.sourceTypes { - if i < len(spec.right.sourceTypes) && leftType != spec.right.sourceTypes[i] { + if i < len(spec.right.sourceTypes) && !leftType.Identical(&spec.right.sourceTypes[i]) { sameSourcesSchema = false } } @@ -459,7 +462,7 @@ func (hj *externalHashJoiner) partitionBatch( for i, colvec := range colVecs { colvec.Copy(coldata.CopySliceArgs{ SliceArgs: coldata.SliceArgs{ - ColType: sourceSpec.sourceTypes[i], + ColType: typeconv.FromColumnType(&sourceSpec.sourceTypes[i]), Src: batch.ColVec(i), Sel: sel, SrcEndIdx: len(sel), @@ -469,7 +472,7 @@ func (hj *externalHashJoiner) partitionBatch( scratchBatch.SetLength(len(sel)) }) if err := partitioner.Enqueue(ctx, partitionIdx, scratchBatch); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } partitionInfo, ok := hj.partitionsToJoinUsingInMemHash[partitionIdx] if !ok { @@ -481,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 += getProportionalBatchMemSize(scratchBatch, int64(len(sel))) + partitionInfo.rightMemSize += colmem.GetProportionalBatchMemSize(scratchBatch, int64(len(sel))) } } } @@ -511,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } if err := hj.rightPartitioner.CloseAllOpenWriteFileDescriptors(ctx); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } hj.inMemHashJoiner.Init() hj.partitionIdxOffset += hj.numBuckets @@ -524,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } hj.fdState.acquiredFDs = toAcquire } @@ -560,7 +563,7 @@ StateChanged: } for { if err := partitioner.Dequeue(ctx, parentPartitionIdx, batch); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } if batch.Length() == 0 { break @@ -570,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } // We're done writing to the newly created partitions. // TODO(yuzefovich): we should not release the descriptors here. The @@ -583,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } } for idx := 0; idx < hj.numBuckets; idx++ { @@ -660,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } if err := hj.rightPartitioner.CloseInactiveReadPartitions(ctx); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } hj.state = externalHJJoinNewPartition continue @@ -692,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } if err := hj.rightPartitioner.CloseInactiveReadPartitions(ctx); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } hj.state = externalHJSortMergeNewPartition continue @@ -704,11 +707,11 @@ StateChanged: case externalHJFinished: if err := hj.idempotentCloseLocked(ctx); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return coldata.ZeroBatch default: - execerror.VectorizedInternalPanic(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 b1e24b477c18..d87826b4a74b 100644 --- a/pkg/sql/colexec/external_hash_joiner_test.go +++ b/pkg/sql/colexec/external_hash_joiner_test.go @@ -16,13 +16,14 @@ import ( "testing" "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/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" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils/colcontainerutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -74,8 +75,8 @@ func TestExternalHashJoiner(t *testing.T) { }(tc.skipAllNullsInjection) tc.skipAllNullsInjection = true } - runHashJoinTestCase(t, tc, func(sources []Operator) (Operator, error) { - sem := 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 @@ -132,33 +133,33 @@ func TestExternalHashJoinerFallbackToSortMergeJoin(t *testing.T) { DiskMonitor: testDiskMonitor, }, } - sourceTypes := []coltypes.T{coltypes.Int64} + sourceTypes := []types.T{*types.Int} batch := testAllocator.NewMemBatch(sourceTypes) // We don't need to set the data since zero values in the columns work. batch.SetLength(coldata.BatchSize()) nBatches := 2 - leftSource := newFiniteBatchSource(batch, nBatches) - rightSource := newFiniteBatchSource(batch, nBatches) + leftSource := newFiniteBatchSource(batch, sourceTypes, nBatches) + rightSource := newFiniteBatchSource(batch, sourceTypes, nBatches) tc := &joinTestCase{ - joinType: sqlbase.JoinType_INNER, - leftPhysTypes: sourceTypes, - leftOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightPhysTypes: sourceTypes, - rightOutCols: []uint32{0}, - rightEqCols: []uint32{0}, + joinType: sqlbase.JoinType_INNER, + leftTypes: sourceTypes, + leftOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightTypes: sourceTypes, + rightOutCols: []uint32{0}, + rightEqCols: []uint32{0}, } tc.init() spec := createSpecForHashJoiner(tc) var spilled bool queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(t, true /* inMem */) defer cleanup() - sem := 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, []Operator{leftSource, rightSource}, + ctx, flowCtx, spec, []colexecbase.Operator{leftSource, rightSource}, func() { spilled = true }, queueCfg, 0 /* numForcedRepartitions */, true, /* delegateFDAcquisitions */ sem, ) @@ -197,10 +198,10 @@ func BenchmarkExternalHashJoiner(b *testing.B) { }, } nCols := 4 - sourceTypes := make([]coltypes.T, nCols) + sourceTypes := make([]types.T, nCols) for colIdx := 0; colIdx < nCols; colIdx++ { - sourceTypes[colIdx] = coltypes.Int64 + sourceTypes[colIdx] = *types.Int } batch := testAllocator.NewMemBatch(sourceTypes) @@ -230,8 +231,8 @@ func BenchmarkExternalHashJoiner(b *testing.B) { } queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(b, false /* inMem */) defer cleanup() - leftSource := newFiniteBatchSource(batch, 0) - rightSource := newFiniteBatchSource(batch, 0) + leftSource := newFiniteBatchSource(batch, sourceTypes, 0) + rightSource := newFiniteBatchSource(batch, sourceTypes, 0) for _, fullOuter := range []bool{false, true} { for _, nBatches := range []int{1 << 2, 1 << 7} { for _, spillForced := range []bool{false, true} { @@ -244,13 +245,13 @@ func BenchmarkExternalHashJoiner(b *testing.B) { joinType = sqlbase.JoinType_FULL_OUTER } tc := &joinTestCase{ - joinType: joinType, - leftPhysTypes: sourceTypes, - leftOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0, 2}, - rightPhysTypes: sourceTypes, - rightOutCols: []uint32{2, 3}, - rightEqCols: []uint32{0, 1}, + joinType: joinType, + leftTypes: sourceTypes, + leftOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0, 2}, + rightTypes: sourceTypes, + rightOutCols: []uint32{2, 3}, + rightEqCols: []uint32{0, 1}, } tc.init() spec := createSpecForHashJoiner(tc) @@ -263,9 +264,9 @@ func BenchmarkExternalHashJoiner(b *testing.B) { leftSource.reset(nBatches) rightSource.reset(nBatches) hj, accounts, monitors, _, err := createDiskBackedHashJoiner( - ctx, flowCtx, spec, []Operator{leftSource, rightSource}, + ctx, flowCtx, spec, []colexecbase.Operator{leftSource, rightSource}, func() {}, queueCfg, 0 /* numForcedRepartitions */, false, /* delegateFDAcquisitions */ - NewTestingSemaphore(VecMaxOpenFDsLimit), + colexecbase.NewTestingSemaphore(VecMaxOpenFDsLimit), ) memAccounts = append(memAccounts, accounts...) memMonitors = append(memMonitors, monitors...) @@ -296,13 +297,13 @@ func createDiskBackedHashJoiner( ctx context.Context, flowCtx *execinfra.FlowCtx, spec *execinfrapb.ProcessorSpec, - inputs []Operator, + inputs []colexecbase.Operator, spillingCallbackFn func(), diskQueueCfg colcontainer.DiskQueueCfg, numForcedRepartitions int, delegateFDAcquisitions bool, testingSemaphore semaphore.Semaphore, -) (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 d1a1596ec140..942909c241b9 100644 --- a/pkg/sql/colexec/external_sort.go +++ b/pkg/sql/colexec/external_sort.go @@ -15,10 +15,12 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" @@ -118,9 +120,9 @@ type externalSorter struct { // Next, which will simplify this model. mu syncutil.Mutex - unlimitedAllocator *Allocator + unlimitedAllocator *colmem.Allocator state externalSorterState - inputTypes []coltypes.T + inputTypes []types.T ordering execinfrapb.Ordering inMemSorter resettableOperator inMemSorterInput *inputPartitioningOperator @@ -138,7 +140,7 @@ type externalSorter struct { acquiredFDs int } - emitter Operator + emitter colexecbase.Operator testingKnobs struct { // delegateFDAcquisitions if true, means that a test wants to force the @@ -171,10 +173,10 @@ var _ closableOperator = &externalSorter{} // them up front in Next. This should only be true in tests. func newExternalSorter( ctx context.Context, - unlimitedAllocator *Allocator, + unlimitedAllocator *colmem.Allocator, standaloneMemAccount *mon.BoundAccount, - input Operator, - inputTypes []coltypes.T, + input colexecbase.Operator, + inputTypes []types.T, ordering execinfrapb.Ordering, memoryLimit int64, maxNumberPartitions int, @@ -182,9 +184,9 @@ func newExternalSorter( diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, diskAcc *mon.BoundAccount, -) Operator { +) colexecbase.Operator { if diskQueueCfg.CacheMode != colcontainer.DiskQueueCacheModeReuseCache { - execerror.VectorizedInternalPanic(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 @@ -200,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 := estimateBatchSizeBytes(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 { @@ -214,7 +216,7 @@ func newExternalSorter( inputTypes, ordering.Columns, ) if err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } partitionedDiskQueueSemaphore := fdSemaphore if !delegateFDAcquisitions { @@ -265,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } s.state = externalSorterSpillPartition continue @@ -293,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } s.fdState.acquiredFDs = toAcquire } if err := s.partitioner.Enqueue(ctx, curPartitionIdx, b); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } continue case externalSorterRepeatedMerging: @@ -311,12 +313,15 @@ func (s *externalSorter) Next(ctx context.Context) coldata.Batch { // with the unlimited allocator and will *not* release that memory // from the allocator, so we have to do it ourselves. before := s.unlimitedAllocator.Used() - merger := s.createMergerForPartitions(s.firstPartitionIdx, s.numPartitions) + merger, err := s.createMergerForPartitions(s.firstPartitionIdx, s.numPartitions) + if err != nil { + 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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } } after := s.unlimitedAllocator.Used() @@ -325,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } s.firstPartitionIdx += s.numPartitions s.numPartitions = 1 @@ -340,7 +345,11 @@ func (s *externalSorter) Next(ctx context.Context) coldata.Batch { s.unlimitedAllocator, s.inputTypes, s.partitioner, s.firstPartitionIdx, ) } else { - s.emitter = s.createMergerForPartitions(s.firstPartitionIdx, s.numPartitions) + var err error + s.emitter, err = s.createMergerForPartitions(s.firstPartitionIdx, s.numPartitions) + if err != nil { + colexecerror.InternalError(err) + } } s.emitter.Init() s.state = externalSorterEmitting @@ -354,11 +363,11 @@ func (s *externalSorter) Next(ctx context.Context) coldata.Batch { return b case externalSorterFinished: if err := s.internalCloseLocked(ctx); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return coldata.ZeroBatch default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unexpected externalSorterState %d", s.state)) + colexecerror.InternalError(fmt.Sprintf("unexpected externalSorterState %d", s.state)) } } } @@ -371,7 +380,7 @@ func (s *externalSorter) reset(ctx context.Context) { s.mu.Lock() defer s.mu.Unlock() if err := s.internalCloseLocked(ctx); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } s.firstPartitionIdx = 0 s.numPartitions = 0 @@ -404,8 +413,10 @@ func (s *externalSorter) IdempotentClose(ctx context.Context) error { // createMergerForPartitions creates an ordered synchronizer that will merge // partitions in [firstIdx, firstIdx+numPartitions) range. -func (s *externalSorter) createMergerForPartitions(firstIdx, numPartitions int) Operator { - syncInputs := make([]Operator, numPartitions) +func (s *externalSorter) createMergerForPartitions( + firstIdx, numPartitions int, +) (colexecbase.Operator, error) { + syncInputs := make([]colexecbase.Operator, numPartitions) for i := range syncInputs { syncInputs[i] = newPartitionerToOperator( s.unlimitedAllocator, s.inputTypes, s.partitioner, firstIdx+i, @@ -420,7 +431,7 @@ func (s *externalSorter) createMergerForPartitions(firstIdx, numPartitions int) } func newInputPartitioningOperator( - input Operator, standaloneMemAccount *mon.BoundAccount, memoryLimit int64, + input colexecbase.Operator, standaloneMemAccount *mon.BoundAccount, memoryLimit int64, ) resettableOperator { return &inputPartitioningOperator{ OneInputNode: NewOneInputNode(input), @@ -480,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 := getProportionalBatchMemSize(b, int64(b.Length())) + batchMemSize := colmem.GetProportionalBatchMemSize(b, int64(b.Length())) if err := o.standaloneMemAccount.Grow(ctx, batchMemSize); err != nil { - execerror.VectorizedInternalPanic(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 16dad4734a82..1353f9f0a107 100644 --- a/pkg/sql/colexec/external_sort_test.go +++ b/pkg/sql/colexec/external_sort_test.go @@ -18,7 +18,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/typeconv" + "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 []Operator) (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 := 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, @@ -93,7 +94,7 @@ func TestExternalSort(t *testing.T) { // result.ToClose) in cases where it is know the sorter will not // be drained. sorter, newAccounts, newMonitors, closers, err := createDiskBackedSorter( - ctx, flowCtx, input, tc.logTypes, tc.ordCols, tc.matchLen, tc.k, func() {}, + ctx, flowCtx, input, tc.typs, tc.ordCols, tc.matchLen, tc.k, func() {}, externalSorterMinPartitions, false /* delegateFDAcquisition */, queueCfg, sem, ) // Check that the sort was added as a Closer. @@ -136,9 +137,9 @@ func TestExternalSortRandomized(t *testing.T) { nTups := coldata.BatchSize()*4 + 1 maxCols := 2 // TODO(yuzefovich): randomize types as well. - logTypes := make([]types.T, maxCols) - for i := range logTypes { - logTypes[i] = *types.Int + typs := make([]types.T, maxCols) + for i := range typs { + typs[i] = *types.Int } queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(t, true /* inMem */) @@ -153,11 +154,9 @@ func TestExternalSortRandomized(t *testing.T) { // 2) The memory limit is dynamically set to repartition twice, this will also // allow the in-memory sorter to spool several batches before hitting the // memory limit. - colTyps, err := typeconv.FromColumnTypes(logTypes) - require.NoError(t, err) // memoryToSort is the total amount of memory that will be sorted in this // test. - memoryToSort := (nTups / coldata.BatchSize()) * estimateBatchSizeBytes(colTyps, 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 @@ -193,11 +192,11 @@ func TestExternalSortRandomized(t *testing.T) { []tuples{tups}, expected, orderedVerifier, - func(input []Operator) (Operator, error) { - sem := 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, logTypes[:nCols], ordCols, + ctx, flowCtx, input, typs[:nCols], ordCols, 0 /* matchLen */, 0 /* k */, func() {}, externalSorterMinPartitions, delegateFDAcquisition, queueCfg, sem) // TODO(asubiotto): Explicitly Close when testing.T is passed into @@ -253,13 +252,11 @@ func BenchmarkExternalSort(b *testing.B) { // 8 (bytes / int64) * nBatches (number of batches) * coldata.BatchSize() (rows / // batch) * nCols (number of columns / row). b.SetBytes(int64(8 * nBatches * coldata.BatchSize() * nCols)) - logTypes := make([]types.T, nCols) - for i := range logTypes { - logTypes[i] = *types.Int + typs := make([]types.T, nCols) + for i := range typs { + typs[i] = *types.Int } - physTypes, err := typeconv.FromColumnTypes(logTypes) - require.NoError(b, err) - batch := testAllocator.NewMemBatch(physTypes) + batch := testAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) ordCols := make([]execinfrapb.Ordering_Column, nCols) for i := range ordCols { @@ -272,15 +269,15 @@ func BenchmarkExternalSort(b *testing.B) { } b.ResetTimer() for n := 0; n < b.N; n++ { - source := newFiniteBatchSource(batch, nBatches) + source := newFiniteBatchSource(batch, typs, nBatches) var spilled bool // TODO(yuzefovich): do not specify maxNumberPartitions (let the // external sorter figure out that number itself) once we pass in // filled-in disk queue config. sorter, accounts, monitors, _, err := createDiskBackedSorter( - ctx, flowCtx, []Operator{source}, logTypes, ordCols, + ctx, flowCtx, []colexecbase.Operator{source}, typs, ordCols, 0 /* matchLen */, 0 /* k */, func() { spilled = true }, - 64 /* maxNumberPartitions */, false /* delegateFDAcquisitions */, queueCfg, &TestingSemaphore{}, + 64 /* maxNumberPartitions */, false /* delegateFDAcquisitions */, queueCfg, &colexecbase.TestingSemaphore{}, ) memAccounts = append(memAccounts, accounts...) memMonitors = append(memMonitors, monitors...) @@ -314,8 +311,8 @@ func BenchmarkExternalSort(b *testing.B) { func createDiskBackedSorter( ctx context.Context, flowCtx *execinfra.FlowCtx, - input []Operator, - logTypes []types.T, + input []colexecbase.Operator, + typs []types.T, ordCols []execinfrapb.Ordering_Column, matchLen int, k uint16, @@ -324,13 +321,13 @@ func createDiskBackedSorter( delegateFDAcquisitions bool, diskQueueCfg colcontainer.DiskQueueCfg, testingSemaphore semaphore.Semaphore, -) (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), } spec := &execinfrapb.ProcessorSpec{ - Input: []execinfrapb.InputSyncSpec{{ColumnTypes: logTypes}}, + Input: []execinfrapb.InputSyncSpec{{ColumnTypes: typs}}, Core: execinfrapb.ProcessorCoreUnion{ Sorter: sorterSpec, }, diff --git a/pkg/sql/colexec/hash_aggregator.go b/pkg/sql/colexec/hash_aggregator.go index be5f7ccd69cf..0c3bd4849d1f 100644 --- a/pkg/sql/colexec/hash_aggregator.go +++ b/pkg/sql/colexec/hash_aggregator.go @@ -15,8 +15,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" ) @@ -55,14 +59,15 @@ const ( type hashAggregator struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator aggCols [][]uint32 - aggTypes [][]coltypes.T + aggTypes [][]types.T aggFuncs []execinfrapb.AggregatorSpec_Func - inputTypes []coltypes.T - outputTypes []coltypes.T + inputTypes []types.T + inputPhysTypes []coltypes.T + outputTypes []types.T // aggFuncMap stores the mapping from hash code to a vector of aggregation // functions. Each aggregation function is stored along with keys that @@ -148,7 +153,7 @@ type hashAggregator struct { groupCols []uint32 // groupCols stores the types of the grouping columns. - groupTypes []coltypes.T + groupTypes []types.T // hashBuffer stores hash values for each tuple in the buffered batch. hashBuffer []uint64 @@ -158,20 +163,20 @@ type hashAggregator struct { decimalScratch decimalOverloadScratch } -var _ 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 *Allocator, - input Operator, - colTypes []coltypes.T, + allocator *colmem.Allocator, + input colexecbase.Operator, + typs []types.T, aggFns []execinfrapb.AggregatorSpec_Func, groupCols []uint32, aggCols [][]uint32, -) (Operator, error) { - aggTyps := extractAggTypes(aggCols, colTypes) +) (colexecbase.Operator, error) { + aggTyps := extractAggTypes(aggCols, typs) outputTypes, err := makeAggregateFuncsOutputTypes(aggTyps, aggFns) if err != nil { return nil, errors.AssertionFailedf( @@ -179,14 +184,15 @@ func NewHashAggregator( ) } - groupTypes := make([]coltypes.T, len(groupCols)) + groupTypes := make([]types.T, len(groupCols)) for i, colIdx := range groupCols { - groupTypes[i] = colTypes[colIdx] + groupTypes[i] = typs[colIdx] } // We picked value this as the result of our benchmark. tupleLimit := coldata.BatchSize() * 2 + inputPhysTypes, err := typeconv.FromColumnTypes(typs) return &hashAggregator{ OneInputNode: NewOneInputNode(input), allocator: allocator, @@ -198,13 +204,14 @@ func NewHashAggregator( batchTupleLimit: tupleLimit, - state: hashAggregatorBuffering, - inputTypes: colTypes, - outputTypes: outputTypes, + state: hashAggregatorBuffering, + inputTypes: typs, + inputPhysTypes: inputPhysTypes, + outputTypes: outputTypes, groupCols: groupCols, groupTypes: groupTypes, - }, nil + }, err } func (op *hashAggregator) Init() { @@ -308,7 +315,7 @@ func (op *hashAggregator) Next(ctx context.Context) coldata.Batch { case hashAggregatorDone: return coldata.ZeroBatch default: - execerror.VectorizedInternalPanic("hash aggregator in unhandled state") + colexecerror.InternalError("hash aggregator in unhandled state") // This code is unreachable, but the compiler cannot infer that. return nil } @@ -341,7 +348,7 @@ func (op *hashAggregator) buildSelectionForEachHashCode(ctx context.Context) { for _, colIdx := range op.groupCols { rehash(ctx, hashBuffer, - op.inputTypes[colIdx], + &op.inputTypes[colIdx], op.scratch.ColVec(int(colIdx)), nKeys, nil, /* sel */ @@ -429,7 +436,7 @@ func (op *hashAggregator) onlineAgg() { // performance. op.keyMapping.ColVec(keyIdx).Append(coldata.SliceArgs{ Src: op.scratch.ColVec(int(colIdx)), - ColType: op.inputTypes[colIdx], + ColType: op.inputPhysTypes[colIdx], DestIdx: aggFunc.keyIdx, SrcStartIdx: remaining[0], SrcEndIdx: remaining[0] + 1, diff --git a/pkg/sql/colexec/hash_aggregator_tmpl.go b/pkg/sql/colexec/hash_aggregator_tmpl.go index 342714d4d14b..87dd71d23a55 100644 --- a/pkg/sql/colexec/hash_aggregator_tmpl.go +++ b/pkg/sql/colexec/hash_aggregator_tmpl.go @@ -26,13 +26,16 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" - // {{/* + "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. @@ -46,10 +49,13 @@ var _ tree.Operator // Dummy import to pull in "math" package. var _ int = math.MaxInt16 +// Dummy import to pull in "coltypes" package. +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 { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} @@ -114,7 +120,7 @@ func (v hashAggFuncs) match( sel []int, b coldata.Batch, keyCols []uint32, - keyTypes []coltypes.T, + keyTypes []types.T, keyMapping coldata.Batch, diff []bool, ) (bool, []int) { @@ -135,7 +141,7 @@ func (v hashAggFuncs) match( keyTyp := keyTypes[keyIdx] - switch keyTyp { + switch typeconv.FromColumnType(&keyTyp) { // {{range .}} case _TYPES_T: lhsCol := lhs._TemplateType() @@ -156,7 +162,7 @@ func (v hashAggFuncs) match( } // {{end}} default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled type %d", keyTyp)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", &keyTyp)) } } diff --git a/pkg/sql/colexec/hash_utils.go b/pkg/sql/colexec/hash_utils.go index 41789e70fd38..d967f23d8139 100644 --- a/pkg/sql/colexec/hash_utils.go +++ b/pkg/sql/colexec/hash_utils.go @@ -14,7 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/types" ) // initHash, rehash, and finalizeHash work together to compute the hash value @@ -124,13 +124,13 @@ func newTupleHashDistributor(initHashValue uint64, numOutputs int) *tupleHashDis } func (d *tupleHashDistributor) distribute( - ctx context.Context, b coldata.Batch, types []coltypes.T, hashCols []uint32, + ctx context.Context, b coldata.Batch, types []types.T, hashCols []uint32, ) [][]int { n := b.Length() initHash(d.buckets, n, d.initHashValue) for _, i := range hashCols { - rehash(ctx, d.buckets, types[i], b.ColVec(int(i)), n, b.Selection(), d.cancelChecker, d.decimalScratch) + rehash(ctx, d.buckets, &types[i], b.ColVec(int(i)), n, b.Selection(), d.cancelChecker, d.decimalScratch) } finalizeHash(d.buckets, n, uint64(len(d.selections))) diff --git a/pkg/sql/colexec/hash_utils_test.go b/pkg/sql/colexec/hash_utils_test.go index 4386720e3d9c..b5b0febcfff8 100644 --- a/pkg/sql/colexec/hash_utils_test.go +++ b/pkg/sql/colexec/hash_utils_test.go @@ -16,7 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) @@ -30,8 +30,8 @@ func TestHashFunctionFamily(t *testing.T) { ctx := context.Background() bucketsA, bucketsB := make([]uint64, coldata.BatchSize()), make([]uint64, coldata.BatchSize()) nKeys := coldata.BatchSize() - keyTypes := []coltypes.T{coltypes.Int64} - keys := []coldata.Vec{testAllocator.NewMemColumn(keyTypes[0], coldata.BatchSize())} + keyTypes := []types.T{*types.Int} + keys := []coldata.Vec{testAllocator.NewMemColumn(&keyTypes[0], coldata.BatchSize())} for i := int64(0); i < int64(coldata.BatchSize()); i++ { keys[0].Int64()[i] = i } @@ -45,7 +45,7 @@ func TestHashFunctionFamily(t *testing.T) { // We need +1 here because 0 is not a valid initial hash value. initHash(buckets, nKeys, uint64(initHashValue+1)) for i, typ := range keyTypes { - rehash(ctx, buckets, typ, keys[i], nKeys, nil /* sel */, cancelChecker, decimalScratch) + rehash(ctx, buckets, &typ, keys[i], nKeys, nil /* sel */, cancelChecker, decimalScratch) } finalizeHash(buckets, nKeys, numBuckets) } diff --git a/pkg/sql/colexec/hash_utils_tmpl.go b/pkg/sql/colexec/hash_utils_tmpl.go index a66247f33844..5c8a516e4cb8 100644 --- a/pkg/sql/colexec/hash_utils_tmpl.go +++ b/pkg/sql/colexec/hash_utils_tmpl.go @@ -28,16 +28,15 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} - // HACK: crlfmt removes the "*/}}" comment if it's the last line in the - // import block. This was picked because it sorts after - // "pkg/sql/colexec/execgen" and has no deps. - _ "github.com/cockroachdb/cockroach/pkg/util/bufalloc" + "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 @@ -49,13 +48,16 @@ var _ reflect.SliceHeader // Dummy import to pull in "math" package. var _ = math.MaxInt64 +// Dummy import to pull in "coltypes" package. +var _ coltypes.T + // _GOTYPESLICE is a template Go type slice variable. 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 { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} @@ -109,14 +111,14 @@ func _REHASH_BODY( func rehash( ctx context.Context, buckets []uint64, - t coltypes.T, + t *types.T, col coldata.Vec, nKeys int, sel []int, cancelChecker CancelChecker, decimalScratch decimalOverloadScratch, ) { - switch t { + switch typeconv.FromColumnType(t) { // {{range $hashType := .}} case _TYPES_T: keys, nulls := col._TemplateType(), col.Nulls() @@ -136,6 +138,6 @@ func rehash( // {{end}} default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled type %d", t)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", t)) } } diff --git a/pkg/sql/colexec/hashjoiner.go b/pkg/sql/colexec/hashjoiner.go index a94e9e5c5a64..1a7ea6cd77a9 100644 --- a/pkg/sql/colexec/hashjoiner.go +++ b/pkg/sql/colexec/hashjoiner.go @@ -14,9 +14,12 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" ) @@ -61,7 +64,7 @@ type hashJoinerSourceSpec struct { // sourceTypes specify the types of the input columns of the source table for // the hash joiner. - sourceTypes []coltypes.T + sourceTypes []types.T // outer specifies whether an outer join is required over the input. outer bool @@ -154,7 +157,7 @@ type hashJoinerSourceSpec struct { type hashJoiner struct { twoInputNode - allocator *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. @@ -172,7 +175,7 @@ type hashJoiner struct { // probeState is used in hjProbing state. probeState struct { // keyTypes stores the types of the equality columns on the probe side. - keyTypes []coltypes.T + keyTypes []types.T // buildIdx and probeIdx represents the matching row indices that are used to // stitch together the join results. @@ -250,7 +253,7 @@ func (hj *hashJoiner) Next(ctx context.Context) coldata.Batch { hj.emitUnmatched() return hj.output default: - execerror.VectorizedInternalPanic("hash joiner in unhandled state") + colexecerror.InternalError("hash joiner in unhandled state") // This code is unreachable, but the compiler cannot infer that. return nil } @@ -298,7 +301,7 @@ func (hj *hashJoiner) emitUnmatched() { } outCols := hj.output.ColVecs()[len(hj.spec.left.sourceTypes) : len(hj.spec.left.sourceTypes)+len(hj.spec.right.sourceTypes)] - for i, colType := range hj.spec.right.sourceTypes { + for i, typ := range hj.spec.right.sourceTypes { outCol := outCols[i] valCol := hj.ht.vals.ColVec(i) // NOTE: this Copy is not accounted for because we don't want for memory @@ -313,7 +316,7 @@ func (hj *hashJoiner) emitUnmatched() { outCol.Copy( coldata.CopySliceArgs{ SliceArgs: coldata.SliceArgs{ - ColType: colType, + ColType: typeconv.FromColumnType(&typ), Src: valCol, SrcEndIdx: nResults, Sel: hj.probeState.buildIdx, @@ -457,7 +460,7 @@ func (hj *hashJoiner) congregate(nResults int, batch coldata.Batch, batchSize in // will be set below. if hj.ht.vals.Length() > 0 { outCols := hj.output.ColVecs()[rightColOffset : rightColOffset+len(hj.spec.right.sourceTypes)] - for i, colType := range hj.spec.right.sourceTypes { + for i, typ := range hj.spec.right.sourceTypes { outCol := outCols[i] valCol := hj.ht.vals.ColVec(i) // Note that if for some index i, probeRowUnmatched[i] is true, then @@ -466,7 +469,7 @@ func (hj *hashJoiner) congregate(nResults int, batch coldata.Batch, batchSize in outCol.Copy( coldata.CopySliceArgs{ SliceArgs: coldata.SliceArgs{ - ColType: colType, + ColType: typeconv.FromColumnType(&typ), Src: valCol, SrcEndIdx: nResults, Sel: hj.probeState.buildIdx, @@ -490,13 +493,13 @@ func (hj *hashJoiner) congregate(nResults int, batch coldata.Batch, batchSize in } outCols := hj.output.ColVecs()[:len(hj.spec.left.sourceTypes)] - for i, colType := range hj.spec.left.sourceTypes { + for i, typ := range hj.spec.left.sourceTypes { outCol := outCols[i] valCol := batch.ColVec(i) outCol.Copy( coldata.CopySliceArgs{ SliceArgs: coldata.SliceArgs{ - ColType: colType, + ColType: typeconv.FromColumnType(&typ), Src: valCol, Sel: hj.probeState.probeIdx, SrcEndIdx: nResults, @@ -524,7 +527,7 @@ func (hj *hashJoiner) congregate(nResults int, batch coldata.Batch, batchSize in hj.output.SetLength(nResults) } -func (hj *hashJoiner) ExportBuffered(input 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 @@ -543,14 +546,14 @@ func (hj *hashJoiner) ExportBuffered(input Operator) coldata.Batch { // We don't need to worry about selection vectors on hj.ht.vals because the // tuples have been already selected during building of the hash table. for i, t := range hj.spec.right.sourceTypes { - window := hj.ht.vals.ColVec(i).Window(t, startIdx, endIdx) + window := hj.ht.vals.ColVec(i).Window(typeconv.FromColumnType(&t), startIdx, endIdx) b.ReplaceCol(window, i) } b.SetLength(endIdx - startIdx) hj.exportBufferedState.rightExported = newRightExported return b } else { - execerror.VectorizedInternalPanic(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. @@ -560,7 +563,7 @@ func (hj *hashJoiner) ExportBuffered(input Operator) coldata.Batch { func (hj *hashJoiner) resetOutput() { if hj.output == nil { - outputTypes := append([]coltypes.T{}, hj.spec.left.sourceTypes...) + outputTypes := append([]types.T{}, hj.spec.left.sourceTypes...) if hj.spec.joinType != sqlbase.LeftSemiJoin && hj.spec.joinType != sqlbase.LeftAntiJoin { outputTypes = append(outputTypes, hj.spec.right.sourceTypes...) } @@ -571,7 +574,7 @@ func (hj *hashJoiner) resetOutput() { } func (hj *hashJoiner) reset(ctx context.Context) { - for _, input := range []Operator{hj.inputOne, hj.inputTwo} { + for _, input := range []colexecbase.Operator{hj.inputOne, hj.inputTwo} { if r, ok := input.(resetter); ok { r.reset(ctx) } @@ -598,8 +601,8 @@ func makeHashJoinerSpec( joinType sqlbase.JoinType, leftEqCols []uint32, rightEqCols []uint32, - leftTypes []coltypes.T, - rightTypes []coltypes.T, + leftTypes []types.T, + rightTypes []types.T, rightDistinct bool, ) (hashJoinerSpec, error) { var ( @@ -651,8 +654,8 @@ func makeHashJoinerSpec( // newHashJoiner creates a new equality hash join operator on the left and // right input tables. func newHashJoiner( - allocator *Allocator, spec hashJoinerSpec, leftSource, rightSource Operator, -) Operator { + allocator *colmem.Allocator, spec hashJoinerSpec, leftSource, rightSource colexecbase.Operator, +) colexecbase.Operator { hj := &hashJoiner{ twoInputNode: newTwoInputNode(leftSource, rightSource), allocator: allocator, @@ -664,7 +667,7 @@ func newHashJoiner( if spec.left.outer { hj.probeState.probeRowUnmatched = make([]bool, coldata.BatchSize()) } - hj.probeState.keyTypes = make([]coltypes.T, len(spec.left.eqCols)) + hj.probeState.keyTypes = make([]types.T, len(spec.left.eqCols)) for i, colIdx := range spec.left.eqCols { hj.probeState.keyTypes[i] = spec.left.sourceTypes[colIdx] } diff --git a/pkg/sql/colexec/hashjoiner_test.go b/pkg/sql/colexec/hashjoiner_test.go index 70cab53bee40..a3c229aee0a6 100644 --- a/pkg/sql/colexec/hashjoiner_test.go +++ b/pkg/sql/colexec/hashjoiner_test.go @@ -19,9 +19,9 @@ import ( "github.com/cockroachdb/apd" "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/colexec/execerror" + "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,15 +43,15 @@ func init() { for i, f := range floats { _, err := decs[i].SetFloat64(f) if err != nil { - execerror.VectorizedInternalPanic(fmt.Sprintf("%v", err)) + colexecerror.InternalError(fmt.Sprintf("%v", err)) } } hjTestCases = []*joinTestCase{ { - description: "0", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + description: "0", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, leftTuples: tuples{ {0}, @@ -85,9 +85,9 @@ func init() { }, }, { - description: "1", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + description: "1", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, // Test an empty build table. leftTuples: tuples{}, @@ -112,9 +112,9 @@ func init() { }, }, { - description: "2", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + description: "2", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, leftTuples: tuples{ {0}, @@ -147,9 +147,9 @@ func init() { }, }, { - description: "3", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + description: "3", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, // Test right outer join. leftTuples: tuples{ @@ -176,9 +176,9 @@ func init() { }, }, { - description: "4", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + description: "4", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, // Test right outer join with non-distinct left build table with an // unmatched row from the right followed by a matched one. This is a @@ -208,9 +208,9 @@ func init() { }, }, { - description: "5", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + description: "5", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, // Test null handling only on probe column. leftTuples: tuples{ @@ -234,9 +234,9 @@ func init() { }, }, { - description: "6", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + description: "6", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, // Test null handling only on build column. leftTuples: tuples{ @@ -265,9 +265,9 @@ func init() { }, }, { - description: "7", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, + description: "7", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, // Test null handling in output columns. leftTuples: tuples{ @@ -299,9 +299,9 @@ func init() { }, }, { - description: "8", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + description: "8", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, // Test null handling in hash join key column. leftTuples: tuples{ @@ -334,9 +334,9 @@ func init() { }, { // Test handling of multiple column non-distinct equality keys. - description: "9", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, + description: "9", + leftTypes: []types.T{*types.Int, *types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int, *types.Int}, leftTuples: tuples{ {0, 0, 1}, @@ -373,9 +373,9 @@ func init() { }, { // Test handling of duplicate equality keys that map to same buckets. - description: "10", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + description: "10", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, leftTuples: tuples{ {0}, @@ -420,9 +420,9 @@ func init() { }, { // Test handling of duplicate equality keys. - description: "11", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + description: "11", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, leftTuples: tuples{ {0}, @@ -459,9 +459,9 @@ func init() { }, { // Test handling of various output column coltypes. - description: "12", - leftPhysTypes: []coltypes.T{coltypes.Bool, coltypes.Int64, coltypes.Bytes, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Float64, coltypes.Int32}, + description: "12", + leftTypes: []types.T{*types.Bool, *types.Int, *types.Bytes, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Float, *types.Int4}, leftTuples: tuples{ {false, 5, "a", 10}, @@ -492,9 +492,9 @@ func init() { }, }, { - description: "13", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + description: "13", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, // Reverse engineering hash table hash heuristic to find key values that // hash to the same bucket. @@ -525,9 +525,9 @@ func init() { }, }, { - description: "14", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + description: "14", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, // Test a N:1 inner join where the right side key has duplicate values. leftTuples: tuples{ @@ -562,9 +562,9 @@ func init() { }, }, { - description: "15", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, + description: "15", + leftTypes: []types.T{*types.Int, *types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int, *types.Int}, // Test inner join on multiple equality columns. leftTuples: tuples{ @@ -599,9 +599,9 @@ func init() { }, }, { - description: "16", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, + description: "16", + leftTypes: []types.T{*types.Int, *types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, // Test multiple column with values that hash to the same bucket. leftTuples: tuples{ @@ -634,9 +634,9 @@ func init() { }, }, { - description: "17", - leftPhysTypes: []coltypes.T{coltypes.Bytes, coltypes.Bool, coltypes.Int16, coltypes.Int32, coltypes.Int64, coltypes.Bytes}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int32, coltypes.Int16, coltypes.Bool, coltypes.Bytes}, + description: "17", + leftTypes: []types.T{*types.Bytes, *types.Bool, *types.Int2, *types.Int4, *types.Int, *types.Bytes}, + rightTypes: []types.T{*types.Int, *types.Int4, *types.Int2, *types.Bool, *types.Bytes}, // Test multiple equality columns of different coltypes. leftTuples: tuples{ @@ -672,9 +672,9 @@ func init() { }, }, { - description: "18", - leftPhysTypes: []coltypes.T{coltypes.Float64}, - rightPhysTypes: []coltypes.T{coltypes.Float64}, + description: "18", + leftTypes: []types.T{*types.Float}, + rightTypes: []types.T{*types.Float}, // Test equality columns of type float. leftTuples: tuples{ @@ -705,9 +705,9 @@ func init() { }, }, { - description: "19", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64, coltypes.Int64}, + description: "19", + leftTypes: []types.T{*types.Int, *types.Int, *types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int, *types.Int, *types.Int}, // Test use right side as build table. leftTuples: tuples{ @@ -737,11 +737,11 @@ func init() { }, }, { - description: "20", - leftPhysTypes: []coltypes.T{coltypes.Decimal}, - rightPhysTypes: []coltypes.T{coltypes.Decimal}, + description: "20", + leftTypes: []types.T{*types.Decimal}, + rightTypes: []types.T{*types.Decimal}, - // Test coltypes.Decimal type as equality column. + // Test *types.Decimal type as equality column. leftTuples: tuples{ {decs[0]}, {decs[1]}, @@ -767,9 +767,9 @@ func init() { }, }, { - description: "21", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + description: "21", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, joinType: sqlbase.JoinType_LEFT_SEMI, @@ -800,9 +800,9 @@ func init() { }, }, { - description: "22", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + description: "22", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, joinType: sqlbase.JoinType_LEFT_ANTI, @@ -831,9 +831,9 @@ func init() { }, }, { - description: "23", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, + description: "23", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, // Test ON expression. leftTuples: tuples{ @@ -864,9 +864,9 @@ func init() { }, }, { - description: "24", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, + description: "24", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, // Test ON expression. leftTuples: tuples{ @@ -912,14 +912,14 @@ func createSpecForHashJoiner(tc *joinTestCase) *execinfrapb.ProcessorSpec { } projection := make([]uint32, 0, len(tc.leftOutCols)+len(tc.rightOutCols)) projection = append(projection, tc.leftOutCols...) - rColOffset := uint32(len(tc.leftPhysTypes)) + rColOffset := uint32(len(tc.leftTypes)) for _, outCol := range tc.rightOutCols { projection = append(projection, rColOffset+outCol) } return &execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{ - {ColumnTypes: tc.leftLogTypes}, - {ColumnTypes: tc.rightLogTypes}, + {ColumnTypes: tc.leftTypes}, + {ColumnTypes: tc.rightTypes}, }, Core: execinfrapb.ProcessorCoreUnion{ HashJoiner: hjSpec, @@ -935,11 +935,13 @@ func createSpecForHashJoiner(tc *joinTestCase) *execinfrapb.ProcessorSpec { // against a hash join operator (either in-memory or disk-backed one) which is // created by the provided constructor. func runHashJoinTestCase( - t *testing.T, tc *joinTestCase, hjOpConstructor func(sources []Operator) (Operator, error), + t *testing.T, + tc *joinTestCase, + hjOpConstructor func(sources []colexecbase.Operator) (colexecbase.Operator, error), ) { tc.init() inputs := []tuples{tc.leftTuples, tc.rightTuples} - typs := [][]coltypes.T{tc.leftPhysTypes, tc.rightPhysTypes} + typs := [][]types.T{tc.leftTypes, tc.rightTypes} var runner testRunner if tc.skipAllNullsInjection { // We're omitting all nulls injection test. See comments for each such @@ -973,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 []Operator) (Operator, error) { + runHashJoinTestCase(t, tc, func(sources []colexecbase.Operator) (colexecbase.Operator, error) { spec := createSpecForHashJoiner(tc) args := NewColOperatorArgs{ Spec: spec, @@ -1000,10 +1002,10 @@ func TestHashJoiner(t *testing.T) { func BenchmarkHashJoiner(b *testing.B) { ctx := context.Background() nCols := 4 - sourceTypes := make([]coltypes.T, nCols) + sourceTypes := make([]types.T, nCols) for colIdx := 0; colIdx < nCols; colIdx++ { - sourceTypes[colIdx] = coltypes.Int64 + sourceTypes[colIdx] = *types.Int } batch := testAllocator.NewMemBatch(sourceTypes) @@ -1043,8 +1045,8 @@ func BenchmarkHashJoiner(b *testing.B) { b.SetBytes(int64(8 * nBatches * coldata.BatchSize() * nCols * 2)) b.ResetTimer() for i := 0; i < b.N; i++ { - leftSource := NewRepeatableBatchSource(testAllocator, batch) - rightSource := newFiniteBatchSource(batch, nBatches) + leftSource := colexecbase.NewRepeatableBatchSource(testAllocator, batch, sourceTypes) + rightSource := newFiniteBatchSource(batch, sourceTypes, nBatches) joinType := sqlbase.JoinType_INNER if fullOuter { joinType = sqlbase.JoinType_FULL_OUTER @@ -1129,9 +1131,7 @@ func TestHashJoinerProjection(t *testing.T) { } leftTypes := []types.T{*types.Bool, *types.Int, *types.Bytes} - leftColTypes := []coltypes.T{coltypes.Bool, coltypes.Int64, coltypes.Bytes} rightTypes := []types.T{*types.Int, *types.Float, *types.Decimal} - rightColTypes := []coltypes.T{coltypes.Int64, coltypes.Float64, coltypes.Decimal} leftTuples := tuples{{false, 1, "foo"}} rightTuples := tuples{{1, 1.1, decs[1]}} @@ -1156,11 +1156,11 @@ func TestHashJoinerProjection(t *testing.T) { }, } - leftSource := newOpTestInput(1, leftTuples, leftColTypes) - rightSource := newOpTestInput(1, rightTuples, rightColTypes) + leftSource := newOpTestInput(1, leftTuples, leftTypes) + rightSource := newOpTestInput(1, rightTuples, rightTypes) args := NewColOperatorArgs{ Spec: spec, - Inputs: []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 a0907dd62846..fc824d51507e 100644 --- a/pkg/sql/colexec/hashtable.go +++ b/pkg/sql/colexec/hashtable.go @@ -15,8 +15,10 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" ) // TODO(yuzefovich): support rehashing instead of large fixed bucket size. @@ -105,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 *Allocator + allocator *colmem.Allocator // buildScratch contains the scratch buffers required for the build table. buildScratch hashTableBuildBuffer @@ -131,9 +133,9 @@ type hashTable struct { // index + 1. vals *appendOnlyBufferedBatch // valTypes stores the corresponding types of the val columns. - valTypes []coltypes.T + valTypes []types.T // keyCols stores the corresponding types of key columns. - keyTypes []coltypes.T + keyTypes []types.T // keyCols stores the indices of vals which are key columns. keyCols []uint32 @@ -155,14 +157,14 @@ type hashTable struct { var _ resetter = &hashTable{} func newHashTable( - allocator *Allocator, + allocator *colmem.Allocator, numBuckets uint64, - sourceTypes []coltypes.T, + sourceTypes []types.T, eqCols []uint32, allowNullEquality bool, mode hashTableMode, ) *hashTable { - keyTypes := make([]coltypes.T, len(eqCols)) + keyTypes := make([]types.T, len(eqCols)) for i, colIdx := range eqCols { keyTypes[i] = sourceTypes[colIdx] } @@ -206,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 Operator) { +func (ht *hashTable) build(ctx context.Context, input colexecbase.Operator) { nKeyCols := len(ht.keyCols) switch ht.mode { @@ -277,7 +279,7 @@ func (ht *hashTable) build(ctx context.Context, input Operator) { ht.buildNextChains(ctx, ht.buildScratch.first, ht.buildScratch.next, numBuffered+1, batch.Length()) } default: - execerror.VectorizedInternalPanic(fmt.Sprintf("hashTable in unhandled state")) + colexecerror.InternalError(fmt.Sprintf("hashTable in unhandled state")) } } @@ -318,7 +320,7 @@ func (ht *hashTable) removeDuplicates( // checkCols performs a column by column checkCol on the key columns. func (ht *hashTable) checkCols( probeVecs, buildVecs []coldata.Vec, - probeKeyTypes []coltypes.T, + probeKeyTypes []types.T, buildKeyCols []uint32, nToCheck uint64, probeSel []int, @@ -327,7 +329,7 @@ func (ht *hashTable) checkCols( for i := range ht.keyCols { probeType := probeKeyTypes[i] buildType := ht.keyTypes[i] - ht.checkCol(probeVecs[i], buildVecs[buildKeyCols[i]], probeType, buildType, + ht.checkCol(probeVecs[i], buildVecs[buildKeyCols[i]], &probeType, &buildType, i, nToCheck, probeSel, buildSel) } } @@ -343,7 +345,7 @@ func (ht *hashTable) checkColsForDistinctTuples( buildVec := buildVecs[ht.keyCols[i]] probeType := ht.keyTypes[i] - ht.checkColForDistinctTuples(probeVec, buildVec, probeType, nToCheck, probeSel) + ht.checkColForDistinctTuples(probeVec, buildVec, &probeType, nToCheck, probeSel) } } @@ -352,7 +354,7 @@ func (ht *hashTable) checkColsForDistinctTuples( func (ht *hashTable) computeBuckets( ctx context.Context, buckets []uint64, - keyTypes []coltypes.T, + keyTypes []types.T, keys []coldata.Vec, nKeys int, sel []int, @@ -365,7 +367,7 @@ func (ht *hashTable) computeBuckets( } for i := range ht.keyCols { - rehash(ctx, buckets, keyTypes[i], keys[i], nKeys, sel, ht.cancelChecker, ht.decimalScratch) + rehash(ctx, buckets, &keyTypes[i], keys[i], nKeys, sel, ht.cancelChecker, ht.decimalScratch) } finalizeHash(buckets, nKeys, ht.numBuckets) @@ -437,7 +439,7 @@ func (ht *hashTable) maybeAllocateSameAndVisited() { // column of the batch and stores the results in groupID. It also initializes // toCheck with all indices in the range [0, batchSize). func (ht *hashTable) lookupInitial( - ctx context.Context, keyTypes []coltypes.T, batchSize int, sel []int, + ctx context.Context, keyTypes []types.T, batchSize int, sel []int, ) { ht.computeBuckets(ctx, ht.probeScratch.buckets, keyTypes, ht.probeScratch.keys, batchSize, sel) for i := 0; i < batchSize; i++ { diff --git a/pkg/sql/colexec/hashtable_tmpl.go b/pkg/sql/colexec/hashtable_tmpl.go index 555e948ecb1b..cb5da33751d7 100644 --- a/pkg/sql/colexec/hashtable_tmpl.go +++ b/pkg/sql/colexec/hashtable_tmpl.go @@ -26,13 +26,16 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" - // {{/* + "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. @@ -47,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 { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // _PROBE_TYPE is the template type variable for coltypes.T. It will be @@ -183,7 +186,7 @@ func _CHECK_COL_WITH_NULLS( // there is no match. func (ht *hashTable) checkCol( probeVec, buildVec coldata.Vec, - probeType, buildType coltypes.T, + probeType, buildType *types.T, keyColIdx int, nToCheck uint64, probeSel []int, @@ -193,10 +196,10 @@ func (ht *hashTable) checkCol( // In order to inline the templated code of overloads, we need to have a // `decimalScratch` local variable of type `decimalOverloadScratch`. decimalScratch := ht.decimalScratch - switch probeType { + switch typeconv.FromColumnType(probeType) { // {{range $lTyp, $rTypToOverload := .}} case _PROBE_TYPE: - switch buildType { + switch typeconv.FromColumnType(buildType) { // {{range $rTyp, $overload := $rTypToOverload}} case _BUILD_TYPE: probeKeys := probeVec._ProbeType() @@ -217,11 +220,11 @@ func (ht *hashTable) checkCol( } // {{end}} default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled type %d", buildType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", buildType)) } // {{end}} default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled type %d", probeType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", probeType)) } } @@ -229,7 +232,7 @@ func (ht *hashTable) checkCol( func _CHECK_COL_FOR_DISTINCT_WITH_NULLS( ht *hashTable, probeVec, buildVec coldata.Vec, - probeType coltypes.T, + probeType *types.T, nToCheck uint16, probeSel []uint16, _USE_PROBE_SEL bool, @@ -255,9 +258,9 @@ func _CHECK_COL_FOR_DISTINCT_WITH_NULLS( } // */}} func (ht *hashTable) checkColForDistinctTuples( - probeVec, buildVec coldata.Vec, probeType coltypes.T, nToCheck uint64, probeSel []int, + probeVec, buildVec coldata.Vec, probeType *types.T, nToCheck uint64, probeSel []int, ) { - switch probeType { + switch typeconv.FromColumnType(probeType) { // {{/* // index directive allows the template to index into indexable types such as // slices or maps. Following code is semantically equivalent to the Go code @@ -283,7 +286,7 @@ func (ht *hashTable) checkColForDistinctTuples( // {{end}} // {{end}} default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled type %d", probeType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", probeType)) } } @@ -344,7 +347,7 @@ func (ht *hashTable) checkBuildForDistinct( probeVecs []coldata.Vec, nToCheck uint64, probeSel []int, ) uint64 { if probeSel == nil { - execerror.VectorizedInternalPanic("invalid selection vector") + colexecerror.InternalError("invalid selection vector") } copy(ht.probeScratch.distinct, zeroBoolColumn) @@ -376,7 +379,7 @@ func (ht *hashTable) checkBuildForDistinct( // new length of toCheck is returned by this function. func (ht *hashTable) check( probeVecs []coldata.Vec, - probeKeyTypes []coltypes.T, + probeKeyTypes []types.T, buildKeyCols []uint32, nToCheck uint64, probeSel []int, @@ -393,7 +396,7 @@ func (ht *hashTable) check( // in the probe table. func (ht *hashTable) checkProbeForDistinct(vecs []coldata.Vec, nToCheck uint64, sel []int) uint64 { for i := range ht.keyCols { - ht.checkCol(vecs[i], vecs[i], ht.keyTypes[i], ht.keyTypes[i], i, nToCheck, sel, sel) + ht.checkCol(vecs[i], vecs[i], &ht.keyTypes[i], &ht.keyTypes[i], i, nToCheck, sel, sel) } nDiffers := uint64(0) @@ -460,7 +463,7 @@ func (ht *hashTable) updateSel(b coldata.Batch) { // list is reconstructed to only hold the indices of the eqCol keys that have // not been found. The new length of toCheck is returned by this function. func (ht *hashTable) distinctCheck( - probeKeyTypes []coltypes.T, nToCheck uint64, probeSel []int, + probeKeyTypes []types.T, nToCheck uint64, probeSel []int, ) uint64 { probeVecs := ht.probeScratch.keys buildVecs := ht.vals.ColVecs() diff --git a/pkg/sql/colexec/invariants_checker.go b/pkg/sql/colexec/invariants_checker.go index cd23902baa12..f585c29b05cc 100644 --- a/pkg/sql/colexec/invariants_checker.go +++ b/pkg/sql/colexec/invariants_checker.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" ) // invariantsChecker is a helper Operator that will check that invariants that @@ -24,10 +25,10 @@ type invariantsChecker struct { OneInputNode } -var _ Operator = invariantsChecker{} +var _ colexecbase.Operator = invariantsChecker{} // NewInvariantsChecker creates a new invariantsChecker. -func NewInvariantsChecker(input Operator) 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 2bd5a68a63f1..faaa03bd4985 100644 --- a/pkg/sql/colexec/is_null_ops.go +++ b/pkg/sql/colexec/is_null_ops.go @@ -14,7 +14,9 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/types" ) // isNullProjOp is an Operator that projects into outputIdx Vec whether the @@ -22,16 +24,16 @@ import ( // If negate is true, it does the opposite - it performs IS NOT NULL check. type isNullProjOp struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator colIdx int outputIdx int negate bool } func newIsNullProjOp( - allocator *Allocator, input Operator, colIdx, outputIdx int, negate bool, -) Operator { - input = newVectorTypeEnforcer(allocator, input, coltypes.Bool, outputIdx) + 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), allocator: allocator, @@ -41,7 +43,7 @@ func newIsNullProjOp( } } -var _ Operator = &isNullProjOp{} +var _ colexecbase.Operator = &isNullProjOp{} func (o *isNullProjOp) Init() { o.input.Init() @@ -95,7 +97,7 @@ type isNullSelOp struct { negate bool } -func newIsNullSelOp(input Operator, colIdx int, negate bool) Operator { +func newIsNullSelOp(input colexecbase.Operator, colIdx int, negate bool) colexecbase.Operator { return &isNullSelOp{ OneInputNode: NewOneInputNode(input), colIdx: colIdx, @@ -103,7 +105,7 @@ func newIsNullSelOp(input Operator, colIdx int, negate bool) Operator { } } -var _ 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 433aee230fd0..a066d7a657be 100644 --- a/pkg/sql/colexec/is_null_ops_test.go +++ b/pkg/sql/colexec/is_null_ops_test.go @@ -16,6 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "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" @@ -82,7 +83,7 @@ func TestIsNullProjOp(t *testing.T) { for _, c := range testCases { t.Run(c.desc, func(t *testing.T) { - opConstructor := func(input []Operator) (Operator, error) { + opConstructor := func(input []colexecbase.Operator) (colexecbase.Operator, error) { projExpr := "IS NULL" if c.negate { projExpr = "IS NOT NULL" @@ -156,7 +157,7 @@ func TestIsNullSelOp(t *testing.T) { for _, c := range testCases { t.Run(c.desc, func(t *testing.T) { - opConstructor := func(input []Operator) (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 3096caa3c7ab..51862686e098 100644 --- a/pkg/sql/colexec/like_ops.go +++ b/pkg/sql/colexec/like_ops.go @@ -13,8 +13,10 @@ package colexec import ( "strings" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "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" ) @@ -87,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 Operator, colIdx int, pattern string, negate bool, -) (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 @@ -172,20 +174,20 @@ 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 *Allocator, + allocator *colmem.Allocator, ctx *tree.EvalContext, - input Operator, + input colexecbase.Operator, colIdx int, resultIdx int, pattern string, negate bool, -) (Operator, error) { +) (colexecbase.Operator, error) { likeOpType, pattern, err := getLikeOperatorType(pattern, negate) if err != nil { return nil, err } pat := []byte(pattern) - input = newVectorTypeEnforcer(allocator, input, coltypes.Bool, resultIdx) + input = newVectorTypeEnforcer(allocator, input, types.Bool, resultIdx) base := projConstOpBase{ OneInputNode: NewOneInputNode(input), allocator: allocator, diff --git a/pkg/sql/colexec/like_ops_test.go b/pkg/sql/colexec/like_ops_test.go index 035a4fc7e3ce..290f8d584f1c 100644 --- a/pkg/sql/colexec/like_ops_test.go +++ b/pkg/sql/colexec/like_ops_test.go @@ -17,9 +17,10 @@ import ( "testing" "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/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/randutil" ) @@ -90,7 +91,7 @@ func TestLikeOperators(t *testing.T) { } { runTests( t, []tuples{tc.tups}, tc.expected, orderedVerifier, - func(input []Operator) (Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { ctx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) return GetLikeOperator(&ctx, input[0], 0, tc.pattern, tc.negate) }) @@ -101,7 +102,8 @@ func BenchmarkLikeOps(b *testing.B) { rng, _ := randutil.NewPseudoRand() ctx := context.Background() - batch := testAllocator.NewMemBatch([]coltypes.T{coltypes.Bytes}) + typs := []types.T{*types.Bytes} + batch := testAllocator.NewMemBatch(typs) col := batch.ColVec(0).Bytes() width := 64 for i := 0; i < coldata.BatchSize(); i++ { @@ -118,7 +120,7 @@ func BenchmarkLikeOps(b *testing.B) { } batch.SetLength(coldata.BatchSize()) - source := NewRepeatableBatchSource(testAllocator, batch) + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) source.Init() base := selConstOpBase{ @@ -141,7 +143,7 @@ func BenchmarkLikeOps(b *testing.B) { testCases := []struct { name string - op 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 2764198b07a5..ff1287a7713c 100644 --- a/pkg/sql/colexec/limit.go +++ b/pkg/sql/colexec/limit.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" ) // limitOp is an operator that implements limit, returning only the first n @@ -30,11 +31,11 @@ type limitOp struct { done bool } -var _ Operator = &limitOp{} +var _ colexecbase.Operator = &limitOp{} var _ closableOperator = &limitOp{} // NewLimitOp returns a new limit operator with the given limit. -func NewLimitOp(input Operator, limit int) 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 5687af4d8f2b..8b71838a9e3d 100644 --- a/pkg/sql/colexec/limit_test.go +++ b/pkg/sql/colexec/limit_test.go @@ -13,6 +13,7 @@ package colexec import ( "testing" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) @@ -63,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 []Operator) (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 95a9026012e8..3a678ea8eebe 100644 --- a/pkg/sql/colexec/main_test.go +++ b/pkg/sql/colexec/main_test.go @@ -18,7 +18,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" @@ -29,7 +30,7 @@ import ( var ( // testAllocator is an Allocator with an unlimited budget for use in tests. - testAllocator *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. @@ -50,7 +51,7 @@ func TestMain(m *testing.M) { defer testMemMonitor.Stop(ctx) memAcc := testMemMonitor.MakeBoundAccount() testMemAcc = &memAcc - testAllocator = NewAllocator(ctx, testMemAcc) + testAllocator = colmem.NewAllocator(ctx, testMemAcc) defer testMemAcc.Close(ctx) testDiskMonitor = execinfra.NewTestDiskMonitor(ctx, cluster.MakeTestingClusterSettings()) @@ -64,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return m.Run() }()) diff --git a/pkg/sql/colexec/materializer.go b/pkg/sql/colexec/materializer.go index c9f37df304e6..b463ec4d126f 100644 --- a/pkg/sql/colexec/materializer.go +++ b/pkg/sql/colexec/materializer.go @@ -15,7 +15,8 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" @@ -28,7 +29,7 @@ type Materializer struct { execinfra.ProcessorBase NonExplainable - input Operator + input colexecbase.Operator da sqlbase.DatumAlloc @@ -76,7 +77,7 @@ const materializerProcName = "materializer" func NewMaterializer( flowCtx *execinfra.FlowCtx, processorID int32, - input Operator, + input colexecbase.Operator, typs []types.T, post *execinfrapb.PostProcessSpec, output execinfra.RowReceiver, @@ -129,7 +130,7 @@ func (m *Materializer) Child(nth int, verbose bool) execinfra.OpNode { if nth == 0 { return m.input } - execerror.VectorizedInternalPanic(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 } @@ -181,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 := execerror.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 f02732b5c821..b75073426003 100644 --- a/pkg/sql/colexec/mergejoinbase_tmpl.go +++ b/pkg/sql/colexec/mergejoinbase_tmpl.go @@ -28,14 +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/colexec/execerror" - // {{/* + "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. @@ -69,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 { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} @@ -95,9 +97,9 @@ func (o *mergeJoinBase) isBufferedGroupFinished( // Check all equality columns in the first row of batch to make sure we're in // the same group. for _, colIdx := range input.eqCols[:len(input.eqCols)] { - colTyp := input.sourceTypes[colIdx] + typ := input.sourceTypes[colIdx] - switch colTyp { + switch typeconv.FromColumnType(&typ) { // {{ range . }} case _TYPES_T: // We perform this null check on every equality column of the first @@ -123,7 +125,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( } // {{end}} default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled type %d", colTyp)) + 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 ebfe15a3243b..634980fa7466 100644 --- a/pkg/sql/colexec/mergejoiner.go +++ b/pkg/sql/colexec/mergejoiner.go @@ -16,10 +16,14 @@ 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/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" @@ -188,17 +192,18 @@ type mergeJoinInput struct { // sourceTypes specify the types of the input columns of the source table for // the merge joiner. - sourceTypes []coltypes.T + sourceTypes []types.T + sourcePhysTypes []coltypes.T // The distincter is used in the finishGroup phase, and is used only to // determine where the current group ends, in the case that the group ended // with a batch. distincterInput *feedOperator - distincter Operator + distincter colexecbase.Operator distinctOutput []bool // source specifies the input operator to the merge join. - source Operator + source colexecbase.Operator } // The merge join operator uses a probe and build approach to generate the @@ -221,15 +226,15 @@ type mergeJoinInput struct { // sources, based on the equality columns, assuming both inputs are in sorted // order. func newMergeJoinOp( - unlimitedAllocator *Allocator, + unlimitedAllocator *colmem.Allocator, memoryLimit int64, diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, joinType sqlbase.JoinType, - left Operator, - right Operator, - leftTypes []coltypes.T, - rightTypes []coltypes.T, + left colexecbase.Operator, + right colexecbase.Operator, + leftTypes []types.T, + rightTypes []types.T, leftOrdering []execinfrapb.Ordering_Column, rightOrdering []execinfrapb.Ordering_Column, diskAcc *mon.BoundAccount, @@ -284,15 +289,15 @@ func (s *mjBuilderCrossProductState) setBuilderColumnState(target mjBuilderCross } func newMergeJoinBase( - unlimitedAllocator *Allocator, + unlimitedAllocator *colmem.Allocator, memoryLimit int64, diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, joinType sqlbase.JoinType, - left Operator, - right Operator, - leftTypes []coltypes.T, - rightTypes []coltypes.T, + left colexecbase.Operator, + right colexecbase.Operator, + leftTypes []types.T, + rightTypes []types.T, leftOrdering []execinfrapb.Ordering_Column, rightOrdering []execinfrapb.Ordering_Column, diskAcc *mon.BoundAccount, @@ -313,6 +318,14 @@ func newMergeJoinBase( diskQueueCfg.CacheMode = colcontainer.DiskQueueCacheModeReuseCache diskQueueCfg.SetDefaultBufferSizeBytesForCacheMode() + leftPhysTypes, err := typeconv.FromColumnTypes(leftTypes) + if err != nil { + return nil, err + } + rightPhysTypes, err := typeconv.FromColumnTypes(rightTypes) + if err != nil { + return nil, err + } base := &mergeJoinBase{ twoInputNode: newTwoInputNode(left, right), unlimitedAllocator: unlimitedAllocator, @@ -321,20 +334,21 @@ func newMergeJoinBase( fdSemaphore: fdSemaphore, joinType: joinType, left: mergeJoinInput{ - source: left, - sourceTypes: leftTypes, - eqCols: lEqCols, - directions: lDirections, + source: left, + sourceTypes: leftTypes, + sourcePhysTypes: leftPhysTypes, + eqCols: lEqCols, + directions: lDirections, }, right: mergeJoinInput{ - source: right, - sourceTypes: rightTypes, - eqCols: rEqCols, - directions: rDirections, + source: right, + sourceTypes: rightTypes, + sourcePhysTypes: rightPhysTypes, + eqCols: rEqCols, + directions: rDirections, }, diskAcc: diskAcc, } - var err error base.left.distincterInput = &feedOperator{} base.left.distincter, base.left.distinctOutput, err = OrderedDistinctColsToOperators( base.left.distincterInput, lEqCols, leftTypes) @@ -362,7 +376,7 @@ type mergeJoinBase struct { // Next, which will simplify this model. mu syncutil.Mutex - unlimitedAllocator *Allocator + unlimitedAllocator *colmem.Allocator memoryLimit int64 diskQueueCfg colcontainer.DiskQueueCfg fdSemaphore semaphore.Semaphore @@ -432,7 +446,7 @@ func (o *mergeJoinBase) Init() { } func (o *mergeJoinBase) initWithOutputBatchSize(outBatchSize int) { - outputTypes := append([]coltypes.T{}, o.left.sourceTypes...) + outputTypes := append([]types.T{}, o.left.sourceTypes...) if o.joinType != sqlbase.LeftSemiJoin && o.joinType != sqlbase.LeftAntiJoin { outputTypes = append(outputTypes, o.right.sourceTypes...) } @@ -453,16 +467,16 @@ func (o *mergeJoinBase) initWithOutputBatchSize(outBatchSize int) { o.diskQueueCfg, o.fdSemaphore, coldata.BatchSize(), o.diskAcc, ) o.proberState.lBufferedGroup.firstTuple = make([]coldata.Vec, len(o.left.sourceTypes)) - for colIdx, colType := range o.left.sourceTypes { - o.proberState.lBufferedGroup.firstTuple[colIdx] = o.unlimitedAllocator.NewMemColumn(colType, 1) + for colIdx, typ := range o.left.sourceTypes { + o.proberState.lBufferedGroup.firstTuple[colIdx] = o.unlimitedAllocator.NewMemColumn(&typ, 1) } o.proberState.rBufferedGroup.spillingQueue = newRewindableSpillingQueue( o.unlimitedAllocator, o.right.sourceTypes, o.memoryLimit, o.diskQueueCfg, o.fdSemaphore, coldata.BatchSize(), o.diskAcc, ) o.proberState.rBufferedGroup.firstTuple = make([]coldata.Vec, len(o.right.sourceTypes)) - for colIdx, colType := range o.right.sourceTypes { - o.proberState.rBufferedGroup.firstTuple[colIdx] = o.unlimitedAllocator.NewMemColumn(colType, 1) + for colIdx, typ := range o.right.sourceTypes { + o.proberState.rBufferedGroup.firstTuple[colIdx] = o.unlimitedAllocator.NewMemColumn(&typ, 1) } o.builderState.lGroups = make([]group, 1) @@ -495,7 +509,7 @@ func (o *mergeJoinBase) appendToBufferedGroup( var ( bufferedGroup *mjBufferedGroup scratchBatch coldata.Batch - sourceTypes []coltypes.T + sourceTypes []types.T ) if input == &o.left { sourceTypes = o.left.sourceTypes @@ -519,11 +533,11 @@ func (o *mergeJoinBase) appendToBufferedGroup( scratchBatch = o.unlimitedAllocator.NewMemBatchWithSize(sourceTypes, groupLength) if bufferedGroup.numTuples == 0 { o.unlimitedAllocator.PerformOperation(bufferedGroup.firstTuple, func() { - for colIdx, colType := range sourceTypes { + for colIdx, typ := range sourceTypes { bufferedGroup.firstTuple[colIdx].Copy( coldata.CopySliceArgs{ SliceArgs: coldata.SliceArgs{ - ColType: colType, + ColType: typeconv.FromColumnType(&typ), Src: batch.ColVec(colIdx), Sel: sel, DestIdx: 0, @@ -538,12 +552,12 @@ func (o *mergeJoinBase) appendToBufferedGroup( bufferedGroup.numTuples += groupLength o.unlimitedAllocator.PerformOperation(scratchBatch.ColVecs(), func() { - for cIdx, cType := range input.sourceTypes { - scratchBatch.ColVec(cIdx).Copy( + for colIdx, typ := range input.sourceTypes { + scratchBatch.ColVec(colIdx).Copy( coldata.CopySliceArgs{ SliceArgs: coldata.SliceArgs{ - ColType: cType, - Src: batch.ColVec(cIdx), + ColType: typeconv.FromColumnType(&typ), + Src: batch.ColVec(colIdx), Sel: sel, DestIdx: 0, SrcStartIdx: groupStartIdx, @@ -556,7 +570,7 @@ func (o *mergeJoinBase) appendToBufferedGroup( scratchBatch.SetSelection(false) scratchBatch.SetLength(groupLength) if err := bufferedGroup.enqueue(ctx, scratchBatch); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } } @@ -704,7 +718,7 @@ func (o *mergeJoinBase) IdempotentClose(ctx context.Context) error { return nil } var lastErr error - for _, op := range []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 9e8e8ba4455f..11c878844f3e 100644 --- a/pkg/sql/colexec/mergejoiner_test.go +++ b/pkg/sql/colexec/mergejoiner_test.go @@ -16,12 +16,15 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "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" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils/colcontainerutils" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -59,14 +62,14 @@ func createSpecForMergeJoiner(tc *joinTestCase) *execinfrapb.ProcessorSpec { } projection := make([]uint32, 0, len(tc.leftOutCols)+len(tc.rightOutCols)) projection = append(projection, tc.leftOutCols...) - rColOffset := uint32(len(tc.leftPhysTypes)) + rColOffset := uint32(len(tc.leftTypes)) for _, outCol := range tc.rightOutCols { projection = append(projection, rColOffset+outCol) } return &execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{ - {ColumnTypes: tc.leftLogTypes}, - {ColumnTypes: tc.rightLogTypes}, + {ColumnTypes: tc.leftTypes}, + {ColumnTypes: tc.rightTypes}, }, Core: execinfrapb.ProcessorCoreUnion{ MergeJoiner: mjSpec, @@ -80,153 +83,153 @@ func createSpecForMergeJoiner(tc *joinTestCase) *execinfrapb.ProcessorSpec { var mjTestCases = []*joinTestCase{ { - description: "basic test", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {2}, {3}, {4}}, - rightTuples: tuples{{1}, {2}, {3}, {4}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {2}, {3}, {4}}, - }, - { - description: "basic test, no out cols", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {2}, {3}, {4}}, - rightTuples: tuples{{1}, {2}, {3}, {4}}, - leftOutCols: []uint32{}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{}, {}, {}, {}}, - }, - { - description: "basic test, out col on left", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {2}, {3}, {4}}, - rightTuples: tuples{{1}, {2}, {3}, {4}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {2}, {3}, {4}}, - }, - { - description: "basic test, out col on right", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {2}, {3}, {4}}, - rightTuples: tuples{{1}, {2}, {3}, {4}}, - leftOutCols: []uint32{}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {2}, {3}, {4}}, - }, - { - description: "basic test, L missing", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {3}, {4}}, - rightTuples: tuples{{1}, {2}, {3}, {4}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {3}, {4}}, - }, - { - description: "basic test, R missing", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {2}, {3}, {4}}, - rightTuples: tuples{{1}, {3}, {4}}, - leftOutCols: []uint32{}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {3}, {4}}, - }, - { - description: "basic test, L duplicate", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {1}, {2}, {3}, {4}}, - rightTuples: tuples{{1}, {2}, {3}, {4}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {1}, {2}, {3}, {4}}, - }, - { - description: "basic test, R duplicate", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {2}, {3}, {4}}, - rightTuples: tuples{{1}, {1}, {2}, {3}, {4}}, - leftOutCols: []uint32{}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {1}, {2}, {3}, {4}}, - }, - { - description: "basic test, R duplicate 2", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {2}}, - rightTuples: tuples{{1}, {1}, {2}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {1}, {2}}, - }, - { - description: "basic test, L+R duplicates", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {1}, {2}, {3}, {4}}, - rightTuples: tuples{{1}, {1}, {2}, {3}, {4}}, - leftOutCols: []uint32{}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {1}, {1}, {1}, {2}, {3}, {4}}, - }, - { - description: "basic test, L+R duplicate, multiple runs", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {2}, {2}, {2}, {3}, {4}}, - rightTuples: tuples{{1}, {1}, {2}, {3}, {4}}, - leftOutCols: []uint32{}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {1}, {2}, {2}, {2}, {3}, {4}}, - }, - { - description: "cross product test, batch size = col.BatchSize()", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {1}, {1}, {1}}, - rightTuples: tuples{{1}, {1}, {1}, {1}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}}, + description: "basic test", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {2}, {3}, {4}}, + rightTuples: tuples{{1}, {2}, {3}, {4}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1}, {2}, {3}, {4}}, + }, + { + description: "basic test, no out cols", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {2}, {3}, {4}}, + rightTuples: tuples{{1}, {2}, {3}, {4}}, + leftOutCols: []uint32{}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{}, {}, {}, {}}, + }, + { + description: "basic test, out col on left", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {2}, {3}, {4}}, + rightTuples: tuples{{1}, {2}, {3}, {4}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1}, {2}, {3}, {4}}, + }, + { + description: "basic test, out col on right", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {2}, {3}, {4}}, + rightTuples: tuples{{1}, {2}, {3}, {4}}, + leftOutCols: []uint32{}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1}, {2}, {3}, {4}}, + }, + { + description: "basic test, L missing", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {3}, {4}}, + rightTuples: tuples{{1}, {2}, {3}, {4}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1}, {3}, {4}}, + }, + { + description: "basic test, R missing", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {2}, {3}, {4}}, + rightTuples: tuples{{1}, {3}, {4}}, + leftOutCols: []uint32{}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1}, {3}, {4}}, + }, + { + description: "basic test, L duplicate", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {1}, {2}, {3}, {4}}, + rightTuples: tuples{{1}, {2}, {3}, {4}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1}, {1}, {2}, {3}, {4}}, + }, + { + description: "basic test, R duplicate", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {2}, {3}, {4}}, + rightTuples: tuples{{1}, {1}, {2}, {3}, {4}}, + leftOutCols: []uint32{}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1}, {1}, {2}, {3}, {4}}, + }, + { + description: "basic test, R duplicate 2", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {2}}, + rightTuples: tuples{{1}, {1}, {2}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1}, {1}, {2}}, + }, + { + description: "basic test, L+R duplicates", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {1}, {2}, {3}, {4}}, + rightTuples: tuples{{1}, {1}, {2}, {3}, {4}}, + leftOutCols: []uint32{}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1}, {1}, {1}, {1}, {2}, {3}, {4}}, + }, + { + description: "basic test, L+R duplicate, multiple runs", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {2}, {2}, {2}, {3}, {4}}, + rightTuples: tuples{{1}, {1}, {2}, {3}, {4}}, + leftOutCols: []uint32{}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1}, {1}, {2}, {2}, {2}, {3}, {4}}, + }, + { + description: "cross product test, batch size = col.BatchSize()", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {1}, {1}, {1}}, + rightTuples: tuples{{1}, {1}, {1}, {1}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}}, }, { description: "cross product test, batch size = 4 (small even)", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, leftTuples: tuples{{1}, {1}, {1}, {1}}, rightTuples: tuples{{1}, {1}, {1}, {1}}, leftOutCols: []uint32{0}, @@ -238,8 +241,8 @@ var mjTestCases = []*joinTestCase{ }, { description: "cross product test, batch size = 3 (small odd)", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, leftTuples: tuples{{1}, {1}, {1}, {1}}, rightTuples: tuples{{1}, {1}, {1}, {1}}, leftOutCols: []uint32{}, @@ -251,8 +254,8 @@ var mjTestCases = []*joinTestCase{ }, { description: "cross product test, batch size = 1 (unit)", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, leftTuples: tuples{{1}, {1}, {1}, {1}}, rightTuples: tuples{{1}, {1}, {1}, {1}}, leftOutCols: []uint32{}, @@ -263,21 +266,21 @@ var mjTestCases = []*joinTestCase{ outputBatchSize: 1, }, { - description: "multi output column test, basic", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 10}, {2, 20}, {3, 30}, {4, 40}}, - rightTuples: tuples{{1, 11}, {2, 12}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1, 10, 1, 11}, {2, 20, 2, 12}, {3, 30, 3, 13}, {4, 40, 4, 14}}, + description: "multi output column test, basic", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 10}, {2, 20}, {3, 30}, {4, 40}}, + rightTuples: tuples{{1, 11}, {2, 12}, {3, 13}, {4, 14}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1, 10, 1, 11}, {2, 20, 2, 12}, {3, 30, 3, 13}, {4, 40, 4, 14}}, }, { description: "multi output column test, batch size = 1", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, leftTuples: tuples{{1, 10}, {2, 20}, {3, 30}, {4, 40}}, rightTuples: tuples{{1, 11}, {2, 12}, {3, 13}, {4, 14}}, leftOutCols: []uint32{0, 1}, @@ -288,45 +291,45 @@ var mjTestCases = []*joinTestCase{ outputBatchSize: 1, }, { - description: "multi output column test, test output coldata projection", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 10}, {2, 20}, {3, 30}, {4, 40}}, - rightTuples: tuples{{1, 11}, {2, 12}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1, 1}, {2, 2}, {3, 3}, {4, 4}}, - }, - { - description: "multi output column test, test output coldata projection", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 10}, {2, 20}, {3, 30}, {4, 40}}, - rightTuples: tuples{{1, 11}, {2, 12}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{1}, - rightOutCols: []uint32{1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{10, 11}, {20, 12}, {30, 13}, {40, 14}}, - }, - { - description: "multi output column test, L run", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 10}, {2, 20}, {2, 21}, {3, 30}, {4, 40}}, - rightTuples: tuples{{1, 11}, {2, 12}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1, 10, 1, 11}, {2, 20, 2, 12}, {2, 21, 2, 12}, {3, 30, 3, 13}, {4, 40, 4, 14}}, + description: "multi output column test, test output coldata projection", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 10}, {2, 20}, {3, 30}, {4, 40}}, + rightTuples: tuples{{1, 11}, {2, 12}, {3, 13}, {4, 14}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1, 1}, {2, 2}, {3, 3}, {4, 4}}, + }, + { + description: "multi output column test, test output coldata projection", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 10}, {2, 20}, {3, 30}, {4, 40}}, + rightTuples: tuples{{1, 11}, {2, 12}, {3, 13}, {4, 14}}, + leftOutCols: []uint32{1}, + rightOutCols: []uint32{1}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{10, 11}, {20, 12}, {30, 13}, {40, 14}}, + }, + { + description: "multi output column test, L run", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 10}, {2, 20}, {2, 21}, {3, 30}, {4, 40}}, + rightTuples: tuples{{1, 11}, {2, 12}, {3, 13}, {4, 14}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1, 10, 1, 11}, {2, 20, 2, 12}, {2, 21, 2, 12}, {3, 30, 3, 13}, {4, 40, 4, 14}}, }, { description: "multi output column test, L run, batch size = 1", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, leftTuples: tuples{{1, 10}, {2, 20}, {2, 21}, {3, 30}, {4, 40}}, rightTuples: tuples{{1, 11}, {2, 12}, {3, 13}, {4, 14}}, leftOutCols: []uint32{0, 1}, @@ -337,21 +340,21 @@ var mjTestCases = []*joinTestCase{ outputBatchSize: 1, }, { - description: "multi output column test, R run", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 10}, {2, 20}, {3, 30}, {4, 40}}, - rightTuples: tuples{{1, 11}, {1, 111}, {2, 12}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1, 10, 1, 11}, {1, 10, 1, 111}, {2, 20, 2, 12}, {3, 30, 3, 13}, {4, 40, 4, 14}}, + description: "multi output column test, R run", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 10}, {2, 20}, {3, 30}, {4, 40}}, + rightTuples: tuples{{1, 11}, {1, 111}, {2, 12}, {3, 13}, {4, 14}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1, 10, 1, 11}, {1, 10, 1, 111}, {2, 20, 2, 12}, {3, 30, 3, 13}, {4, 40, 4, 14}}, }, { description: "multi output column test, R run, batch size = 1", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, leftTuples: tuples{{1, 10}, {2, 20}, {3, 30}, {4, 40}}, rightTuples: tuples{{1, 11}, {1, 111}, {2, 12}, {3, 13}, {4, 14}}, leftOutCols: []uint32{0, 1}, @@ -362,27 +365,27 @@ var mjTestCases = []*joinTestCase{ outputBatchSize: 1, }, { - description: "logic test", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{-1, -1}, {0, 4}, {2, 1}, {3, 4}, {5, 4}}, - rightTuples: tuples{{0, 5}, {1, 3}, {3, 2}, {4, 6}}, - leftOutCols: []uint32{1}, - rightOutCols: []uint32{1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{4, 5}, {4, 2}}, - }, - { - description: "multi output column test, batch size = 1 and runs (to test saved output), reordered out columns", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 10}, {1, 10}, {1, 10}, {2, 20}, {3, 30}, {4, 40}}, - rightTuples: tuples{{1, 11}, {1, 11}, {2, 12}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{1, 0}, - rightOutCols: []uint32{1, 0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, + description: "logic test", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{-1, -1}, {0, 4}, {2, 1}, {3, 4}, {5, 4}}, + rightTuples: tuples{{0, 5}, {1, 3}, {3, 2}, {4, 6}}, + leftOutCols: []uint32{1}, + rightOutCols: []uint32{1}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{4, 5}, {4, 2}}, + }, + { + description: "multi output column test, batch size = 1 and runs (to test saved output), reordered out columns", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 10}, {1, 10}, {1, 10}, {2, 20}, {3, 30}, {4, 40}}, + rightTuples: tuples{{1, 11}, {1, 11}, {2, 12}, {3, 13}, {4, 14}}, + leftOutCols: []uint32{1, 0}, + rightOutCols: []uint32{1, 0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, expected: tuples{ {10, 1, 11, 1}, {10, 1, 11, 1}, @@ -397,15 +400,15 @@ var mjTestCases = []*joinTestCase{ outputBatchSize: 1, }, { - description: "multi output column test, batch size = 1 and runs (to test saved output), reordered out columns that dont start at 0", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 10}, {1, 10}, {1, 10}, {2, 20}, {3, 30}, {4, 40}}, - rightTuples: tuples{{1, 11}, {1, 11}, {2, 12}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{1, 0}, - rightOutCols: []uint32{1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, + description: "multi output column test, batch size = 1 and runs (to test saved output), reordered out columns that dont start at 0", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 10}, {1, 10}, {1, 10}, {2, 20}, {3, 30}, {4, 40}}, + rightTuples: tuples{{1, 11}, {1, 11}, {2, 12}, {3, 13}, {4, 14}}, + leftOutCols: []uint32{1, 0}, + rightOutCols: []uint32{1}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, expected: tuples{ {10, 1, 11}, {10, 1, 11}, @@ -420,15 +423,15 @@ var mjTestCases = []*joinTestCase{ outputBatchSize: 1, }, { - description: "equality column is correctly indexed", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{10, 1}, {10, 1}, {10, 1}, {20, 2}, {30, 3}, {40, 4}}, - rightTuples: tuples{{1, 11}, {1, 11}, {2, 12}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{1, 0}, - rightOutCols: []uint32{1}, - leftEqCols: []uint32{1}, - rightEqCols: []uint32{0}, + description: "equality column is correctly indexed", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{10, 1}, {10, 1}, {10, 1}, {20, 2}, {30, 3}, {40, 4}}, + rightTuples: tuples{{1, 11}, {1, 11}, {2, 12}, {3, 13}, {4, 14}}, + leftOutCols: []uint32{1, 0}, + rightOutCols: []uint32{1}, + leftEqCols: []uint32{1}, + rightEqCols: []uint32{0}, expected: tuples{ {1, 10, 11}, {1, 10, 11}, @@ -442,30 +445,30 @@ var mjTestCases = []*joinTestCase{ }, }, { - description: "multi column equality basic test", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 10}, {2, 20}, {3, 30}, {4, 40}}, - rightTuples: tuples{{1, 10}, {2, 20}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, + description: "multi column equality basic test", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 10}, {2, 20}, {3, 30}, {4, 40}}, + rightTuples: tuples{{1, 10}, {2, 20}, {3, 13}, {4, 14}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, expected: tuples{ {1, 10, 1, 10}, {2, 20, 2, 20}, }, }, { - description: "multi column equality runs", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 10}, {1, 10}, {1, 10}, {2, 20}, {3, 30}, {4, 40}}, - rightTuples: tuples{{1, 10}, {1, 10}, {2, 20}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, + description: "multi column equality runs", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 10}, {1, 10}, {1, 10}, {2, 20}, {3, 30}, {4, 40}}, + rightTuples: tuples{{1, 10}, {1, 10}, {2, 20}, {3, 13}, {4, 14}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, expected: tuples{ {1, 10, 1, 10}, {1, 10, 1, 10}, @@ -477,30 +480,30 @@ var mjTestCases = []*joinTestCase{ }, }, { - description: "multi column non-consecutive equality cols", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 123, 1}, {1, 234, 10}}, - rightTuples: tuples{{1, 1, 345}, {1, 10, 456}}, - leftOutCols: []uint32{0, 2, 1}, - rightOutCols: []uint32{0, 2, 1}, - leftEqCols: []uint32{0, 2}, - rightEqCols: []uint32{0, 1}, + description: "multi column non-consecutive equality cols", + leftTypes: []types.T{*types.Int, *types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int, *types.Int}, + leftTuples: tuples{{1, 123, 1}, {1, 234, 10}}, + rightTuples: tuples{{1, 1, 345}, {1, 10, 456}}, + leftOutCols: []uint32{0, 2, 1}, + rightOutCols: []uint32{0, 2, 1}, + leftEqCols: []uint32{0, 2}, + rightEqCols: []uint32{0, 1}, expected: tuples{ {1, 1, 123, 1, 345, 1}, {1, 10, 234, 1, 456, 10}, }, }, { - description: "multi column equality: new batch ends run", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 1}, {1, 1}, {3, 3}, {4, 3}}, - rightTuples: tuples{{1, 1}, {1, 2}, {3, 3}, {3, 3}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, + description: "multi column equality: new batch ends run", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 1}, {1, 1}, {3, 3}, {4, 3}}, + rightTuples: tuples{{1, 1}, {1, 2}, {3, 3}, {3, 3}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, expected: tuples{ {1, 1, 1, 1}, {1, 1, 1, 1}, @@ -509,15 +512,15 @@ var mjTestCases = []*joinTestCase{ }, }, { - description: "multi column equality: reordered eq columns", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 1}, {1, 1}, {3, 3}, {4, 3}}, - rightTuples: tuples{{1, 1}, {1, 2}, {3, 3}, {3, 3}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{1, 0}, + description: "multi column equality: reordered eq columns", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 1}, {1, 1}, {3, 3}, {4, 3}}, + rightTuples: tuples{{1, 1}, {1, 2}, {3, 3}, {3, 3}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{1, 0}, expected: tuples{ {1, 1, 1, 1}, {1, 1, 1, 1}, @@ -526,15 +529,15 @@ var mjTestCases = []*joinTestCase{ }, }, { - description: "cross batch, distinct group", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 2}, {1, 2}, {1, 2}, {2, 2}}, - rightTuples: tuples{{1, 2}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, + description: "cross batch, distinct group", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 2}, {1, 2}, {1, 2}, {2, 2}}, + rightTuples: tuples{{1, 2}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, expected: tuples{ {1, 2, 1, 2}, {1, 2, 1, 2}, @@ -542,29 +545,29 @@ var mjTestCases = []*joinTestCase{ }, }, { - description: "templating basic test", - leftPhysTypes: []coltypes.T{coltypes.Bool, coltypes.Int16, coltypes.Float64}, - rightPhysTypes: []coltypes.T{coltypes.Bool, coltypes.Int16, coltypes.Float64}, - leftTuples: tuples{{true, int16(10), 1.2}, {true, int16(20), 2.2}, {true, int16(30), 3.2}}, - rightTuples: tuples{{true, int16(10), 1.2}, {false, int16(20), 2.2}, {true, int16(30), 3.9}}, - leftOutCols: []uint32{0, 1, 2}, - rightOutCols: []uint32{0, 1, 2}, - leftEqCols: []uint32{0, 1, 2}, - rightEqCols: []uint32{0, 1, 2}, + description: "templating basic test", + leftTypes: []types.T{*types.Bool, *types.Int2, *types.Float}, + rightTypes: []types.T{*types.Bool, *types.Int2, *types.Float}, + leftTuples: tuples{{true, int16(10), 1.2}, {true, int16(20), 2.2}, {true, int16(30), 3.2}}, + rightTuples: tuples{{true, int16(10), 1.2}, {false, int16(20), 2.2}, {true, int16(30), 3.9}}, + leftOutCols: []uint32{0, 1, 2}, + rightOutCols: []uint32{0, 1, 2}, + leftEqCols: []uint32{0, 1, 2}, + rightEqCols: []uint32{0, 1, 2}, expected: tuples{ {true, 10, 1.2, true, 10, 1.2}, }, }, { - description: "templating cross product test", - leftPhysTypes: []coltypes.T{coltypes.Bool, coltypes.Int16, coltypes.Float64}, - rightPhysTypes: []coltypes.T{coltypes.Bool, coltypes.Int16, coltypes.Float64}, - leftTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.2}, {true, int16(30), 3.2}}, - rightTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.3}, {true, int16(20), 2.4}, {true, int16(31), 3.9}}, - leftOutCols: []uint32{0, 1, 2}, - rightOutCols: []uint32{0, 1, 2}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, + description: "templating cross product test", + leftTypes: []types.T{*types.Bool, *types.Int2, *types.Float}, + rightTypes: []types.T{*types.Bool, *types.Int2, *types.Float}, + leftTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.2}, {true, int16(30), 3.2}}, + rightTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.3}, {true, int16(20), 2.4}, {true, int16(31), 3.9}}, + leftOutCols: []uint32{0, 1, 2}, + rightOutCols: []uint32{0, 1, 2}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, expected: tuples{ {false, 10, 1.2, false, 10, 1.2}, {true, 20, 2.2, true, 20, 2.3}, @@ -572,15 +575,15 @@ var mjTestCases = []*joinTestCase{ }, }, { - description: "templating cross product test, output batch size 1", - leftPhysTypes: []coltypes.T{coltypes.Bool, coltypes.Int16, coltypes.Float64}, - rightPhysTypes: []coltypes.T{coltypes.Bool, coltypes.Int16, coltypes.Float64}, - leftTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.2}, {true, int16(30), 3.2}}, - rightTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.3}, {true, int16(20), 2.4}, {true, int16(31), 3.9}}, - leftOutCols: []uint32{0, 1, 2}, - rightOutCols: []uint32{0, 1, 2}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, + description: "templating cross product test, output batch size 1", + leftTypes: []types.T{*types.Bool, *types.Int2, *types.Float}, + rightTypes: []types.T{*types.Bool, *types.Int2, *types.Float}, + leftTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.2}, {true, int16(30), 3.2}}, + rightTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.3}, {true, int16(20), 2.4}, {true, int16(31), 3.9}}, + leftOutCols: []uint32{0, 1, 2}, + rightOutCols: []uint32{0, 1, 2}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, expected: tuples{ {false, 10, 1.2, false, 10, 1.2}, {true, 20, 2.2, true, 20, 2.3}, @@ -589,15 +592,15 @@ var mjTestCases = []*joinTestCase{ outputBatchSize: 1, }, { - description: "templating cross product test, output batch size 2", - leftPhysTypes: []coltypes.T{coltypes.Bool, coltypes.Int16, coltypes.Float64}, - rightPhysTypes: []coltypes.T{coltypes.Bool, coltypes.Int16, coltypes.Float64}, - leftTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.2}, {true, int16(30), 3.2}}, - rightTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.3}, {true, int16(20), 2.4}, {true, int16(31), 3.9}}, - leftOutCols: []uint32{0, 1, 2}, - rightOutCols: []uint32{0, 1, 2}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, + description: "templating cross product test, output batch size 2", + leftTypes: []types.T{*types.Bool, *types.Int2, *types.Float}, + rightTypes: []types.T{*types.Bool, *types.Int2, *types.Float}, + leftTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.2}, {true, int16(30), 3.2}}, + rightTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.3}, {true, int16(20), 2.4}, {true, int16(31), 3.9}}, + leftOutCols: []uint32{0, 1, 2}, + rightOutCols: []uint32{0, 1, 2}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, expected: tuples{ {false, 10, 1.2, false, 10, 1.2}, {true, 20, 2.2, true, 20, 2.3}, @@ -606,15 +609,15 @@ var mjTestCases = []*joinTestCase{ outputBatchSize: 2, }, { - description: "templating reordered eq columns", - leftPhysTypes: []coltypes.T{coltypes.Bool, coltypes.Int16, coltypes.Float64}, - rightPhysTypes: []coltypes.T{coltypes.Bool, coltypes.Int16, coltypes.Float64}, - leftTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.2}, {true, int16(30), 3.2}}, - rightTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.3}, {true, int16(20), 2.4}, {true, int16(31), 3.9}}, - leftOutCols: []uint32{0, 1, 2}, - rightOutCols: []uint32{0, 1, 2}, - leftEqCols: []uint32{1, 0}, - rightEqCols: []uint32{1, 0}, + description: "templating reordered eq columns", + leftTypes: []types.T{*types.Bool, *types.Int2, *types.Float}, + rightTypes: []types.T{*types.Bool, *types.Int2, *types.Float}, + leftTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.2}, {true, int16(30), 3.2}}, + rightTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.3}, {true, int16(20), 2.4}, {true, int16(31), 3.9}}, + leftOutCols: []uint32{0, 1, 2}, + rightOutCols: []uint32{0, 1, 2}, + leftEqCols: []uint32{1, 0}, + rightEqCols: []uint32{1, 0}, expected: tuples{ {false, 10, 1.2, false, 10, 1.2}, {true, 20, 2.2, true, 20, 2.3}, @@ -622,15 +625,15 @@ var mjTestCases = []*joinTestCase{ }, }, { - description: "templating reordered eq columns non symmetrical", - leftPhysTypes: []coltypes.T{coltypes.Bool, coltypes.Int16, coltypes.Float64}, - rightPhysTypes: []coltypes.T{coltypes.Int16, coltypes.Float64, coltypes.Bool}, - leftTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.2}, {true, int16(30), 3.2}}, - rightTuples: tuples{{int16(10), 1.2, false}, {int16(20), 2.2, true}, {int16(21), 2.2, true}, {int16(30), 3.2, false}}, - leftOutCols: []uint32{0, 1, 2}, - rightOutCols: []uint32{0, 1, 2}, - leftEqCols: []uint32{2, 0}, - rightEqCols: []uint32{1, 2}, + description: "templating reordered eq columns non symmetrical", + leftTypes: []types.T{*types.Bool, *types.Int2, *types.Float}, + rightTypes: []types.T{*types.Int2, *types.Float, *types.Bool}, + leftTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.2}, {true, int16(30), 3.2}}, + rightTuples: tuples{{int16(10), 1.2, false}, {int16(20), 2.2, true}, {int16(21), 2.2, true}, {int16(30), 3.2, false}}, + leftOutCols: []uint32{0, 1, 2}, + rightOutCols: []uint32{0, 1, 2}, + leftEqCols: []uint32{2, 0}, + rightEqCols: []uint32{1, 2}, expected: tuples{ {false, 10, 1.2, 10, 1.2, false}, {true, 20, 2.2, 20, 2.2, true}, @@ -638,200 +641,200 @@ var mjTestCases = []*joinTestCase{ }, }, { - description: "null handling", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{nil}, {0}}, - rightTuples: tuples{{nil}, {0}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, + description: "null handling", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{nil}, {0}}, + rightTuples: tuples{{nil}, {0}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, expected: tuples{ {0, 0}, }, }, { - description: "null handling multi column, nulls on left", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{nil, 0}, {0, nil}}, - rightTuples: tuples{{nil, nil}, {0, 1}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, + description: "null handling multi column, nulls on left", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{nil, 0}, {0, nil}}, + rightTuples: tuples{{nil, nil}, {0, 1}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, expected: tuples{ {0, nil, 0, 1}, }, }, { - description: "null handling multi column, nulls on right", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{nil, 0}, {0, 1}}, - rightTuples: tuples{{nil, nil}, {0, nil}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, + description: "null handling multi column, nulls on right", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{nil, 0}, {0, 1}}, + rightTuples: tuples{{nil, nil}, {0, nil}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, expected: tuples{ {0, 1, 0, nil}, }, }, { - description: "desc test", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{4}, {3}, {2}, {1}}, - rightTuples: tuples{{4}, {2}, {1}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{4, 4}, {2, 2}, {1, 1}}, + description: "desc test", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{4}, {3}, {2}, {1}}, + rightTuples: tuples{{4}, {2}, {1}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{4, 4}, {2, 2}, {1, 1}}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, }, { - description: "desc nulls test", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{4}, {3}, {nil}, {1}}, - rightTuples: tuples{{4}, {nil}, {2}, {1}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{4, 4}, {1, 1}}, + description: "desc nulls test", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{4}, {3}, {nil}, {1}}, + rightTuples: tuples{{4}, {nil}, {2}, {1}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{4, 4}, {1, 1}}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, }, { - description: "desc nulls test end on 0", - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{9}, {9}, {8}, {0}, {nil}}, - rightTuples: tuples{{9}, {9}, {8}, {0}, {nil}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{9, 9}, {9, 9}, {9, 9}, {9, 9}, {8, 8}, {0, 0}}, + description: "desc nulls test end on 0", + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{9}, {9}, {8}, {0}, {nil}}, + rightTuples: tuples{{9}, {9}, {8}, {0}, {nil}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{9, 9}, {9, 9}, {9, 9}, {9, 9}, {8, 8}, {0, 0}}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, }, { - description: "non-equality columns with nulls", - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, nil}, {2, 2}, {2, 2}, {3, nil}, {4, nil}}, - rightTuples: tuples{{1, 1}, {2, nil}, {2, nil}, {3, nil}, {4, 4}, {4, 4}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1, nil, 1, 1}, {2, 2, 2, nil}, {2, 2, 2, nil}, {2, 2, 2, nil}, {2, 2, 2, nil}, {3, nil, 3, nil}, {4, nil, 4, 4}, {4, nil, 4, 4}}, - }, - { - description: "basic LEFT OUTER JOIN test, L and R exhausted at the same time", - joinType: sqlbase.JoinType_LEFT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {2}, {3}, {4}, {4}}, - rightTuples: tuples{{0}, {2}, {3}, {4}, {4}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1, nil}, {2, 2}, {3, 3}, {4, 4}, {4, 4}, {4, 4}, {4, 4}}, - }, - { - description: "basic LEFT OUTER JOIN test, R exhausted first", - joinType: sqlbase.JoinType_LEFT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {1}, {3}, {5}, {6}, {7}}, - rightTuples: tuples{{2}, {3}, {4}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1, nil}, {1, nil}, {3, 3}, {5, nil}, {6, nil}, {7, nil}}, - }, - { - description: "basic LEFT OUTER JOIN test, L exhausted first", - joinType: sqlbase.JoinType_LEFT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{3}, {5}, {6}, {7}}, - rightTuples: tuples{{2}, {3}, {4}, {6}, {8}, {9}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{3, 3}, {5, nil}, {6, 6}, {7, nil}}, - }, - { - description: "multi output column LEFT OUTER JOIN test with nulls", - joinType: sqlbase.JoinType_LEFT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 10}, {2, 20}, {3, nil}, {4, 40}}, - rightTuples: tuples{{1, nil}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1, 10, 1, nil}, {2, 20, nil, nil}, {3, nil, 3, 13}, {4, 40, 4, 14}}, - }, - { - description: "null in equality column LEFT OUTER JOIN", - joinType: sqlbase.JoinType_LEFT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{nil}, {nil}, {1}, {3}}, - rightTuples: tuples{{nil, 1}, {1, 1}, {2, 2}, {3, 3}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{nil, nil, nil}, {nil, nil, nil}, {1, 1, 1}, {3, 3, 3}}, - }, - { - description: "multi equality column LEFT OUTER JOIN test with nulls", - joinType: sqlbase.JoinType_LEFT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, 10}, {2, 20}, {4, 40}}, - rightTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, nil}, {2, 20}, {3, 30}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, - expected: tuples{{nil, nil, nil, nil}, {nil, 10, nil, nil}, {1, nil, nil, nil}, {1, 10, nil, nil}, {2, 20, 2, 20}, {4, 40, nil, nil}}, - }, - { - description: "multi equality column (long runs on left) LEFT OUTER JOIN test with nulls", - joinType: sqlbase.JoinType_LEFT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 9}, {1, 10}, {1, 10}, {1, 11}, {2, 20}, {2, 20}, {2, 21}, {2, 22}, {2, 22}}, - rightTuples: tuples{{1, 8}, {1, 11}, {1, 11}, {2, 21}, {2, 23}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, - expected: tuples{{1, 9, nil, nil}, {1, 10, nil, nil}, {1, 10, nil, nil}, {1, 11, 1, 11}, {1, 11, 1, 11}, {2, 20, nil, nil}, {2, 20, nil, nil}, {2, 21, 2, 21}, {2, 22, nil, nil}, {2, 22, nil, nil}}, + description: "non-equality columns with nulls", + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, nil}, {2, 2}, {2, 2}, {3, nil}, {4, nil}}, + rightTuples: tuples{{1, 1}, {2, nil}, {2, nil}, {3, nil}, {4, 4}, {4, 4}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1, nil, 1, 1}, {2, 2, 2, nil}, {2, 2, 2, nil}, {2, 2, 2, nil}, {2, 2, 2, nil}, {3, nil, 3, nil}, {4, nil, 4, 4}, {4, nil, 4, 4}}, + }, + { + description: "basic LEFT OUTER JOIN test, L and R exhausted at the same time", + joinType: sqlbase.JoinType_LEFT_OUTER, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {2}, {3}, {4}, {4}}, + rightTuples: tuples{{0}, {2}, {3}, {4}, {4}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1, nil}, {2, 2}, {3, 3}, {4, 4}, {4, 4}, {4, 4}, {4, 4}}, + }, + { + description: "basic LEFT OUTER JOIN test, R exhausted first", + joinType: sqlbase.JoinType_LEFT_OUTER, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {1}, {3}, {5}, {6}, {7}}, + rightTuples: tuples{{2}, {3}, {4}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1, nil}, {1, nil}, {3, 3}, {5, nil}, {6, nil}, {7, nil}}, + }, + { + description: "basic LEFT OUTER JOIN test, L exhausted first", + joinType: sqlbase.JoinType_LEFT_OUTER, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{3}, {5}, {6}, {7}}, + rightTuples: tuples{{2}, {3}, {4}, {6}, {8}, {9}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{3, 3}, {5, nil}, {6, 6}, {7, nil}}, + }, + { + description: "multi output column LEFT OUTER JOIN test with nulls", + joinType: sqlbase.JoinType_LEFT_OUTER, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 10}, {2, 20}, {3, nil}, {4, 40}}, + rightTuples: tuples{{1, nil}, {3, 13}, {4, 14}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1, 10, 1, nil}, {2, 20, nil, nil}, {3, nil, 3, 13}, {4, 40, 4, 14}}, + }, + { + description: "null in equality column LEFT OUTER JOIN", + joinType: sqlbase.JoinType_LEFT_OUTER, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{nil}, {nil}, {1}, {3}}, + rightTuples: tuples{{nil, 1}, {1, 1}, {2, 2}, {3, 3}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{nil, nil, nil}, {nil, nil, nil}, {1, 1, 1}, {3, 3, 3}}, + }, + { + description: "multi equality column LEFT OUTER JOIN test with nulls", + joinType: sqlbase.JoinType_LEFT_OUTER, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, 10}, {2, 20}, {4, 40}}, + rightTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, nil}, {2, 20}, {3, 30}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, + expected: tuples{{nil, nil, nil, nil}, {nil, 10, nil, nil}, {1, nil, nil, nil}, {1, 10, nil, nil}, {2, 20, 2, 20}, {4, 40, nil, nil}}, + }, + { + description: "multi equality column (long runs on left) LEFT OUTER JOIN test with nulls", + joinType: sqlbase.JoinType_LEFT_OUTER, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 9}, {1, 10}, {1, 10}, {1, 11}, {2, 20}, {2, 20}, {2, 21}, {2, 22}, {2, 22}}, + rightTuples: tuples{{1, 8}, {1, 11}, {1, 11}, {2, 21}, {2, 23}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, + expected: tuples{{1, 9, nil, nil}, {1, 10, nil, nil}, {1, 10, nil, nil}, {1, 11, 1, 11}, {1, 11, 1, 11}, {2, 20, nil, nil}, {2, 20, nil, nil}, {2, 21, 2, 21}, {2, 22, nil, nil}, {2, 22, nil, nil}}, }, { description: "3 equality column LEFT OUTER JOIN test with nulls DESC ordering", joinType: sqlbase.JoinType_LEFT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, + leftTypes: []types.T{*types.Int, *types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int, *types.Int}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC}, leftTuples: tuples{{2, 3, 1}, {2, nil, 1}, {nil, 1, 3}}, @@ -845,8 +848,8 @@ var mjTestCases = []*joinTestCase{ { description: "3 equality column LEFT OUTER JOIN test with nulls mixed ordering", joinType: sqlbase.JoinType_LEFT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, + leftTypes: []types.T{*types.Int, *types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int, *types.Int}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_ASC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_ASC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC}, leftTuples: tuples{{2, 3, 1}, {2, nil, 1}, {nil, 1, 3}}, @@ -860,8 +863,8 @@ var mjTestCases = []*joinTestCase{ { description: "single column DESC with nulls on the left LEFT OUTER JOIN", joinType: sqlbase.JoinType_LEFT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, leftTuples: tuples{{1}, {1}, {1}, {nil}, {nil}, {nil}}, @@ -873,101 +876,101 @@ var mjTestCases = []*joinTestCase{ expected: tuples{{1, 1}, {1, 1}, {1, 1}, {nil, nil}, {nil, nil}, {nil, nil}}, }, { - description: "basic RIGHT OUTER JOIN test, L and R exhausted at the same time", - joinType: sqlbase.JoinType_RIGHT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{-1}, {2}, {3}, {4}, {4}}, - rightTuples: tuples{{1}, {2}, {3}, {4}, {4}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{nil, 1}, {2, 2}, {3, 3}, {4, 4}, {4, 4}, {4, 4}, {4, 4}}, - }, - { - description: "basic RIGHT OUTER JOIN test, R exhausted first", - joinType: sqlbase.JoinType_RIGHT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {1}, {3}, {5}, {6}, {7}}, - rightTuples: tuples{{2}, {3}, {4}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{nil, 2}, {3, 3}, {nil, 4}}, - }, - { - description: "basic RIGHT OUTER JOIN test, L exhausted first", - joinType: sqlbase.JoinType_RIGHT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{3}, {5}, {6}, {7}}, - rightTuples: tuples{{2}, {3}, {4}, {6}, {8}, {9}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{nil, 2}, {3, 3}, {nil, 4}, {6, 6}, {nil, 8}, {nil, 9}}, - }, - { - description: "multi output column RIGHT OUTER JOIN test with nulls", - joinType: sqlbase.JoinType_RIGHT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, nil}, {3, 13}, {4, 14}}, - rightTuples: tuples{{1, 10}, {2, 20}, {3, nil}, {4, 40}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1, nil, 1, 10}, {nil, nil, 2, 20}, {3, 13, 3, nil}, {4, 14, 4, 40}}, - }, - { - description: "null in equality column RIGHT OUTER JOIN", - joinType: sqlbase.JoinType_RIGHT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{nil, 1}, {1, 1}, {2, 2}, {3, 3}}, - rightTuples: tuples{{nil}, {nil}, {1}, {3}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{nil, nil, nil}, {nil, nil, nil}, {1, 1, 1}, {3, 3, 3}}, - }, - { - description: "multi equality column RIGHT OUTER JOIN test with nulls", - joinType: sqlbase.JoinType_RIGHT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, nil}, {2, 20}, {3, 30}}, - rightTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, 10}, {2, 20}, {4, 40}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, - expected: tuples{{nil, nil, nil, nil}, {nil, nil, nil, 10}, {nil, nil, 1, nil}, {nil, nil, 1, 10}, {2, 20, 2, 20}, {nil, nil, 4, 40}}, - }, - { - description: "multi equality column (long runs on right) RIGHT OUTER JOIN test with nulls", - joinType: sqlbase.JoinType_RIGHT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 8}, {1, 11}, {1, 11}, {2, 21}, {2, 23}}, - rightTuples: tuples{{1, 9}, {1, 10}, {1, 10}, {1, 11}, {2, 20}, {2, 20}, {2, 21}, {2, 22}, {2, 22}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, - expected: tuples{{nil, nil, 1, 9}, {nil, nil, 1, 10}, {nil, nil, 1, 10}, {1, 11, 1, 11}, {1, 11, 1, 11}, {nil, nil, 2, 20}, {nil, nil, 2, 20}, {2, 21, 2, 21}, {nil, nil, 2, 22}, {nil, nil, 2, 22}}, + description: "basic RIGHT OUTER JOIN test, L and R exhausted at the same time", + joinType: sqlbase.JoinType_RIGHT_OUTER, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{-1}, {2}, {3}, {4}, {4}}, + rightTuples: tuples{{1}, {2}, {3}, {4}, {4}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{nil, 1}, {2, 2}, {3, 3}, {4, 4}, {4, 4}, {4, 4}, {4, 4}}, + }, + { + description: "basic RIGHT OUTER JOIN test, R exhausted first", + joinType: sqlbase.JoinType_RIGHT_OUTER, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {1}, {3}, {5}, {6}, {7}}, + rightTuples: tuples{{2}, {3}, {4}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{nil, 2}, {3, 3}, {nil, 4}}, + }, + { + description: "basic RIGHT OUTER JOIN test, L exhausted first", + joinType: sqlbase.JoinType_RIGHT_OUTER, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{3}, {5}, {6}, {7}}, + rightTuples: tuples{{2}, {3}, {4}, {6}, {8}, {9}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{nil, 2}, {3, 3}, {nil, 4}, {6, 6}, {nil, 8}, {nil, 9}}, + }, + { + description: "multi output column RIGHT OUTER JOIN test with nulls", + joinType: sqlbase.JoinType_RIGHT_OUTER, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, nil}, {3, 13}, {4, 14}}, + rightTuples: tuples{{1, 10}, {2, 20}, {3, nil}, {4, 40}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1, nil, 1, 10}, {nil, nil, 2, 20}, {3, 13, 3, nil}, {4, 14, 4, 40}}, + }, + { + description: "null in equality column RIGHT OUTER JOIN", + joinType: sqlbase.JoinType_RIGHT_OUTER, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{nil, 1}, {1, 1}, {2, 2}, {3, 3}}, + rightTuples: tuples{{nil}, {nil}, {1}, {3}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{nil, nil, nil}, {nil, nil, nil}, {1, 1, 1}, {3, 3, 3}}, + }, + { + description: "multi equality column RIGHT OUTER JOIN test with nulls", + joinType: sqlbase.JoinType_RIGHT_OUTER, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, nil}, {2, 20}, {3, 30}}, + rightTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, 10}, {2, 20}, {4, 40}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, + expected: tuples{{nil, nil, nil, nil}, {nil, nil, nil, 10}, {nil, nil, 1, nil}, {nil, nil, 1, 10}, {2, 20, 2, 20}, {nil, nil, 4, 40}}, + }, + { + description: "multi equality column (long runs on right) RIGHT OUTER JOIN test with nulls", + joinType: sqlbase.JoinType_RIGHT_OUTER, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 8}, {1, 11}, {1, 11}, {2, 21}, {2, 23}}, + rightTuples: tuples{{1, 9}, {1, 10}, {1, 10}, {1, 11}, {2, 20}, {2, 20}, {2, 21}, {2, 22}, {2, 22}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, + expected: tuples{{nil, nil, 1, 9}, {nil, nil, 1, 10}, {nil, nil, 1, 10}, {1, 11, 1, 11}, {1, 11, 1, 11}, {nil, nil, 2, 20}, {nil, nil, 2, 20}, {2, 21, 2, 21}, {nil, nil, 2, 22}, {nil, nil, 2, 22}}, }, { description: "3 equality column RIGHT OUTER JOIN test with nulls DESC ordering", joinType: sqlbase.JoinType_RIGHT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, + leftTypes: []types.T{*types.Int, *types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int, *types.Int}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC}, leftTuples: tuples{{4, 3, 3}, {nil, 2, nil}, {nil, 1, 3}}, @@ -981,8 +984,8 @@ var mjTestCases = []*joinTestCase{ { description: "3 equality column RIGHT OUTER JOIN test with nulls mixed ordering", joinType: sqlbase.JoinType_RIGHT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, + leftTypes: []types.T{*types.Int, *types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int, *types.Int}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_ASC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_ASC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC}, leftTuples: tuples{{4, 3, 3}, {nil, 2, nil}, {nil, 1, 3}}, @@ -996,8 +999,8 @@ var mjTestCases = []*joinTestCase{ { description: "single column DESC with nulls on the right RIGHT OUTER JOIN", joinType: sqlbase.JoinType_RIGHT_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, leftTuples: tuples{{1}}, @@ -1009,101 +1012,101 @@ var mjTestCases = []*joinTestCase{ expected: tuples{{1, 1}, {1, 1}, {1, 1}, {nil, nil}, {nil, nil}, {nil, nil}}, }, { - description: "basic FULL OUTER JOIN test, L and R exhausted at the same time", - joinType: sqlbase.JoinType_FULL_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{-1}, {2}, {3}, {4}, {4}}, - rightTuples: tuples{{1}, {2}, {3}, {4}, {4}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{-1, nil}, {nil, 1}, {2, 2}, {3, 3}, {4, 4}, {4, 4}, {4, 4}, {4, 4}}, - }, - { - description: "basic FULL OUTER JOIN test, R exhausted first", - joinType: sqlbase.JoinType_FULL_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {1}, {3}, {5}, {6}, {7}}, - rightTuples: tuples{{2}, {3}, {4}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1, nil}, {1, nil}, {nil, 2}, {3, 3}, {nil, 4}, {5, nil}, {6, nil}, {7, nil}}, - }, - { - description: "basic FULL OUTER JOIN test, L exhausted first", - joinType: sqlbase.JoinType_FULL_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{3}, {5}, {6}, {7}}, - rightTuples: tuples{{2}, {3}, {4}, {6}, {8}, {9}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{nil, 2}, {3, 3}, {nil, 4}, {5, nil}, {6, 6}, {7, nil}, {nil, 8}, {nil, 9}}, - }, - { - description: "multi output column FULL OUTER JOIN test with nulls", - joinType: sqlbase.JoinType_FULL_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, nil}, {3, 13}, {4, 14}}, - rightTuples: tuples{{1, 10}, {2, 20}, {3, nil}, {4, 40}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1, nil, 1, 10}, {nil, nil, 2, 20}, {3, 13, 3, nil}, {4, 14, 4, 40}}, - }, - { - description: "null in equality column FULL OUTER JOIN", - joinType: sqlbase.JoinType_FULL_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{nil, 1}, {1, 1}, {2, 2}, {3, 3}}, - rightTuples: tuples{{nil}, {nil}, {1}, {3}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{nil, 1, nil}, {nil, nil, nil}, {nil, nil, nil}, {1, 1, 1}, {2, 2, nil}, {3, 3, 3}}, - }, - { - description: "multi equality column FULL OUTER JOIN test with nulls", - joinType: sqlbase.JoinType_FULL_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, nil}, {2, 20}, {3, 30}}, - rightTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, 10}, {2, 20}, {4, 40}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, - expected: tuples{{nil, nil, nil, nil}, {nil, nil, nil, nil}, {nil, 10, nil, nil}, {nil, nil, nil, 10}, {1, nil, nil, nil}, {1, nil, nil, nil}, {nil, nil, 1, nil}, {nil, nil, 1, 10}, {2, 20, 2, 20}, {3, 30, nil, nil}, {nil, nil, 4, 40}}, - }, - { - description: "multi equality column (long runs on right) FULL OUTER JOIN test with nulls", - joinType: sqlbase.JoinType_FULL_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 8}, {1, 11}, {1, 11}, {2, 21}, {2, 23}}, - rightTuples: tuples{{1, 9}, {1, 10}, {1, 10}, {1, 11}, {2, 20}, {2, 20}, {2, 21}, {2, 22}, {2, 22}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, - expected: tuples{{1, 8, nil, nil}, {nil, nil, 1, 9}, {nil, nil, 1, 10}, {nil, nil, 1, 10}, {1, 11, 1, 11}, {1, 11, 1, 11}, {nil, nil, 2, 20}, {nil, nil, 2, 20}, {2, 21, 2, 21}, {nil, nil, 2, 22}, {nil, nil, 2, 22}, {2, 23, nil, nil}}, + description: "basic FULL OUTER JOIN test, L and R exhausted at the same time", + joinType: sqlbase.JoinType_FULL_OUTER, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{-1}, {2}, {3}, {4}, {4}}, + rightTuples: tuples{{1}, {2}, {3}, {4}, {4}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{-1, nil}, {nil, 1}, {2, 2}, {3, 3}, {4, 4}, {4, 4}, {4, 4}, {4, 4}}, + }, + { + description: "basic FULL OUTER JOIN test, R exhausted first", + joinType: sqlbase.JoinType_FULL_OUTER, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {1}, {3}, {5}, {6}, {7}}, + rightTuples: tuples{{2}, {3}, {4}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1, nil}, {1, nil}, {nil, 2}, {3, 3}, {nil, 4}, {5, nil}, {6, nil}, {7, nil}}, + }, + { + description: "basic FULL OUTER JOIN test, L exhausted first", + joinType: sqlbase.JoinType_FULL_OUTER, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{3}, {5}, {6}, {7}}, + rightTuples: tuples{{2}, {3}, {4}, {6}, {8}, {9}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{nil, 2}, {3, 3}, {nil, 4}, {5, nil}, {6, 6}, {7, nil}, {nil, 8}, {nil, 9}}, + }, + { + description: "multi output column FULL OUTER JOIN test with nulls", + joinType: sqlbase.JoinType_FULL_OUTER, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, nil}, {3, 13}, {4, 14}}, + rightTuples: tuples{{1, 10}, {2, 20}, {3, nil}, {4, 40}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1, nil, 1, 10}, {nil, nil, 2, 20}, {3, 13, 3, nil}, {4, 14, 4, 40}}, + }, + { + description: "null in equality column FULL OUTER JOIN", + joinType: sqlbase.JoinType_FULL_OUTER, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{nil, 1}, {1, 1}, {2, 2}, {3, 3}}, + rightTuples: tuples{{nil}, {nil}, {1}, {3}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{nil, 1, nil}, {nil, nil, nil}, {nil, nil, nil}, {1, 1, 1}, {2, 2, nil}, {3, 3, 3}}, + }, + { + description: "multi equality column FULL OUTER JOIN test with nulls", + joinType: sqlbase.JoinType_FULL_OUTER, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, nil}, {2, 20}, {3, 30}}, + rightTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, 10}, {2, 20}, {4, 40}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, + expected: tuples{{nil, nil, nil, nil}, {nil, nil, nil, nil}, {nil, 10, nil, nil}, {nil, nil, nil, 10}, {1, nil, nil, nil}, {1, nil, nil, nil}, {nil, nil, 1, nil}, {nil, nil, 1, 10}, {2, 20, 2, 20}, {3, 30, nil, nil}, {nil, nil, 4, 40}}, + }, + { + description: "multi equality column (long runs on right) FULL OUTER JOIN test with nulls", + joinType: sqlbase.JoinType_FULL_OUTER, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 8}, {1, 11}, {1, 11}, {2, 21}, {2, 23}}, + rightTuples: tuples{{1, 9}, {1, 10}, {1, 10}, {1, 11}, {2, 20}, {2, 20}, {2, 21}, {2, 22}, {2, 22}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, + expected: tuples{{1, 8, nil, nil}, {nil, nil, 1, 9}, {nil, nil, 1, 10}, {nil, nil, 1, 10}, {1, 11, 1, 11}, {1, 11, 1, 11}, {nil, nil, 2, 20}, {nil, nil, 2, 20}, {2, 21, 2, 21}, {nil, nil, 2, 22}, {nil, nil, 2, 22}, {2, 23, nil, nil}}, }, { description: "3 equality column FULL OUTER JOIN test with nulls DESC ordering", joinType: sqlbase.JoinType_FULL_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, + leftTypes: []types.T{*types.Int, *types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int, *types.Int}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC}, leftTuples: tuples{{4, 3, 3}, {nil, 2, nil}, {nil, 1, 3}}, @@ -1117,8 +1120,8 @@ var mjTestCases = []*joinTestCase{ { description: "3 equality column FULL OUTER JOIN test with nulls mixed ordering", joinType: sqlbase.JoinType_FULL_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, + leftTypes: []types.T{*types.Int, *types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int, *types.Int}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_ASC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_ASC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC}, leftTuples: tuples{{4, 3, 3}, {nil, 2, nil}, {nil, 1, 3}}, @@ -1132,8 +1135,8 @@ var mjTestCases = []*joinTestCase{ { description: "single column DESC with nulls on the right FULL OUTER JOIN", joinType: sqlbase.JoinType_FULL_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, leftTuples: tuples{{1}}, @@ -1145,114 +1148,114 @@ var mjTestCases = []*joinTestCase{ expected: tuples{{1, 1}, {1, 1}, {1, 1}, {nil, nil}, {nil, nil}, {nil, nil}}, }, { - description: "FULL OUTER JOIN test with nulls and Bytes", - joinType: sqlbase.JoinType_FULL_OUTER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Bytes}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Bytes}, - leftTuples: tuples{{nil, "0"}, {1, "10"}, {2, "20"}, {3, nil}, {4, "40"}}, - rightTuples: tuples{{1, nil}, {3, "13"}, {4, nil}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{nil, "0", nil}, {1, "10", nil}, {2, "20", nil}, {3, nil, "13"}, {4, "40", nil}}, - }, - { - description: "basic LEFT SEMI JOIN test, L and R exhausted at the same time", - joinType: sqlbase.JoinType_LEFT_SEMI, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {2}, {3}, {4}, {4}}, - rightTuples: tuples{{-1}, {2}, {3}, {4}, {4}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{2}, {3}, {4}, {4}}, - }, - { - description: "basic LEFT SEMI JOIN test, R exhausted first", - joinType: sqlbase.JoinType_LEFT_SEMI, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {1}, {3}, {5}, {6}, {7}}, - rightTuples: tuples{{2}, {3}, {4}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{3}}, - }, - { - description: "basic LEFT SEMI JOIN test, L exhausted first", - joinType: sqlbase.JoinType_LEFT_SEMI, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{3}, {5}, {6}, {7}}, - rightTuples: tuples{{2}, {3}, {3}, {3}, {4}, {6}, {8}, {9}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{3}, {6}}, - }, - { - description: "multi output column LEFT SEMI JOIN test with nulls", - joinType: sqlbase.JoinType_LEFT_SEMI, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 10}, {2, 20}, {3, nil}, {4, 40}}, - rightTuples: tuples{{1, nil}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1, 10}, {3, nil}, {4, 40}}, - }, - { - description: "null in equality column LEFT SEMI JOIN", - joinType: sqlbase.JoinType_LEFT_SEMI, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{nil}, {nil}, {1}, {3}}, - rightTuples: tuples{{nil, 1}, {1, 1}, {2, 2}, {3, 3}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {3}}, - }, - { - description: "multi equality column LEFT SEMI JOIN test with nulls", - joinType: sqlbase.JoinType_LEFT_SEMI, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, 10}, {2, 20}, {4, 40}}, - rightTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, nil}, {2, 20}, {3, 30}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, - expected: tuples{{2, 20}}, - }, - { - description: "multi equality column (long runs on left) LEFT SEMI JOIN test with nulls", - joinType: sqlbase.JoinType_LEFT_SEMI, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 9}, {1, 10}, {1, 10}, {1, 11}, {1, 11}, {1, 11}, {2, 20}, {2, 20}, {2, 21}, {2, 22}, {2, 22}}, - rightTuples: tuples{{1, 8}, {1, 11}, {1, 11}, {2, 21}, {2, 23}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, - expected: tuples{{1, 11}, {1, 11}, {1, 11}, {2, 21}}, + description: "FULL OUTER JOIN test with nulls and Bytes", + joinType: sqlbase.JoinType_FULL_OUTER, + leftTypes: []types.T{*types.Int, *types.Bytes}, + rightTypes: []types.T{*types.Int, *types.Bytes}, + leftTuples: tuples{{nil, "0"}, {1, "10"}, {2, "20"}, {3, nil}, {4, "40"}}, + rightTuples: tuples{{1, nil}, {3, "13"}, {4, nil}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{1}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{nil, "0", nil}, {1, "10", nil}, {2, "20", nil}, {3, nil, "13"}, {4, "40", nil}}, + }, + { + description: "basic LEFT SEMI JOIN test, L and R exhausted at the same time", + joinType: sqlbase.JoinType_LEFT_SEMI, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {2}, {3}, {4}, {4}}, + rightTuples: tuples{{-1}, {2}, {3}, {4}, {4}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{2}, {3}, {4}, {4}}, + }, + { + description: "basic LEFT SEMI JOIN test, R exhausted first", + joinType: sqlbase.JoinType_LEFT_SEMI, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {1}, {3}, {5}, {6}, {7}}, + rightTuples: tuples{{2}, {3}, {4}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{3}}, + }, + { + description: "basic LEFT SEMI JOIN test, L exhausted first", + joinType: sqlbase.JoinType_LEFT_SEMI, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{3}, {5}, {6}, {7}}, + rightTuples: tuples{{2}, {3}, {3}, {3}, {4}, {6}, {8}, {9}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{3}, {6}}, + }, + { + description: "multi output column LEFT SEMI JOIN test with nulls", + joinType: sqlbase.JoinType_LEFT_SEMI, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 10}, {2, 20}, {3, nil}, {4, 40}}, + rightTuples: tuples{{1, nil}, {3, 13}, {4, 14}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1, 10}, {3, nil}, {4, 40}}, + }, + { + description: "null in equality column LEFT SEMI JOIN", + joinType: sqlbase.JoinType_LEFT_SEMI, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{nil}, {nil}, {1}, {3}}, + rightTuples: tuples{{nil, 1}, {1, 1}, {2, 2}, {3, 3}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1}, {3}}, + }, + { + description: "multi equality column LEFT SEMI JOIN test with nulls", + joinType: sqlbase.JoinType_LEFT_SEMI, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, 10}, {2, 20}, {4, 40}}, + rightTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, nil}, {2, 20}, {3, 30}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, + expected: tuples{{2, 20}}, + }, + { + description: "multi equality column (long runs on left) LEFT SEMI JOIN test with nulls", + joinType: sqlbase.JoinType_LEFT_SEMI, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 9}, {1, 10}, {1, 10}, {1, 11}, {1, 11}, {1, 11}, {2, 20}, {2, 20}, {2, 21}, {2, 22}, {2, 22}}, + rightTuples: tuples{{1, 8}, {1, 11}, {1, 11}, {2, 21}, {2, 23}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, + expected: tuples{{1, 11}, {1, 11}, {1, 11}, {2, 21}}, }, { description: "3 equality column LEFT SEMI JOIN test with nulls DESC ordering", joinType: sqlbase.JoinType_LEFT_SEMI, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, + leftTypes: []types.T{*types.Int, *types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int, *types.Int}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC}, leftTuples: tuples{{2, 3, 1}, {2, nil, 1}, {nil, 1, 3}}, @@ -1269,8 +1272,8 @@ var mjTestCases = []*joinTestCase{ { description: "3 equality column LEFT SEMI JOIN test with nulls mixed ordering", joinType: sqlbase.JoinType_LEFT_SEMI, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, + leftTypes: []types.T{*types.Int, *types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int, *types.Int}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_ASC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_ASC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC}, leftTuples: tuples{{2, 3, 1}, {2, nil, 1}, {nil, 1, 3}}, @@ -1287,8 +1290,8 @@ var mjTestCases = []*joinTestCase{ { description: "single column DESC with nulls on the left LEFT SEMI JOIN", joinType: sqlbase.JoinType_LEFT_SEMI, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, leftTuples: tuples{{1}, {1}, {1}, {nil}, {nil}, {nil}}, @@ -1300,101 +1303,101 @@ var mjTestCases = []*joinTestCase{ expected: tuples{{1}, {1}, {1}}, }, { - description: "basic LEFT ANTI JOIN test, L and R exhausted at the same time", - joinType: sqlbase.JoinType_LEFT_ANTI, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {2}, {3}, {4}, {4}}, - rightTuples: tuples{{-1}, {2}, {4}, {4}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {3}}, - }, - { - description: "basic LEFT ANTI JOIN test, R exhausted first", - joinType: sqlbase.JoinType_LEFT_ANTI, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{1}, {1}, {3}, {5}, {6}, {7}}, - rightTuples: tuples{{2}, {3}, {4}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {1}, {5}, {6}, {7}}, - }, - { - description: "basic LEFT ANTI JOIN test, L exhausted first", - joinType: sqlbase.JoinType_LEFT_ANTI, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, - leftTuples: tuples{{3}, {5}, {6}, {7}}, - rightTuples: tuples{{2}, {3}, {3}, {3}, {4}, {6}, {8}, {9}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{5}, {7}}, - }, - { - description: "multi output column LEFT ANTI JOIN test with nulls", - joinType: sqlbase.JoinType_LEFT_ANTI, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 10}, {2, 20}, {3, nil}, {4, 40}}, - rightTuples: tuples{{1, nil}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{2, 20}}, - }, - { - description: "null in equality column LEFT ANTI JOIN", - joinType: sqlbase.JoinType_LEFT_ANTI, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{nil}, {nil}, {1}, {3}}, - rightTuples: tuples{{nil, 1}, {1, 1}, {2, 2}, {3, 3}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{nil}, {nil}}, - }, - { - description: "multi equality column LEFT ANTI JOIN test with nulls", - joinType: sqlbase.JoinType_LEFT_ANTI, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, 10}, {2, 20}, {4, 40}}, - rightTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, nil}, {2, 20}, {3, 30}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, - expected: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, 10}, {4, 40}}, - }, - { - description: "multi equality column (long runs on left) LEFT ANTI JOIN test with nulls", - joinType: sqlbase.JoinType_LEFT_ANTI, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{1, 9}, {1, 10}, {1, 10}, {1, 11}, {1, 11}, {1, 11}, {2, 20}, {2, 20}, {2, 21}, {2, 22}, {2, 22}}, - rightTuples: tuples{{1, 8}, {1, 11}, {1, 11}, {2, 21}, {2, 23}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, - expected: tuples{{1, 9}, {1, 10}, {1, 10}, {2, 20}, {2, 20}, {2, 22}, {2, 22}}, + description: "basic LEFT ANTI JOIN test, L and R exhausted at the same time", + joinType: sqlbase.JoinType_LEFT_ANTI, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {2}, {3}, {4}, {4}}, + rightTuples: tuples{{-1}, {2}, {4}, {4}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1}, {3}}, + }, + { + description: "basic LEFT ANTI JOIN test, R exhausted first", + joinType: sqlbase.JoinType_LEFT_ANTI, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{1}, {1}, {3}, {5}, {6}, {7}}, + rightTuples: tuples{{2}, {3}, {4}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{1}, {1}, {5}, {6}, {7}}, + }, + { + description: "basic LEFT ANTI JOIN test, L exhausted first", + joinType: sqlbase.JoinType_LEFT_ANTI, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, + leftTuples: tuples{{3}, {5}, {6}, {7}}, + rightTuples: tuples{{2}, {3}, {3}, {3}, {4}, {6}, {8}, {9}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{5}, {7}}, + }, + { + description: "multi output column LEFT ANTI JOIN test with nulls", + joinType: sqlbase.JoinType_LEFT_ANTI, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 10}, {2, 20}, {3, nil}, {4, 40}}, + rightTuples: tuples{{1, nil}, {3, 13}, {4, 14}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{2, 20}}, + }, + { + description: "null in equality column LEFT ANTI JOIN", + joinType: sqlbase.JoinType_LEFT_ANTI, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{nil}, {nil}, {1}, {3}}, + rightTuples: tuples{{nil, 1}, {1, 1}, {2, 2}, {3, 3}}, + leftOutCols: []uint32{0}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + expected: tuples{{nil}, {nil}}, + }, + { + description: "multi equality column LEFT ANTI JOIN test with nulls", + joinType: sqlbase.JoinType_LEFT_ANTI, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, 10}, {2, 20}, {4, 40}}, + rightTuples: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, nil}, {2, 20}, {3, 30}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, + expected: tuples{{nil, nil}, {nil, 10}, {1, nil}, {1, 10}, {4, 40}}, + }, + { + description: "multi equality column (long runs on left) LEFT ANTI JOIN test with nulls", + joinType: sqlbase.JoinType_LEFT_ANTI, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{1, 9}, {1, 10}, {1, 10}, {1, 11}, {1, 11}, {1, 11}, {2, 20}, {2, 20}, {2, 21}, {2, 22}, {2, 22}}, + rightTuples: tuples{{1, 8}, {1, 11}, {1, 11}, {2, 21}, {2, 23}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, + expected: tuples{{1, 9}, {1, 10}, {1, 10}, {2, 20}, {2, 20}, {2, 22}, {2, 22}}, }, { description: "3 equality column LEFT ANTI JOIN test with nulls DESC ordering", joinType: sqlbase.JoinType_LEFT_ANTI, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, + leftTypes: []types.T{*types.Int, *types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int, *types.Int}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC}, leftTuples: tuples{{2, 3, 1}, {2, nil, 1}, {nil, 1, 3}}, @@ -1408,8 +1411,8 @@ var mjTestCases = []*joinTestCase{ { description: "3 equality column LEFT ANTI JOIN test with nulls mixed ordering", joinType: sqlbase.JoinType_LEFT_ANTI, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, + leftTypes: []types.T{*types.Int, *types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int, *types.Int}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_ASC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_ASC, execinfrapb.Ordering_Column_DESC, execinfrapb.Ordering_Column_DESC}, leftTuples: tuples{{2, 3, 1}, {2, nil, 1}, {nil, 1, 3}}, @@ -1423,8 +1426,8 @@ var mjTestCases = []*joinTestCase{ { description: "single column DESC with nulls on the left LEFT ANTI JOIN", joinType: sqlbase.JoinType_LEFT_ANTI, - leftPhysTypes: []coltypes.T{coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64}, + leftTypes: []types.T{*types.Int}, + rightTypes: []types.T{*types.Int}, leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, leftTuples: tuples{{1}, {1}, {1}, {nil}, {nil}, {nil}}, @@ -1436,46 +1439,46 @@ var mjTestCases = []*joinTestCase{ expected: tuples{{nil}, {nil}, {nil}}, }, { - description: "INNER JOIN test with ON expression (filter only on left)", - joinType: sqlbase.JoinType_INNER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{nil, 0}, {1, 10}, {2, 20}, {3, nil}, {4, 40}}, - rightTuples: tuples{{1, nil}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - onExpr: execinfrapb.Expression{Expr: "@1 < 4"}, - expected: tuples{{1, 10}, {3, nil}}, - }, - { - description: "INNER JOIN test with ON expression (filter only on right)", - joinType: sqlbase.JoinType_INNER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{nil, 0}, {1, 10}, {2, 20}, {3, nil}, {4, 40}}, - rightTuples: tuples{{1, nil}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - onExpr: execinfrapb.Expression{Expr: "@4 < 14"}, - expected: tuples{{3, nil}}, - }, - { - description: "INNER JOIN test with ON expression (filter on both)", - joinType: sqlbase.JoinType_INNER, - leftPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - rightPhysTypes: []coltypes.T{coltypes.Int64, coltypes.Int64}, - leftTuples: tuples{{nil, 0}, {1, 10}, {2, 20}, {3, nil}, {4, 40}}, - rightTuples: tuples{{1, nil}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - onExpr: execinfrapb.Expression{Expr: "@2 + @3 < 50"}, - expected: tuples{{1, 10}, {4, 40}}, + description: "INNER JOIN test with ON expression (filter only on left)", + joinType: sqlbase.JoinType_INNER, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{nil, 0}, {1, 10}, {2, 20}, {3, nil}, {4, 40}}, + rightTuples: tuples{{1, nil}, {3, 13}, {4, 14}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + onExpr: execinfrapb.Expression{Expr: "@1 < 4"}, + expected: tuples{{1, 10}, {3, nil}}, + }, + { + description: "INNER JOIN test with ON expression (filter only on right)", + joinType: sqlbase.JoinType_INNER, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{nil, 0}, {1, 10}, {2, 20}, {3, nil}, {4, 40}}, + rightTuples: tuples{{1, nil}, {3, 13}, {4, 14}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + onExpr: execinfrapb.Expression{Expr: "@4 < 14"}, + expected: tuples{{3, nil}}, + }, + { + description: "INNER JOIN test with ON expression (filter on both)", + joinType: sqlbase.JoinType_INNER, + leftTypes: []types.T{*types.Int, *types.Int}, + rightTypes: []types.T{*types.Int, *types.Int}, + leftTuples: tuples{{nil, 0}, {1, 10}, {2, 20}, {3, nil}, {4, 40}}, + rightTuples: tuples{{1, nil}, {3, 13}, {4, 14}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{0}, + onExpr: execinfrapb.Expression{Expr: "@2 + @3 < 50"}, + expected: tuples{{1, 10}, {4, 40}}, }, } @@ -1536,16 +1539,16 @@ func TestMergeJoiner(t *testing.T) { for _, memoryLimit := range []int64{1, defaultMemoryLimit} { t.Run(fmt.Sprintf("MemoryLimit=%s/%s", humanizeutil.IBytes(memoryLimit), tc.description), func(t *testing.T) { runner(t, []tuples{tc.leftTuples, tc.rightTuples}, - [][]coltypes.T{tc.leftPhysTypes, tc.rightPhysTypes}, + [][]types.T{tc.leftTypes, tc.rightTypes}, tc.expected, mergeJoinVerifier, - func(input []Operator) (Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { spec := createSpecForMergeJoiner(tc) args := NewColOperatorArgs{ Spec: spec, Inputs: input, StreamingMemAccount: testMemAcc, DiskQueueCfg: queueCfg, - FDSemaphore: NewTestingSemaphore(mjFDLimit), + FDSemaphore: colexecbase.NewTestingSemaphore(mjFDLimit), } args.TestingKnobs.UseStreamingMemAccountForBuffering = true flowCtx.Cfg.TestingKnobs.MemoryLimitBytes = memoryLimit @@ -1587,9 +1590,9 @@ func TestFullOuterMergeJoinWithMaximumNumberOfGroups(t *testing.T) { for _, outBatchSize := range []int{1, 16, coldata.BatchSize() - 1, coldata.BatchSize(), coldata.BatchSize() + 1} { t.Run(fmt.Sprintf("outBatchSize=%d", outBatchSize), func(t *testing.T) { - typs := []coltypes.T{coltypes.Int64} - colsLeft := []coldata.Vec{testAllocator.NewMemColumn(typs[0], nTuples)} - colsRight := []coldata.Vec{testAllocator.NewMemColumn(typs[0], nTuples)} + typs := []types.T{*types.Int} + colsLeft := []coldata.Vec{testAllocator.NewMemColumn(&typs[0], nTuples)} + colsRight := []coldata.Vec{testAllocator.NewMemColumn(&typs[0], nTuples)} groupsLeft := colsLeft[0].Int64() groupsRight := colsRight[0].Int64() for i := range groupsLeft { @@ -1600,7 +1603,7 @@ func TestFullOuterMergeJoinWithMaximumNumberOfGroups(t *testing.T) { rightSource := newChunkingBatchSource(typs, colsRight, nTuples) a, err := newMergeJoinOp( testAllocator, defaultMemoryLimit, queueCfg, - 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}}, @@ -1672,12 +1675,12 @@ func TestMergeJoinCrossProduct(t *testing.T) { for _, outBatchSize := range []int{1, 17, coldata.BatchSize() - 1, coldata.BatchSize(), coldata.BatchSize() + 1} { t.Run(fmt.Sprintf("outBatchSize=%d", outBatchSize), func(t *testing.T) { - typs := []coltypes.T{coltypes.Int64, coltypes.Bytes, coltypes.Decimal} + typs := []types.T{*types.Int, *types.Bytes, *types.Decimal} colsLeft := make([]coldata.Vec, len(typs)) colsRight := make([]coldata.Vec, len(typs)) for i, typ := range typs { - colsLeft[i] = testAllocator.NewMemColumn(typ, nTuples) - colsRight[i] = testAllocator.NewMemColumn(typ, nTuples) + colsLeft[i] = testAllocator.NewMemColumn(&typ, nTuples) + colsRight[i] = testAllocator.NewMemColumn(&typ, nTuples) } groupsLeft := colsLeft[0].Int64() groupsRight := colsRight[0].Int64() @@ -1693,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) @@ -1702,7 +1705,7 @@ func TestMergeJoinCrossProduct(t *testing.T) { rightHJSource := newChunkingBatchSource(typs, colsRight, nTuples) mj, err := newMergeJoinOp( testAllocator, defaultMemoryLimit, queueCfg, - 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}}, @@ -1759,8 +1762,8 @@ func TestMergeJoinerMultiBatch(t *testing.T) { t.Run(fmt.Sprintf("numInputBatches=%d", numInputBatches), func(t *testing.T) { nTuples := coldata.BatchSize() * numInputBatches - typs := []coltypes.T{coltypes.Int64} - cols := []coldata.Vec{testAllocator.NewMemColumn(typs[0], nTuples)} + typs := []types.T{*types.Int} + cols := []coldata.Vec{testAllocator.NewMemColumn(&typs[0], nTuples)} groups := cols[0].Int64() for i := range groups { groups[i] = int64(i) @@ -1771,7 +1774,7 @@ func TestMergeJoinerMultiBatch(t *testing.T) { a, err := newMergeJoinOp( testAllocator, defaultMemoryLimit, - queueCfg, 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}}, @@ -1836,10 +1839,10 @@ func TestMergeJoinerMultiBatchRuns(t *testing.T) { // group will be of size 0. lastGroupSize := nTuples % groupSize expCount := nTuples/groupSize*(groupSize*groupSize) + lastGroupSize*lastGroupSize - typs := []coltypes.T{coltypes.Int64, coltypes.Int64} + typs := []types.T{*types.Int, *types.Int} cols := []coldata.Vec{ - testAllocator.NewMemColumn(typs[0], nTuples), - testAllocator.NewMemColumn(typs[1], nTuples), + testAllocator.NewMemColumn(&typs[0], nTuples), + testAllocator.NewMemColumn(&typs[1], nTuples), } for i := range cols[0].Int64() { cols[0].Int64()[i] = int64(i / groupSize) @@ -1851,7 +1854,7 @@ func TestMergeJoinerMultiBatchRuns(t *testing.T) { a, err := newMergeJoinOp( testAllocator, defaultMemoryLimit, - queueCfg, 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}}, @@ -1904,12 +1907,12 @@ type expectedGroup struct { } func newBatchesOfRandIntRows( - nTuples int, typs []coltypes.T, maxRunLength int64, skipValues bool, randomIncrement int64, + nTuples int, typs []types.T, maxRunLength int64, skipValues bool, randomIncrement int64, ) ([]coldata.Vec, []coldata.Vec, []expectedGroup) { rng, _ := randutil.NewPseudoRand() - lCols := []coldata.Vec{testAllocator.NewMemColumn(typs[0], nTuples)} + lCols := []coldata.Vec{testAllocator.NewMemColumn(&typs[0], nTuples)} lCol := lCols[0].Int64() - rCols := []coldata.Vec{testAllocator.NewMemColumn(typs[0], nTuples)} + rCols := []coldata.Vec{testAllocator.NewMemColumn(&typs[0], nTuples)} rCol := rCols[0].Int64() exp := make([]expectedGroup, nTuples) val := int64(0) @@ -1974,14 +1977,14 @@ func TestMergeJoinerRandomized(t *testing.T) { t.Run(fmt.Sprintf("numInputBatches=%dmaxRunLength=%dskipValues=%trandomIncrement=%d", numInputBatches, maxRunLength, skipValues, randomIncrement), func(t *testing.T) { nTuples := coldata.BatchSize() * numInputBatches - typs := []coltypes.T{coltypes.Int64} + typs := []types.T{*types.Int} lCols, rCols, exp := newBatchesOfRandIntRows(nTuples, typs, maxRunLength, skipValues, randomIncrement) leftSource := newChunkingBatchSource(typs, lCols, nTuples) rightSource := newChunkingBatchSource(typs, rCols, nTuples) a, err := newMergeJoinOp( testAllocator, defaultMemoryLimit, - queueCfg, 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}}, @@ -2059,10 +2062,10 @@ func newBatchOfRepeatedIntRows(nCols int, batch coldata.Batch, numRepeats int) c func BenchmarkMergeJoiner(b *testing.B) { ctx := context.Background() nCols := 4 - sourceTypes := make([]coltypes.T, nCols) + sourceTypes := make([]types.T, nCols) for colIdx := 0; colIdx < nCols; colIdx++ { - sourceTypes[colIdx] = coltypes.Int64 + sourceTypes[colIdx] = *types.Int } batch := testAllocator.NewMemBatch(sourceTypes) @@ -2079,12 +2082,12 @@ func BenchmarkMergeJoiner(b *testing.B) { b.SetBytes(int64(8 * nBatches * coldata.BatchSize() * nCols * 2)) b.ResetTimer() for i := 0; i < b.N; i++ { - leftSource := newFiniteBatchSource(newBatchOfIntRows(nCols, batch), nBatches) - rightSource := newFiniteBatchSource(newBatchOfIntRows(nCols, batch), nBatches) + leftSource := newFiniteBatchSource(newBatchOfIntRows(nCols, batch), sourceTypes, nBatches) + rightSource := newFiniteBatchSource(newBatchOfIntRows(nCols, batch), sourceTypes, nBatches) benchMemAccount.Clear(ctx) base, err := newMergeJoinBase( - NewAllocator(ctx, &benchMemAccount), defaultMemoryLimit, queueCfg, 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}}, @@ -2110,12 +2113,12 @@ func BenchmarkMergeJoiner(b *testing.B) { b.SetBytes(int64(8 * nBatches * coldata.BatchSize() * nCols * 2)) b.ResetTimer() for i := 0; i < b.N; i++ { - leftSource := newFiniteBatchSource(newBatchOfRepeatedIntRows(nCols, batch, nBatches), nBatches) - rightSource := newFiniteBatchSource(newBatchOfIntRows(nCols, batch), nBatches) + leftSource := newFiniteBatchSource(newBatchOfRepeatedIntRows(nCols, batch, nBatches), sourceTypes, nBatches) + rightSource := newFiniteBatchSource(newBatchOfIntRows(nCols, batch), sourceTypes, nBatches) benchMemAccount.Clear(ctx) base, err := newMergeJoinBase( - NewAllocator(ctx, &benchMemAccount), defaultMemoryLimit, queueCfg, 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}}, @@ -2143,12 +2146,12 @@ func BenchmarkMergeJoiner(b *testing.B) { b.SetBytes(int64(8 * nBatches * coldata.BatchSize() * nCols * 2)) b.ResetTimer() for i := 0; i < b.N; i++ { - leftSource := newFiniteBatchSource(newBatchOfRepeatedIntRows(nCols, batch, numRepeats), nBatches) - rightSource := newFiniteBatchSource(newBatchOfRepeatedIntRows(nCols, batch, numRepeats), nBatches) + leftSource := newFiniteBatchSource(newBatchOfRepeatedIntRows(nCols, batch, numRepeats), sourceTypes, nBatches) + rightSource := newFiniteBatchSource(newBatchOfRepeatedIntRows(nCols, batch, numRepeats), sourceTypes, nBatches) benchMemAccount.Clear(ctx) base, err := newMergeJoinBase( - NewAllocator(ctx, &benchMemAccount), defaultMemoryLimit, queueCfg, 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 7f833f7e0ea1..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/colexec/execerror" - // {{/* "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 { - execerror.VectorizedInternalPanic("") + 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 { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // _L_SEL_IND is the template type variable for the loop variable that @@ -271,7 +272,7 @@ func _PROBE_SWITCH( } // {{end}} default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled type %d", 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 { - execerror.VectorizedInternalPanic(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 { - execerror.VectorizedInternalPanic(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. @@ -553,8 +554,10 @@ EqLoop: rightColIdx := o.right.eqCols[eqColIdx] lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) - leftPhysType := o.left.sourceTypes[leftColIdx] - rightPhysType := o.right.sourceTypes[rightColIdx] + leftType := o.left.sourceTypes[leftColIdx] + leftPhysType := o.left.sourcePhysTypes[leftColIdx] + rightType := o.right.sourceTypes[rightColIdx] + rightPhysType := o.right.sourcePhysTypes[rightColIdx] colType := leftPhysType // Merge joiner only supports the case when the physical types in the // equality columns in both inputs are the same. If that is not the case, @@ -577,23 +580,26 @@ EqLoop: case coltypes.Float64: castLeftToRight = rightPhysType == coltypes.Decimal } - toType := leftPhysType + + toType, toPhysType := leftType, leftPhysType if castLeftToRight { - toType = rightPhysType + toType, toPhysType = rightType, rightPhysType } - tempVec := o.scratch.tempVecByType[toType] + tempVec := o.scratch.tempVecByType[toPhysType] if tempVec == nil { - tempVec = o.unlimitedAllocator.NewMemColumn(toType, coldata.BatchSize()) - o.scratch.tempVecByType[toType] = tempVec + // TODO(yuzefovich): this will need to be changed once we fully + // support coltypes.Datum. + tempVec = o.unlimitedAllocator.NewMemColumn(&toType, coldata.BatchSize()) + o.scratch.tempVecByType[toPhysType] = tempVec } else { tempVec.Nulls().UnsetNulls() } if castLeftToRight { - cast(leftPhysType, rightPhysType, lVec, tempVec, o.proberState.lBatch.Length(), lSel) + cast(&leftType, &rightType, lVec, tempVec, o.proberState.lBatch.Length(), lSel) lVec = tempVec - colType = o.right.sourceTypes[rightColIdx] + colType = rightPhysType } else { - cast(rightPhysType, leftPhysType, rVec, tempVec, o.proberState.rBatch.Length(), rSel) + cast(&rightType, &leftType, rVec, tempVec, o.proberState.rBatch.Length(), rSel) rVec = tempVec } } @@ -710,7 +716,7 @@ func _LEFT_SWITCH(_JOIN_TYPE joinTypeInfo, _HAS_SELECTION bool, _HAS_NULLS bool) o.builderState.left.groupsIdx = zeroMJCPGroupsIdx // {{end}} default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled type %d", colType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) } // {{end}} // {{/* @@ -750,7 +756,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildLeftGroupsFromBatch( func() { // Loop over every column. LeftColLoop: - for colIdx, colType := range input.sourceTypes { + for colIdx, colType := range input.sourcePhysTypes { outStartIdx := destStartIdx out := o.output.ColVec(colIdx) var src coldata.Vec @@ -798,7 +804,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildLeftBufferedGroup( if currentBatch == nil { currentBatch, err = bufferedGroup.dequeue(ctx) if err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } o.builderState.lBufferedGroupBatch = currentBatch o.builderState.left.curSrcStartIdx = 0 @@ -813,7 +819,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildLeftBufferedGroup( var updatedDestStartIdx int // Loop over every column. LeftColLoop: - for colIdx, colType := range input.sourceTypes { + for colIdx, colType := range input.sourcePhysTypes { outStartIdx := destStartIdx src := currentBatch.ColVec(colIdx) out := o.output.ColVec(colIdx) @@ -870,7 +876,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildLeftBufferedGroup( } // {{end}} default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled type %d", colType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) } updatedDestStartIdx = outStartIdx o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -884,7 +890,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildLeftBufferedGroup( o.unlimitedAllocator.ReleaseBatch(currentBatch) currentBatch, err = bufferedGroup.dequeue(ctx) if err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } o.builderState.lBufferedGroupBatch = currentBatch batchLength = currentBatch.Length() @@ -1004,7 +1010,7 @@ func _RIGHT_SWITCH(_JOIN_TYPE joinTypeInfo, _HAS_SELECTION bool, _HAS_NULLS bool o.builderState.right.groupsIdx = zeroMJCPGroupsIdx // {{end}} default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled type %d", colType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) } // {{end}} // {{/* @@ -1043,7 +1049,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildRightGroupsFromBatch( func() { // Loop over every column. RightColLoop: - for colIdx, colType := range input.sourceTypes { + for colIdx, colType := range input.sourcePhysTypes { outStartIdx := destStartIdx out := o.output.ColVec(colIdx + colOffset) var src coldata.Vec @@ -1098,7 +1104,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildRightBufferedGroup( if currentBatch == nil { currentBatch, err = bufferedGroup.dequeue(ctx) if err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } o.builderState.rBufferedGroupBatch = currentBatch o.builderState.right.curSrcStartIdx = 0 @@ -1111,7 +1117,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildRightBufferedGroup( } // Loop over every column. - for colIdx, colType := range input.sourceTypes { + for colIdx, colType := range input.sourcePhysTypes { out := o.output.ColVec(colIdx + colOffset) src := currentBatch.ColVec(colIdx) switch colType { @@ -1144,7 +1150,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildRightBufferedGroup( } // {{end}} default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled type %d", colType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) } } outStartIdx += toAppend @@ -1160,7 +1166,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildRightBufferedGroup( o.unlimitedAllocator.ReleaseBatch(currentBatch) currentBatch, err = bufferedGroup.dequeue(ctx) if err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } o.builderState.rBufferedGroupBatch = currentBatch batchLength = currentBatch.Length() @@ -1169,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } o.builderState.rBufferedGroupBatch = nil } @@ -1381,7 +1387,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) build(ctx context.Context) { // {{ end }} default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) + colexecerror.InternalError(fmt.Sprintf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) } } } @@ -1501,7 +1507,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) Next(ctx context.Context) coldata.Batch { } return coldata.ZeroBatch default: - execerror.VectorizedInternalPanic(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 ea3e9608e936..a12061c69550 100644 --- a/pkg/sql/colexec/min_max_agg_tmpl.go +++ b/pkg/sql/colexec/min_max_agg_tmpl.go @@ -27,15 +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/colexec/execerror" + "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. @@ -57,6 +64,9 @@ var _ tree.Datum // Dummy import to pull in "math" package. var _ = math.MaxInt64 +// Dummy import to pull in "coltypes" package. +var _ = coltypes.T + // _GOTYPESLICE is the template Go type slice variable for this operator. It // will be replaced by the Go slice representation for each type in coltypes.T, for // example []int64 for coltypes.Int64. @@ -66,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 { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} @@ -76,8 +86,8 @@ 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 *Allocator, t coltypes.T) (aggregateFunc, error) { - switch t { +func new_AGG_TITLEAgg(allocator *colmem.Allocator, t *types.T) (aggregateFunc, error) { + switch typeconv.FromColumnType(t) { // {{range .Overloads}} case _TYPES_T: return &_AGG_TYPEAgg{allocator: allocator}, nil @@ -90,7 +100,7 @@ func new_AGG_TITLEAgg(allocator *Allocator, t coltypes.T) (aggregateFunc, error) // {{range .Overloads}} type _AGG_TYPEAgg struct { - allocator *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 8fc1ce221e2f..f90f5c0a0177 100644 --- a/pkg/sql/colexec/offset.go +++ b/pkg/sql/colexec/offset.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" ) // offsetOp is an operator that implements offset, returning everything @@ -27,10 +28,10 @@ type offsetOp struct { seen int } -var _ Operator = &offsetOp{} +var _ colexecbase.Operator = &offsetOp{} // NewOffsetOp returns a new offset operator with the given offset. -func NewOffsetOp(input Operator, offset int) 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 a1de98298608..793a785d1ef9 100644 --- a/pkg/sql/colexec/offset_test.go +++ b/pkg/sql/colexec/offset_test.go @@ -15,7 +15,8 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) @@ -56,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 []Operator) (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 }) } @@ -64,9 +65,10 @@ func TestOffset(t *testing.T) { func BenchmarkOffset(b *testing.B) { ctx := context.Background() - batch := testAllocator.NewMemBatch([]coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}) + typs := []types.T{*types.Int, *types.Int, *types.Int} + batch := testAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) - source := NewRepeatableBatchSource(testAllocator, batch) + 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 0b6d16b2d3f7..835cd997b2f1 100644 --- a/pkg/sql/colexec/one_shot.go +++ b/pkg/sql/colexec/one_shot.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" ) // oneShotOp is an operator that does an arbitrary operation on the first batch @@ -22,12 +23,12 @@ import ( type oneShotOp struct { OneInputNode - outputSourceRef *Operator + outputSourceRef *colexecbase.Operator fn func(batch coldata.Batch) } -var _ 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 3aa6349afcf1..394ea088f745 100644 --- a/pkg/sql/colexec/operator.go +++ b/pkg/sql/colexec/operator.go @@ -15,38 +15,13 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" ) -// Operator is a column vector operator that produces a Batch as output. -type Operator interface { - // Init initializes this operator. Will be called once at operator setup - // time. If an operator has an input operator, it's responsible for calling - // Init on that input operator as well. - // TODO(yuzefovich): we might need to clarify whether it is ok to call - // Init() multiple times before the first call to Next(). It is possible to - // hit the memory limit during Init(), and a disk-backed operator needs to - // make sure that the input has been initialized. We could also in case that - // Init() doesn't succeed for bufferingInMemoryOperator - which should only - // happen when 'workmem' setting is too low - just bail, even if we have - // disk spilling for that operator. - Init() - - // Next returns the next Batch from this operator. Once the operator is - // finished, it will return a Batch with length 0. Subsequent calls to - // Next at that point will always return a Batch with length 0. - // - // Calling Next may invalidate the contents of the last Batch returned by - // Next. - // Canceling the provided context results in forceful termination of - // execution. - Next(context.Context) coldata.Batch - - execinfra.OpNode -} - // OperatorInitStatus indicates whether Init method has already been called on // an Operator. type OperatorInitStatus int @@ -67,13 +42,13 @@ type NonExplainable interface { } // NewOneInputNode returns an execinfra.OpNode with a single Operator input. -func NewOneInputNode(input 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 Operator + input colexecbase.Operator } // ChildCount implements the execinfra.OpNode interface. @@ -86,39 +61,24 @@ func (n OneInputNode) Child(nth int, verbose bool) execinfra.OpNode { if nth == 0 { return n.input } - execerror.VectorizedInternalPanic(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() Operator { +func (n OneInputNode) Input() colexecbase.Operator { return n.input } -// ZeroInputNode is an execinfra.OpNode with no inputs. -type ZeroInputNode struct{} - -// ChildCount implements the execinfra.OpNode interface. -func (ZeroInputNode) ChildCount(verbose bool) int { - return 0 -} - -// Child implements the execinfra.OpNode interface. -func (ZeroInputNode) Child(nth int, verbose bool) execinfra.OpNode { - execerror.VectorizedInternalPanic(fmt.Sprintf("invalid index %d", nth)) - // This code is unreachable, but the compiler cannot infer that. - return nil -} - // newTwoInputNode returns an execinfra.OpNode with two Operator inputs. -func newTwoInputNode(inputOne, inputTwo Operator) twoInputNode { +func newTwoInputNode(inputOne, inputTwo colexecbase.Operator) twoInputNode { return twoInputNode{inputOne: inputOne, inputTwo: inputTwo} } type twoInputNode struct { - inputOne Operator - inputTwo Operator + inputOne colexecbase.Operator + inputTwo colexecbase.Operator } func (twoInputNode) ChildCount(verbose bool) int { @@ -132,7 +92,7 @@ func (n *twoInputNode) Child(nth int, verbose bool) execinfra.OpNode { case 1: return n.inputTwo } - execerror.VectorizedInternalPanic(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 } @@ -145,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 { - Operator + colexecbase.Operator // InternalMemoryUsage reports the internal memory usage (in bytes) of an // operator. InternalMemoryUsage() int @@ -159,7 +119,7 @@ type resetter interface { // resettableOperator is an Operator that can be reset. type resettableOperator interface { - Operator + colexecbase.Operator resetter } @@ -188,7 +148,7 @@ func (c *closerHelper) close() bool { } type closableOperator interface { - Operator + colexecbase.Operator IdempotentCloser } @@ -197,10 +157,10 @@ type noopOperator struct { NonExplainable } -var _ Operator = &noopOperator{} +var _ colexecbase.Operator = &noopOperator{} // NewNoop returns a new noop Operator. -func NewNoop(input Operator) Operator { +func NewNoop(input colexecbase.Operator) colexecbase.Operator { return &noopOperator{OneInputNode: NewOneInputNode(input)} } @@ -223,10 +183,10 @@ type zeroOperator struct { NonExplainable } -var _ Operator = &zeroOperator{} +var _ colexecbase.Operator = &zeroOperator{} // NewZeroOp creates a new operator which just returns an empty batch. -func NewZeroOp(input Operator) Operator { +func NewZeroOp(input colexecbase.Operator) colexecbase.Operator { return &zeroOperator{OneInputNode: NewOneInputNode(input)} } @@ -241,18 +201,18 @@ func (s *zeroOperator) Next(ctx context.Context) coldata.Batch { } type singleTupleNoInputOperator struct { - ZeroInputNode + colexecbase.ZeroInputNode NonExplainable batch coldata.Batch nexted bool } -var _ 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 *Allocator) Operator { +func NewSingleTupleNoInputOp(allocator *colmem.Allocator) colexecbase.Operator { return &singleTupleNoInputOperator{ batch: allocator.NewMemBatchWithSize(nil /* types */, 1 /* size */), } @@ -274,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 { - ZeroInputNode + colexecbase.ZeroInputNode NonExplainable batch coldata.Batch } @@ -285,7 +245,7 @@ func (o *feedOperator) Next(context.Context) coldata.Batch { return o.batch } -var _ 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 @@ -315,14 +275,16 @@ type vectorTypeEnforcer struct { OneInputNode NonExplainable - allocator *Allocator - typ coltypes.T + allocator *colmem.Allocator + typ *types.T idx int } -var _ Operator = &vectorTypeEnforcer{} +var _ colexecbase.Operator = &vectorTypeEnforcer{} -func newVectorTypeEnforcer(allocator *Allocator, input Operator, typ coltypes.T, idx int) Operator { +func newVectorTypeEnforcer( + allocator *colmem.Allocator, input colexecbase.Operator, typ *types.T, idx int, +) colexecbase.Operator { return &vectorTypeEnforcer{ OneInputNode: NewOneInputNode(input), allocator: allocator, @@ -340,7 +302,7 @@ func (e *vectorTypeEnforcer) Next(ctx context.Context) coldata.Batch { if b.Length() == 0 { return b } - e.allocator.maybeAppendColumn(b, e.typ, e.idx) + e.allocator.MaybeAppendColumn(b, e.typ, e.idx) return b } @@ -358,14 +320,14 @@ type batchSchemaPrefixEnforcer struct { OneInputNode NonExplainable - allocator *Allocator - typs []coltypes.T + allocator *colmem.Allocator + typs []types.T } -var _ Operator = &batchSchemaPrefixEnforcer{} +var _ colexecbase.Operator = &batchSchemaPrefixEnforcer{} func newBatchSchemaPrefixEnforcer( - allocator *Allocator, input Operator, typs []coltypes.T, + allocator *colmem.Allocator, input colexecbase.Operator, typs []types.T, ) *batchSchemaPrefixEnforcer { return &batchSchemaPrefixEnforcer{ OneInputNode: NewOneInputNode(input), @@ -384,7 +346,7 @@ func (e *batchSchemaPrefixEnforcer) Next(ctx context.Context) coldata.Batch { return b } for i, typ := range e.typs { - e.allocator.maybeAppendColumn(b, typ, i) + e.allocator.MaybeAppendColumn(b, &typ, i) } return b } diff --git a/pkg/sql/colexec/orderedsynchronizer_test.go b/pkg/sql/colexec/orderedsynchronizer_test.go index d3053a9f194c..1c4c43772266 100644 --- a/pkg/sql/colexec/orderedsynchronizer_test.go +++ b/pkg/sql/colexec/orderedsynchronizer_test.go @@ -17,10 +17,12 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "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" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" ) // Adapted from the same-named test in the rowflow package. @@ -137,17 +139,12 @@ func TestOrderedSync(t *testing.T) { } for _, tc := range testCases { numCols := len(tc.sources[0][0]) - columnTypes := make([]coltypes.T, numCols) - for i := range columnTypes { - columnTypes[i] = coltypes.Int64 + typs := make([]types.T, numCols) + for i := range typs { + typs[i] = *types.Int } - runTests(t, tc.sources, tc.expected, orderedVerifier, func(inputs []Operator) (Operator, error) { - return &OrderedSynchronizer{ - allocator: testAllocator, - inputs: inputs, - ordering: tc.ordering, - columnTypes: columnTypes, - }, nil + runTests(t, tc.sources, tc.expected, orderedVerifier, func(inputs []colexecbase.Operator) (colexecbase.Operator, error) { + return NewOrderedSynchronizer(testAllocator, inputs, typs, tc.ordering) }) } } @@ -156,10 +153,11 @@ func TestOrderedSyncRandomInput(t *testing.T) { defer leaktest.AfterTest(t)() numInputs := 3 inputLen := 1024 - batchSize := int(16) + batchSize := 16 if batchSize > coldata.BatchSize() { batchSize = coldata.BatchSize() } + typs := []types.T{*types.Int} // Generate a random slice of sorted ints. randInts := make([]int, inputLen) @@ -181,24 +179,15 @@ func TestOrderedSyncRandomInput(t *testing.T) { } sources[sourceIdx] = append(sources[sourceIdx], t) } - inputs := make([]Operator, numInputs) + inputs := make([]colexecbase.Operator, numInputs) for i := range inputs { - inputs[i] = newOpTestInput(batchSize, sources[i], nil /* typs */) - } - - op := OrderedSynchronizer{ - allocator: testAllocator, - inputs: inputs, - ordering: sqlbase.ColumnOrdering{ - { - ColIdx: 0, - Direction: encoding.Ascending, - }, - }, - columnTypes: []coltypes.T{coltypes.Int64}, + inputs[i] = newOpTestInput(batchSize, sources[i], typs) } + ordering := sqlbase.ColumnOrdering{{ColIdx: 0, Direction: encoding.Ascending}} + op, err := NewOrderedSynchronizer(testAllocator, inputs, typs, ordering) + require.NoError(t, err) op.Init() - out := newOpTestOutput(&op, expected) + out := newOpTestOutput(op, expected) if err := out.Verify(); err != nil { t.Error(err) } @@ -208,9 +197,10 @@ func BenchmarkOrderedSynchronizer(b *testing.B) { ctx := context.Background() numInputs := int64(3) + typs := []types.T{*types.Int} batches := make([]coldata.Batch, numInputs) for i := range batches { - batches[i] = testAllocator.NewMemBatch([]coltypes.T{coltypes.Int64}) + batches[i] = testAllocator.NewMemBatch(typs) batches[i].SetLength(coldata.BatchSize()) } for i := int64(0); i < int64(coldata.BatchSize())*numInputs; i++ { @@ -218,19 +208,14 @@ func BenchmarkOrderedSynchronizer(b *testing.B) { batch.ColVec(0).Int64()[i/numInputs] = i } - inputs := make([]Operator, len(batches)) + inputs := make([]colexecbase.Operator, len(batches)) for i := range batches { - inputs[i] = NewRepeatableBatchSource(testAllocator, batches[i]) + inputs[i] = colexecbase.NewRepeatableBatchSource(testAllocator, batches[i], typs) } - op := OrderedSynchronizer{ - allocator: testAllocator, - inputs: inputs, - ordering: sqlbase.ColumnOrdering{ - {ColIdx: 0, Direction: encoding.Ascending}, - }, - columnTypes: []coltypes.T{coltypes.Int64}, - } + ordering := sqlbase.ColumnOrdering{{ColIdx: 0, Direction: encoding.Ascending}} + op, err := NewOrderedSynchronizer(testAllocator, inputs, typs, ordering) + require.NoError(b, err) op.Init() b.SetBytes(8 * int64(coldata.BatchSize()) * numInputs) diff --git a/pkg/sql/colexec/orderedsynchronizer_tmpl.go b/pkg/sql/colexec/orderedsynchronizer_tmpl.go index 47c4f3cebd19..e9f5c12d908f 100644 --- a/pkg/sql/colexec/orderedsynchronizer_tmpl.go +++ b/pkg/sql/colexec/orderedsynchronizer_tmpl.go @@ -28,16 +28,21 @@ 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/colexec/execerror" - // {{/* + "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" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/encoding" ) +// Remove unused warning. +var _ = execgen.UNSAFEGET + // {{/* // Declarations to make the template compile properly. @@ -65,10 +70,11 @@ 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 *Allocator - inputs []Operator - ordering sqlbase.ColumnOrdering - columnTypes []coltypes.T + allocator *colmem.Allocator + inputs []colexecbase.Operator + ordering sqlbase.ColumnOrdering + typs []types.T + physTypes []coltypes.T // inputBatches stores the current batch for each input. inputBatches []coldata.Batch @@ -103,7 +109,7 @@ type OrderedSynchronizer struct { outColsMap []int } -var _ Operator = &OrderedSynchronizer{} +var _ colexecbase.Operator = &OrderedSynchronizer{} // ChildCount implements the execinfrapb.OpNode interface. func (o *OrderedSynchronizer) ChildCount(verbose bool) int { @@ -117,14 +123,19 @@ func (o *OrderedSynchronizer) Child(nth int, verbose bool) execinfra.OpNode { // NewOrderedSynchronizer creates a new OrderedSynchronizer. func NewOrderedSynchronizer( - allocator *Allocator, inputs []Operator, typs []coltypes.T, ordering sqlbase.ColumnOrdering, -) *OrderedSynchronizer { + allocator *colmem.Allocator, + inputs []colexecbase.Operator, + typs []types.T, + ordering sqlbase.ColumnOrdering, +) (*OrderedSynchronizer, error) { + physTypes, err := typeconv.FromColumnTypes(typs) return &OrderedSynchronizer{ - allocator: allocator, - inputs: inputs, - ordering: ordering, - columnTypes: typs, - } + allocator: allocator, + inputs: inputs, + ordering: ordering, + typs: typs, + physTypes: physTypes, + }, err } // Next is part of the Operator interface. @@ -157,7 +168,7 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { if sel := batch.Selection(); sel != nil { srcRowIdx = sel[srcRowIdx] } - for i, physType := range o.columnTypes { + for i, physType := range o.physTypes { vec := batch.ColVec(i) if vec.Nulls().MaybeHasNulls() && vec.Nulls().NullAt(srcRowIdx) { o.outNulls[i].SetNull(outputIdx) @@ -171,7 +182,7 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { execgen.SET(outCol, outputIdx, v) // {{end}} default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled type %d", physType)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", physType)) } } } @@ -201,19 +212,19 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { // Init is part of the Operator interface. func (o *OrderedSynchronizer) Init() { o.inputIndices = make([]int, len(o.inputs)) - o.output = o.allocator.NewMemBatch(o.columnTypes) - o.outNulls = make([]*coldata.Nulls, len(o.columnTypes)) - o.outColsMap = make([]int, len(o.columnTypes)) + o.output = o.allocator.NewMemBatch(o.typs) + o.outNulls = make([]*coldata.Nulls, len(o.typs)) + o.outColsMap = make([]int, len(o.typs)) for i, outVec := range o.output.ColVecs() { o.outNulls[i] = outVec.Nulls() - switch o.columnTypes[i] { + switch o.physTypes[i] { // {{range .}} case _TYPES_T: o.outColsMap[i] = len(o.out_TYPECols) o.out_TYPECols = append(o.out_TYPECols, outVec._TYPE()) // {{end}} default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled type %d", o.columnTypes[i])) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", &o.typs[i])) } } for i := range o.inputs { @@ -221,8 +232,8 @@ func (o *OrderedSynchronizer) Init() { } o.comparators = make([]vecComparator, len(o.ordering)) for i := range o.ordering { - typ := o.columnTypes[o.ordering[i].ColIdx] - o.comparators[i] = GetVecComparator(typ, len(o.inputs)) + typ := o.typs[o.ordering[i].ColIdx] + o.comparators[i] = GetVecComparator(&typ, len(o.inputs)) } } @@ -247,7 +258,7 @@ func (o *OrderedSynchronizer) compareRow(batchIdx1 int, batchIdx2 int) int { case encoding.Descending: return -res default: - execerror.VectorizedInternalPanic(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 7deffce5b59d..74243fa189f4 100644 --- a/pkg/sql/colexec/ordinality.go +++ b/pkg/sql/colexec/ordinality.go @@ -14,7 +14,9 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/types" ) // ordinalityOp is an operator that implements WITH ORDINALITY, which adds @@ -22,7 +24,7 @@ import ( type ordinalityOp struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator // outputIdx is the index of the column in which ordinalityOp will write the // ordinal number. outputIdx int @@ -30,11 +32,13 @@ type ordinalityOp struct { counter int64 } -var _ Operator = &ordinalityOp{} +var _ colexecbase.Operator = &ordinalityOp{} // NewOrdinalityOp returns a new WITH ORDINALITY operator. -func NewOrdinalityOp(allocator *Allocator, input Operator, outputIdx int) Operator { - input = newVectorTypeEnforcer(allocator, input, coltypes.Int64, outputIdx) +func NewOrdinalityOp( + allocator *colmem.Allocator, input colexecbase.Operator, outputIdx int, +) colexecbase.Operator { + input = newVectorTypeEnforcer(allocator, input, types.Int, outputIdx) c := &ordinalityOp{ OneInputNode: NewOneInputNode(input), allocator: allocator, diff --git a/pkg/sql/colexec/ordinality_test.go b/pkg/sql/colexec/ordinality_test.go index 5cf9ec93350b..c31d6643b22d 100644 --- a/pkg/sql/colexec/ordinality_test.go +++ b/pkg/sql/colexec/ordinality_test.go @@ -15,8 +15,8 @@ import ( "testing" "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/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 []Operator) (Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return createTestOrdinalityOperator(ctx, flowCtx, input[0], tc.inputTypes) }) } @@ -84,9 +84,10 @@ func BenchmarkOrdinality(b *testing.B) { }, } - batch := testAllocator.NewMemBatch([]coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}) + typs := []types.T{*types.Int, *types.Int, *types.Int} + batch := testAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) - source := NewRepeatableBatchSource(testAllocator, batch) + 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() @@ -98,8 +99,8 @@ func BenchmarkOrdinality(b *testing.B) { } func createTestOrdinalityOperator( - ctx context.Context, flowCtx *execinfra.FlowCtx, input Operator, inputTypes []types.T, -) (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{ @@ -108,7 +109,7 @@ func createTestOrdinalityOperator( } args := NewColOperatorArgs{ Spec: spec, - Inputs: []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 4e5d9ba3bbdf..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/colexec/execerror" + "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(execerror.CatchVectorizedRuntimeError(func() { performPlusInt16Int16(1, math.MaxInt16) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.CatchVectorizedRuntimeError(func() { performPlusInt16Int16(-1, math.MinInt16) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.CatchVectorizedRuntimeError(func() { performPlusInt16Int16(math.MaxInt16, 1) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.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(execerror.CatchVectorizedRuntimeError(func() { performMinusInt16Int16(1, -math.MaxInt16) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.CatchVectorizedRuntimeError(func() { performMinusInt16Int16(-2, math.MaxInt16) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.CatchVectorizedRuntimeError(func() { performMinusInt16Int16(math.MaxInt16, -1) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.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(execerror.CatchVectorizedRuntimeError(func() { performDivInt16Int16(10, 0) }), tree.ErrDivByZero)) - require.True(t, errors.Is(execerror.CatchVectorizedRuntimeError(func() { performDivInt32Int32(10, 0) }), tree.ErrDivByZero)) - require.True(t, errors.Is(execerror.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(execerror.CatchVectorizedRuntimeError(func() { performMultInt16Int16(math.MaxInt16-1, 100) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.CatchVectorizedRuntimeError(func() { performMultInt16Int16(math.MaxInt16-1, 3) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.CatchVectorizedRuntimeError(func() { performMultInt16Int16(math.MinInt16+1, 3) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.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(execerror.CatchVectorizedRuntimeError(func() { performMultInt32Int32(math.MaxInt32-1, 100) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.CatchVectorizedRuntimeError(func() { performMultInt32Int32(math.MaxInt32-1, 3) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.CatchVectorizedRuntimeError(func() { performMultInt32Int32(math.MinInt32+1, 3) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.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(execerror.CatchVectorizedRuntimeError(func() { performMultInt64Int64(math.MaxInt64-1, 100) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.CatchVectorizedRuntimeError(func() { performMultInt64Int64(math.MaxInt64-1, 3) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.CatchVectorizedRuntimeError(func() { performMultInt64Int64(math.MinInt64+1, 3) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.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(execerror.CatchVectorizedRuntimeError(func() { performMultInt16Int16(math.MinInt16, -1) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.CatchVectorizedRuntimeError(func() { performMultInt32Int32(math.MinInt32, -1) }), tree.ErrIntOutOfRange)) - require.True(t, errors.Is(execerror.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(execerror.CatchVectorizedRuntimeError(func() { performDivDecimalInt16(nonZeroDec, 0) }), tree.ErrDivByZero)) - require.True(t, errors.Is(execerror.CatchVectorizedRuntimeError(func() { performDivDecimalInt32(nonZeroDec, 0) }), tree.ErrDivByZero)) - require.True(t, errors.Is(execerror.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(execerror.CatchVectorizedRuntimeError(func() { performDivInt64Decimal(2, zeroDec) }), tree.ErrDivByZero)) - require.True(t, errors.Is(execerror.CatchVectorizedRuntimeError(func() { performDivInt32Decimal(2, zeroDec) }), tree.ErrDivByZero)) - require.True(t, errors.Is(execerror.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(execerror.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 348e401a8144..398364ab2c4d 100644 --- a/pkg/sql/colexec/parallel_unordered_synchronizer.go +++ b/pkg/sql/colexec/parallel_unordered_synchronizer.go @@ -16,9 +16,10 @@ import ( "sync/atomic" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" ) @@ -30,13 +31,13 @@ type unorderedSynchronizerMsg struct { b coldata.Batch } -var _ 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 []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 @@ -90,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 []Operator, typs []coltypes.T, wg *sync.WaitGroup, + inputs []colexecbase.Operator, typs []types.T, wg *sync.WaitGroup, ) *ParallelUnorderedSynchronizer { readNextBatch := make([]chan struct{}, len(inputs)) for i := range readNextBatch { @@ -131,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 Operator, inputIdx int) func() { + s.nextBatch[i] = func(input colexecbase.Operator, inputIdx int) func() { return func() { s.batches[inputIdx] = input.Next(ctx) } @@ -141,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 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) @@ -153,7 +154,7 @@ func (s *ParallelUnorderedSynchronizer) init(ctx context.Context) { inputIdx: inputIdx, } for { - if err := execerror.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. @@ -216,7 +217,7 @@ func (s *ParallelUnorderedSynchronizer) Next(ctx context.Context) coldata.Batch // propagate this error through a panic. s.cancelFn() s.internalWaitGroup.Wait() - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } case msg := <-s.batchCh: if msg == nil { @@ -226,7 +227,7 @@ func (s *ParallelUnorderedSynchronizer) Next(ctx context.Context) coldata.Batch select { case err := <-s.errCh: if err != nil { - execerror.VectorizedInternalPanic(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 0389f452fd0b..420287adad0f 100644 --- a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go +++ b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go @@ -18,8 +18,10 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -36,16 +38,17 @@ func TestParallelUnorderedSynchronizer(t *testing.T) { var ( rng, _ = randutil.NewPseudoRand() - typs = []coltypes.T{coltypes.Int64} + typs = []types.T{*types.Int} numInputs = rng.Intn(maxInputs) + 1 numBatches = rng.Intn(maxBatches) + 1 ) - inputs := make([]Operator, numInputs) + inputs := make([]colexecbase.Operator, numInputs) for i := range inputs { - source := NewRepeatableBatchSource( + source := colexecbase.NewRepeatableBatchSource( testAllocator, - 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) inputs[i] = source @@ -75,7 +78,7 @@ func TestParallelUnorderedSynchronizer(t *testing.T) { batchesReturned := 0 for { var b coldata.Batch - if err := execerror.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 @@ -99,17 +102,17 @@ func TestUnorderedSynchronizerNoLeaksOnError(t *testing.T) { const expectedErr = "first input error" - inputs := make([]Operator, 6) - inputs[0] = &CallbackOperator{NextCb: func(context.Context) coldata.Batch { - execerror.VectorizedInternalPanic(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] = &CallbackOperator{ + inputs[i] = &colexecbase.CallbackOperator{ NextCb: func(ctx context.Context) coldata.Batch { <-ctx.Done() - execerror.VectorizedInternalPanic(ctx.Err()) + colexecerror.InternalError(ctx.Err()) // This code is unreachable, but the compiler cannot infer that. return nil }, @@ -120,8 +123,8 @@ func TestUnorderedSynchronizerNoLeaksOnError(t *testing.T) { ctx = context.Background() wg sync.WaitGroup ) - s := NewParallelUnorderedSynchronizer(inputs, []coltypes.T{coltypes.Int64}, &wg) - err := execerror.CatchVectorizedRuntimeError(func() { _ = s.Next(ctx) }) + s := NewParallelUnorderedSynchronizer(inputs, []types.T{*types.Int}, &wg) + 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) @@ -130,12 +133,12 @@ func TestUnorderedSynchronizerNoLeaksOnError(t *testing.T) { func BenchmarkParallelUnorderedSynchronizer(b *testing.B) { const numInputs = 6 - typs := []coltypes.T{coltypes.Int64} - inputs := make([]Operator, numInputs) + typs := []types.T{*types.Int} + inputs := make([]colexecbase.Operator, numInputs) for i := range inputs { batch := testAllocator.NewMemBatchWithSize(typs, coldata.BatchSize()) batch.SetLength(coldata.BatchSize()) - inputs[i] = NewRepeatableBatchSource(testAllocator, batch) + 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 be8b04f31512..859e6b4a74c1 100644 --- a/pkg/sql/colexec/partially_ordered_distinct.go +++ b/pkg/sql/colexec/partially_ordered_distinct.go @@ -15,9 +15,12 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" ) @@ -28,22 +31,22 @@ const partiallyOrderedDistinctNumHashBuckets = 1024 // distinct columns when we have partial ordering on some of the distinct // columns. func newPartiallyOrderedDistinct( - allocator *Allocator, - input Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, distinctCols []uint32, orderedCols []uint32, - colTypes []coltypes.T, -) (Operator, error) { + typs []types.T, +) (colexecbase.Operator, error) { if len(orderedCols) == 0 || len(orderedCols) == len(distinctCols) { return nil, errors.AssertionFailedf( "partially ordered distinct wrongfully planned: numDistinctCols=%d "+ "numOrderedCols=%d", len(distinctCols), len(orderedCols)) } - chunker, err := newChunker(allocator, input, colTypes, orderedCols) + chunker, err := newChunker(allocator, input, typs, orderedCols) if err != nil { return nil, err } - chunkerOperator := newChunkerOperator(allocator, chunker, colTypes) + chunkerOperator := newChunkerOperator(allocator, chunker, typs) // distinctUnorderedCols will contain distinct columns that are not present // among orderedCols. The unordered distinct operator will use these columns // to find distinct tuples within "chunks" of tuples that are the same on the @@ -62,7 +65,7 @@ func newPartiallyOrderedDistinct( } } distinct := NewUnorderedDistinct( - allocator, chunkerOperator, distinctUnorderedCols, colTypes, + allocator, chunkerOperator, distinctUnorderedCols, typs, partiallyOrderedDistinctNumHashBuckets, ) return &partiallyOrderedDistinct{ @@ -80,7 +83,7 @@ type partiallyOrderedDistinct struct { distinct resettableOperator } -var _ Operator = &partiallyOrderedDistinct{} +var _ colexecbase.Operator = &partiallyOrderedDistinct{} func (p *partiallyOrderedDistinct) ChildCount(bool) int { return 1 @@ -90,7 +93,7 @@ func (p *partiallyOrderedDistinct) Child(nth int, _ bool) execinfra.OpNode { if nth == 0 { return p.input } - execerror.VectorizedInternalPanic(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 } @@ -116,7 +119,7 @@ func (p *partiallyOrderedDistinct) Next(ctx context.Context) coldata.Batch { } func newChunkerOperator( - allocator *Allocator, input *chunker, inputTypes []coltypes.T, + allocator *colmem.Allocator, input *chunker, inputTypes []types.T, ) *chunkerOperator { return &chunkerOperator{ input: input, @@ -134,7 +137,7 @@ func newChunkerOperator( // that the input has been fully processed). type chunkerOperator struct { input *chunker - inputTypes []coltypes.T + inputTypes []types.T // haveChunksToEmit indicates whether we have spooled input and still there // are more chunks to emit. haveChunksToEmit bool @@ -167,7 +170,7 @@ func (c *chunkerOperator) Child(nth int, _ bool) execinfra.OpNode { if nth == 0 { return c.input } - execerror.VectorizedInternalPanic(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 } @@ -211,7 +214,7 @@ func (c *chunkerOperator) Next(ctx context.Context) coldata.Batch { c.currentChunkFinished = true } for i, typ := range c.inputTypes { - window := c.input.getValues(i).Window(typ, c.outputTupleStartIdx, outputTupleEndIdx) + window := c.input.getValues(i).Window(typeconv.FromColumnType(&typ), c.outputTupleStartIdx, outputTupleEndIdx) c.windowedBatch.ReplaceCol(window, i) } c.windowedBatch.SetSelection(false) diff --git a/pkg/sql/colexec/partitioner.go b/pkg/sql/colexec/partitioner.go index aaf3ad9e0d17..c8b1ed99ed22 100644 --- a/pkg/sql/colexec/partitioner.go +++ b/pkg/sql/colexec/partitioner.go @@ -14,8 +14,10 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "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" ) // NewWindowSortingPartitioner creates a new colexec.Operator that orders input @@ -24,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 *Allocator, - input Operator, - inputTyps []coltypes.T, + allocator *colmem.Allocator, + input colexecbase.Operator, + inputTyps []types.T, partitionIdxs []uint32, ordCols []execinfrapb.Ordering_Column, partitionColIdx int, - createDiskBackedSorter func(input Operator, inputTypes []coltypes.T, orderingCols []execinfrapb.Ordering_Column) (Operator, error), -) (op 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} @@ -48,7 +50,7 @@ func NewWindowSortingPartitioner( return nil, err } - input = newVectorTypeEnforcer(allocator, input, coltypes.Bool, partitionColIdx) + input = newVectorTypeEnforcer(allocator, input, types.Bool, partitionColIdx) return &windowSortingPartitioner{ OneInputNode: NewOneInputNode(input), allocator: allocator, @@ -60,7 +62,7 @@ func NewWindowSortingPartitioner( type windowSortingPartitioner struct { OneInputNode - allocator *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 9ac2b2c342a6..4b5fa0354d9c 100644 --- a/pkg/sql/colexec/proj_const_ops_tmpl.go +++ b/pkg/sql/colexec/proj_const_ops_tmpl.go @@ -28,17 +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/colexec/execerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} - "github.com/cockroachdb/cockroach/pkg/sql/colexec/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/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. @@ -66,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{}) { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // _RET_UNSAFEGET is the template function that will be replaced by // "execgen.UNSAFEGET" which uses _RET_TYP. func _RET_UNSAFEGET(_, _ interface{}) interface{} { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} @@ -200,17 +203,17 @@ 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 *Allocator, - leftColType *types.T, - rightColType *types.T, - outputPhysType coltypes.T, + allocator *colmem.Allocator, + leftType *types.T, + rightType *types.T, + outputType *types.T, op tree.Operator, - input Operator, + input colexecbase.Operator, colIdx int, constArg tree.Datum, outputIdx int, -) (Operator, error) { - input = newVectorTypeEnforcer(allocator, input, outputPhysType, outputIdx) +) (colexecbase.Operator, error) { + input = newVectorTypeEnforcer(allocator, input, outputType, outputIdx) projConstOpBase := projConstOpBase{ OneInputNode: NewOneInputNode(input), allocator: allocator, @@ -218,17 +221,17 @@ func GetProjection_CONST_SIDEConstOperator( outputIdx: outputIdx, } // {{if _IS_CONST_LEFT}} - c, err := typeconv.GetDatumToPhysicalFn(leftColType)(constArg) + c, err := getDatumToPhysicalFn(leftType)(constArg) // {{else}} - c, err := typeconv.GetDatumToPhysicalFn(rightColType)(constArg) + c, err := getDatumToPhysicalFn(rightType)(constArg) // {{end}} if err != nil { return nil, err } - switch leftType := typeconv.FromColumnType(leftColType); leftType { + switch typeconv.FromColumnType(leftType) { // {{range $lTyp, $rTypToOverloads := .}} case coltypes._L_TYP_VAR: - switch rightType := typeconv.FromColumnType(rightColType); rightType { + switch typeconv.FromColumnType(rightType) { // {{range $rTyp, $overloads := $rTypToOverloads}} case coltypes._R_TYP_VAR: switch op.(type) { @@ -269,7 +272,7 @@ func GetProjection_CONST_SIDEConstOperator( return nil, errors.Errorf("unhandled comparison operator: %s", op) } default: - return nil, errors.New("unhandled operator type") + return nil, errors.Errorf("unhandled operator type: %s", op) } // {{end}} default: diff --git a/pkg/sql/colexec/proj_non_const_ops_tmpl.go b/pkg/sql/colexec/proj_non_const_ops_tmpl.go index 396ba902252e..749ce3637f07 100644 --- a/pkg/sql/colexec/proj_non_const_ops_tmpl.go +++ b/pkg/sql/colexec/proj_non_const_ops_tmpl.go @@ -26,17 +26,20 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} - "github.com/cockroachdb/cockroach/pkg/sql/colexec/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/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. @@ -58,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{}) { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // _L_UNSAFEGET is the template function that will be replaced by // "execgen.UNSAFEGET" which uses _L_TYP. func _L_UNSAFEGET(_, _ interface{}) interface{} { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // _R_UNSAFEGET is the template function that will be replaced by // "execgen.UNSAFEGET" which uses _R_TYP. func _R_UNSAFEGET(_, _ interface{}) interface{} { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // _RET_UNSAFEGET is the template function that will be replaced by // "execgen.UNSAFEGET" which uses _RET_TYP. func _RET_UNSAFEGET(_, _ interface{}) interface{} { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} @@ -88,7 +91,7 @@ func _RET_UNSAFEGET(_, _ interface{}) interface{} { // around the problem we specify it here. type projConstOpBase struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator colIdx int outputIdx int decimalScratch decimalOverloadScratch @@ -97,7 +100,7 @@ type projConstOpBase struct { // projOpBase contains all of the fields for non-constant binary projections. type projOpBase struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator col1Idx int col2Idx int outputIdx int @@ -226,17 +229,17 @@ 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 *Allocator, - leftColType *types.T, - rightColType *types.T, - outputPhysType coltypes.T, + allocator *colmem.Allocator, + leftType *types.T, + rightType *types.T, + outputType *types.T, op tree.Operator, - input Operator, + input colexecbase.Operator, col1Idx int, col2Idx int, outputIdx int, -) (Operator, error) { - input = newVectorTypeEnforcer(allocator, input, outputPhysType, outputIdx) +) (colexecbase.Operator, error) { + input = newVectorTypeEnforcer(allocator, input, outputType, outputIdx) projOpBase := projOpBase{ OneInputNode: NewOneInputNode(input), allocator: allocator, @@ -244,10 +247,10 @@ func GetProjectionOperator( col2Idx: col2Idx, outputIdx: outputIdx, } - switch leftType := typeconv.FromColumnType(leftColType); leftType { + switch typeconv.FromColumnType(leftType) { // {{range $lTyp, $rTypToOverloads := .}} case coltypes._L_TYP_VAR: - switch rightType := typeconv.FromColumnType(rightColType); rightType { + switch typeconv.FromColumnType(rightType) { // {{range $rTyp, $overloads := $rTypToOverloads}} case coltypes._R_TYP_VAR: switch op.(type) { diff --git a/pkg/sql/colexec/projection_ops_test.go b/pkg/sql/colexec/projection_ops_test.go index acf475f8399e..3858d1af08d2 100644 --- a/pkg/sql/colexec/projection_ops_test.go +++ b/pkg/sql/colexec/projection_ops_test.go @@ -18,9 +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/colexec/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" @@ -44,7 +46,7 @@ func TestProjPlusInt64Int64ConstOp(t *testing.T) { }, } runTests(t, []tuples{{{1}, {2}, {nil}}}, tuples{{1, 2}, {2, 3}, {nil, nil}}, orderedVerifier, - func(input []Operator) (Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return createTestProjectingOperator( ctx, flowCtx, input[0], []types.T{*types.Int}, "@1 + 1" /* projectingExpr */, false, /* canFallbackToRowexec */ @@ -66,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 []Operator) (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 */ @@ -88,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 []Operator) (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 */ @@ -107,7 +109,8 @@ func benchmarkProjPlusInt64Int64ConstOp(b *testing.B, useSelectionVector bool, h Settings: st, }, } - batch := testAllocator.NewMemBatch([]coltypes.T{coltypes.Int64, coltypes.Int64}) + typs := []types.T{*types.Int, *types.Int} + batch := testAllocator.NewMemBatch(typs) col := batch.ColVec(0).Int64() for i := 0; i < coldata.BatchSize(); i++ { col[i] = 1 @@ -127,7 +130,7 @@ func benchmarkProjPlusInt64Int64ConstOp(b *testing.B, useSelectionVector bool, h sel[i] = i } } - source := NewRepeatableBatchSource(testAllocator, batch) + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) plusOp, err := createTestProjectingOperator( ctx, flowCtx, source, []types.T{*types.Int}, "@1 + 1" /* projectingExpr */, false, /* canFallbackToRowexec */ @@ -154,13 +157,13 @@ func BenchmarkProjPlusInt64Int64ConstOp(b *testing.B) { func TestGetProjectionConstOperator(t *testing.T) { defer leaktest.AfterTest(t)() binOp := tree.Mult - var input Operator + var input colexecbase.Operator colIdx := 3 constVal := 31.37 constArg := tree.NewDFloat(tree.DFloat(constVal)) outputIdx := 5 op, err := GetProjectionRConstOperator( - testAllocator, types.Float, types.Float, coltypes.Float64, + testAllocator, types.Float, types.Float, types.Float, binOp, input, colIdx, constArg, outputIdx, ) if err != nil { @@ -183,13 +186,13 @@ func TestGetProjectionConstOperator(t *testing.T) { func TestGetProjectionConstMixedTypeOperator(t *testing.T) { defer leaktest.AfterTest(t)() binOp := tree.GE - var input Operator + var input colexecbase.Operator colIdx := 3 constVal := int16(31) constArg := tree.NewDInt(tree.DInt(constVal)) outputIdx := 5 op, err := GetProjectionRConstOperator( - testAllocator, types.Int, types.Int2, coltypes.Int64, + testAllocator, types.Int, types.Int2, types.Int, binOp, input, colIdx, constArg, outputIdx, ) if err != nil { @@ -231,29 +234,28 @@ func TestRandomComparisons(t *testing.T) { var da sqlbase.DatumAlloc lDatums := make([]tree.Datum, numTuples) rDatums := make([]tree.Datum, numTuples) - for _, ct := range types.Scalar { - if ct.Family() == types.DateFamily { + for _, typ := range types.Scalar { + if typ.Family() == types.DateFamily { // TODO(jordan): #40354 tracks failure to compare infinite dates. continue } - typ := typeconv.FromColumnType(ct) - if typ == coltypes.Unhandled { + if typeconv.FromColumnType(typ) == coltypes.Unhandled { continue } - typs := []coltypes.T{typ, typ, coltypes.Bool} + typs := []types.T{*typ, *typ, *types.Bool} bytesFixedLength := 0 - if ct.Family() == types.UuidFamily { + if typ.Family() == types.UuidFamily { bytesFixedLength = 16 } b := testAllocator.NewMemBatchWithSize(typs, numTuples) 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, ct) - rDatums[i] = PhysicalTypeColElemToDatum(rVec, i, da, ct) + lDatums[i] = PhysicalTypeColElemToDatum(lVec, i, da, typ) + rDatums[i] = PhysicalTypeColElemToDatum(rVec, i, da, typ) } for _, cmpOp := range []tree.ComparisonOperator{tree.EQ, tree.NE, tree.LT, tree.LE, tree.GT, tree.GE} { for i := range lDatums { @@ -277,7 +279,7 @@ func TestRandomComparisons(t *testing.T) { } input := newChunkingBatchSource(typs, []coldata.Vec{lVec, rVec, ret}, numTuples) op, err := createTestProjectingOperator( - ctx, flowCtx, input, []types.T{*ct, *ct}, + ctx, flowCtx, input, []types.T{*typ, *typ}, fmt.Sprintf("@1 %s @2", cmpOp), false, /* canFallbackToRowexec */ ) require.NoError(t, err) @@ -290,7 +292,7 @@ func TestRandomComparisons(t *testing.T) { for i := 0; i < batch.Length(); i++ { absIdx := idx + i assert.Equal(t, expected[absIdx], batch.ColVec(2).Bool()[i], - "expected %s %s %s (%s[%d]) to be %t found %t", lDatums[absIdx], cmpOp, rDatums[absIdx], ct, absIdx, + "expected %s %s %s (%s[%d]) to be %t found %t", lDatums[absIdx], cmpOp, rDatums[absIdx], typ, absIdx, expected[absIdx], ret.Bool()[i]) } idx += batch.Length() @@ -301,14 +303,14 @@ func TestRandomComparisons(t *testing.T) { func TestGetProjectionOperator(t *testing.T) { defer leaktest.AfterTest(t)() - ct := types.Int2 + typ := types.Int2 binOp := tree.Mult - var input Operator + var input colexecbase.Operator col1Idx := 5 col2Idx := 7 outputIdx := 9 op, err := GetProjectionOperator( - testAllocator, ct, ct, coltypes.Int16, + testAllocator, typ, typ, types.Int2, binOp, input, col1Idx, col2Idx, outputIdx, ) if err != nil { @@ -330,15 +332,16 @@ func TestGetProjectionOperator(t *testing.T) { func benchmarkProjOp( b *testing.B, - makeProjOp func(source *RepeatableBatchSource, intType coltypes.T) (Operator, error), + makeProjOp func(source *colexecbase.RepeatableBatchSource, intWidth int32) (colexecbase.Operator, error), useSelectionVector bool, hasNulls bool, - intType coltypes.T, + intType *types.T, ) { ctx := context.Background() - batch := testAllocator.NewMemBatch([]coltypes.T{intType, intType}) - switch intType { + typs := []types.T{*intType, *intType} + batch := testAllocator.NewMemBatch(typs) + switch typeconv.FromColumnType(intType) { case coltypes.Int64: col1 := batch.ColVec(0).Int64() col2 := batch.ColVec(1).Int64() @@ -374,8 +377,8 @@ func benchmarkProjOp( sel[i] = i } } - source := NewRepeatableBatchSource(testAllocator, batch) - op, err := makeProjOp(source, intType) + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) + op, err := makeProjOp(source, intType.Width()) require.NoError(b, err) op.Init() @@ -396,51 +399,51 @@ func BenchmarkProjOp(b *testing.B) { Settings: st, }, } - getInputTypesForColtype := func(intType coltypes.T) []types.T { - switch intType { - case coltypes.Int64: + getInputTypesForIntWidth := func(width int32) []types.T { + switch width { + case 0, 64: return []types.T{*types.Int, *types.Int} - case coltypes.Int32: + case 32: return []types.T{*types.Int4, *types.Int4} default: - b.Fatalf("unsupported type: %s", intType) + b.Fatalf("unsupported int width: %d", width) return nil } } - projOpMap := map[string]func(*RepeatableBatchSource, coltypes.T) (Operator, error){ - "projPlusIntIntOp": func(source *RepeatableBatchSource, intType coltypes.T) (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, getInputTypesForColtype(intType), + ctx, flowCtx, source, getInputTypesForIntWidth(width), "@1 + @2" /* projectingExpr */, false, /* canFallbackToRowexec */ ) }, - "projMinusIntIntOp": func(source *RepeatableBatchSource, intType coltypes.T) (Operator, error) { + "projMinusIntIntOp": func(source *colexecbase.RepeatableBatchSource, width int32) (colexecbase.Operator, error) { return createTestProjectingOperator( - ctx, flowCtx, source, getInputTypesForColtype(intType), + ctx, flowCtx, source, getInputTypesForIntWidth(width), "@1 - @2" /* projectingExpr */, false, /* canFallbackToRowexec */ ) }, - "projMultIntIntOp": func(source *RepeatableBatchSource, intType coltypes.T) (Operator, error) { + "projMultIntIntOp": func(source *colexecbase.RepeatableBatchSource, width int32) (colexecbase.Operator, error) { return createTestProjectingOperator( - ctx, flowCtx, source, getInputTypesForColtype(intType), + ctx, flowCtx, source, getInputTypesForIntWidth(width), "@1 * @2" /* projectingExpr */, false, /* canFallbackToRowexec */ ) }, - "projDivIntIntOp": func(source *RepeatableBatchSource, intType coltypes.T) (Operator, error) { + "projDivIntIntOp": func(source *colexecbase.RepeatableBatchSource, width int32) (colexecbase.Operator, error) { return createTestProjectingOperator( - ctx, flowCtx, source, getInputTypesForColtype(intType), + ctx, flowCtx, source, getInputTypesForIntWidth(width), "@1 / @2" /* projectingExpr */, false, /* canFallbackToRowexec */ ) }, } for projOp, makeProjOp := range projOpMap { - for _, intType := range []coltypes.T{coltypes.Int64, coltypes.Int32} { + for _, intType := range []types.T{*types.Int, *types.Int4} { for _, useSel := range []bool{true, false} { for _, hasNulls := range []bool{true, false} { b.Run(fmt.Sprintf("op=%s/type=%s/useSel=%t/hasNulls=%t", - projOp, intType, useSel, hasNulls), func(b *testing.B) { - benchmarkProjOp(b, makeProjOp, useSel, hasNulls, intType) + projOp, &intType, useSel, hasNulls), func(b *testing.B) { + benchmarkProjOp(b, makeProjOp, useSel, hasNulls, &intType) }) } } diff --git a/pkg/sql/colexec/rank_tmpl.go b/pkg/sql/colexec/rank_tmpl.go index 85022ebbc880..2533a075ec49 100644 --- a/pkg/sql/colexec/rank_tmpl.go +++ b/pkg/sql/colexec/rank_tmpl.go @@ -23,14 +23,17 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - // {{/* - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" - // */}} + "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 @@ -38,18 +41,18 @@ 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 *Allocator, - input Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, windowFn execinfrapb.WindowerSpec_WindowFunc, orderingCols []execinfrapb.Ordering_Column, outputColIdx int, partitionColIdx int, peersColIdx int, -) (Operator, error) { +) (colexecbase.Operator, error) { if len(orderingCols) == 0 { - return NewConstOp(allocator, input, coltypes.Int64, int64(1), outputColIdx) + return NewConstOp(allocator, input, types.Int, int64(1), outputColIdx) } - input = newVectorTypeEnforcer(allocator, input, coltypes.Int64, outputColIdx) + input = newVectorTypeEnforcer(allocator, input, types.Int, outputColIdx) initFields := rankInitFields{ OneInputNode: NewOneInputNode(input), allocator: allocator, @@ -78,13 +81,13 @@ func NewRankOperator( // _UPDATE_RANK is the template function for updating the state of rank // operators. func _UPDATE_RANK() { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // _UPDATE_RANK_INCREMENT is the template function for updating the state of // rank operators. func _UPDATE_RANK_INCREMENT() { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} @@ -92,7 +95,7 @@ func _UPDATE_RANK_INCREMENT() { type rankInitFields struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator outputColIdx int partitionColIdx int peersColIdx int @@ -110,7 +113,7 @@ type _RANK_STRINGOp struct { rankIncrement int64 } -var _ 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 cecf57cd993b..126d773108e3 100644 --- a/pkg/sql/colexec/relative_rank_tmpl.go +++ b/pkg/sql/colexec/relative_rank_tmpl.go @@ -23,10 +23,13 @@ import ( "context" "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/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" @@ -40,25 +43,25 @@ 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 *Allocator, + unlimitedAllocator *colmem.Allocator, memoryLimit int64, diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, - input Operator, - inputTypes []coltypes.T, + input colexecbase.Operator, + inputTypes []types.T, windowFn execinfrapb.WindowerSpec_WindowFunc, orderingCols []execinfrapb.Ordering_Column, outputColIdx int, partitionColIdx int, peersColIdx int, diskAcc *mon.BoundAccount, -) (Operator, error) { +) (colexecbase.Operator, error) { if len(orderingCols) == 0 { constValue := float64(0) if windowFn == execinfrapb.WindowerSpec_CUME_DIST { constValue = 1 } - return NewConstOp(unlimitedAllocator, input, coltypes.Float64, constValue, outputColIdx) + return NewConstOp(unlimitedAllocator, input, types.Float, constValue, outputColIdx) } rrInitFields := relativeRankInitFields{ rankInitFields: rankInitFields{ @@ -142,7 +145,7 @@ func _COMPUTE_PARTITIONS_SIZES() { // */}} // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatch([]coltypes.T{coltypes.Int64}) + r.partitionsState.runningSizes = r.allocator.NewMemBatch([]types.T{*types.Int}) runningPartitionsSizesCol = r.partitionsState.runningSizes.ColVec(0).Int64() } if r.numTuplesInPartition > 0 { @@ -153,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } r.partitionsState.runningSizes = nil r.partitionsState.idx = 0 @@ -180,7 +183,7 @@ func _COMPUTE_PEER_GROUPS_SIZES() { // */}} // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatch([]coltypes.T{coltypes.Int64}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatch([]types.T{*types.Int}) runningPeerGroupsSizesCol = r.peerGroupsState.runningSizes.ColVec(0).Int64() } if r.numPeers > 0 { @@ -191,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } r.peerGroupsState.runningSizes = nil r.peerGroupsState.idx = 0 @@ -211,7 +214,7 @@ type relativeRankInitFields struct { memoryLimit int64 diskQueueCfg colcontainer.DiskQueueCfg fdSemaphore semaphore.Semaphore - inputTypes []coltypes.T + inputTypes []types.T diskAcc *mon.BoundAccount } @@ -285,7 +288,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Init() { usedMemoryLimitFraction := 0.0 // {{if .HasPartition}} r.partitionsState.spillingQueue = newSpillingQueue( - r.allocator, []coltypes.T{coltypes.Int64}, + r.allocator, []types.T{*types.Int}, int64(float64(r.memoryLimit)*relativeRankUtilityQueueMemLimitFraction), r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, ) @@ -293,7 +296,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Init() { // {{end}} // {{if .IsCumeDist}} r.peerGroupsState.spillingQueue = newSpillingQueue( - r.allocator, []coltypes.T{coltypes.Int64}, + r.allocator, []types.T{*types.Int}, int64(float64(r.memoryLimit)*relativeRankUtilityQueueMemLimitFraction), r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, ) @@ -304,7 +307,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Init() { int64(float64(r.memoryLimit)*(1.0-usedMemoryLimitFraction)), r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, ) - r.output = r.allocator.NewMemBatch(append(r.inputTypes, coltypes.Float64)) + r.output = r.allocator.NewMemBatch(append(r.inputTypes, *types.Float)) // {{if .IsPercentRank}} // All rank functions start counting from 1. Before we assign the rank to a // tuple in the batch, we first increment r.rank, so setting this @@ -355,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } // {{if .HasPartition}} // We need to flush the last vector of the running partitions @@ -364,17 +367,17 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatch([]coltypes.T{coltypes.Int64}) + r.partitionsState.runningSizes = r.allocator.NewMemBatch([]types.T{*types.Int}) } runningPartitionsSizesCol := r.partitionsState.runningSizes.ColVec(0).Int64() runningPartitionsSizesCol[r.partitionsState.idx] = r.numTuplesInPartition r.partitionsState.idx++ r.partitionsState.runningSizes.SetLength(r.partitionsState.idx) if err := r.partitionsState.enqueue(ctx, r.partitionsState.runningSizes); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } if err := r.partitionsState.enqueue(ctx, coldata.ZeroBatch); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } // {{end}} // {{if .IsCumeDist}} @@ -384,17 +387,17 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatch([]coltypes.T{coltypes.Int64}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatch([]types.T{*types.Int}) } runningPeerGroupsSizesCol := r.peerGroupsState.runningSizes.ColVec(0).Int64() runningPeerGroupsSizesCol[r.peerGroupsState.idx] = r.numPeers r.peerGroupsState.idx++ r.peerGroupsState.runningSizes.SetLength(r.peerGroupsState.idx) if err := r.peerGroupsState.enqueue(ctx, r.peerGroupsState.runningSizes); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } if err := r.peerGroupsState.enqueue(ctx, coldata.ZeroBatch); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } // {{end}} // We have fully consumed the input, so now we can populate the output. @@ -422,7 +425,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { for colIdx, vec := range r.scratch.ColVecs() { vec.Append( coldata.SliceArgs{ - ColType: r.inputTypes[colIdx], + ColType: typeconv.FromColumnType(&r.inputTypes[colIdx]), Src: batch.ColVec(colIdx), Sel: sel, SrcEndIdx: n, @@ -432,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } // Then, we need to update the sizes of the partitions. @@ -477,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } n := r.scratch.Length() if n == 0 { @@ -488,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } r.partitionsState.idx = 0 r.numTuplesInPartition = 0 @@ -498,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } r.peerGroupsState.idx = 0 r.numPeers = 0 @@ -511,7 +514,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { for colIdx, vec := range r.output.ColVecs()[:r.outputColIdx] { vec.Append( coldata.SliceArgs{ - ColType: r.inputTypes[colIdx], + ColType: typeconv.FromColumnType(&r.inputTypes[colIdx]), Src: r.scratch.ColVec(colIdx), SrcEndIdx: n, }, @@ -536,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } r.partitionsState.idx = 0 } @@ -572,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } r.peerGroupsState.idx = 0 } @@ -601,12 +604,12 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { case relativeRankFinished: if err := r.idempotentCloseLocked(ctx); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return coldata.ZeroBatch default: - execerror.VectorizedInternalPanic("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 498b4c1a0208..12ec56fa7a41 100644 --- a/pkg/sql/colexec/routers.go +++ b/pkg/sql/colexec/routers.go @@ -16,11 +16,14 @@ import ( "sync" "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/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -68,7 +71,7 @@ type routerOutputOp struct { // input is a reference to our router. input execinfra.OpNode - types []coltypes.T + types []types.T // unblockedEventsChan is signaled when a routerOutput changes state from // blocked to unblocked. @@ -106,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 *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 @@ -152,12 +155,12 @@ func (o *routerOutputOp) Child(nth int, verbose bool) execinfra.OpNode { if nth == 0 { return o.input } - execerror.VectorizedInternalPanic(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 _ 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 @@ -165,8 +168,8 @@ var _ Operator = &routerOutputOp{} // memoryLimit will act as a soft limit to allow the router output to use disk // when it is exceeded. func newRouterOutputOp( - unlimitedAllocator *Allocator, - types []coltypes.T, + unlimitedAllocator *colmem.Allocator, + types []types.T, unblockedEventsChan chan<- struct{}, memoryLimit int64, cfg colcontainer.DiskQueueCfg, @@ -177,8 +180,8 @@ func newRouterOutputOp( } func newRouterOutputOpWithBlockedThresholdAndBatchSize( - unlimitedAllocator *Allocator, - types []coltypes.T, + unlimitedAllocator *colmem.Allocator, + types []types.T, unblockedEventsChan chan<- struct{}, memoryLimit int64, cfg colcontainer.DiskQueueCfg, @@ -293,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 - execerror.VectorizedInternalPanic(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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } } o.mu.pendingBatch = coldata.ZeroBatch @@ -329,7 +332,7 @@ func (o *routerOutputOp) addBatch(ctx context.Context, batch coldata.Batch, sele o.mu.pendingBatch.ColVec(i).Copy( coldata.CopySliceArgs{ SliceArgs: coldata.SliceArgs{ - ColType: t, + ColType: typeconv.FromColumnType(&t), Src: batch.ColVec(i), Sel: selection[:numAppended], DestIdx: o.mu.pendingBatch.Length(), @@ -344,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } o.mu.pendingBatch = nil } @@ -400,7 +403,7 @@ func (o *routerOutputOp) reset(ctx context.Context) { type HashRouter struct { OneInputNode // types are the input coltypes. - types []coltypes.T + types []types.T // hashCols is a slice of indices of the columns used for hashing. hashCols []uint32 @@ -434,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 []*Allocator, - input Operator, - types []coltypes.T, + unlimitedAllocators []*colmem.Allocator, + input colexecbase.Operator, + types []types.T, hashCols []uint32, memoryLimit int64, diskQueueCfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, diskAccounts []*mon.BoundAccount, -) (*HashRouter, []Operator) { +) (*HashRouter, []colexecbase.Operator) { if diskQueueCfg.CacheMode != colcontainer.DiskQueueCacheModeDefault { - execerror.VectorizedInternalPanic(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([]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 @@ -470,8 +473,8 @@ func NewHashRouter( } func newHashRouterWithOutputs( - input Operator, - types []coltypes.T, + input colexecbase.Operator, + types []types.T, hashCols []uint32, unblockEventsChan <-chan struct{}, outputs []routerOutput, @@ -500,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 := execerror.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 @@ -512,7 +515,7 @@ func (r *HashRouter) Run(ctx context.Context) { bufferErr(err) } for _, o := range r.outputs { - if err := execerror.CatchVectorizedRuntimeError(func() { + if err := colexecerror.CatchVectorizedRuntimeError(func() { o.cancel(ctx) }); err != nil { bufferErr(err) @@ -555,7 +558,7 @@ func (r *HashRouter) Run(ctx context.Context) { } } - if err := execerror.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 fc2ea784708b..41d34a5f5685 100644 --- a/pkg/sql/colexec/routers_test.go +++ b/pkg/sql/colexec/routers_test.go @@ -20,9 +20,12 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "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" "github.com/cockroachdb/cockroach/pkg/testutils/colcontainerutils" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" @@ -69,23 +72,23 @@ func getDiskQueueCfgAndMemoryTestCases( } // getDataAndFullSelection is a test helper that generates tuples representing -// a one-column coltypes.Int64 batch where each element is its ordinal and an +// a batch with single int64 column where each element is its ordinal and an // accompanying selection vector that selects every index in tuples. -func getDataAndFullSelection() (tuples, []int) { +func getDataAndFullSelection() (tuples, []types.T, []int) { data := make(tuples, coldata.BatchSize()) fullSelection := make([]int, coldata.BatchSize()) for i := range data { data[i] = tuple{i} fullSelection[i] = i } - return data, fullSelection + return data, []types.T{*types.Int}, fullSelection } func TestRouterOutputAddBatch(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() - data, fullSelection := getDataAndFullSelection() + data, typs, fullSelection := getDataAndFullSelection() // Since the actual data doesn't matter, we will just be reusing data for each // test case. @@ -141,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, []coltypes.T{coltypes.Int64}, unblockEventsChan, mtc.bytes, queueCfg, 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() @@ -176,17 +179,17 @@ func TestRouterOutputNext(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() - data, fullSelection := getDataAndFullSelection() + data, typs, fullSelection := getDataAndFullSelection() testCases := []struct { - unblockEvent func(in 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 Operator, o *routerOutputOp) { + unblockEvent: func(in colexecbase.Operator, o *routerOutputOp) { for { b := in.Next(ctx) o.addBatch(ctx, b, fullSelection) @@ -201,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(_ Operator, o *routerOutputOp) { + unblockEvent: func(_ colexecbase.Operator, o *routerOutputOp) { o.addBatch(ctx, coldata.ZeroBatch, nil /* selection */) }, expected: tuples{}, @@ -210,7 +213,7 @@ func TestRouterOutputNext(t *testing.T) { { // CancelUnblocksReader verifies that calling cancel on an output unblocks // a reader. - unblockEvent: func(_ Operator, o *routerOutputOp) { + unblockEvent: func(_ colexecbase.Operator, o *routerOutputOp) { o.cancel(ctx) }, expected: tuples{}, @@ -234,7 +237,7 @@ func TestRouterOutputNext(t *testing.T) { if queueCfg.FS == nil { t.Fatal("FS was nil") } - o := newRouterOutputOp(testAllocator, []coltypes.T{coltypes.Int64}, unblockedEventsChan, mtc.bytes, queueCfg, 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) @@ -261,11 +264,11 @@ func TestRouterOutputNext(t *testing.T) { tc.unblockEvent(in, o) // Should have data available, pushed by our reader goroutine. - batches := NewBatchBuffer() + batches := colexecbase.NewBatchBuffer() out := newOpTestOutput(batches, tc.expected) for { b := <-batchChan - batches.Add(b) + batches.Add(b, typs) if b.Length() == 0 { break } @@ -284,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, []coltypes.T{coltypes.Int64}, unblockedEventsChan, mtc.bytes, queueCfg, 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) @@ -323,7 +326,7 @@ func TestRouterOutputNext(t *testing.T) { } ch := make(chan struct{}, 2) - o := newRouterOutputOpWithBlockedThresholdAndBatchSize(testAllocator, []coltypes.T{coltypes.Int64}, ch, mtc.bytes, queueCfg, 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() @@ -375,7 +378,7 @@ func TestRouterOutputRandom(t *testing.T) { outputSize = 1 + rng.Intn(maxValues-1) ) - typs := []coltypes.T{coltypes.Int64, coltypes.Int64} + typs := []types.T{*types.Int, *types.Int} dataLen := 1 + rng.Intn(maxValues-1) data := make(tuples, dataLen) @@ -394,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 []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, 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)) @@ -416,7 +419,7 @@ func TestRouterOutputRandom(t *testing.T) { b := inputs[0].Next(ctx) selection := b.Selection() if selection == nil { - selection = randomSel(rng, b.Length(), rng.Float64()) + selection = coldatatestutils.RandomSel(rng, b.Length(), rng.Float64()) } selection = selection[:b.Length()] @@ -467,20 +470,14 @@ func TestRouterOutputRandom(t *testing.T) { } }() - actual := 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 := NewAllocator(ctx, &acc) - defer acc.Close(ctx) for { b := o.Next(ctx) - actual.Add(CopyBatch(allocator, b)) + actual.Add(coldatatestutils.CopyBatch(b, typs), typs) if b.Length() == 0 { wg.Done() return @@ -507,7 +504,7 @@ func TestRouterOutputRandom(t *testing.T) { } type callbackRouterOutput struct { - ZeroInputNode + colexecbase.ZeroInputNode addBatchCb func(coldata.Batch, []int) bool cancelCb func() } @@ -564,6 +561,7 @@ func TestHashRouterComputesDestination(t *testing.T) { expectedNumVals = []int{273, 252, 287, 212} numOutputs = 4 valsPushed = make([]int, numOutputs) + typs = []types.T{*types.Int} ) outputs := make([]routerOutput, numOutputs) @@ -590,7 +588,7 @@ func TestHashRouterComputesDestination(t *testing.T) { } } - r := newHashRouterWithOutputs(in, []coltypes.T{coltypes.Int64}, []uint32{0}, nil /* ch */, outputs) + r := newHashRouterWithOutputs(in, typs, []uint32{0}, nil /* ch */, outputs) for r.processNextBatch(ctx) { } @@ -622,13 +620,14 @@ func TestHashRouterCancellation(t *testing.T) { } } + typs := []types.T{*types.Int} // Never-ending input of 0s. - batch := testAllocator.NewMemBatch([]coltypes.T{coltypes.Int64}) + batch := testAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) - in := NewRepeatableBatchSource(testAllocator, batch) + in := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) unbufferedCh := make(chan struct{}) - r := newHashRouterWithOutputs(in, []coltypes.T{coltypes.Int64}, []uint32{0}, unbufferedCh, outputs) + r := newHashRouterWithOutputs(in, typs, []uint32{0}, unbufferedCh, outputs) t.Run("BeforeRun", func(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) @@ -713,10 +712,9 @@ func TestHashRouterOneOutput(t *testing.T) { rng, _ := randutil.NewPseudoRand() - sel := randomSel(rng, coldata.BatchSize(), rng.Float64()) + sel := coldatatestutils.RandomSel(rng, coldata.BatchSize(), rng.Float64()) - data, _ := getDataAndFullSelection() - typs := []coltypes.T{coltypes.Int64} + data, typs, _ := getDataAndFullSelection() expected := make(tuples, 0, len(data)) for _, i := range sel { @@ -733,8 +731,8 @@ func TestHashRouterOneOutput(t *testing.T) { diskAcc := testDiskMonitor.MakeBoundAccount() defer diskAcc.Close(ctx) r, routerOutputs := NewHashRouter( - []*Allocator{testAllocator}, newOpFixedSelTestInput(sel, len(sel), data), - typs, []uint32{0}, mtc.bytes, queueCfg, NewTestingSemaphore(2), + []*colmem.Allocator{testAllocator}, newOpFixedSelTestInput(sel, len(sel), data), + typs, []uint32{0}, mtc.bytes, queueCfg, colexecbase.NewTestingSemaphore(2), []*mon.BoundAccount{&diskAcc}, ) @@ -788,7 +786,7 @@ func TestHashRouterRandom(t *testing.T) { numOutputs = 1 + rng.Intn(maxOutputs-1) ) - typs := []coltypes.T{coltypes.Int64, coltypes.Int64} + typs := []types.T{*types.Int, *types.Int} dataLen := 1 + rng.Intn(maxValues-1) data := make(tuples, dataLen) for i := range data { @@ -831,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 []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([]Operator, numOutputs) + outputsAsOps := make([]colexecbase.Operator, numOutputs) memoryLimitPerOutput := mtc.bytes / int64(len(outputs)) for i := range outputs { // Create separate monitoring infrastructure as well as @@ -844,8 +842,8 @@ func TestHashRouterRandom(t *testing.T) { defer acc.Close(ctx) diskAcc := testDiskMonitor.MakeBoundAccount() defer diskAcc.Close(ctx) - allocator := NewAllocator(ctx, &acc) - op := newRouterOutputOpWithBlockedThresholdAndBatchSize(allocator, typs, unblockEventsChan, memoryLimitPerOutput, queueCfg, 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 } @@ -927,13 +925,12 @@ func BenchmarkHashRouter(b *testing.B) { defer leaktest.AfterTest(b)() ctx := context.Background() - types := []coltypes.T{coltypes.Int64} - // Use only one type. Note: the more types you use, the more you inflate the // numbers. - batch := testAllocator.NewMemBatch(types) + typs := []types.T{*types.Int} + batch := testAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) - input := NewRepeatableBatchSource(testAllocator, batch) + input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(b, true /* inMem */) defer cleanup() @@ -942,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([]*Allocator, numOutputs) + allocators := make([]*colmem.Allocator, numOutputs) diskAccounts := make([]*mon.BoundAccount, numOutputs) for i := range allocators { acc := testMemMonitor.MakeBoundAccount() - allocators[i] = 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, types, []uint32{0}, 64<<20, queueCfg, &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 70e740c2afd2..b080ceec7baf 100644 --- a/pkg/sql/colexec/row_number_tmpl.go +++ b/pkg/sql/colexec/row_number_tmpl.go @@ -23,7 +23,9 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/types" ) // TODO(yuzefovich): add benchmarks. @@ -32,9 +34,9 @@ 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 *Allocator, input Operator, outputColIdx int, partitionColIdx int, -) Operator { - input = newVectorTypeEnforcer(allocator, input, coltypes.Int64, outputColIdx) + allocator *colmem.Allocator, input colexecbase.Operator, outputColIdx int, partitionColIdx int, +) colexecbase.Operator { + input = newVectorTypeEnforcer(allocator, input, types.Int, outputColIdx) base := rowNumberBase{ OneInputNode: NewOneInputNode(input), allocator: allocator, @@ -52,7 +54,7 @@ func NewRowNumberOperator( // and should not be used directly. type rowNumberBase struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator outputColIdx int partitionColIdx int @@ -69,7 +71,7 @@ type _ROW_NUMBER_STRINGOp struct { rowNumberBase } -var _ 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_test.go b/pkg/sql/colexec/rowstovec_test.go index df206f07d4b8..2016a9af5647 100644 --- a/pkg/sql/colexec/rowstovec_test.go +++ b/pkg/sql/colexec/rowstovec_test.go @@ -14,7 +14,6 @@ import ( "reflect" "testing" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -36,14 +35,14 @@ func TestEncDatumRowsToColVecBool(t *testing.T) { sqlbase.EncDatum{Datum: tree.DBoolFalse}, }, } - vec := testAllocator.NewMemColumn(coltypes.Bool, 2) + vec := testAllocator.NewMemColumn(types.Bool, 2) ct := types.Bool // Test converting column 0. if err := EncDatumRowsToColVec(testAllocator, rows, vec, 0 /* columnIdx */, ct, &alloc); err != nil { t.Fatal(err) } - expected := testAllocator.NewMemColumn(coltypes.Bool, 2) + expected := testAllocator.NewMemColumn(types.Bool, 2) expected.Bool()[0] = false expected.Bool()[1] = true if !reflect.DeepEqual(vec, expected) { @@ -67,11 +66,11 @@ func TestEncDatumRowsToColVecInt16(t *testing.T) { sqlbase.EncDatumRow{sqlbase.EncDatum{Datum: tree.NewDInt(17)}}, sqlbase.EncDatumRow{sqlbase.EncDatum{Datum: tree.NewDInt(42)}}, } - vec := testAllocator.NewMemColumn(coltypes.Int16, 2) + vec := testAllocator.NewMemColumn(types.Int2, 2) if err := EncDatumRowsToColVec(testAllocator, rows, vec, 0 /* columnIdx */, types.Int2, &alloc); err != nil { t.Fatal(err) } - expected := testAllocator.NewMemColumn(coltypes.Int16, 2) + expected := testAllocator.NewMemColumn(types.Int2, 2) expected.Int16()[0] = 17 expected.Int16()[1] = 42 if !reflect.DeepEqual(vec, expected) { @@ -85,14 +84,14 @@ func TestEncDatumRowsToColVecString(t *testing.T) { sqlbase.EncDatumRow{sqlbase.EncDatum{Datum: tree.NewDString("foo")}}, sqlbase.EncDatumRow{sqlbase.EncDatum{Datum: tree.NewDString("bar")}}, } - vec := testAllocator.NewMemColumn(coltypes.Bytes, 2) + vec := testAllocator.NewMemColumn(types.Bytes, 2) for _, width := range []int32{0, 25} { ct := types.MakeString(width) vec.Bytes().Reset() if err := EncDatumRowsToColVec(testAllocator, rows, vec, 0 /* columnIdx */, ct, &alloc); err != nil { t.Fatal(err) } - expected := testAllocator.NewMemColumn(coltypes.Bytes, 2) + expected := testAllocator.NewMemColumn(types.Bytes, 2) expected.Bytes().Set(0, []byte("foo")) expected.Bytes().Set(1, []byte("bar")) if !reflect.DeepEqual(vec, expected) { @@ -105,7 +104,7 @@ func TestEncDatumRowsToColVecDecimal(t *testing.T) { defer leaktest.AfterTest(t)() nRows := 3 rows := make(sqlbase.EncDatumRows, nRows) - expected := testAllocator.NewMemColumn(coltypes.Decimal, 3) + expected := testAllocator.NewMemColumn(types.Decimal, 3) for i, s := range []string{"1.0000", "-3.12", "NaN"} { var err error dec, err := tree.ParseDDecimal(s) @@ -115,7 +114,7 @@ func TestEncDatumRowsToColVecDecimal(t *testing.T) { rows[i] = sqlbase.EncDatumRow{sqlbase.EncDatum{Datum: dec}} expected.Decimal()[i] = dec.Decimal } - vec := testAllocator.NewMemColumn(coltypes.Decimal, 3) + vec := testAllocator.NewMemColumn(types.Decimal, 3) ct := types.Decimal if err := EncDatumRowsToColVec(testAllocator, rows, vec, 0 /* columnIdx */, ct, &alloc); err != nil { t.Fatal(err) diff --git a/pkg/sql/colexec/rowstovec_tmpl.go b/pkg/sql/colexec/rowstovec_tmpl.go index 9ee1f76fcfe4..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/colexec/execerror" - // {{/* "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} - "github.com/cockroachdb/cockroach/pkg/sql/colexec/typeconv" + "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. @@ -59,11 +60,11 @@ func _ROWS_TO_COL_VEC( ) error { // */}} // {{define "rowsToColVec" -}} col := vec._TemplateType() - datumToPhysicalFn := typeconv.GetDatumToPhysicalFn(columnType) + datumToPhysicalFn := getDatumToPhysicalFn(typ) for i := range rows { row := rows[i] if row[columnIdx].Datum == nil { - if err = row[columnIdx].EnsureDecoded(columnType, alloc); err != nil { + if err = row[columnIdx].EnsureDecoded(typ, alloc); err != nil { return } } @@ -89,35 +90,35 @@ 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 *Allocator, + allocator *colmem.Allocator, rows sqlbase.EncDatumRows, vec coldata.Vec, columnIdx int, - columnType *types.T, + typ *types.T, alloc *sqlbase.DatumAlloc, ) error { var err error allocator.PerformOperation( []coldata.Vec{vec}, func() { - switch columnType.Family() { + switch typ.Family() { // {{range .}} case _FAMILY: // {{ if .Widths }} - switch columnType.Width() { + switch typ.Width() { // {{range .Widths}} case _WIDTH: - _ROWS_TO_COL_VEC(rows, vec, columnIdx, columnType, alloc) + _ROWS_TO_COL_VEC(rows, vec, columnIdx, typ, alloc) // {{end}} default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unsupported width %d for column type %s", columnType.Width(), columnType.String())) + colexecerror.InternalError(fmt.Sprintf("unsupported width %d for type %s", typ.Width(), typ.String())) } // {{ else }} - _ROWS_TO_COL_VEC(rows, vec, columnIdx, columnType, alloc) + _ROWS_TO_COL_VEC(rows, vec, columnIdx, typ, alloc) // {{end}} // {{end}} default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unsupported column type %s", columnType.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 d9aecc4ec8e2..93ae730b0af0 100644 --- a/pkg/sql/colexec/select_in_test.go +++ b/pkg/sql/colexec/select_in_test.go @@ -17,8 +17,8 @@ import ( "testing" "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/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 []Operator) (Operator, error) { + opConstructor := func(input []colexecbase.Operator) (colexecbase.Operator, error) { op := selectInOpInt64{ OneInputNode: NewOneInputNode(input[0]), colIdx: 0, @@ -101,7 +101,8 @@ func TestSelectInInt64(t *testing.T) { func benchmarkSelectInInt64(b *testing.B, useSelectionVector bool, hasNulls bool) { ctx := context.Background() - batch := testAllocator.NewMemBatch([]coltypes.T{coltypes.Int64}) + typs := []types.T{*types.Int} + batch := testAllocator.NewMemBatch(typs) col1 := batch.ColVec(0).Int64() for i := 0; i < coldata.BatchSize(); i++ { @@ -130,7 +131,7 @@ func benchmarkSelectInInt64(b *testing.B, useSelectionVector bool, hasNulls bool } } - source := NewRepeatableBatchSource(testAllocator, batch) + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) source.Init() inOp := &selectInOpInt64{ OneInputNode: NewOneInputNode(source), @@ -215,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 []Operator) (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 5d973749267b..4c354bf0c0f6 100644 --- a/pkg/sql/colexec/select_in_tmpl.go +++ b/pkg/sql/colexec/select_in_tmpl.go @@ -28,17 +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/colexec/execerror" + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} - "github.com/cockroachdb/cockroach/pkg/sql/colexec/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/types" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/pkg/errors" ) +// Remove unused warnings. +var ( + _ = execgen.UNSAFEGET + _ = colexecerror.InternalError +) + // {{/* type _GOTYPE interface{} @@ -63,7 +69,7 @@ var _ bytes.Buffer var _ = math.MaxInt64 func _ASSIGN_EQ(_, _, _ interface{}) int { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} @@ -78,17 +84,17 @@ const ( ) func GetInProjectionOperator( - allocator *Allocator, - ct *types.T, - input Operator, + allocator *colmem.Allocator, + typ *types.T, + input colexecbase.Operator, colIdx int, resultIdx int, datumTuple *tree.DTuple, negate bool, -) (Operator, error) { - input = newVectorTypeEnforcer(allocator, input, coltypes.Bool, resultIdx) +) (colexecbase.Operator, error) { + input = newVectorTypeEnforcer(allocator, input, types.Bool, resultIdx) var err error - switch t := typeconv.FromColumnType(ct); t { + switch typeconv.FromColumnType(typ) { // {{range .}} case coltypes._TYPE: obj := &projectInOp_TYPE{ @@ -98,22 +104,22 @@ func GetInProjectionOperator( outputIdx: resultIdx, negate: negate, } - obj.filterRow, obj.hasNulls, err = fillDatumRow_TYPE(ct, datumTuple) + obj.filterRow, obj.hasNulls, err = fillDatumRow_TYPE(typ, datumTuple) if err != nil { return nil, err } return obj, nil // {{end}} default: - return nil, errors.Errorf("unhandled type: %s", t) + return nil, errors.Errorf("unhandled type: %s", typ) } } func GetInOperator( - ct *types.T, input Operator, colIdx int, datumTuple *tree.DTuple, negate bool, -) (Operator, error) { + typ *types.T, input colexecbase.Operator, colIdx int, datumTuple *tree.DTuple, negate bool, +) (colexecbase.Operator, error) { var err error - switch t := typeconv.FromColumnType(ct); t { + switch typeconv.FromColumnType(typ) { // {{range .}} case coltypes._TYPE: obj := &selectInOp_TYPE{ @@ -121,14 +127,14 @@ func GetInOperator( colIdx: colIdx, negate: negate, } - obj.filterRow, obj.hasNulls, err = fillDatumRow_TYPE(ct, datumTuple) + obj.filterRow, obj.hasNulls, err = fillDatumRow_TYPE(typ, datumTuple) if err != nil { return nil, err } return obj, nil // {{end}} default: - return nil, errors.Errorf("unhandled type: %s", t) + return nil, errors.Errorf("unhandled type: %s", typ) } } @@ -144,7 +150,7 @@ type selectInOp_TYPE struct { type projectInOp_TYPE struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator colIdx int outputIdx int filterRow []_GOTYPE @@ -152,10 +158,10 @@ type projectInOp_TYPE struct { negate bool } -var _ Operator = &projectInOp_TYPE{} +var _ colexecbase.Operator = &projectInOp_TYPE{} -func fillDatumRow_TYPE(ct *types.T, datumTuple *tree.DTuple) ([]_GOTYPE, bool, error) { - conv := typeconv.GetDatumToPhysicalFn(ct) +func fillDatumRow_TYPE(typ *types.T, datumTuple *tree.DTuple) ([]_GOTYPE, bool, error) { + conv := getDatumToPhysicalFn(typ) var result []_GOTYPE hasNulls := false for _, d := range datumTuple.D { diff --git a/pkg/sql/colexec/selection_ops_test.go b/pkg/sql/colexec/selection_ops_test.go index 5ffac24e9591..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/col/coltypes" + "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 []Operator) (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 []Operator) (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 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 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 Operator + var input colexecbase.Operator col1Idx := 5 col2Idx := 7 op, err := GetSelectionOperator(ct, ct, cmpOp, input, col1Idx, col2Idx) @@ -138,7 +138,8 @@ func TestGetSelectionOperator(t *testing.T) { func benchmarkSelLTInt64Int64ConstOp(b *testing.B, useSelectionVector bool, hasNulls bool) { ctx := context.Background() - batch := testAllocator.NewMemBatch([]coltypes.T{coltypes.Int64}) + typs := []types.T{*types.Int} + batch := testAllocator.NewMemBatch(typs) col := batch.ColVec(0).Int64() for i := 0; i < coldata.BatchSize(); i++ { if float64(i) < float64(coldata.BatchSize())*selectivity { @@ -162,7 +163,7 @@ func benchmarkSelLTInt64Int64ConstOp(b *testing.B, useSelectionVector bool, hasN sel[i] = i } } - source := NewRepeatableBatchSource(testAllocator, batch) + source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) source.Init() plusOp := &selLTInt64Int64ConstOp{ @@ -194,7 +195,8 @@ func BenchmarkSelLTInt64Int64ConstOp(b *testing.B) { func benchmarkSelLTInt64Int64Op(b *testing.B, useSelectionVector bool, hasNulls bool) { ctx := context.Background() - batch := testAllocator.NewMemBatch([]coltypes.T{coltypes.Int64, coltypes.Int64}) + typs := []types.T{*types.Int, *types.Int} + batch := testAllocator.NewMemBatch(typs) col1 := batch.ColVec(0).Int64() col2 := batch.ColVec(1).Int64() for i := 0; i < coldata.BatchSize(); i++ { @@ -222,7 +224,7 @@ func benchmarkSelLTInt64Int64Op(b *testing.B, useSelectionVector bool, hasNulls sel[i] = i } } - source := NewRepeatableBatchSource(testAllocator, batch) + 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 43f8ec369b29..a145a2f3b625 100644 --- a/pkg/sql/colexec/selection_ops_tmpl.go +++ b/pkg/sql/colexec/selection_ops_tmpl.go @@ -28,17 +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/colexec/execerror" - // {{/* + "github.com/cockroachdb/cockroach/pkg/col/coltypes/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" - // */}} - "github.com/cockroachdb/cockroach/pkg/sql/colexec/typeconv" + "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. @@ -61,12 +63,12 @@ var _ time.Time var _ duration.Duration // Dummy import to pull in "coltypes" package. -var _ = coltypes.Bool +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 { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} @@ -289,14 +291,14 @@ func (p *_OP_NAME) Init() { // GetSelectionConstOperator returns the appropriate constant selection operator // for the given left and right column types and comparison. func GetSelectionConstOperator( - leftColType *types.T, - constColType *types.T, + leftType *types.T, + constType *types.T, cmpOp tree.ComparisonOperator, - input Operator, + input colexecbase.Operator, colIdx int, constArg tree.Datum, -) (Operator, error) { - c, err := typeconv.GetDatumToPhysicalFn(constColType)(constArg) +) (colexecbase.Operator, error) { + c, err := getDatumToPhysicalFn(constType)(constArg) if err != nil { return nil, err } @@ -304,10 +306,10 @@ func GetSelectionConstOperator( OneInputNode: NewOneInputNode(input), colIdx: colIdx, } - switch leftType := typeconv.FromColumnType(leftColType); leftType { + switch typeconv.FromColumnType(leftType) { // {{range $lTyp, $rTypToOverloads := .}} case coltypes._L_TYP_VAR: - switch rightType := typeconv.FromColumnType(constColType); rightType { + switch typeconv.FromColumnType(constType) { // {{range $rTyp, $overloads := $rTypToOverloads}} case coltypes._R_TYP_VAR: switch cmpOp { @@ -320,7 +322,7 @@ func GetSelectionConstOperator( } // {{end}} default: - return nil, errors.Errorf("unhandled right type: %s", rightType) + return nil, errors.Errorf("unhandled const type: %s", constType) } // {{end}} default: @@ -331,22 +333,22 @@ func GetSelectionConstOperator( // GetSelectionOperator returns the appropriate two column selection operator // for the given left and right column types and comparison. func GetSelectionOperator( - leftColType *types.T, - rightColType *types.T, + leftType *types.T, + rightType *types.T, cmpOp tree.ComparisonOperator, - input Operator, + input colexecbase.Operator, col1Idx int, col2Idx int, -) (Operator, error) { +) (colexecbase.Operator, error) { selOpBase := selOpBase{ OneInputNode: NewOneInputNode(input), col1Idx: col1Idx, col2Idx: col2Idx, } - switch leftType := typeconv.FromColumnType(leftColType); leftType { + switch typeconv.FromColumnType(leftType) { // {{range $lTyp, $rTypToOverloads := .}} case coltypes._L_TYP_VAR: - switch rightType := typeconv.FromColumnType(rightColType); rightType { + switch typeconv.FromColumnType(rightType) { // {{range $rTyp, $overloads := $rTypToOverloads}} case coltypes._R_TYP_VAR: switch cmpOp { diff --git a/pkg/sql/colexec/serial_unordered_synchronizer.go b/pkg/sql/colexec/serial_unordered_synchronizer.go index fc453cae6bde..f48595aa8e9a 100644 --- a/pkg/sql/colexec/serial_unordered_synchronizer.go +++ b/pkg/sql/colexec/serial_unordered_synchronizer.go @@ -14,8 +14,9 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/types" ) // SerialUnorderedSynchronizer is an Operator that combines multiple Operator @@ -25,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 []Operator + inputs []colexecbase.Operator // curSerialInputIdx indicates the index of the current input being consumed. curSerialInputIdx int } -var _ Operator = &SerialUnorderedSynchronizer{} +var _ colexecbase.Operator = &SerialUnorderedSynchronizer{} var _ execinfra.OpNode = &SerialUnorderedSynchronizer{} // ChildCount implements the execinfra.OpNode interface. @@ -45,7 +46,7 @@ func (s *SerialUnorderedSynchronizer) Child(nth int, verbose bool) execinfra.OpN // NewSerialUnorderedSynchronizer creates a new SerialUnorderedSynchronizer. func NewSerialUnorderedSynchronizer( - inputs []Operator, typs []coltypes.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 e0fafdf2d1db..8e8887abf6ef 100644 --- a/pkg/sql/colexec/serial_unordered_synchronizer_test.go +++ b/pkg/sql/colexec/serial_unordered_synchronizer_test.go @@ -15,7 +15,9 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "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" "github.com/stretchr/testify/require" @@ -29,11 +31,11 @@ func TestSerialUnorderedSynchronizer(t *testing.T) { const numInputs = 3 const numBatches = 4 - typs := []coltypes.T{coltypes.Int64} - inputs := make([]Operator, numInputs) + typs := []types.T{*types.Int} + inputs := make([]colexecbase.Operator, numInputs) for i := range inputs { - batch := RandomBatch(testAllocator, rng, typs, coldata.BatchSize(), 0 /* length */, rng.Float64()) - source := NewRepeatableBatchSource(testAllocator, batch) + 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 e75f9de6bc50..36bbac23a55c 100644 --- a/pkg/sql/colexec/simple_project.go +++ b/pkg/sql/colexec/simple_project.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -90,7 +91,9 @@ func (b *projectingBatch) ReplaceCol(col coldata.Vec, idx int) { // only the columns in the projection slice, in order. In a degenerate case // when input already outputs batches that satisfy the projection, a // simpleProjectOp is not planned and input is returned. -func NewSimpleProjectOp(input Operator, numInputCols int, projection []uint32) Operator { +func NewSimpleProjectOp( + 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 8c8dc227622f..4123fe30faf6 100644 --- a/pkg/sql/colexec/simple_project_test.go +++ b/pkg/sql/colexec/simple_project_test.go @@ -14,7 +14,8 @@ import ( "sync" "testing" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "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" ) @@ -61,7 +62,7 @@ func TestSimpleProjectOp(t *testing.T) { }, } for _, tc := range tcs { - runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, func(input []Operator) (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 }) } @@ -69,13 +70,13 @@ 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 []Operator) (Operator, error) { + func(input []colexecbase.Operator) (colexecbase.Operator, error) { return NewSimpleProjectOp(input[0], 3 /* numInputCols */, nil), nil }) t.Run("RedundantProjectionIsNotPlanned", func(t *testing.T) { - typs := []coltypes.T{coltypes.Int64, coltypes.Int64} - input := newFiniteBatchSource(testAllocator.NewMemBatch(typs), 1 /* usableCount */) + typs := []types.T{*types.Int, *types.Int} + input := newFiniteBatchSource(testAllocator.NewMemBatch(typs), typs, 1 /* usableCount */) projectOp := NewSimpleProjectOp(input, len(typs), []uint32{0, 1}) require.IsType(t, input, projectOp) }) @@ -93,7 +94,7 @@ func TestSimpleProjectOp(t *testing.T) { // batches. See #45686 for detailed discussion. func TestSimpleProjectOpWithUnorderedSynchronizer(t *testing.T) { defer leaktest.AfterTest(t)() - inputTypes := []coltypes.T{coltypes.Bytes, coltypes.Float64} + inputTypes := []types.T{*types.Bytes, *types.Float} constVal := int64(42) var wg sync.WaitGroup inputTuples := []tuples{ @@ -106,12 +107,12 @@ func TestSimpleProjectOpWithUnorderedSynchronizer(t *testing.T) { {"b", constVal}, {"bb", constVal}, } - runTestsWithoutAllNullsInjection(t, inputTuples, [][]coltypes.T{inputTypes, inputTypes}, expected, - unorderedVerifier, func(inputs []Operator) (Operator, error) { - var input Operator + runTestsWithoutAllNullsInjection(t, inputTuples, [][]types.T{inputTypes, inputTypes}, expected, + 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, coltypes.Int64, constVal, 1) + return NewConstOp(testAllocator, input, types.Int, constVal, 1) }) wg.Wait() } diff --git a/pkg/sql/colexec/sort.go b/pkg/sql/colexec/sort.go index 64e9fcb62804..c88214037a05 100644 --- a/pkg/sql/colexec/sort.go +++ b/pkg/sql/colexec/sort.go @@ -15,10 +15,13 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" "github.com/pkg/errors" ) @@ -26,29 +29,29 @@ import ( // given in orderingCols. The inputTypes must correspond 1-1 with the columns // in the input operator. func NewSorter( - allocator *Allocator, - input Operator, - inputTypes []coltypes.T, + allocator *colmem.Allocator, + input colexecbase.Operator, + inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column, -) (Operator, error) { +) (colexecbase.Operator, error) { return newSorter(allocator, newAllSpooler(allocator, input, inputTypes), inputTypes, orderingCols) } func newSorter( - allocator *Allocator, + allocator *colmem.Allocator, input spooler, - inputTypes []coltypes.T, + inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column, ) (resettableOperator, error) { partitioners := make([]partitioner, len(orderingCols)-1) var err error for i, ord := range orderingCols { - if !isSorterSupported(inputTypes[ord.ColIdx], ord.Direction) { - return nil, errors.Errorf("sorter for type: %s and direction: %s not supported", inputTypes[ord.ColIdx], ord.Direction) + if !isSorterSupported(&inputTypes[ord.ColIdx], ord.Direction) { + return nil, errors.Errorf("sorter for type: %s and direction: %s not supported", &inputTypes[ord.ColIdx], ord.Direction) } if i < len(orderingCols)-1 { - partitioners[i], err = newPartitioner(inputTypes[ord.ColIdx]) + partitioners[i], err = newPartitioner(&inputTypes[ord.ColIdx]) if err != nil { return nil, err } @@ -99,9 +102,9 @@ type allSpooler struct { OneInputNode NonExplainable - allocator *Allocator + allocator *colmem.Allocator // inputTypes contains the types of all of the columns from the input. - inputTypes []coltypes.T + inputTypes []types.T // bufferedTuples stores all the values from the input after spooling. Each // Vec in this batch is the entire column from the input. bufferedTuples *appendOnlyBufferedBatch @@ -113,7 +116,9 @@ type allSpooler struct { var _ spooler = &allSpooler{} var _ resetter = &allSpooler{} -func newAllSpooler(allocator *Allocator, input Operator, inputTypes []coltypes.T) spooler { +func newAllSpooler( + allocator *colmem.Allocator, input colexecbase.Operator, inputTypes []types.T, +) spooler { return &allSpooler{ OneInputNode: NewOneInputNode(input), allocator: allocator, @@ -131,7 +136,7 @@ func (p *allSpooler) init() { func (p *allSpooler) spool(ctx context.Context) { if p.spooled { - execerror.VectorizedInternalPanic("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) { @@ -143,7 +148,7 @@ func (p *allSpooler) spool(ctx context.Context) { func (p *allSpooler) getValues(i int) coldata.Vec { if !p.spooled { - execerror.VectorizedInternalPanic("getValues() is called before spool()") + colexecerror.InternalError("getValues() is called before spool()") } return p.bufferedTuples.ColVec(i) } @@ -154,7 +159,7 @@ func (p *allSpooler) getNumTuples() int { func (p *allSpooler) getPartitionsCol() []bool { if !p.spooled { - execerror.VectorizedInternalPanic("getPartitionsCol() is called before spool()") + colexecerror.InternalError("getPartitionsCol() is called before spool()") } return nil } @@ -164,7 +169,7 @@ func (p *allSpooler) getWindowedBatch(startIdx, endIdx int) coldata.Batch { // present on the original input batches, they have been removed when we were // buffering up tuples. for i, t := range p.inputTypes { - window := p.bufferedTuples.ColVec(i).Window(t, startIdx, endIdx) + window := p.bufferedTuples.ColVec(i).Window(typeconv.FromColumnType(&t), startIdx, endIdx) p.windowedBatch.ReplaceCol(window, i) } p.windowedBatch.SetSelection(false) @@ -182,11 +187,11 @@ func (p *allSpooler) reset(ctx context.Context) { } type sortOp struct { - allocator *Allocator + allocator *colmem.Allocator input spooler // inputTypes contains the types of all of the columns from input. - inputTypes []coltypes.T + inputTypes []types.T // orderingCols is the ordered list of column orderings that the sorter should // sort on. orderingCols []execinfrapb.Ordering_Column @@ -278,7 +283,7 @@ func (p *sortOp) Next(ctx context.Context) coldata.Batch { p.output.ColVec(j).Copy( coldata.CopySliceArgs{ SliceArgs: coldata.SliceArgs{ - ColType: p.inputTypes[j], + ColType: typeconv.FromColumnType(&p.inputTypes[j]), Sel: p.order, Src: p.input.getValues(j), SrcStartIdx: p.emitted, @@ -291,7 +296,7 @@ func (p *sortOp) Next(ctx context.Context) coldata.Batch { p.emitted = newEmitted return p.output } - execerror.VectorizedInternalPanic(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 } @@ -318,7 +323,7 @@ func (p *sortOp) sort(ctx context.Context) { for i := range p.orderingCols { inputVec := p.input.getValues(int(p.orderingCols[i].ColIdx)) - p.sorters[i] = newSingleSorter(p.inputTypes[p.orderingCols[i].ColIdx], p.orderingCols[i].Direction, inputVec.MaybeHasNulls()) + p.sorters[i] = newSingleSorter(&p.inputTypes[p.orderingCols[i].ColIdx], p.orderingCols[i].Direction, inputVec.MaybeHasNulls()) p.sorters[i].init(inputVec, p.order) } @@ -418,12 +423,12 @@ func (p *sortOp) Child(nth int, verbose bool) execinfra.OpNode { if nth == 0 { return p.input } - execerror.VectorizedInternalPanic(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(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 146ccd5eac49..97a526e1bf11 100644 --- a/pkg/sql/colexec/sort_chunks.go +++ b/pkg/sql/colexec/sort_chunks.go @@ -15,10 +15,13 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" ) // NewSortChunks returns a new sort chunks operator, which sorts its input on @@ -26,14 +29,14 @@ import ( // the columns in the input operator. The input tuples must be sorted on first // matchLen columns. func NewSortChunks( - allocator *Allocator, - input Operator, - inputTypes []coltypes.T, + allocator *colmem.Allocator, + input colexecbase.Operator, + inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column, matchLen int, -) (Operator, error) { +) (colexecbase.Operator, error) { if matchLen < 1 || matchLen == len(orderingCols) { - execerror.VectorizedInternalPanic(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)) @@ -54,7 +57,7 @@ func NewSortChunks( } type sortChunksOp struct { - allocator *Allocator + allocator *colmem.Allocator input *chunker sorter resettableOperator @@ -63,7 +66,7 @@ type sortChunksOp struct { windowedBatch coldata.Batch } -var _ Operator = &sortChunksOp{} +var _ colexecbase.Operator = &sortChunksOp{} var _ bufferingInMemoryOperator = &sortChunksOp{} func (c *sortChunksOp) ChildCount(verbose bool) int { @@ -74,7 +77,7 @@ func (c *sortChunksOp) Child(nth int, verbose bool) execinfra.OpNode { if nth == 0 { return c.input } - execerror.VectorizedInternalPanic(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 } @@ -108,7 +111,7 @@ func (c *sortChunksOp) Next(ctx context.Context) coldata.Batch { } } -func (c *sortChunksOp) ExportBuffered(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 { @@ -118,7 +121,7 @@ func (c *sortChunksOp) ExportBuffered(Operator) coldata.Batch { newExportedFromBuffer = c.input.bufferedTuples.Length() } for i, t := range c.input.inputTypes { - window := c.input.bufferedTuples.ColVec(i).Window(t, c.exportedFromBuffer, newExportedFromBuffer) + window := c.input.bufferedTuples.ColVec(i).Window(typeconv.FromColumnType(&t), c.exportedFromBuffer, newExportedFromBuffer) c.windowedBatch.ReplaceCol(window, i) } c.windowedBatch.SetLength(newExportedFromBuffer - c.exportedFromBuffer) @@ -199,9 +202,9 @@ type chunker struct { OneInputNode NonExplainable - allocator *Allocator + allocator *colmem.Allocator // inputTypes contains the types of all of the columns from input. - inputTypes []coltypes.T + inputTypes []types.T // inputDone indicates whether input has been fully consumed. inputDone bool // alreadySortedCols indicates the columns on which the input is already @@ -250,12 +253,15 @@ type chunker struct { var _ spooler = &chunker{} func newChunker( - allocator *Allocator, input Operator, inputTypes []coltypes.T, alreadySortedCols []uint32, + allocator *colmem.Allocator, + input colexecbase.Operator, + inputTypes []types.T, + alreadySortedCols []uint32, ) (*chunker, error) { var err error partitioners := make([]partitioner, len(alreadySortedCols)) for i, col := range alreadySortedCols { - partitioners[i], err = newPartitioner(inputTypes[col]) + partitioners[i], err = newPartitioner(&inputTypes[col]) if err != nil { return nil, err } @@ -308,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. - execerror.VectorizedInternalPanic(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 @@ -342,7 +348,7 @@ func (s *chunker) prepareNextChunks(ctx context.Context) chunkerReadingState { i := 0 for !differ && i < len(s.alreadySortedCols) { differ = valuesDiffer( - s.inputTypes[s.alreadySortedCols[i]], + &s.inputTypes[s.alreadySortedCols[i]], s.bufferedTuples.ColVec(int(s.alreadySortedCols[i])), 0, /*aValueIdx */ s.batch.ColVec(int(s.alreadySortedCols[i])), @@ -402,11 +408,11 @@ func (s *chunker) prepareNextChunks(ctx context.Context) chunkerReadingState { if s.inputDone { return chunkerDone } - execerror.VectorizedInternalPanic(fmt.Sprintf("unexpected: chunkerEmittingFromBatch state" + + colexecerror.InternalError(fmt.Sprintf("unexpected: chunkerEmittingFromBatch state" + "when s.chunks is fully processed and input is not done")) } default: - execerror.VectorizedInternalPanic(fmt.Sprintf("invalid chunker spooler state %v", s.state)) + colexecerror.InternalError(fmt.Sprintf("invalid chunker spooler state %v", s.state)) } } } @@ -430,11 +436,11 @@ func (s *chunker) spool(ctx context.Context) { func (s *chunker) getValues(i int) coldata.Vec { switch s.readFrom { case chunkerReadFromBuffer: - return s.bufferedTuples.ColVec(i).Window(s.inputTypes[i], 0 /* start */, s.bufferedTuples.Length()) + return s.bufferedTuples.ColVec(i).Window(typeconv.FromColumnType(&s.inputTypes[i]), 0 /* start */, s.bufferedTuples.Length()) case chunkerReadFromBatch: - return s.batch.ColVec(i).Window(s.inputTypes[i], s.chunks[s.chunksStartIdx], s.chunks[len(s.chunks)-1]) + return s.batch.ColVec(i).Window(typeconv.FromColumnType(&s.inputTypes[i]), s.chunks[s.chunksStartIdx], s.chunks[len(s.chunks)-1]) default: - execerror.VectorizedInternalPanic(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 } @@ -449,7 +455,7 @@ func (s *chunker) getNumTuples() int { case chunkerDone: return 0 default: - execerror.VectorizedInternalPanic(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 } @@ -477,14 +483,14 @@ func (s *chunker) getPartitionsCol() []bool { case chunkerDone: return nil default: - execerror.VectorizedInternalPanic(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 { - execerror.VectorizedInternalPanic("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 119500d2ec05..8b79efa1162d 100644 --- a/pkg/sql/colexec/sort_chunks_test.go +++ b/pkg/sql/colexec/sort_chunks_test.go @@ -17,8 +17,8 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/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/cockroach/pkg/util/leaktest" @@ -33,7 +33,7 @@ func init() { description: `three chunks`, tuples: tuples{{1, 2}, {1, 2}, {1, 3}, {1, 1}, {5, 5}, {6, 6}, {6, 1}}, expected: tuples{{1, 1}, {1, 2}, {1, 2}, {1, 3}, {5, 5}, {6, 1}, {6, 6}}, - logTypes: []types.T{*types.Int, *types.Int}, + typs: []types.T{*types.Int, *types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}, {ColIdx: 1}}, matchLen: 1, }, @@ -41,7 +41,7 @@ func init() { description: `simple nulls asc`, tuples: tuples{{1, 2}, {1, nil}, {1, 3}, {1, 1}, {5, 5}, {6, 6}, {6, nil}}, expected: tuples{{1, nil}, {1, 1}, {1, 2}, {1, 3}, {5, 5}, {6, nil}, {6, 6}}, - logTypes: []types.T{*types.Int, *types.Int}, + typs: []types.T{*types.Int, *types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}, {ColIdx: 1}}, matchLen: 1, }, @@ -49,7 +49,7 @@ func init() { description: `simple nulls desc`, tuples: tuples{{1, 2}, {1, nil}, {1, 3}, {1, 1}, {5, 5}, {6, 6}, {6, nil}}, expected: tuples{{1, 3}, {1, 2}, {1, 1}, {1, nil}, {5, 5}, {6, 6}, {6, nil}}, - logTypes: []types.T{*types.Int, *types.Int}, + typs: []types.T{*types.Int, *types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}, {ColIdx: 1, Direction: execinfrapb.Ordering_Column_DESC}}, matchLen: 1, }, @@ -69,7 +69,7 @@ func init() { {0, 2, 0}, {0, 2, 1}, }, - logTypes: []types.T{*types.Int, *types.Int, *types.Int}, + typs: []types.T{*types.Int, *types.Int, *types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}, {ColIdx: 1}, {ColIdx: 2}}, matchLen: 1, }, @@ -89,7 +89,7 @@ func init() { {1, 1, 1}, {1, 2, 1}, }, - logTypes: []types.T{*types.Int, *types.Int, *types.Int}, + typs: []types.T{*types.Int, *types.Int, *types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}, {ColIdx: 1}, {ColIdx: 2}}, matchLen: 1, }, @@ -109,7 +109,7 @@ func init() { {0, 2, 0}, {0, 2, 1}, }, - logTypes: []types.T{*types.Int, *types.Int, *types.Int}, + typs: []types.T{*types.Int, *types.Int, *types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}, {ColIdx: 1}, {ColIdx: 2}}, matchLen: 2, }, @@ -129,7 +129,7 @@ func init() { {1, 1, 1}, {1, 2, 1}, }, - logTypes: []types.T{*types.Int, *types.Int, *types.Int}, + typs: []types.T{*types.Int, *types.Int, *types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}, {ColIdx: 1}, {ColIdx: 2}}, matchLen: 2, }, @@ -149,7 +149,7 @@ func init() { {1, 1, 1}, {0, 1, 2}, }, - logTypes: []types.T{*types.Int, *types.Int, *types.Int}, + typs: []types.T{*types.Int, *types.Int, *types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 2}, {ColIdx: 1}, {ColIdx: 0}}, matchLen: 1, }, @@ -173,7 +173,7 @@ func init() { {1, 1, 2}, {1, 2, 2}, }, - logTypes: []types.T{*types.Int, *types.Int, *types.Int}, + typs: []types.T{*types.Int, *types.Int, *types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 2}, {ColIdx: 0}, {ColIdx: 1}}, matchLen: 2, }, @@ -184,12 +184,8 @@ func TestSortChunks(t *testing.T) { defer leaktest.AfterTest(t)() for _, tc := range sortChunksTestCases { - runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, func(input []Operator) (Operator, error) { - physTypes, err := typeconv.FromColumnTypes(tc.logTypes) - if err != nil { - return nil, err - } - return NewSortChunks(testAllocator, input[0], physTypes, tc.ordCols, tc.matchLen) + 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) }) } } @@ -200,9 +196,9 @@ func TestSortChunksRandomized(t *testing.T) { nTups := 8 maxCols := 5 // TODO(yuzefovich): randomize types as well. - typs := make([]coltypes.T, maxCols) + typs := make([]types.T, maxCols) for i := range typs { - typs[i] = coltypes.Int64 + typs[i] = *types.Int } for nCols := 1; nCols < maxCols; nCols++ { @@ -229,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 []Operator) (Operator, error) { + runTests(t, []tuples{sortedTups}, expected, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { return NewSortChunks(testAllocator, input[0], typs[:nCols], ordCols, matchLen) }) } @@ -241,9 +237,9 @@ func BenchmarkSortChunks(b *testing.B) { rng, _ := randutil.NewPseudoRand() ctx := context.Background() - sorterConstructors := []func(*Allocator, Operator, []coltypes.T, []execinfrapb.Ordering_Column, int) (Operator, error){ + sorterConstructors := []func(*colmem.Allocator, colexecbase.Operator, []types.T, []execinfrapb.Ordering_Column, int) (colexecbase.Operator, error){ NewSortChunks, - func(allocator *Allocator, input Operator, inputTypes []coltypes.T, orderingCols []execinfrapb.Ordering_Column, _ int) (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) }, } @@ -263,9 +259,9 @@ func BenchmarkSortChunks(b *testing.B) { // 8 (bytes / int64) * nBatches (number of batches) * coldata.BatchSize() (rows / // batch) * nCols (number of columns / row). b.SetBytes(int64(8 * nBatches * coldata.BatchSize() * nCols)) - typs := make([]coltypes.T, nCols) + typs := make([]types.T, nCols) for i := range typs { - typs[i] = coltypes.Int64 + typs[i] = *types.Int } batch := testAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) @@ -293,7 +289,7 @@ func BenchmarkSortChunks(b *testing.B) { } b.ResetTimer() for n := 0; n < b.N; n++ { - source := newFiniteChunksSource(batch, nBatches, matchLen) + source := newFiniteChunksSource(batch, typs, nBatches, matchLen) sorter, err := sorterConstructor(testAllocator, source, typs, ordCols, matchLen) if err != nil { b.Fatal(err) diff --git a/pkg/sql/colexec/sort_test.go b/pkg/sql/colexec/sort_test.go index 98748f1bcaf9..75115599eccb 100644 --- a/pkg/sql/colexec/sort_test.go +++ b/pkg/sql/colexec/sort_test.go @@ -19,9 +19,8 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/typeconv" + "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" @@ -35,80 +34,80 @@ func init() { { tuples: tuples{{1}, {2}, {nil}, {4}, {5}, {nil}}, expected: tuples{{nil}, {nil}, {1}, {2}, {4}, {5}}, - logTypes: []types.T{*types.Int}, + typs: []types.T{*types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}}, }, { tuples: tuples{{1, 2}, {1, 1}, {1, nil}, {2, nil}, {2, 3}, {2, nil}, {5, 1}}, expected: tuples{{1, nil}, {1, 1}, {1, 2}, {2, nil}, {2, nil}, {2, 3}, {5, 1}}, - logTypes: []types.T{*types.Int, *types.Int}, + typs: []types.T{*types.Int, *types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}, {ColIdx: 1}}, }, { tuples: tuples{{1, 2}, {1, 1}, {1, nil}, {2, nil}, {2, 3}, {2, nil}, {5, 1}}, expected: tuples{{5, 1}, {2, 3}, {2, nil}, {2, nil}, {1, 2}, {1, 1}, {1, nil}}, - logTypes: []types.T{*types.Int, *types.Int}, + typs: []types.T{*types.Int, *types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_DESC}, {ColIdx: 1, Direction: execinfrapb.Ordering_Column_DESC}}, }, { tuples: tuples{{nil, nil}, {nil, 3}, {1, nil}, {nil, 1}, {1, 2}, {nil, nil}, {5, nil}}, expected: tuples{{nil, nil}, {nil, nil}, {nil, 1}, {nil, 3}, {1, nil}, {1, 2}, {5, nil}}, - logTypes: []types.T{*types.Int, *types.Int}, + typs: []types.T{*types.Int, *types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}, {ColIdx: 1}}, }, { tuples: tuples{{1}, {2}, {3}, {4}, {5}, {6}, {7}}, expected: tuples{{1}, {2}, {3}, {4}, {5}, {6}, {7}}, - logTypes: []types.T{*types.Int}, + typs: []types.T{*types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}}, }, { tuples: tuples{{1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}}, expected: tuples{{1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}}, - logTypes: []types.T{*types.Int}, + typs: []types.T{*types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}}, }, { tuples: tuples{{1, 1}, {3, 2}, {2, 3}, {4, 4}, {5, 5}, {6, 6}, {7, 7}}, expected: tuples{{1, 1}, {2, 3}, {3, 2}, {4, 4}, {5, 5}, {6, 6}, {7, 7}}, - logTypes: []types.T{*types.Int, *types.Int}, + typs: []types.T{*types.Int, *types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}}, }, { tuples: tuples{{1, 1}, {5, 2}, {3, 3}, {7, 4}, {2, 5}, {6, 6}, {4, 7}}, expected: tuples{{1, 1}, {2, 5}, {3, 3}, {4, 7}, {5, 2}, {6, 6}, {7, 4}}, - logTypes: []types.T{*types.Int, *types.Int}, + typs: []types.T{*types.Int, *types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}}, }, { tuples: tuples{{1}, {5}, {3}, {3}, {2}, {6}, {4}}, expected: tuples{{1}, {2}, {3}, {3}, {4}, {5}, {6}}, - logTypes: []types.T{*types.Int}, + typs: []types.T{*types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}}, }, { tuples: tuples{{false}, {true}}, expected: tuples{{false}, {true}}, - logTypes: []types.T{*types.Bool}, + typs: []types.T{*types.Bool}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}}, }, { tuples: tuples{{true}, {false}}, expected: tuples{{false}, {true}}, - logTypes: []types.T{*types.Bool}, + typs: []types.T{*types.Bool}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}}, }, { tuples: tuples{{3.2}, {2.0}, {2.4}, {math.NaN()}, {math.Inf(-1)}, {math.Inf(1)}}, expected: tuples{{math.NaN()}, {math.Inf(-1)}, {2.0}, {2.4}, {3.2}, {math.Inf(1)}}, - logTypes: []types.T{*types.Float}, + typs: []types.T{*types.Float}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}}, }, { tuples: tuples{{0, 1, 0}, {1, 2, 0}, {2, 3, 2}, {3, 7, 1}, {4, 2, 2}}, expected: tuples{{0, 1, 0}, {1, 2, 0}, {3, 7, 1}, {4, 2, 2}, {2, 3, 2}}, - logTypes: []types.T{*types.Int, *types.Int, *types.Int}, + typs: []types.T{*types.Int, *types.Int, *types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 2}, {ColIdx: 1}}, }, @@ -130,8 +129,8 @@ func init() { {0, 1, 0}, {0, 1, 1}, }, - logTypes: []types.T{*types.Int, *types.Int, *types.Int}, - ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}, {ColIdx: 1}, {ColIdx: 2}}, + typs: []types.T{*types.Int, *types.Int, *types.Int}, + ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}, {ColIdx: 1}, {ColIdx: 2}}, }, } } @@ -139,12 +138,8 @@ 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 []Operator) (Operator, error) { - physTypes, err := typeconv.FromColumnTypes(tc.logTypes) - if err != nil { - return nil, err - } - return NewSorter(testAllocator, input[0], physTypes, tc.ordCols) + runTests(t, []tuples{tc.tuples}, tc.expected, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { + return NewSorter(testAllocator, input[0], tc.typs, tc.ordCols) }) } } @@ -155,9 +150,9 @@ func TestSortRandomized(t *testing.T) { nTups := coldata.BatchSize()*2 + 1 maxCols := 3 // TODO(yuzefovich): randomize types as well. - typs := make([]coltypes.T, maxCols) + typs := make([]types.T, maxCols) for i := range typs { - typs[i] = coltypes.Int64 + typs[i] = *types.Int } for nCols := 1; nCols < maxCols; nCols++ { for nOrderingCols := 1; nOrderingCols <= nCols; nOrderingCols++ { @@ -169,7 +164,7 @@ func TestSortRandomized(t *testing.T) { if topK { expected = expected[:k] } - runTests(t, []tuples{tups}, expected, orderedVerifier, func(input []Operator) (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 } @@ -217,31 +212,31 @@ func TestAllSpooler(t *testing.T) { tcs := []struct { tuples tuples - typ []coltypes.T + typ []types.T }{ { tuples: tuples{{1}, {2}, {3}, {4}, {5}, {6}, {7}}, - typ: []coltypes.T{coltypes.Int64}, + typ: []types.T{*types.Int}, }, { tuples: tuples{{1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}}, - typ: []coltypes.T{coltypes.Int64}, + typ: []types.T{*types.Int}, }, { tuples: tuples{{1, 1}, {3, 2}, {2, 3}, {4, 4}, {5, 5}, {6, 6}, {7, 7}}, - typ: []coltypes.T{coltypes.Int64, coltypes.Int64}, + typ: []types.T{*types.Int, *types.Int}, }, { tuples: tuples{{1, 1}, {5, 2}, {3, 3}, {7, 4}, {2, 5}, {6, 6}, {4, 7}}, - typ: []coltypes.T{coltypes.Int64, coltypes.Int64}, + typ: []types.T{*types.Int, *types.Int}, }, { tuples: tuples{{1}, {5}, {3}, {3}, {2}, {6}, {4}}, - typ: []coltypes.T{coltypes.Int64}, + typ: []types.T{*types.Int}, }, { tuples: tuples{{0, 1, 0}, {1, 2, 0}, {2, 3, 2}, {3, 7, 1}, {4, 2, 2}}, - typ: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, + typ: []types.T{*types.Int, *types.Int, *types.Int}, }, { tuples: tuples{ @@ -251,11 +246,11 @@ func TestAllSpooler(t *testing.T) { {0, 0, 1}, {0, 0, 0}, }, - typ: []coltypes.T{coltypes.Int64, coltypes.Int64, coltypes.Int64}, + typ: []types.T{*types.Int, *types.Int, *types.Int}, }, } for _, tc := range tcs { - runTestsWithFn(t, []tuples{tc.tuples}, nil /* typs */, func(t *testing.T, input []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()) @@ -291,9 +286,9 @@ func BenchmarkSort(b *testing.B) { // 8 (bytes / int64) * nBatches (number of batches) * coldata.BatchSize() (rows / // batch) * nCols (number of columns / row). b.SetBytes(int64(8 * nBatches * coldata.BatchSize() * nCols)) - typs := make([]coltypes.T, nCols) + typs := make([]types.T, nCols) for i := range typs { - typs[i] = coltypes.Int64 + typs[i] = *types.Int } batch := testAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) @@ -309,8 +304,8 @@ func BenchmarkSort(b *testing.B) { } b.ResetTimer() for n := 0; n < b.N; n++ { - source := newFiniteBatchSource(batch, nBatches) - var sorter Operator + source := newFiniteBatchSource(batch, typs, nBatches) + var sorter colexecbase.Operator if topK { sorter = NewTopKSorter(testAllocator, source, typs, ordCols, k) } else { @@ -340,9 +335,9 @@ func BenchmarkAllSpooler(b *testing.B) { // 8 (bytes / int64) * nBatches (number of batches) * col.BatchSize() (rows / // batch) * nCols (number of columns / row). b.SetBytes(int64(8 * nBatches * coldata.BatchSize() * nCols)) - typs := make([]coltypes.T, nCols) + typs := make([]types.T, nCols) for i := range typs { - typs[i] = coltypes.Int64 + typs[i] = *types.Int } batch := testAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) @@ -354,7 +349,7 @@ func BenchmarkAllSpooler(b *testing.B) { } b.ResetTimer() for n := 0; n < b.N; n++ { - source := newFiniteBatchSource(batch, nBatches) + source := newFiniteBatchSource(batch, typs, nBatches) allSpooler := newAllSpooler(testAllocator, source, typs) allSpooler.init() allSpooler.spool(ctx) @@ -403,7 +398,7 @@ func generateColumnOrdering( rng *rand.Rand, nCols int, nOrderingCols int, ) []execinfrapb.Ordering_Column { if nOrderingCols > nCols { - execerror.VectorizedInternalPanic("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 5732729e9766..a01021173e65 100644 --- a/pkg/sql/colexec/sort_tmpl.go +++ b/pkg/sql/colexec/sort_tmpl.go @@ -29,15 +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/colexec/execerror" - // {{/* + "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. @@ -76,13 +79,13 @@ 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 { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} -func isSorterSupported(t coltypes.T, dir execinfrapb.Ordering_Column_Direction) bool { - switch t { +func isSorterSupported(t *types.T, dir execinfrapb.Ordering_Column_Direction) bool { + switch typeconv.FromColumnType(t) { // {{range $typ, $ := . }} {{/* for each type */}} case _TYPES_T: switch dir { @@ -100,9 +103,9 @@ func isSorterSupported(t coltypes.T, dir execinfrapb.Ordering_Column_Direction) } func newSingleSorter( - t coltypes.T, dir execinfrapb.Ordering_Column_Direction, hasNulls bool, + t *types.T, dir execinfrapb.Ordering_Column_Direction, hasNulls bool, ) colSorter { - switch t { + switch typeconv.FromColumnType(t) { // {{range $typ, $ := . }} {{/* for each type */}} case _TYPES_T: switch hasNulls { @@ -114,15 +117,15 @@ func newSingleSorter( return &sort_TYPE_DIR_HANDLES_NULLSOp{} // {{end}} default: - execerror.VectorizedInternalPanic("nulls switch failed") + colexecerror.InternalError("nulls switch failed") } // {{end}} default: - execerror.VectorizedInternalPanic("nulls switch failed") + colexecerror.InternalError("nulls switch failed") } // {{end}} default: - execerror.VectorizedInternalPanic("nulls switch failed") + colexecerror.InternalError("nulls switch failed") } // This code is unreachable, but the compiler cannot infer that. return nil @@ -152,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 { - execerror.VectorizedInternalPanic(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 7dbaa79ad9f1..3a4205fa18e1 100644 --- a/pkg/sql/colexec/sorttopk.go +++ b/pkg/sql/colexec/sorttopk.go @@ -16,9 +16,12 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" ) const ( @@ -30,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 *Allocator, - input Operator, - inputTypes []coltypes.T, + allocator *colmem.Allocator, + input colexecbase.Operator, + inputTypes []types.T, orderingCols []execinfrapb.Ordering_Column, k uint16, -) Operator { +) colexecbase.Operator { return &topKSorter{ allocator: allocator, OneInputNode: NewOneInputNode(input), @@ -62,9 +65,9 @@ const ( type topKSorter struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator orderingCols []execinfrapb.Ordering_Column - inputTypes []coltypes.T + inputTypes []types.T k uint16 // TODO(solon): support larger k values // state is the current state of the sort. @@ -97,9 +100,8 @@ func (t *topKSorter) Init() { t.allocator, t.inputTypes, 0, /* initialSize */ ) t.comparators = make([]vecComparator, len(t.inputTypes)) - for i := range t.inputTypes { - typ := t.inputTypes[i] - t.comparators[i] = GetVecComparator(typ, 2) + for i, typ := range t.inputTypes { + t.comparators[i] = GetVecComparator(&typ, 2) } // TODO(yuzefovich): switch to calling this method on allocator. This will // require plumbing unlimited allocator to work correctly in tests with @@ -116,7 +118,7 @@ func (t *topKSorter) Next(ctx context.Context) coldata.Batch { case topKSortEmitting: return t.emit() } - execerror.VectorizedInternalPanic(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 } @@ -214,7 +216,7 @@ func (t *topKSorter) emit() coldata.Batch { if toEmit > coldata.BatchSize() { toEmit = coldata.BatchSize() } - for i := range t.inputTypes { + for i, typ := range t.inputTypes { vec := t.output.ColVec(i) // At this point, we have already fully sorted the input. It is ok to do // this Copy outside of the allocator - the work has been done, but @@ -224,7 +226,7 @@ func (t *topKSorter) emit() coldata.Batch { vec.Copy( coldata.CopySliceArgs{ SliceArgs: coldata.SliceArgs{ - ColType: t.inputTypes[i], + ColType: typeconv.FromColumnType(&typ), Src: t.topK.ColVec(i), Sel: t.sel, SrcStartIdx: t.emitted, @@ -249,7 +251,7 @@ func (t *topKSorter) compareRow(vecIdx1, vecIdx2 int, rowIdx1, rowIdx2 int) int case execinfrapb.Ordering_Column_DESC: return -res default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unexpected direction value %d", d)) + colexecerror.InternalError(fmt.Sprintf("unexpected direction value %d", d)) } } } @@ -262,7 +264,7 @@ func (t *topKSorter) updateComparators(vecIdx int, batch coldata.Batch) { } } -func (t *topKSorter) ExportBuffered(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 { @@ -271,7 +273,7 @@ func (t *topKSorter) ExportBuffered(Operator) coldata.Batch { newExportedFromTopK = topKLen } for i, typ := range t.inputTypes { - window := t.topK.ColVec(i).Window(typ, t.exportedFromTopK, newExportedFromTopK) + window := t.topK.ColVec(i).Window(typeconv.FromColumnType(&typ), t.exportedFromTopK, newExportedFromTopK) t.windowedBatch.ReplaceCol(window, i) } t.windowedBatch.SetSelection(false) diff --git a/pkg/sql/colexec/sorttopk_test.go b/pkg/sql/colexec/sorttopk_test.go index 34f279a1b2f9..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/colexec/typeconv" + "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" @@ -27,7 +27,7 @@ func init() { description: "k < input length", tuples: tuples{{1}, {2}, {3}, {4}, {5}, {6}, {7}}, expected: tuples{{1}, {2}, {3}}, - logTypes: []types.T{*types.Int}, + typs: []types.T{*types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}}, k: 3, }, @@ -35,7 +35,7 @@ func init() { description: "k > input length", tuples: tuples{{1}, {2}, {3}, {4}, {5}, {6}, {7}}, expected: tuples{{1}, {2}, {3}, {4}, {5}, {6}, {7}}, - logTypes: []types.T{*types.Int}, + typs: []types.T{*types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}}, k: 10, }, @@ -43,7 +43,7 @@ func init() { description: "nulls", tuples: tuples{{1}, {2}, {nil}, {3}, {4}, {5}, {6}, {7}, {nil}}, expected: tuples{{nil}, {nil}, {1}}, - logTypes: []types.T{*types.Int}, + typs: []types.T{*types.Int}, ordCols: []execinfrapb.Ordering_Column{{ColIdx: 0}}, k: 3, }, @@ -51,7 +51,7 @@ func init() { description: "descending", tuples: tuples{{0, 1}, {0, 2}, {0, 3}, {0, 4}, {0, 5}, {1, 5}}, expected: tuples{{0, 5}, {1, 5}, {0, 4}}, - logTypes: []types.T{*types.Int, *types.Int}, + typs: []types.T{*types.Int, *types.Int}, ordCols: []execinfrapb.Ordering_Column{ {ColIdx: 1, Direction: execinfrapb.Ordering_Column_DESC}, {ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}, @@ -66,12 +66,8 @@ 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 []Operator) (Operator, error) { - physTypes, err := typeconv.FromColumnTypes(tc.logTypes) - if err != nil { - return nil, err - } - return NewTopKSorter(testAllocator, input[0], physTypes, tc.ordCols, tc.k), nil + 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 a0e9bb0804ec..ed5cd78f14a7 100644 --- a/pkg/sql/colexec/spilling_queue.go +++ b/pkg/sql/colexec/spilling_queue.go @@ -15,9 +15,10 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" "github.com/cockroachdb/errors" @@ -39,10 +40,10 @@ 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 *Allocator + unlimitedAllocator *colmem.Allocator maxMemoryLimit int64 - typs []coltypes.T + typs []types.T items []coldata.Batch curHeadIdx int curTailIdx int @@ -69,8 +70,8 @@ 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 *Allocator, - typs []coltypes.T, + unlimitedAllocator *colmem.Allocator, + typs []types.T, memoryLimit int64, cfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, @@ -83,7 +84,7 @@ func newSpillingQueue( if memoryLimit < 0 { memoryLimit = 0 } - itemsLen := memoryLimit / int64(estimateBatchSizeBytes(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 @@ -108,8 +109,8 @@ 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 *Allocator, - typs []coltypes.T, + unlimitedAllocator *colmem.Allocator, + typs []types.T, memoryLimit int64, cfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, @@ -167,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 { - execerror.VectorizedInternalPanic(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 @@ -183,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. - execerror.VectorizedInternalPanic("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) @@ -296,7 +297,7 @@ func (q *spillingQueue) rewind() error { func (q *spillingQueue) reset(ctx context.Context) { if err := q.close(ctx); err != nil { - execerror.VectorizedInternalPanic(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 a85b472cd47a..f4913131e63b 100644 --- a/pkg/sql/colexec/spilling_queue_test.go +++ b/pkg/sql/colexec/spilling_queue_test.go @@ -16,7 +16,10 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils/colcontainerutils" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -56,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 := NewRandomDataOp(testAllocator, rng, RandomDataOpArgs{ + op := coldatatestutils.NewRandomDataOp(testAllocator, rng, coldatatestutils.RandomDataOpArgs{ NumBatches: cap(batches), BatchSize: 1 + rng.Intn(coldata.BatchSize()), Nulls: true, - BatchAccumulator: func(b coldata.Batch) { - batches = append(batches, CopyBatch(testAllocator, b)) + BatchAccumulator: func(b coldata.Batch, typs []types.T) { + batches = append(batches, coldatatestutils.CopyBatch(b, typs)) }, }) typs := op.Typs() @@ -75,13 +78,13 @@ func TestSpillingQueue(t *testing.T) { if rewindable { q = newRewindableSpillingQueue( testAllocator, typs, memoryLimit, queueCfg, - NewTestingSemaphore(2), coldata.BatchSize(), + colexecbase.NewTestingSemaphore(2), coldata.BatchSize(), testDiskAcc, ) } else { q = newSpillingQueue( testAllocator, typs, memoryLimit, queueCfg, - 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 2e78c81bbb32..2707d0fcc154 100644 --- a/pkg/sql/colexec/stats.go +++ b/pkg/sql/colexec/stats.go @@ -14,8 +14,9 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" "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" ) @@ -26,7 +27,7 @@ import ( // corresponding VectorizedStatsCollectors are also "connected" by sharing a // StopWatch. type VectorizedStatsCollector struct { - Operator + colexecbase.Operator NonExplainable execpb.VectorizedStats @@ -44,14 +45,14 @@ type VectorizedStatsCollector struct { diskMonitors []*mon.BytesMonitor } -var _ 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 Operator, + op colexecbase.Operator, id int32, isStall bool, inputWatch *timeutil.StopWatch, @@ -59,7 +60,7 @@ func NewVectorizedStatsCollector( diskMonitors []*mon.BytesMonitor, ) *VectorizedStatsCollector { if inputWatch == nil { - execerror.VectorizedInternalPanic("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 7e32ff2c1a75..07421cbee739 100644 --- a/pkg/sql/colexec/stats_test.go +++ b/pkg/sql/colexec/stats_test.go @@ -16,9 +16,10 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "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" "github.com/cockroachdb/cockroach/pkg/testutils/colcontainerutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -91,8 +92,8 @@ func TestVectorizedStatsCollector(t *testing.T) { mergeJoiner, err := newMergeJoinOp( testAllocator, defaultMemoryLimit, queueCfg, - NewTestingSemaphore(4), sqlbase.InnerJoin, leftInput, rightInput, - []coltypes.T{coltypes.Int64}, []coltypes.T{coltypes.Int64}, + 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}}, testDiskAcc, @@ -132,14 +133,15 @@ func TestVectorizedStatsCollector(t *testing.T) { } } -func makeFiniteChunksSourceWithBatchSize(nBatches int, batchSize int) Operator { - batch := testAllocator.NewMemBatchWithSize([]coltypes.T{coltypes.Int64}, batchSize) +func makeFiniteChunksSourceWithBatchSize(nBatches int, batchSize int) colexecbase.Operator { + typs := []types.T{*types.Int} + batch := testAllocator.NewMemBatchWithSize(typs, batchSize) vec := batch.ColVec(0).Int64() for i := 0; i < batchSize; i++ { vec[i] = int64(i) } batch.SetLength(batchSize) - return newFiniteChunksSource(batch, nBatches, 1 /* matchLen */) + return newFiniteChunksSource(batch, typs, nBatches, 1 /* matchLen */) } // timeAdvancingOperator is an Operator that advances the time source upon @@ -150,7 +152,7 @@ type timeAdvancingOperator struct { timeSource *timeutil.TestTimeSource } -var _ 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 1e3f2e1b468d..bc45fe2fb82b 100644 --- a/pkg/sql/colexec/substring_tmpl.go +++ b/pkg/sql/colexec/substring_tmpl.go @@ -24,8 +24,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/typeconv" + "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" ) @@ -39,32 +41,36 @@ var _ coltypes.T // */}} func newSubstringOperator( - allocator *Allocator, columnTypes []types.T, argumentCols []int, outputIdx int, input Operator, -) Operator { - startType := typeconv.FromColumnType(&columnTypes[argumentCols[1]]) - lengthType := typeconv.FromColumnType(&columnTypes[argumentCols[2]]) + allocator *colmem.Allocator, + typs []types.T, + argumentCols []int, + outputIdx int, + input colexecbase.Operator, +) colexecbase.Operator { + startType := &typs[argumentCols[1]] + lengthType := &typs[argumentCols[2]] base := substringFunctionBase{ OneInputNode: NewOneInputNode(input), allocator: allocator, argumentCols: argumentCols, outputIdx: outputIdx, } - switch startType { + switch typeconv.FromColumnType(startType) { // {{range $startType, $lengthTypes := .}} case _StartType_T: - switch lengthType { + switch typeconv.FromColumnType(lengthType) { // {{range $lengthType := $lengthTypes}} case _LengthType_T: return &substring_StartType_LengthTypeOperator{base} // {{end}} default: - execerror.VectorizedInternalPanic(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: - execerror.VectorizedInternalPanic(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 } @@ -72,7 +78,7 @@ func newSubstringOperator( type substringFunctionBase struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator argumentCols []int outputIdx int } @@ -88,7 +94,7 @@ type substring_StartType_LengthTypeOperator struct { substringFunctionBase } -var _ 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) @@ -131,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 { - execerror.NonVectorizedPanic(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 a1901b036cf6..f7f25b2e33a3 100644 --- a/pkg/sql/colexec/sum_agg_tmpl.go +++ b/pkg/sql/colexec/sum_agg_tmpl.go @@ -23,8 +23,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/colexec/execerror" + "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" "github.com/pkg/errors" ) @@ -41,16 +43,19 @@ var _ tree.Datum // Dummy import to pull in "duration" package. var _ duration.Duration +// Dummy import to pull in "coltypes" package. +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) { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} -func newSumAgg(t coltypes.T) (aggregateFunc, error) { - switch t { +func newSumAgg(t *types.T) (aggregateFunc, error) { + switch typeconv.FromColumnType(t) { // {{range .}} case _TYPES_T: return &sum_TYPEAgg{}, nil diff --git a/pkg/sql/colexec/supported_sql_types.go b/pkg/sql/colexec/supported_sql_types.go deleted file mode 100644 index 8fe5a15e8c0a..000000000000 --- a/pkg/sql/colexec/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 colexec - -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/colexec/typeconv/typeconv.go b/pkg/sql/colexec/typeconv/typeconv.go deleted file mode 100644 index d2568f7a65e1..000000000000 --- a/pkg/sql/colexec/typeconv/typeconv.go +++ /dev/null @@ -1,207 +0,0 @@ -// Copyright 2018 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 typeconv - -import ( - "fmt" - "reflect" - - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/encoding" - "github.com/pkg/errors" -) - -// 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. -func FromColumnType(ct *types.T) coltypes.T { - switch ct.Family() { - case types.BoolFamily: - return coltypes.Bool - case types.BytesFamily, types.StringFamily, types.UuidFamily: - return coltypes.Bytes - case types.DateFamily, types.OidFamily: - return coltypes.Int64 - case types.DecimalFamily: - return coltypes.Decimal - case types.IntFamily: - switch ct.Width() { - case 16: - return coltypes.Int16 - case 32: - return coltypes.Int32 - case 0, 64: - return coltypes.Int64 - } - execerror.VectorizedInternalPanic(fmt.Sprintf("integer with unknown width %d", ct.Width())) - case types.FloatFamily: - return coltypes.Float64 - case types.TimestampFamily: - return coltypes.Timestamp - case types.TimestampTZFamily: - return coltypes.Timestamp - case types.IntervalFamily: - return coltypes.Interval - } - return coltypes.Unhandled -} - -// FromColumnTypes calls FromColumnType on each element of cts, returning the -// resulting slice. -func FromColumnTypes(cts []types.T) ([]coltypes.T, error) { - typs := make([]coltypes.T, len(cts)) - for i := range typs { - typs[i] = FromColumnType(&cts[i]) - if typs[i] == coltypes.Unhandled { - return nil, errors.Errorf("unsupported type %s", cts[i].String()) - } - } - return typs, nil -} - -// GetDatumToPhysicalFn returns a function for converting a datum of the given -// ColumnType to the corresponding Go type. -func GetDatumToPhysicalFn(ct *types.T) func(tree.Datum) (interface{}, error) { - switch ct.Family() { - case types.BoolFamily: - return func(datum tree.Datum) (interface{}, error) { - d, ok := datum.(*tree.DBool) - if !ok { - return nil, errors.Errorf("expected *tree.DBool, found %s", reflect.TypeOf(datum)) - } - return bool(*d), nil - } - case types.BytesFamily: - return func(datum tree.Datum) (interface{}, error) { - d, ok := datum.(*tree.DBytes) - if !ok { - return nil, errors.Errorf("expected *tree.DBytes, found %s", reflect.TypeOf(datum)) - } - return encoding.UnsafeConvertStringToBytes(string(*d)), nil - } - case types.IntFamily: - switch ct.Width() { - case 16: - return func(datum tree.Datum) (interface{}, error) { - d, ok := datum.(*tree.DInt) - if !ok { - return nil, errors.Errorf("expected *tree.DInt, found %s", reflect.TypeOf(datum)) - } - return int16(*d), nil - } - case 32: - return func(datum tree.Datum) (interface{}, error) { - d, ok := datum.(*tree.DInt) - if !ok { - return nil, errors.Errorf("expected *tree.DInt, found %s", reflect.TypeOf(datum)) - } - return int32(*d), nil - } - case 0, 64: - return func(datum tree.Datum) (interface{}, error) { - d, ok := datum.(*tree.DInt) - if !ok { - return nil, errors.Errorf("expected *tree.DInt, found %s", reflect.TypeOf(datum)) - } - return int64(*d), nil - } - } - execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled INT width %d", ct.Width())) - case types.DateFamily: - return func(datum tree.Datum) (interface{}, error) { - d, ok := datum.(*tree.DDate) - if !ok { - return nil, errors.Errorf("expected *tree.DDate, found %s", reflect.TypeOf(datum)) - } - return d.UnixEpochDaysWithOrig(), nil - } - case types.FloatFamily: - return func(datum tree.Datum) (interface{}, error) { - d, ok := datum.(*tree.DFloat) - if !ok { - return nil, errors.Errorf("expected *tree.DFloat, found %s", reflect.TypeOf(datum)) - } - return float64(*d), nil - } - case types.OidFamily: - return func(datum tree.Datum) (interface{}, error) { - d, ok := datum.(*tree.DOid) - if !ok { - return nil, errors.Errorf("expected *tree.DOid, found %s", reflect.TypeOf(datum)) - } - return int64(d.DInt), nil - } - case types.StringFamily: - return func(datum tree.Datum) (interface{}, error) { - // Handle other STRING-related OID types, like oid.T_name. - wrapper, ok := datum.(*tree.DOidWrapper) - if ok { - datum = wrapper.Wrapped - } - - d, ok := datum.(*tree.DString) - if !ok { - return nil, errors.Errorf("expected *tree.DString, found %s", reflect.TypeOf(datum)) - } - return encoding.UnsafeConvertStringToBytes(string(*d)), nil - } - case types.DecimalFamily: - return func(datum tree.Datum) (interface{}, error) { - d, ok := datum.(*tree.DDecimal) - if !ok { - return nil, errors.Errorf("expected *tree.DDecimal, found %s", reflect.TypeOf(datum)) - } - return d.Decimal, nil - } - case types.UuidFamily: - return func(datum tree.Datum) (interface{}, error) { - d, ok := datum.(*tree.DUuid) - if !ok { - return nil, errors.Errorf("expected *tree.DUuid, found %s", reflect.TypeOf(datum)) - } - return d.UUID.GetBytesMut(), nil - } - case types.TimestampFamily: - return func(datum tree.Datum) (interface{}, error) { - d, ok := datum.(*tree.DTimestamp) - if !ok { - return nil, errors.Errorf("expected *tree.DTimestamp, found %s", reflect.TypeOf(datum)) - } - return d.Time, nil - } - case types.TimestampTZFamily: - return func(datum tree.Datum) (interface{}, error) { - d, ok := datum.(*tree.DTimestampTZ) - if !ok { - return nil, errors.Errorf("expected *tree.DTimestampTZ, found %s", reflect.TypeOf(datum)) - } - return d.Time, nil - } - case types.IntervalFamily: - return func(datum tree.Datum) (interface{}, error) { - d, ok := datum.(*tree.DInterval) - if !ok { - return nil, errors.Errorf("expected *tree.DInterval, found %s", reflect.TypeOf(datum)) - } - return d.Duration, nil - } - } - // It would probably be more correct to return an error here, rather than a - // function which always returns an error. But since the function tends to be - // invoked immediately after GetDatumToPhysicalFn is called, this works just - // as well and makes the error handling less messy for the caller. - return func(datum tree.Datum) (interface{}, error) { - return nil, errors.Errorf("unhandled type %s", ct.DebugString()) - } -} diff --git a/pkg/sql/colexec/types_integration_test.go b/pkg/sql/colexec/types_integration_test.go index 150f3da3a3fd..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/colexec/execerror" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/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/sem/tree" @@ -58,7 +59,7 @@ func TestSupportedSQLTypesIntegration(t *testing.T) { var da sqlbase.DatumAlloc rng, _ := randutil.NewPseudoRand() - for _, typ := range allSupportedSQLTypes { + for _, typ := range typeconv.AllSupportedSQLTypes { for _, numRows := range []int{ // A few interesting sizes. 1, @@ -77,11 +78,9 @@ func TestSupportedSQLTypesIntegration(t *testing.T) { columnarizer, err := NewColumnarizer(ctx, testAllocator, flowCtx, 0 /* processorID */, source) require.NoError(t, err) - coltyps, err := typeconv.FromColumnTypes(typs) + c, err := colserde.NewArrowBatchConverter(typs) require.NoError(t, err) - c, err := colserde.NewArrowBatchConverter(coltyps) - require.NoError(t, err) - r, err := colserde.NewRecordBatchSerializer(coltyps) + r, err := colserde.NewRecordBatchSerializer(typs) require.NoError(t, err) arrowOp := newArrowTestOperator(columnarizer, c, r) @@ -128,11 +127,11 @@ type arrowTestOperator struct { r *colserde.RecordBatchSerializer } -var _ Operator = &arrowTestOperator{} +var _ colexecbase.Operator = &arrowTestOperator{} func newArrowTestOperator( - input Operator, c *colserde.ArrowBatchConverter, r *colserde.RecordBatchSerializer, -) Operator { + input colexecbase.Operator, c *colserde.ArrowBatchConverter, r *colserde.RecordBatchSerializer, +) colexecbase.Operator { return &arrowTestOperator{ OneInputNode: NewOneInputNode(input), c: c, @@ -150,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } _, _, err = a.r.Serialize(&buf, arrowDataIn) if err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } var arrowDataOut []*array.Data if err := a.r.Deserialize(&arrowDataOut, buf.Bytes()); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } batchOut := testAllocator.NewMemBatchWithSize(nil, 0) if err := a.c.ArrowToBatch(arrowDataOut, batchOut); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return batchOut } diff --git a/pkg/sql/colexec/unordered_distinct.go b/pkg/sql/colexec/unordered_distinct.go index 9769d8bc16c4..ef539199a3ec 100644 --- a/pkg/sql/colexec/unordered_distinct.go +++ b/pkg/sql/colexec/unordered_distinct.go @@ -14,7 +14,10 @@ import ( "context" "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/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/types" ) // NewUnorderedDistinct creates an unordered distinct on the given distinct @@ -22,16 +25,16 @@ import ( // numHashBuckets determines the number of buckets that the hash table is // created with. func NewUnorderedDistinct( - allocator *Allocator, - input Operator, + allocator *colmem.Allocator, + input colexecbase.Operator, distinctCols []uint32, - colTypes []coltypes.T, + typs []types.T, numHashBuckets uint64, -) Operator { +) colexecbase.Operator { ht := newHashTable( allocator, numHashBuckets, - colTypes, + typs, distinctCols, true, /* allowNullEquality */ hashTableDistinctMode, @@ -41,7 +44,7 @@ func NewUnorderedDistinct( OneInputNode: NewOneInputNode(input), allocator: allocator, ht: ht, - output: allocator.NewMemBatch(colTypes), + output: allocator.NewMemBatch(typs), } } @@ -55,7 +58,7 @@ func NewUnorderedDistinct( type unorderedDistinct struct { OneInputNode - allocator *Allocator + allocator *colmem.Allocator ht *hashTable buildFinished bool @@ -65,7 +68,7 @@ type unorderedDistinct struct { outputBatchStart int } -var _ Operator = &unorderedDistinct{} +var _ colexecbase.Operator = &unorderedDistinct{} func (op *unorderedDistinct) Init() { op.input.Init() @@ -100,7 +103,7 @@ func (op *unorderedDistinct) Next(ctx context.Context) coldata.Batch { toCol.Copy( coldata.CopySliceArgs{ SliceArgs: coldata.SliceArgs{ - ColType: typ, + ColType: typeconv.FromColumnType(&typ), Src: fromCol, SrcStartIdx: op.outputBatchStart, SrcEndIdx: batchEnd, diff --git a/pkg/sql/colexec/utils.go b/pkg/sql/colexec/utils.go index 4e089e9b46a4..0b1015f10fb3 100644 --- a/pkg/sql/colexec/utils.go +++ b/pkg/sql/colexec/utils.go @@ -12,13 +12,21 @@ package colexec import ( "context" + "fmt" + "reflect" "github.com/cockroachdb/apd" "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/sql/colcontainer" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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/cockroachdb/cockroach/pkg/util/encoding" + "github.com/pkg/errors" ) var ( @@ -42,36 +50,12 @@ type decimalOverloadScratch struct { tmpDec1, tmpDec2 apd.Decimal } -// 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) coldata.Batch { - typs := make([]coltypes.T, original.Width()) - for i, vec := range original.ColVecs() { - typs[i] = vec.Type() - } - b := allocator.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: typs[colIdx], - Src: col, - SrcEndIdx: original.Length(), - }, - }) - } - }) - return b -} - // makeWindowIntoBatch updates windowedBatch so that it provides a "window" // into inputBatch starting at tuple index startIdx. It handles selection // vectors on inputBatch as well (in which case windowedBatch will also have a // "windowed" selection vector). func makeWindowIntoBatch( - windowedBatch, inputBatch coldata.Batch, startIdx int, inputTypes []coltypes.T, + windowedBatch, inputBatch coldata.Batch, startIdx int, inputTypes []types.T, ) { inputBatchLen := inputBatch.Length() windowStart := startIdx @@ -95,15 +79,15 @@ func makeWindowIntoBatch( windowedBatch.SetSelection(false) } for i, typ := range inputTypes { - window := inputBatch.ColVec(i).Window(typ, windowStart, windowEnd) + window := inputBatch.ColVec(i).Window(typeconv.FromColumnType(&typ), windowStart, windowEnd) windowedBatch.ReplaceCol(window, i) } windowedBatch.SetLength(inputBatchLen - startIdx) } func newPartitionerToOperator( - allocator *Allocator, - types []coltypes.T, + allocator *colmem.Allocator, + types []types.T, partitioner colcontainer.PartitionedQueue, partitionIdx int, ) *partitionerToOperator { @@ -118,7 +102,7 @@ func newPartitionerToOperator( // partition on every call to Next. It is a converter from filled in // PartitionedQueue to Operator. type partitionerToOperator struct { - ZeroInputNode + colexecbase.ZeroInputNode NonExplainable partitioner colcontainer.PartitionedQueue @@ -126,19 +110,19 @@ type partitionerToOperator struct { batch coldata.Batch } -var _ 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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return p.batch } func newAppendOnlyBufferedBatch( - allocator *Allocator, typs []coltypes.T, initialSize int, + allocator *colmem.Allocator, typs []types.T, initialSize int, ) *appendOnlyBufferedBatch { batch := allocator.NewMemBatchWithSize(typs, initialSize) return &appendOnlyBufferedBatch{ @@ -152,7 +136,7 @@ func newAppendOnlyBufferedBatch( // used by operators that buffer many tuples into a single batch by appending // to it. It stores the length of the batch separately and intercepts calls to // Length() and SetLength() in order to avoid updating offsets on vectors of -// coltypes.Bytes type - which would result in a quadratic behavior - because +// *types.Bytes type - which would result in a quadratic behavior - because // it is not necessary since coldata.Vec.Append maintains the correct offsets. // // Note: "appendOnly" in the name indicates that the tuples should *only* be @@ -164,7 +148,7 @@ type appendOnlyBufferedBatch struct { length int colVecs []coldata.Vec - typs []coltypes.T + typs []types.T } var _ coldata.Batch = &appendOnlyBufferedBatch{} @@ -186,11 +170,11 @@ func (b *appendOnlyBufferedBatch) ColVecs() []coldata.Vec { } func (b *appendOnlyBufferedBatch) AppendCol(coldata.Vec) { - execerror.VectorizedInternalPanic("AppendCol is prohibited on appendOnlyBufferedBatch") + colexecerror.InternalError("AppendCol is prohibited on appendOnlyBufferedBatch") } func (b *appendOnlyBufferedBatch) ReplaceCol(coldata.Vec, int) { - execerror.VectorizedInternalPanic("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 @@ -201,7 +185,7 @@ func (b *appendOnlyBufferedBatch) append(batch coldata.Batch, startIdx, endIdx i for i, colVec := range b.colVecs { colVec.Append( coldata.SliceArgs{ - ColType: b.typs[i], + ColType: typeconv.FromColumnType(&b.typs[i]), Src: batch.ColVec(i), Sel: batch.Selection(), DestIdx: b.length, @@ -212,3 +196,139 @@ func (b *appendOnlyBufferedBatch) append(batch coldata.Batch, startIdx, endIdx i } b.length += endIdx - startIdx } + +// getDatumToPhysicalFn returns a function for converting a datum of the given +// ColumnType to the corresponding Go type. +func getDatumToPhysicalFn(ct *types.T) func(tree.Datum) (interface{}, error) { + switch ct.Family() { + case types.BoolFamily: + return func(datum tree.Datum) (interface{}, error) { + d, ok := datum.(*tree.DBool) + if !ok { + return nil, errors.Errorf("expected *tree.DBool, found %s", reflect.TypeOf(datum)) + } + return bool(*d), nil + } + case types.BytesFamily: + return func(datum tree.Datum) (interface{}, error) { + d, ok := datum.(*tree.DBytes) + if !ok { + return nil, errors.Errorf("expected *tree.DBytes, found %s", reflect.TypeOf(datum)) + } + return encoding.UnsafeConvertStringToBytes(string(*d)), nil + } + case types.IntFamily: + switch ct.Width() { + case 16: + return func(datum tree.Datum) (interface{}, error) { + d, ok := datum.(*tree.DInt) + if !ok { + return nil, errors.Errorf("expected *tree.DInt, found %s", reflect.TypeOf(datum)) + } + return int16(*d), nil + } + case 32: + return func(datum tree.Datum) (interface{}, error) { + d, ok := datum.(*tree.DInt) + if !ok { + return nil, errors.Errorf("expected *tree.DInt, found %s", reflect.TypeOf(datum)) + } + return int32(*d), nil + } + case 0, 64: + return func(datum tree.Datum) (interface{}, error) { + d, ok := datum.(*tree.DInt) + if !ok { + return nil, errors.Errorf("expected *tree.DInt, found %s", reflect.TypeOf(datum)) + } + return int64(*d), nil + } + } + 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) + if !ok { + return nil, errors.Errorf("expected *tree.DDate, found %s", reflect.TypeOf(datum)) + } + return d.UnixEpochDaysWithOrig(), nil + } + case types.FloatFamily: + return func(datum tree.Datum) (interface{}, error) { + d, ok := datum.(*tree.DFloat) + if !ok { + return nil, errors.Errorf("expected *tree.DFloat, found %s", reflect.TypeOf(datum)) + } + return float64(*d), nil + } + case types.OidFamily: + return func(datum tree.Datum) (interface{}, error) { + d, ok := datum.(*tree.DOid) + if !ok { + return nil, errors.Errorf("expected *tree.DOid, found %s", reflect.TypeOf(datum)) + } + return int64(d.DInt), nil + } + case types.StringFamily: + return func(datum tree.Datum) (interface{}, error) { + // Handle other STRING-related OID types, like oid.T_name. + wrapper, ok := datum.(*tree.DOidWrapper) + if ok { + datum = wrapper.Wrapped + } + + d, ok := datum.(*tree.DString) + if !ok { + return nil, errors.Errorf("expected *tree.DString, found %s", reflect.TypeOf(datum)) + } + return encoding.UnsafeConvertStringToBytes(string(*d)), nil + } + case types.DecimalFamily: + return func(datum tree.Datum) (interface{}, error) { + d, ok := datum.(*tree.DDecimal) + if !ok { + return nil, errors.Errorf("expected *tree.DDecimal, found %s", reflect.TypeOf(datum)) + } + return d.Decimal, nil + } + case types.UuidFamily: + return func(datum tree.Datum) (interface{}, error) { + d, ok := datum.(*tree.DUuid) + if !ok { + return nil, errors.Errorf("expected *tree.DUuid, found %s", reflect.TypeOf(datum)) + } + return d.UUID.GetBytesMut(), nil + } + case types.TimestampFamily: + return func(datum tree.Datum) (interface{}, error) { + d, ok := datum.(*tree.DTimestamp) + if !ok { + return nil, errors.Errorf("expected *tree.DTimestamp, found %s", reflect.TypeOf(datum)) + } + return d.Time, nil + } + case types.TimestampTZFamily: + return func(datum tree.Datum) (interface{}, error) { + d, ok := datum.(*tree.DTimestampTZ) + if !ok { + return nil, errors.Errorf("expected *tree.DTimestampTZ, found %s", reflect.TypeOf(datum)) + } + return d.Time, nil + } + case types.IntervalFamily: + return func(datum tree.Datum) (interface{}, error) { + d, ok := datum.(*tree.DInterval) + if !ok { + return nil, errors.Errorf("expected *tree.DInterval, found %s", reflect.TypeOf(datum)) + } + return d.Duration, nil + } + } + // It would probably be more correct to return an error here, rather than a + // function which always returns an error. But since the function tends to be + // invoked immediately after getDatumToPhysicalFn is called, this works just + // as well and makes the error handling less messy for the caller. + return func(datum tree.Datum) (interface{}, error) { + return nil, errors.Errorf("unhandled type %s", ct.DebugString()) + } +} diff --git a/pkg/sql/colexec/utils_test.go b/pkg/sql/colexec/utils_test.go index 1672104a3c99..eff6c37a1141 100644 --- a/pkg/sql/colexec/utils_test.go +++ b/pkg/sql/colexec/utils_test.go @@ -23,9 +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/colexec/execerror" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/typeconv" + "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" @@ -124,7 +126,7 @@ func (t tuple) less(other tuple) bool { case "string": return lhsVal.String() < rhsVal.String() default: - execerror.VectorizedInternalPanic(fmt.Sprintf("Unhandled comparison type: %s", typ)) + colexecerror.InternalError(fmt.Sprintf("Unhandled comparison type: %s", typ)) } } return false @@ -205,7 +207,7 @@ func maybeHasNulls(b coldata.Batch) bool { return false } -type testRunner func(*testing.T, []tuples, [][]coltypes.T, tuples, interface{}, func([]Operator) (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 @@ -225,7 +227,7 @@ func runTests( tups []tuples, expected tuples, verifier interface{}, - constructor func(inputs []Operator) (Operator, error), + constructor func(inputs []colexecbase.Operator) (colexecbase.Operator, error), ) { runTestsWithTyps(t, tups, nil /* typs */, expected, verifier, constructor) } @@ -238,10 +240,10 @@ func runTests( func runTestsWithTyps( t *testing.T, tups []tuples, - typs [][]coltypes.T, + typs [][]types.T, expected tuples, verifier interface{}, - constructor func(inputs []Operator) (Operator, error), + constructor func(inputs []colexecbase.Operator) (colexecbase.Operator, error), ) { runTestsWithoutAllNullsInjection(t, tups, typs, expected, verifier, constructor) @@ -261,9 +263,9 @@ func runTestsWithTyps( } } } - opConstructor := func(injectAllNulls bool) Operator { - inputSources := make([]Operator, len(tups)) - var inputTypes []coltypes.T + opConstructor := func(injectAllNulls bool) colexecbase.Operator { + inputSources := make([]colexecbase.Operator, len(tups)) + var inputTypes []types.T for i, tup := range tups { if typs != nil { inputTypes = typs[i] @@ -331,10 +333,10 @@ func runTestsWithTyps( func runTestsWithoutAllNullsInjection( t *testing.T, tups []tuples, - typs [][]coltypes.T, + typs [][]types.T, expected tuples, verifier interface{}, - constructor func(inputs []Operator) (Operator, error), + constructor func(inputs []colexecbase.Operator) (colexecbase.Operator, error), ) { skipVerifySelAndNullsResets := true var verifyFn verifierFn @@ -350,12 +352,12 @@ func runTestsWithoutAllNullsInjection( case unorderedVerifier: verifyFn = (*opTestOutput).VerifyAnyOrder default: - execerror.VectorizedInternalPanic(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 []Operator) { + runTestsWithFn(t, tups, typs, func(t *testing.T, inputs []colexecbase.Operator) { op, err := constructor(inputs) if err != nil { t.Fatal(err) @@ -380,10 +382,10 @@ func runTestsWithoutAllNullsInjection( // doesn't have to restore anything on a zero-length batch). var ( secondBatchHasSelection, secondBatchHasNulls bool - inputTypes []coltypes.T + inputTypes []types.T ) for round := 0; round < 2; round++ { - inputSources := make([]Operator, len(tups)) + inputSources := make([]colexecbase.Operator, len(tups)) for i, tup := range tups { if typs != nil { inputTypes = typs[i] @@ -453,8 +455,8 @@ 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([]Operator, len(tups)) - var inputTypes []coltypes.T + inputSources := make([]colexecbase.Operator, len(tups)) + var inputTypes []types.T for i, tup := range tups { if typs != nil { inputTypes = typs[i] @@ -489,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 [][]coltypes.T, test func(t *testing.T, inputs []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(). @@ -504,8 +509,8 @@ 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([]Operator, len(tups)) - var inputTypes []coltypes.T + inputSources := make([]colexecbase.Operator, len(tups)) + var inputTypes []types.T if useSel { for i, tup := range tups { if typs != nil { @@ -533,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 []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([]Operator, len(tups)) + inputSources := make([]colexecbase.Operator, len(tups)) for i, tup := range tups { inputSources[i] = newOpFixedSelTestInput(sel, batchSize, tup) } @@ -568,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 { - execerror.VectorizedInternalPanic( + colexecerror.InternalError( fmt.Sprintf("unable to set decimal %f: %v", floatVal, err)) } // .Set is used here instead of assignment to ensure the pointer address @@ -589,16 +594,16 @@ func setColVal(vec coldata.Vec, idx int, val interface{}) { // {1,2,3.3,true}, // {5,6,7.0,false}, // } -// tupleSource := newOpTestInput(inputTuples, coltypes.Bool) +// tupleSource := newOpTestInput(inputTuples, *types.Bool) // opUnderTest := newFooOp(tupleSource, ...) // output := newOpTestOutput(opUnderTest, expectedOutputTuples) // if err := output.Verify(); err != nil { // t.Fatal(err) // } type opTestInput struct { - ZeroInputNode + colexecbase.ZeroInputNode - typs []coltypes.T + typs []types.T batchSize int tuples tuples @@ -616,12 +621,12 @@ type opTestInput struct { injectRandomNulls bool } -var _ 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 // types automatically, using simple rules (e.g. integers always become Int64). -func newOpTestInput(batchSize int, tuples tuples, typs []coltypes.T) *opTestInput { +func newOpTestInput(batchSize int, tuples tuples, typs []types.T) *opTestInput { ret := &opTestInput{ batchSize: batchSize, tuples: tuples, @@ -630,9 +635,7 @@ func newOpTestInput(batchSize int, tuples tuples, typs []coltypes.T) *opTestInpu return ret } -func newOpTestSelInput( - rng *rand.Rand, batchSize int, tuples tuples, typs []coltypes.T, -) *opTestInput { +func newOpTestSelInput(rng *rand.Rand, batchSize int, tuples tuples, typs []types.T) *opTestInput { ret := &opTestInput{ useSel: true, rng: rng, @@ -646,19 +649,23 @@ func newOpTestSelInput( func (s *opTestInput) Init() { if s.typs == nil { if len(s.tuples) == 0 { - execerror.VectorizedInternalPanic("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 // the input tuple. - s.typs = make([]coltypes.T, len(s.tuples[0])) + s.typs = make([]types.T, len(s.tuples[0])) for i := range s.typs { // Default type for test cases is Int64 in case the entire column is null // and the type is indeterminate. - s.typs[i] = coltypes.Int64 + s.typs[i] = *types.Int for _, tup := range s.tuples { if tup[i] != nil { - s.typs[i] = coltypes.FromGoType(tup[i]) + t, err := typeconv.UnsafeToSQLType(coltypes.FromGoType(tup[i])) + if err != nil { + colexecerror.InternalError(err) + } + s.typs[i] = *t break } } @@ -687,7 +694,7 @@ func (s *opTestInput) Next(context.Context) coldata.Batch { tupleLen := len(tups[0]) for i := range tups { if len(tups[i]) != tupleLen { - execerror.VectorizedInternalPanic(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)) } } @@ -758,7 +765,7 @@ func (s *opTestInput) Next(context.Context) coldata.Batch { d := apd.Decimal{} _, err := d.SetFloat64(rng.Float64()) if err != nil { - execerror.VectorizedInternalPanic(fmt.Sprintf("%v", err)) + colexecerror.InternalError(fmt.Sprintf("%v", err)) } col.Index(outputIdx).Set(reflect.ValueOf(d)) } else if typ == coltypes.Bytes { @@ -768,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 { - execerror.VectorizedInternalPanic(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 { @@ -782,9 +789,9 @@ func (s *opTestInput) Next(context.Context) coldata.Batch { } type opFixedSelTestInput struct { - ZeroInputNode + colexecbase.ZeroInputNode - typs []coltypes.T + typs []types.T batchSize int tuples tuples @@ -796,7 +803,7 @@ type opFixedSelTestInput struct { idx int } -var _ 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 @@ -812,28 +819,31 @@ func newOpFixedSelTestInput(sel []int, batchSize int, tuples tuples) *opFixedSel func (s *opFixedSelTestInput) Init() { if len(s.tuples) == 0 { - execerror.VectorizedInternalPanic("empty tuple source") + colexecerror.InternalError("empty tuple source") } - typs := make([]coltypes.T, len(s.tuples[0])) - for i := range typs { + s.typs = make([]types.T, len(s.tuples[0])) + for i := range s.typs { // Default type for test cases is Int64 in case the entire column is null // and the type is indeterminate. - typs[i] = coltypes.Int64 + s.typs[i] = *types.Int for _, tup := range s.tuples { if tup[i] != nil { - typs[i] = coltypes.FromGoType(tup[i]) + t, err := typeconv.UnsafeToSQLType(coltypes.FromGoType(tup[i])) + if err != nil { + colexecerror.InternalError(err) + } + s.typs[i] = *t break } } } - s.typs = typs s.batch = testAllocator.NewMemBatch(s.typs) tupleLen := len(s.tuples[0]) for _, i := range s.sel { if len(s.tuples[i]) != tupleLen { - execerror.VectorizedInternalPanic(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)) } } @@ -915,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 Operator, expected tuples) *opTestOutput { +func newOpTestOutput(input colexecbase.Operator, expected tuples) *opTestOutput { input.Init() return &opTestOutput{ @@ -1098,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 { - ZeroInputNode + colexecbase.ZeroInputNode - repeatableBatch *RepeatableBatchSource + repeatableBatch *colexecbase.RepeatableBatchSource usableCount int } -var _ 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, usableCount int) *finiteBatchSource { +func newFiniteBatchSource(batch coldata.Batch, typs []types.T, usableCount int) *finiteBatchSource { return &finiteBatchSource{ - repeatableBatch: NewRepeatableBatchSource(testAllocator, batch), + repeatableBatch: colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs), usableCount: usableCount, } } @@ -1137,19 +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 { - ZeroInputNode - repeatableBatch *RepeatableBatchSource + colexecbase.ZeroInputNode + repeatableBatch *colexecbase.RepeatableBatchSource usableCount int matchLen int adjustment []int64 } -var _ Operator = &finiteChunksSource{} +var _ colexecbase.Operator = &finiteChunksSource{} -func newFiniteChunksSource(batch coldata.Batch, usableCount int, matchLen int) *finiteChunksSource { +func newFiniteChunksSource( + batch coldata.Batch, typs []types.T, usableCount int, matchLen int, +) *finiteChunksSource { return &finiteChunksSource{ - repeatableBatch: NewRepeatableBatchSource(testAllocator, batch), + repeatableBatch: colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs), usableCount: usableCount, matchLen: matchLen, } @@ -1203,7 +1215,7 @@ func TestOpTestInputOutput(t *testing.T) { {1, 5, 0}, }, } - runTestsWithFn(t, inputs, nil /* typs */, func(t *testing.T, sources []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 { @@ -1214,13 +1226,14 @@ func TestOpTestInputOutput(t *testing.T) { func TestRepeatableBatchSource(t *testing.T) { defer leaktest.AfterTest(t)() - batch := testAllocator.NewMemBatch([]coltypes.T{coltypes.Int64}) + typs := []types.T{*types.Int} + batch := testAllocator.NewMemBatch(typs) batchLen := 10 if coldata.BatchSize() < batchLen { batchLen = coldata.BatchSize() } batch.SetLength(batchLen) - input := NewRepeatableBatchSource(testAllocator, batch) + input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) b := input.Next(context.Background()) b.SetLength(0) @@ -1237,18 +1250,19 @@ func TestRepeatableBatchSource(t *testing.T) { func TestRepeatableBatchSourceWithFixedSel(t *testing.T) { defer leaktest.AfterTest(t)() - batch := testAllocator.NewMemBatch([]coltypes.T{coltypes.Int64}) + typs := []types.T{*types.Int} + batch := testAllocator.NewMemBatch(typs) rng, _ := randutil.NewPseudoRand() batchSize := 10 if batchSize > coldata.BatchSize() { batchSize = coldata.BatchSize() } - sel := 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 := NewRepeatableBatchSource(testAllocator, batch) + input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) b := input.Next(context.Background()) b.SetLength(0) @@ -1267,7 +1281,7 @@ func TestRepeatableBatchSourceWithFixedSel(t *testing.T) { } } - newSel := 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) @@ -1290,8 +1304,8 @@ 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 { - ZeroInputNode - typs []coltypes.T + colexecbase.ZeroInputNode + typs []types.T cols []coldata.Vec len int @@ -1299,11 +1313,11 @@ type chunkingBatchSource struct { batch coldata.Batch } -var _ Operator = &chunkingBatchSource{} +var _ colexecbase.Operator = &chunkingBatchSource{} // newChunkingBatchSource returns a new chunkingBatchSource with the given // column types, columns, and length. -func newChunkingBatchSource(typs []coltypes.T, cols []coldata.Vec, len int) *chunkingBatchSource { +func newChunkingBatchSource(typs []types.T, cols []coldata.Vec, len int) *chunkingBatchSource { return &chunkingBatchSource{ typs: typs, cols: cols, @@ -1333,7 +1347,7 @@ func (c *chunkingBatchSource) Next(context.Context) coldata.Batch { lastIdx = c.len } for i, vec := range c.batch.ColVecs() { - vec.SetCol(c.cols[i].Window(c.typs[i], c.curIdx, lastIdx).Col()) + vec.SetCol(c.cols[i].Window(typeconv.FromColumnType(&c.typs[i]), c.curIdx, lastIdx).Col()) nullsSlice := c.cols[i].Nulls().Slice(c.curIdx, lastIdx) vec.SetNulls(&nullsSlice) } @@ -1349,22 +1363,16 @@ func (c *chunkingBatchSource) reset() { // joinTestCase is a helper struct shared by the hash and merge join unit // tests. Not all fields have to be filled in, but init() method *must* be // called. -// NOTE: either logical or physical types *must* be filled in for both inputs. -// Note, however, that if physical types are provided, we will be using lossful -// type conversion during init(). If you want to use specific logical types, -// you should set logical types then. type joinTestCase struct { description string joinType sqlbase.JoinType leftTuples tuples - leftLogTypes []types.T - leftPhysTypes []coltypes.T + leftTypes []types.T leftOutCols []uint32 leftEqCols []uint32 leftDirections []execinfrapb.Ordering_Column_Direction rightTuples tuples - rightLogTypes []types.T - rightPhysTypes []coltypes.T + rightTypes []types.T rightOutCols []uint32 rightEqCols []uint32 rightDirections []execinfrapb.Ordering_Column_Direction @@ -1382,70 +1390,18 @@ func (tc *joinTestCase) init() { } if len(tc.leftDirections) == 0 { - tc.leftDirections = make([]execinfrapb.Ordering_Column_Direction, len(tc.leftPhysTypes)) + tc.leftDirections = make([]execinfrapb.Ordering_Column_Direction, len(tc.leftTypes)) for i := range tc.leftDirections { tc.leftDirections[i] = execinfrapb.Ordering_Column_ASC } } if len(tc.rightDirections) == 0 { - tc.rightDirections = make([]execinfrapb.Ordering_Column_Direction, len(tc.rightPhysTypes)) + tc.rightDirections = make([]execinfrapb.Ordering_Column_Direction, len(tc.rightTypes)) for i := range tc.rightDirections { tc.rightDirections[i] = execinfrapb.Ordering_Column_ASC } } - - toLogType := func(t coltypes.T) *types.T { - switch t { - case coltypes.Bool: - return types.Bool - case coltypes.Bytes: - return types.Bytes - case coltypes.Decimal: - return types.Decimal - case coltypes.Int16: - return types.Int2 - case coltypes.Int32: - return types.Int4 - case coltypes.Int64: - return types.Int - case coltypes.Float64: - return types.Float - case coltypes.Timestamp: - return types.Timestamp - case coltypes.Interval: - return types.Interval - } - execerror.VectorizedInternalPanic(fmt.Sprintf("unexpected coltype %s", t.String())) - return nil - } - toLogTypes := func(typs []coltypes.T) []types.T { - cts := make([]types.T, len(typs)) - for i := range cts { - t := toLogType(typs[i]) - cts[i] = *t - } - return cts - } - var err error - if tc.leftPhysTypes == nil { - tc.leftPhysTypes, err = typeconv.FromColumnTypes(tc.leftLogTypes) - if err != nil { - execerror.VectorizedInternalPanic(err) - } - } - if tc.leftLogTypes == nil { - tc.leftLogTypes = toLogTypes(tc.leftPhysTypes) - } - if tc.rightPhysTypes == nil { - tc.rightPhysTypes, err = typeconv.FromColumnTypes(tc.rightLogTypes) - if err != nil { - execerror.VectorizedInternalPanic(err) - } - } - if tc.rightLogTypes == nil { - tc.rightLogTypes = toLogTypes(tc.rightPhysTypes) - } } // mutateTypes returns a slice of joinTestCases with varied types. Assumes @@ -1453,8 +1409,8 @@ func (tc *joinTestCase) init() { func (tc *joinTestCase) mutateTypes() []*joinTestCase { ret := []*joinTestCase{tc} - for _, typ := range []coltypes.T{coltypes.Decimal, coltypes.Bytes} { - if typ == coltypes.Bytes { + for _, typ := range []types.T{*types.Decimal, *types.Bytes} { + if typ.Identical(types.Bytes) { // Skip test cases with ON conditions for now, since those expect // numeric inputs. if !tc.onExpr.Empty() { @@ -1462,15 +1418,13 @@ func (tc *joinTestCase) mutateTypes() []*joinTestCase { } } newTc := *tc - newTc.leftPhysTypes = make([]coltypes.T, len(tc.leftPhysTypes)) - newTc.rightPhysTypes = make([]coltypes.T, len(tc.rightPhysTypes)) - newTc.leftLogTypes = nil - newTc.rightLogTypes = nil - copy(newTc.leftPhysTypes, tc.leftPhysTypes) - copy(newTc.rightPhysTypes, tc.rightPhysTypes) - for _, typs := range [][]coltypes.T{newTc.leftPhysTypes, newTc.rightPhysTypes} { + newTc.leftTypes = make([]types.T, len(tc.leftTypes)) + newTc.rightTypes = make([]types.T, len(tc.rightTypes)) + copy(newTc.leftTypes, tc.leftTypes) + copy(newTc.rightTypes, tc.rightTypes) + for _, typs := range [][]types.T{newTc.leftTypes, newTc.rightTypes} { for i := range typs { - if typs[i] != coltypes.Int64 { + if !typ.Identical(types.Int) { // We currently can only mutate test cases that are made up of int64 // only. return ret @@ -1488,7 +1442,7 @@ func (tc *joinTestCase) mutateTypes() []*joinTestCase { if tups[i][j] == nil { continue } - switch typ { + switch typeconv.FromColumnType(&typ) { case coltypes.Decimal: var d apd.Decimal _, _ = d.SetFloat64(float64(tups[i][j].(int))) @@ -1508,7 +1462,7 @@ type sortTestCase struct { description string tuples tuples expected tuples - logTypes []types.T + typs []types.T ordCols []execinfrapb.Ordering_Column matchLen int k uint16 @@ -1545,11 +1499,11 @@ func (p *mockTypeContext) IndexedVarNodeFormatter(idx int) tree.NodeFormatter { func createTestProjectingOperator( ctx context.Context, flowCtx *execinfra.FlowCtx, - input Operator, + input colexecbase.Operator, inputTypes []types.T, projectingExpr string, canFallbackToRowexec bool, -) (Operator, error) { +) (colexecbase.Operator, error) { expr, err := parser.ParseExpr(projectingExpr) if err != nil { return nil, err @@ -1575,7 +1529,7 @@ func createTestProjectingOperator( } args := NewColOperatorArgs{ Spec: spec, - Inputs: []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 f95e85fa54ef..7bf243586788 100644 --- a/pkg/sql/colexec/values_differ_tmpl.go +++ b/pkg/sql/colexec/values_differ_tmpl.go @@ -26,13 +26,16 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" - // {{/* + "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. @@ -58,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 { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} @@ -67,9 +70,9 @@ func _ASSIGN_NE(_, _, _ string) bool { // the values differ. This function pays attention to NULLs, and two NULL // values do *not* differ. func valuesDiffer( - t coltypes.T, aColVec coldata.Vec, aValueIdx int, bColVec coldata.Vec, bValueIdx int, + t *types.T, aColVec coldata.Vec, aValueIdx int, bColVec coldata.Vec, bValueIdx int, ) bool { - switch t { + switch typeconv.FromColumnType(t) { // {{range .}} case _TYPES_T: aCol := aColVec._TemplateType() @@ -90,7 +93,7 @@ func valuesDiffer( return unique // {{end}} default: - execerror.VectorizedInternalPanic(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 67d6c7a60acb..7e4f5a8f8e14 100644 --- a/pkg/sql/colexec/vec_comparators_tmpl.go +++ b/pkg/sql/colexec/vec_comparators_tmpl.go @@ -28,14 +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/colexec/execerror" - // {{/* + "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. @@ -58,10 +61,13 @@ var _ tree.Datum // Dummy import to pull in "math" package. var _ = math.MaxInt64 +// Dummy import to pull in "coltypes" package. +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 { - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } // */}} @@ -135,8 +141,8 @@ func (c *_TYPEVecComparator) set(srcVecIdx, dstVecIdx int, srcIdx, dstIdx int) { // {{end}} -func GetVecComparator(t coltypes.T, numVecs int) vecComparator { - switch t { +func GetVecComparator(t *types.T, numVecs int) vecComparator { + switch typeconv.FromColumnType(t) { // {{range .}} case coltypes._TYPE: return &_TYPEVecComparator{ @@ -145,7 +151,7 @@ func GetVecComparator(t coltypes.T, numVecs int) vecComparator { } // {{end}} } - execerror.VectorizedInternalPanic(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 54d3ee3dbf54..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/colexec/execerror" + "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 { - execerror.VectorizedInternalPanic(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: - execerror.VectorizedInternalPanic(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 a613910ca64b..87d4b5e36374 100644 --- a/pkg/sql/colexec/window_functions_test.go +++ b/pkg/sql/colexec/window_functions_test.go @@ -16,6 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "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" @@ -271,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 []Operator) (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 { @@ -283,7 +284,7 @@ func TestWindowFunctions(t *testing.T) { Windower: &tc.windowerSpec, }, } - sem := 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 cc88fd58fc82..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/colexec/execerror" + "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: - execerror.VectorizedInternalPanic(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 8a5ffe1777d5..34305ec69e70 100644 --- a/pkg/sql/colexec/window_peer_grouper_tmpl.go +++ b/pkg/sql/colexec/window_peer_grouper_tmpl.go @@ -23,8 +23,10 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "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" ) // NewWindowPeerGrouper creates a new Operator that puts 'true' in @@ -36,13 +38,13 @@ import ( // 'true' indicates the start of a new partition. // NOTE: the input *must* already be ordered on ordCols. func NewWindowPeerGrouper( - allocator *Allocator, - input Operator, - inputTyps []coltypes.T, + allocator *colmem.Allocator, + input colexecbase.Operator, + typs []types.T, orderingCols []execinfrapb.Ordering_Column, partitionColIdx int, outputColIdx int, -) (op Operator, err error) { +) (op colexecbase.Operator, err error) { allPeers := len(orderingCols) == 0 var distinctCol []bool if !allPeers { @@ -51,13 +53,13 @@ func NewWindowPeerGrouper( orderIdxs[i] = ordCol.ColIdx } input, distinctCol, err = OrderedDistinctColsToOperators( - input, orderIdxs, inputTyps, + input, orderIdxs, typs, ) if err != nil { return nil, err } } - input = newVectorTypeEnforcer(allocator, input, coltypes.Bool, outputColIdx) + input = newVectorTypeEnforcer(allocator, input, types.Bool, outputColIdx) initFields := windowPeerGrouperInitFields{ OneInputNode: NewOneInputNode(input), allocator: allocator, @@ -88,7 +90,7 @@ func NewWindowPeerGrouper( type windowPeerGrouperInitFields struct { OneInputNode - allocator *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 @@ -106,7 +108,7 @@ type _PEER_GROUPER_STRINGOp struct { // {{end}} } -var _ Operator = &_PEER_GROUPER_STRINGOp{} +var _ colexecbase.Operator = &_PEER_GROUPER_STRINGOp{} func (p *_PEER_GROUPER_STRINGOp) Init() { p.input.Init() diff --git a/pkg/sql/colexec/execerror/error.go b/pkg/sql/colexecbase/colexecerror/error.go similarity index 61% rename from pkg/sql/colexec/execerror/error.go rename to pkg/sql/colexecbase/colexecerror/error.go index c2606d9a76a2..bcf71d063b84 100644 --- a/pkg/sql/colexec/execerror/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 execerror +package colexecerror import ( "bufio" @@ -78,18 +78,18 @@ func CatchVectorizedRuntimeError(operation func()) (retErr error) { } annotateErrorWithoutCode := true - var nvie *notVectorizedInternalError - if errors.As(err, &nvie) { - // A notVectorizedInternalError was not caused by the - // vectorized engine and represents an error that we don't - // want to annotate in case it doesn't have a valid PG code. + var nie *notInternalError + if errors.As(err, &nie) { + // A notInternalError was not caused by the vectorized engine and + // represents an error that we don't want to annotate in case it + // doesn't have a valid PG code. annotateErrorWithoutCode = false } if code := pgerror.GetPGCode(err); annotateErrorWithoutCode && code == pgcode.Uncategorized { // Any error without a code already is "surprising" and // needs to be annotated to indicate that it was // unexpected. - retErr = errors.NewAssertionErrorWithWrappedErrf(err, "unexpected error from the vectorized runtime") + retErr = errors.NewAssertionErrorWithWrappedErrf(err, "unexpected error from the vectorized engine") } }() operation() @@ -97,13 +97,11 @@ func CatchVectorizedRuntimeError(operation func()) (retErr error) { } const ( - colPackagePrefix = "github.com/cockroachdb/cockroach/pkg/col" - colcontainerPackagePrefix = "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" - colexecPackagePrefix = "github.com/cockroachdb/cockroach/pkg/sql/colexec" - colflowsetupPackagePrefix = "github.com/cockroachdb/cockroach/pkg/sql/colflow" - execinfraPackagePrefix = "github.com/cockroachdb/cockroach/pkg/sql/execinfra" - rowexecPackagePrefix = "github.com/cockroachdb/cockroach/pkg/sql/rowexec" - treePackagePrefix = "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + colPackagesPrefix = "github.com/cockroachdb/cockroach/pkg/col" + execinfraPackagePrefix = "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + rowexecPackagePrefix = "github.com/cockroachdb/cockroach/pkg/sql/rowexec" + sqlColPackagesPrefix = "github.com/cockroachdb/cockroach/pkg/sql/col" + treePackagePrefix = "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" ) // isPanicFromVectorizedEngine checks whether the panic that was emitted from @@ -118,12 +116,10 @@ func isPanicFromVectorizedEngine(panicEmittedFrom string) bool { // we say that the panic is not from the vectorized engine. return false } - return strings.HasPrefix(panicEmittedFrom, colPackagePrefix) || - strings.HasPrefix(panicEmittedFrom, colcontainerPackagePrefix) || - strings.HasPrefix(panicEmittedFrom, colexecPackagePrefix) || - strings.HasPrefix(panicEmittedFrom, colflowsetupPackagePrefix) || + return strings.HasPrefix(panicEmittedFrom, colPackagesPrefix) || strings.HasPrefix(panicEmittedFrom, execinfraPackagePrefix) || strings.HasPrefix(panicEmittedFrom, rowexecPackagePrefix) || + strings.HasPrefix(panicEmittedFrom, sqlColPackagesPrefix) || strings.HasPrefix(panicEmittedFrom, treePackagePrefix) } @@ -145,65 +141,58 @@ func NewStorageError(err error) *StorageError { return &StorageError{error: err} } -// notVectorizedInternalError is an error that originated outside of the -// vectorized engine (for example, it was caused by a non-columnar builtin). -// notVectorizedInternalError will be returned to the client not as an +// notInternalError is an error that occurs not because the vectorized engine +// happens to be in an unexpected state (for example, it was caused by a +// non-columnar builtin). +// notInternalError will be returned to the client not as an // "internal error" and without the stack trace. -type notVectorizedInternalError struct { +type notInternalError struct { cause error } -func newNotVectorizedInternalError(err error) *notVectorizedInternalError { - return ¬VectorizedInternalError{cause: err} +func newNotInternalError(err error) *notInternalError { + return ¬InternalError{cause: err} } var ( - _ causer.Causer = ¬VectorizedInternalError{} - _ errors.Wrapper = ¬VectorizedInternalError{} + _ causer.Causer = ¬InternalError{} + _ errors.Wrapper = ¬InternalError{} ) -func (e *notVectorizedInternalError) Error() string { +func (e *notInternalError) Error() string { return e.cause.Error() } -func (e *notVectorizedInternalError) Cause() error { +func (e *notInternalError) Cause() error { return e.cause } -func (e *notVectorizedInternalError) Unwrap() error { +func (e *notInternalError) Unwrap() error { return e.Cause() } -func decodeNotVectorizedInternalError( +func decodeNotInternalError( _ context.Context, cause error, _ string, _ []string, _ proto.Message, ) error { - return newNotVectorizedInternalError(cause) + return newNotInternalError(cause) } func init() { - errors.RegisterWrapperDecoder(errors.GetTypeKey((*notVectorizedInternalError)(nil)), decodeNotVectorizedInternalError) + errors.RegisterWrapperDecoder(errors.GetTypeKey((*notInternalError)(nil)), decodeNotInternalError) } -// VectorizedInternalPanic simply panics with the provided object. It will -// always be returned as internal error to the client with the corresponding -// stack trace. This method should be called to propagate all *unexpected* -// errors that originated within the vectorized engine. -func VectorizedInternalPanic(err interface{}) { +// InternalError simply panics with the provided object. It will always be +// caught and returned as internal error to the client with the corresponding +// stack trace. This method should be called to propagate errors that resulted +// in the vectorized engine being in an *unexpected* state. +func InternalError(err interface{}) { panic(err) } -// VectorizedExpectedInternalPanic is the same as NonVectorizedPanic. It should -// be called to propagate all *expected* errors that originated within the -// vectorized engine. -func VectorizedExpectedInternalPanic(err error) { - NonVectorizedPanic(err) -} - -// NonVectorizedPanic panics with the error that is wrapped by -// notVectorizedInternalError which will not be treated as internal error and +// ExpectedError panics with the error that is wrapped by +// notInternalError which will not be treated as internal error and // will not have a printed out stack trace. This method should be called to -// propagate all errors that originated outside of the vectorized engine and -// all expected errors from the vectorized engine. -func NonVectorizedPanic(err error) { - panic(newNotVectorizedInternalError(err)) +// propagate errors that the vectorized engine *expects* to occur. +func ExpectedError(err error) { + panic(newNotInternalError(err)) } diff --git a/pkg/sql/colexecbase/dep_test.go b/pkg/sql/colexecbase/dep_test.go new file mode 100644 index 000000000000..f9e2b8431674 --- /dev/null +++ b/pkg/sql/colexecbase/dep_test.go @@ -0,0 +1,33 @@ +// 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 colexecbase + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" +) + +func TestNoLinkForbidden(t *testing.T) { + defer leaktest.AfterTest(t)() + + buildutil.VerifyNoImports(t, + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase", true, + []string{ + "github.com/cockroachdb/cockroach/pkg/sql/colcontainer", + "github.com/cockroachdb/cockroach/pkg/sql/colexec", + "github.com/cockroachdb/cockroach/pkg/sql/colflow", + "github.com/cockroachdb/cockroach/pkg/sql/rowexec", + "github.com/cockroachdb/cockroach/pkg/sql/rowflow", + }, nil, + ) +} diff --git a/pkg/sql/colexecbase/operator.go b/pkg/sql/colexecbase/operator.go new file mode 100644 index 000000000000..9d6ea1fd0e90 --- /dev/null +++ b/pkg/sql/colexecbase/operator.go @@ -0,0 +1,62 @@ +// 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 colexecbase + +import ( + "context" + "fmt" + + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" +) + +// Operator is a column vector operator that produces a Batch as output. +type Operator interface { + // Init initializes this operator. Will be called once at operator setup + // time. If an operator has an input operator, it's responsible for calling + // Init on that input operator as well. + // TODO(yuzefovich): we might need to clarify whether it is ok to call + // Init() multiple times before the first call to Next(). It is possible to + // hit the memory limit during Init(), and a disk-backed operator needs to + // make sure that the input has been initialized. We could also in case that + // Init() doesn't succeed for bufferingInMemoryOperator - which should only + // happen when 'workmem' setting is too low - just bail, even if we have + // disk spilling for that operator. + Init() + + // Next returns the next Batch from this operator. Once the operator is + // finished, it will return a Batch with length 0. Subsequent calls to + // Next at that point will always return a Batch with length 0. + // + // Calling Next may invalidate the contents of the last Batch returned by + // Next. + // Canceling the provided context results in forceful termination of + // execution. + Next(context.Context) coldata.Batch + + execinfra.OpNode +} + +// ZeroInputNode is an execinfra.OpNode with no inputs. +type ZeroInputNode struct{} + +// ChildCount implements the execinfra.OpNode interface. +func (ZeroInputNode) ChildCount(verbose bool) int { + return 0 +} + +// Child implements the execinfra.OpNode interface. +func (ZeroInputNode) Child(nth int, verbose bool) execinfra.OpNode { + 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/testutils.go b/pkg/sql/colexecbase/testutils.go similarity index 86% rename from pkg/sql/colexec/testutils.go rename to pkg/sql/colexecbase/testutils.go index 451c2eac206e..7a80f96c9901 100644 --- a/pkg/sql/colexec/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 colexec +package colexecbase import ( "context" @@ -16,7 +16,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" ) @@ -37,7 +40,7 @@ func NewBatchBuffer() *BatchBuffer { } // Add adds a batch to the buffer. -func (b *BatchBuffer) Add(batch coldata.Batch) { +func (b *BatchBuffer) Add(batch coldata.Batch, _ []types.T) { b.buffer = append(b.buffer, batch) } @@ -55,10 +58,11 @@ func (b *BatchBuffer) Next(context.Context) coldata.Batch { type RepeatableBatchSource struct { ZeroInputNode - colVecs []coldata.Vec - typs []coltypes.T - sel []int - batchLen int + colVecs []coldata.Vec + typs []types.T + physTypes []coltypes.T + sel []int + batchLen int // numToCopy indicates the number of tuples that needs to be copied. It is // equal to batchLen when sel is nil and is equal to maxSelIdx+1 when sel is // non-nil. @@ -75,11 +79,9 @@ var _ Operator = &RepeatableBatchSource{} // input batch forever. Note that it stores the contents of the input batch and // 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) *RepeatableBatchSource { - typs := make([]coltypes.T, batch.Width()) - for i, vec := range batch.ColVecs() { - typs[i] = vec.Type() - } +func NewRepeatableBatchSource( + allocator *colmem.Allocator, batch coldata.Batch, typs []types.T, +) *RepeatableBatchSource { sel := batch.Selection() batchLen := batch.Length() numToCopy := batchLen @@ -93,9 +95,14 @@ func NewRepeatableBatchSource(allocator *Allocator, batch coldata.Batch) *Repeat numToCopy = maxIdx + 1 } output := allocator.NewMemBatchWithSize(typs, numToCopy) + physTypes, err := typeconv.FromColumnTypes(typs) + if err != nil { + colexecerror.InternalError(err) + } src := &RepeatableBatchSource{ colVecs: batch.ColVecs(), typs: typs, + physTypes: physTypes, sel: sel, batchLen: batchLen, numToCopy: numToCopy, @@ -114,13 +121,13 @@ func (s *RepeatableBatchSource) Next(context.Context) coldata.Batch { if s.sel != nil { copy(s.output.Selection()[:s.batchLen], s.sel[:s.batchLen]) } - for i, typ := range s.typs { + for i, physType := range s.physTypes { // This Copy is outside of the allocator since the RepeatableBatchSource is // a test utility which is often used in the benchmarks, and we want to // reduce the performance impact of this operator. s.output.ColVec(i).Copy(coldata.CopySliceArgs{ SliceArgs: coldata.SliceArgs{ - ColType: typ, + ColType: physType, Src: s.colVecs[i], SrcEndIdx: s.numToCopy, }, @@ -194,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 { - execerror.VectorizedInternalPanic("releasing a negative amount") + colexecerror.InternalError("releasing a negative amount") } if s.count-n < 0 { - execerror.VectorizedInternalPanic(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 2b27ccc19810..bf324667f620 100644 --- a/pkg/sql/colflow/colrpc/colrpc_test.go +++ b/pkg/sql/colflow/colrpc/colrpc_test.go @@ -22,12 +22,16 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "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/colexec" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -185,8 +189,8 @@ func TestOutboxInbox(t *testing.T) { // Do the actual testing. t.Run(fmt.Sprintf("cancellationScenario=%s", cancellationScenarioName), func(t *testing.T) { var ( - typs = []coltypes.T{coltypes.Int64} - inputBuffer = colexec.NewBatchBuffer() + typs = []types.T{*types.Int} + 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 @@ -198,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 := colexec.RandomDataOpArgs{ + args := coldatatestutils.RandomDataOpArgs{ DeterministicTyps: typs, NumBatches: 64, Selection: true, @@ -214,19 +218,19 @@ func TestOutboxInbox(t *testing.T) { } inputMemAcc := testMemMonitor.MakeBoundAccount() defer inputMemAcc.Close(ctx) - input := colexec.NewRandomDataOp( - colexec.NewAllocator(ctx, &inputMemAcc), rng, args, + input := coldatatestutils.NewRandomDataOp( + colmem.NewAllocator(ctx, &inputMemAcc), rng, args, ) outboxMemAcc := testMemMonitor.MakeBoundAccount() defer outboxMemAcc.Close(ctx) - outbox, err := NewOutbox(colexec.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( - colexec.NewAllocator(ctx, &inboxMemAcc), typs, execinfrapb.StreamID(0), + colmem.NewAllocator(ctx, &inboxMemAcc), typs, execinfrapb.StreamID(0), ) require.NoError(t, err) @@ -265,14 +269,14 @@ func TestOutboxInbox(t *testing.T) { deselectorMemAcc := testMemMonitor.MakeBoundAccount() defer deselectorMemAcc.Close(ctx) inputBatches := colexec.NewDeselectorOp( - colexec.NewAllocator(ctx, &deselectorMemAcc), inputBuffer, typs, + colmem.NewAllocator(ctx, &deselectorMemAcc), inputBuffer, typs, ) inputBatches.Init() - outputBatches := colexec.NewBatchBuffer() + outputBatches := colexecbase.NewBatchBuffer() var readerErr error for { var outputBatch coldata.Batch - if err := execerror.CatchVectorizedRuntimeError(func() { + if err := colexecerror.CatchVectorizedRuntimeError(func() { outputBatch = inbox.Next(readerCtx) }); err != nil { readerErr = err @@ -282,14 +286,14 @@ func TestOutboxInbox(t *testing.T) { // Accumulate batches to check for correctness. // Copy batch since it's not safe to reuse after calling Next. if outputBatch == coldata.ZeroBatch { - outputBatches.Add(coldata.ZeroBatch) + outputBatches.Add(coldata.ZeroBatch, typs) } else { batchCopy := testAllocator.NewMemBatchWithSize(typs, outputBatch.Length()) testAllocator.PerformOperation(batchCopy.ColVecs(), func() { for i := range typs { batchCopy.ColVec(i).Append( coldata.SliceArgs{ - ColType: typs[i], + ColType: typeconv.FromColumnType(&typs[i]), Src: outputBatch.ColVec(i), SrcEndIdx: outputBatch.Length(), }, @@ -297,7 +301,7 @@ func TestOutboxInbox(t *testing.T) { } }) batchCopy.SetLength(outputBatch.Length()) - outputBatches.Add(batchCopy) + outputBatches.Add(batchCopy, typs) } } if outputBatch.Length() == 0 { @@ -332,8 +336,8 @@ func TestOutboxInbox(t *testing.T) { for i := range typs { require.Equal( t, - inputBatch.ColVec(i).Window(typs[i], 0, inputBatch.Length()), - outputBatch.ColVec(i).Window(typs[i], 0, outputBatch.Length()), + inputBatch.ColVec(i).Window(typeconv.FromColumnType(&typs[i]), 0, inputBatch.Length()), + outputBatch.ColVec(i).Window(typeconv.FromColumnType(&typs[i]), 0, outputBatch.Length()), "batchNum: %d", batchNum, ) } @@ -442,11 +446,11 @@ func TestOutboxInboxMetadataPropagation(t *testing.T) { var ( serverStreamNotification = <-mockServer.InboundStreams serverStream = serverStreamNotification.Stream - typs = []coltypes.T{coltypes.Int64} - input = colexec.NewRandomDataOp( + typs = []types.T{*types.Int} + input = coldatatestutils.NewRandomDataOp( testAllocator, rng, - colexec.RandomDataOpArgs{ + coldatatestutils.RandomDataOpArgs{ DeterministicTyps: typs, NumBatches: tc.numBatches, Selection: true, @@ -458,7 +462,7 @@ func TestOutboxInboxMetadataPropagation(t *testing.T) { outboxMemAcc := testMemMonitor.MakeBoundAccount() defer outboxMemAcc.Close(ctx) - outbox, err := NewOutbox(colexec.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)}} @@ -470,7 +474,7 @@ func TestOutboxInboxMetadataPropagation(t *testing.T) { inboxMemAcc := testMemMonitor.MakeBoundAccount() defer inboxMemAcc.Close(ctx) inbox, err := NewInbox( - colexec.NewAllocator(ctx, &inboxMemAcc), + colmem.NewAllocator(ctx, &inboxMemAcc), typs, execinfrapb.StreamID(0), ) require.NoError(t, err) @@ -523,22 +527,22 @@ func BenchmarkOutboxInbox(b *testing.B) { serverStreamNotification := <-mockServer.InboundStreams serverStream := serverStreamNotification.Stream - typs := []coltypes.T{coltypes.Int64} + typs := []types.T{*types.Int} batch := testAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) - input := colexec.NewRepeatableBatchSource(testAllocator, batch) + input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) outboxMemAcc := testMemMonitor.MakeBoundAccount() defer outboxMemAcc.Close(ctx) - outbox, err := NewOutbox(colexec.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( - colexec.NewAllocator(ctx, &inboxMemAcc), typs, execinfrapb.StreamID(0), + colmem.NewAllocator(ctx, &inboxMemAcc), typs, execinfrapb.StreamID(0), ) require.NoError(b, err) @@ -581,12 +585,12 @@ func TestOutboxStreamIDPropagation(t *testing.T) { dialer := &execinfrapb.MockDialer{Addr: addr} defer dialer.Close() - typs := []coltypes.T{coltypes.Int64} + typs := []types.T{*types.Int} var inTags *logtags.Buffer nextDone := make(chan struct{}) - input := &colexec.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) @@ -596,7 +600,7 @@ func TestOutboxStreamIDPropagation(t *testing.T) { outboxMemAcc := testMemMonitor.MakeBoundAccount() defer outboxMemAcc.Close(ctx) - outbox, err := NewOutbox(colexec.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{}) @@ -658,7 +662,7 @@ func TestInboxCtxStreamIDTagging(t *testing.T) { rpcLayer := makeMockFlowStreamRPCLayer() - typs := []coltypes.T{coltypes.Int64} + typs := []types.T{*types.Int} inbox, err := NewInbox(testAllocator, typs, streamID) require.NoError(t, err) diff --git a/pkg/sql/colflow/colrpc/inbox.go b/pkg/sql/colflow/colrpc/inbox.go index f8ded390febb..b73f0c754f2b 100644 --- a/pkg/sql/colflow/colrpc/inbox.go +++ b/pkg/sql/colflow/colrpc/inbox.go @@ -19,10 +19,11 @@ 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" - "github.com/cockroachdb/cockroach/pkg/sql/colexec" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/logtags" @@ -45,8 +46,8 @@ type flowStreamServer interface { // RunWithStream (or more specifically, the RPC handler) will unblock Next by // closing the stream. type Inbox struct { - colexec.ZeroInputNode - typs []coltypes.T + colexecbase.ZeroInputNode + typs []types.T converter *colserde.ArrowBatchConverter serializer *colserde.RecordBatchSerializer @@ -113,11 +114,11 @@ type Inbox struct { } } -var _ colexec.Operator = &Inbox{} +var _ colexecbase.Operator = &Inbox{} // NewInbox creates a new Inbox. func NewInbox( - allocator *colexec.Allocator, typs []coltypes.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() - execerror.VectorizedInternalPanic(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. - execerror.VectorizedExpectedInternalPanic(err) + colexecerror.ExpectedError(err) } for { @@ -306,7 +307,7 @@ func (i *Inbox) Next(ctx context.Context) coldata.Batch { return coldata.ZeroBatch } i.errCh <- err - execerror.VectorizedInternalPanic(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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } if err := i.converter.ArrowToBatch(i.scratch.data, i.scratch.b); err != nil { - execerror.VectorizedInternalPanic(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 078a60316d7e..4383fc8c5932 100644 --- a/pkg/sql/colflow/colrpc/inbox_test.go +++ b/pkg/sql/colflow/colrpc/inbox_test.go @@ -22,11 +22,12 @@ 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/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -58,7 +59,7 @@ var _ flowStreamServer = callbackFlowStreamServer{} func TestInboxCancellation(t *testing.T) { defer leaktest.AfterTest(t)() - typs := []coltypes.T{coltypes.Int64} + typs := []types.T{*types.Int} t.Run("ReaderWaitingForStreamHandler", func(t *testing.T) { inbox, err := NewInbox(testAllocator, typs, execinfrapb.StreamID(0)) require.NoError(t, err) @@ -66,7 +67,7 @@ func TestInboxCancellation(t *testing.T) { // Cancel the context. cancelFn() // Next should not block if the context is canceled. - err = execerror.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{}) @@ -124,7 +125,7 @@ func TestInboxCancellation(t *testing.T) { func TestInboxNextPanicDoesntLeakGoroutines(t *testing.T) { defer leaktest.AfterTest(t)() - inbox, err := NewInbox(testAllocator, []coltypes.T{coltypes.Int64}, execinfrapb.StreamID(0)) + inbox, err := NewInbox(testAllocator, []types.T{*types.Int}, execinfrapb.StreamID(0)) require.NoError(t, err) rpcLayer := makeMockFlowStreamRPCLayer() @@ -151,7 +152,7 @@ func TestInboxTimeout(t *testing.T) { ctx := context.Background() - inbox, err := NewInbox(testAllocator, []coltypes.T{coltypes.Int64}, execinfrapb.StreamID(0)) + inbox, err := NewInbox(testAllocator, []types.T{*types.Int}, execinfrapb.StreamID(0)) require.NoError(t, err) var ( @@ -159,7 +160,7 @@ func TestInboxTimeout(t *testing.T) { rpcLayer = makeMockFlowStreamRPCLayer() ) go func() { - readerErrCh <- execerror.CatchVectorizedRuntimeError(func() { inbox.Next(ctx) }) + readerErrCh <- colexecerror.CatchVectorizedRuntimeError(func() { inbox.Next(ctx) }) }() // Timeout the inbox. @@ -199,8 +200,8 @@ func TestInboxShutdown(t *testing.T) { drainMetaSleep = time.Millisecond * time.Duration(rng.Intn(10)) nextSleep = time.Millisecond * time.Duration(rng.Intn(10)) runWithStreamSleep = time.Millisecond * time.Duration(rng.Intn(10)) - typs = []coltypes.T{coltypes.Int64} - batch = colexec.RandomBatch(testAllocator, rng, typs, coldata.BatchSize(), 0 /* length */, rng.Float64()) + typs = []types.T{*types.Int} + 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( - colexec.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 = execerror.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 ea8c58add8e2..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/colexec" + "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 *colexec.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 = colexec.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 f47bb12e2e2e..e47a1671e8b0 100644 --- a/pkg/sql/colflow/colrpc/outbox.go +++ b/pkg/sql/colflow/colrpc/outbox.go @@ -18,12 +18,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/colserde" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/sql/colexec" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" "github.com/cockroachdb/logtags" "google.golang.org/grpc" @@ -49,7 +51,7 @@ type Dialer interface { type Outbox struct { colexec.OneInputNode - typs []coltypes.T + typs []types.T // batch is the last batch received from the input. batch coldata.Batch @@ -74,9 +76,9 @@ type Outbox struct { // NewOutbox creates a new Outbox. func NewOutbox( - allocator *colexec.Allocator, - input colexec.Operator, - typs []coltypes.T, + allocator *colmem.Allocator, + input colexecbase.Operator, + typs []types.T, metadataSources []execinfrapb.MetadataSource, toClose []colexec.IdempotentCloser, ) (*Outbox, error) { @@ -246,7 +248,7 @@ func (o *Outbox) sendBatches( return true, nil } - if err := execerror.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 cde20cba21d4..7b28c1600184 100644 --- a/pkg/sql/colflow/colrpc/outbox_test.go +++ b/pkg/sql/colflow/colrpc/outbox_test.go @@ -16,9 +16,10 @@ import ( "sync/atomic" "testing" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "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/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/stretchr/testify/require" @@ -30,8 +31,8 @@ func TestOutboxCatchesPanics(t *testing.T) { ctx := context.Background() var ( - input = colexec.NewBatchBuffer() - typs = []coltypes.T{coltypes.Int64} + input = colexecbase.NewBatchBuffer() + typs = []types.T{*types.Int} rpcLayer = makeMockFlowStreamRPCLayer() ) outbox, err := NewOutbox(testAllocator, input, typs, nil /* metadataSources */, nil /* toClose */) @@ -53,7 +54,7 @@ func TestOutboxCatchesPanics(t *testing.T) { inboxMemAccount := testMemMonitor.MakeBoundAccount() defer inboxMemAccount.Close(ctx) inbox, err := NewInbox( - colexec.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 = colexec.NewBatchBuffer() - typs = []coltypes.T{coltypes.Int64} + 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 *colexec.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,13 +108,13 @@ func TestOutboxDrainsMetadataSources(t *testing.T) { outboxMemAccount := testMemMonitor.MakeBoundAccount() defer outboxMemAccount.Close(ctx) outbox, sourceDrained, err := newOutboxWithMetaSources( - colexec.NewAllocator(ctx, &outboxMemAccount), + colmem.NewAllocator(ctx, &outboxMemAccount), ) require.NoError(t, err) b := testAllocator.NewMemBatch(typs) b.SetLength(0) - input.Add(b) + input.Add(b, typs) // Close the csChan to unblock the Recv goroutine (we don't need it for this // test). diff --git a/pkg/sql/colflow/main_test.go b/pkg/sql/colflow/main_test.go index 2d1c470b805d..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/colexec" + "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 *colexec.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 = colexec.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 91f320102125..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" + "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/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexec" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/typeconv" + "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 { - execerror.VectorizedInternalPanic(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 colexec.Operator, - inputs []colexec.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 colexec.Operator + rootOperator colexecbase.Operator metadataSources []execinfrapb.MetadataSource toClose []colexec.IdempotentCloser } @@ -449,21 +450,21 @@ type opDAGWithMetaSources struct { // several components in a remote flow. Mostly for testing purposes. type remoteComponentCreator interface { newOutbox( - allocator *colexec.Allocator, - input colexec.Operator, - typs []coltypes.T, + allocator *colmem.Allocator, + input colexecbase.Operator, + typs []types.T, metadataSources []execinfrapb.MetadataSource, toClose []colexec.IdempotentCloser, ) (*colrpc.Outbox, error) - newInbox(allocator *colexec.Allocator, typs []coltypes.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 *colexec.Allocator, - input colexec.Operator, - typs []coltypes.T, + allocator *colmem.Allocator, + input colexecbase.Operator, + typs []types.T, metadataSources []execinfrapb.MetadataSource, toClose []colexec.IdempotentCloser, ) (*colrpc.Outbox, error) { @@ -471,7 +472,7 @@ func (vectorizedRemoteComponentCreator) newOutbox( } func (vectorizedRemoteComponentCreator) newInbox( - allocator *colexec.Allocator, typs []coltypes.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 colexec.Operator, - outputTyps []coltypes.T, + op colexecbase.Operator, + outputTyps []types.T, stream *execinfrapb.StreamEndpointSpec, metadataSourcesQueue []execinfrapb.MetadataSource, toClose []colexec.IdempotentCloser, ) (execinfra.OpNode, error) { outbox, err := s.remoteComponentCreator.newOutbox( - colexec.NewAllocator(ctx, s.newStreamingMemAccount(flowCtx)), + colmem.NewAllocator(ctx, s.newStreamingMemAccount(flowCtx)), op, outputTyps, metadataSourcesQueue, toClose, ) if err != nil { @@ -639,8 +640,8 @@ func (s *vectorizedFlowCreator) setupRemoteOutputStream( func (s *vectorizedFlowCreator) setupRouter( ctx context.Context, flowCtx *execinfra.FlowCtx, - input colexec.Operator, - outputTyps []coltypes.T, + input colexecbase.Operator, + outputTyps []types.T, output *execinfrapb.OutputRouterSpec, metadataSourcesQueue []execinfrapb.MetadataSource, toClose []colexec.IdempotentCloser, @@ -657,10 +658,10 @@ func (s *vectorizedFlowCreator) setupRouter( mmName := "hash-router-[" + strings.Join(streamIDs, ",") + "]" hashRouterMemMonitor := s.createBufferingUnlimitedMemMonitor(ctx, flowCtx, mmName) - allocators := make([]*colexec.Allocator, len(output.Streams)) + allocators := make([]*colmem.Allocator, len(output.Streams)) for i := range allocators { acc := hashRouterMemMonitor.MakeBoundAccount() - allocators[i] = colexec.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 colexec.Operator, _ []execinfrapb.MetadataSource, _ error) { - inputStreamOps := make([]colexec.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 { @@ -750,13 +751,12 @@ func (s *vectorizedFlowCreator) setupInput( if err := s.checkInboundStreamID(inputStream.StreamID); err != nil { return nil, nil, err } - typs, err := typeconv.FromColumnTypes(input.ColumnTypes) - if err != nil { + if _, err := typeconv.FromColumnTypes(input.ColumnTypes); err != nil { return nil, nil, err } inbox, err := s.remoteComponentCreator.newInbox( - colexec.NewAllocator(ctx, s.newStreamingMemAccount(flowCtx)), - typs, inputStream.StreamID, + colmem.NewAllocator(ctx, s.newStreamingMemAccount(flowCtx)), + input.ColumnTypes, inputStream.StreamID, ) if err != nil { return nil, nil, err @@ -788,20 +788,23 @@ func (s *vectorizedFlowCreator) setupInput( op = inputStreamOps[0] if len(inputStreamOps) > 1 { statsInputs := inputStreamOps - typs, err := typeconv.FromColumnTypes(input.ColumnTypes) - if err != nil { + if _, err := typeconv.FromColumnTypes(input.ColumnTypes); err != nil { return nil, nil, err } if input.Type == execinfrapb.InputSyncSpec_ORDERED { - op = colexec.NewOrderedSynchronizer( - colexec.NewAllocator(ctx, s.newStreamingMemAccount(flowCtx)), - inputStreamOps, typs, execinfrapb.ConvertToColumnOrdering(input.Ordering), + var err error + op, err = colexec.NewOrderedSynchronizer( + colmem.NewAllocator(ctx, s.newStreamingMemAccount(flowCtx)), + inputStreamOps, input.ColumnTypes, execinfrapb.ConvertToColumnOrdering(input.Ordering), ) + if err != nil { + return nil, nil, err + } } else { if opt == flowinfra.FuseAggressively { - op = colexec.NewSerialUnorderedSynchronizer(inputStreamOps, typs) + op = colexec.NewSerialUnorderedSynchronizer(inputStreamOps, input.ColumnTypes) } else { - op = colexec.NewParallelUnorderedSynchronizer(inputStreamOps, typs, s.waitGroup) + op = colexec.NewParallelUnorderedSynchronizer(inputStreamOps, input.ColumnTypes, s.waitGroup) s.operatorConcurrency = true } // Don't use the unordered synchronizer's inputs for stats collection @@ -832,8 +835,8 @@ func (s *vectorizedFlowCreator) setupOutput( ctx context.Context, flowCtx *execinfra.FlowCtx, pspec *execinfrapb.ProcessorSpec, - op colexec.Operator, - opOutputTypes []coltypes.T, + op colexecbase.Operator, + opOutputTypes []types.T, metadataSourcesQueue []execinfrapb.MetadataSource, toClose []colexec.IdempotentCloser, ) error { @@ -966,7 +969,7 @@ func (s *vectorizedFlowCreator) setupFlow( queue = append(queue, i) } - inputs := make([]colexec.Operator, 0, 2) + inputs := make([]colexecbase.Operator, 0, 2) for len(queue) > 0 { pspec := &processorSpecs[queue[0]] queue = queue[1:] @@ -1045,12 +1048,11 @@ func (s *vectorizedFlowCreator) setupFlow( // disk. vectorize=on does support this. return nil, errors.Errorf("hash router encountered when vectorize=auto") } - opOutputTypes, err := typeconv.FromColumnTypes(result.ColumnTypes) - if err != nil { + if _, err := typeconv.FromColumnTypes(result.ColumnTypes); err != nil { return nil, err } if err = s.setupOutput( - ctx, flowCtx, pspec, op, opOutputTypes, metadataSourcesQueue, toClose, + ctx, flowCtx, pspec, op, result.ColumnTypes, metadataSourcesQueue, toClose, ); err != nil { return nil, err } @@ -1073,7 +1075,7 @@ func (s *vectorizedFlowCreator) setupFlow( for l := range outputSpec.Input[k].Streams { inputStream := outputSpec.Input[k].Streams[l] if inputStream.StreamID == outputStream.StreamID { - if err := assertTypesMatch(outputSpec.Input[k].ColumnTypes, opOutputTypes); err != nil { + if err := assertTypesMatch(outputSpec.Input[k].ColumnTypes, result.ColumnTypes); err != nil { return nil, err } } @@ -1097,22 +1099,18 @@ func (s *vectorizedFlowCreator) setupFlow( } if len(s.vectorizedStatsCollectorsQueue) > 0 { - execerror.VectorizedInternalPanic("not all vectorized stats collectors have been processed") + colexecerror.InternalError("not all vectorized stats collectors have been processed") } return s.leaves, nil } -// assertTypesMatch checks whether expected logical types match with actual -// physical types and returns an error if not. -func assertTypesMatch(expected []types.T, actual []coltypes.T) error { - converted, err := typeconv.FromColumnTypes(expected) - if err != nil { - return err - } - for i := range converted { - if converted[i] != actual[i] { - return errors.Errorf("mismatched physical types at index %d: expected %v\tactual %v ", - i, converted, actual, +// assertTypesMatch checks whether expected types match with actual types and +// returns an error if not. +func assertTypesMatch(expected []types.T, actual []types.T) error { + for i := range expected { + if !expected[i].Identical(&actual[i]) { + return errors.Errorf("mismatched types at index %d: expected %v\tactual %v ", + i, expected, actual, ) } } @@ -1265,7 +1263,7 @@ func SupportsVectorized( mon.Stop(ctx) } }() - if vecErr := execerror.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 c4c1814898ad..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/coltypes" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/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" @@ -143,12 +145,11 @@ func TestVectorizedFlowShutdown(t *testing.T) { var ( err error wg sync.WaitGroup - typs = []coltypes.T{coltypes.Int64} - semtyps = []types.T{*types.Int} - hashRouterInput = colexec.NewRandomDataOp( + typs = []types.T{*types.Int} + hashRouterInput = coldatatestutils.NewRandomDataOp( testAllocator, rng, - colexec.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. @@ -160,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([]colexec.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([]*colexec.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] = colexec.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, &colexec.TestingSemaphore{}, diskAccounts, + queueCfg, &colexecbase.TestingSemaphore{}, diskAccounts, ) for i := 0; i < numInboxes; i++ { inboxMemAccount := testMemMonitor.MakeBoundAccount() defer inboxMemAccount.Close(ctxLocal) inbox, err := colrpc.NewInbox( - colexec.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, colexec.Operator(inbox)) + synchronizerInputs = append(synchronizerInputs, colexecbase.Operator(inbox)) } synchronizer := colexec.NewParallelUnorderedSynchronizer(synchronizerInputs, typs, &wg) flowID := execinfrapb.FlowID{UUID: uuid.MakeV4()} @@ -205,7 +206,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { ctx context.Context, cancelFn context.CancelFunc, outboxMemAcc *mon.BoundAccount, - outboxInput colexec.Operator, + outboxInput colexecbase.Operator, inbox *colrpc.Inbox, id int, outboxMetadataSources []execinfrapb.MetadataSource, @@ -213,7 +214,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { idToClosed.Lock() idToClosed.mapping[id] = false idToClosed.Unlock() - outbox, err := colrpc.NewOutbox(colexec.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)}} @@ -263,22 +264,22 @@ func TestVectorizedFlowShutdown(t *testing.T) { } else { sourceMemAccount := testMemMonitor.MakeBoundAccount() defer sourceMemAccount.Close(ctxRemote) - remoteAllocator := colexec.NewAllocator(ctxRemote, &sourceMemAccount) + remoteAllocator := colmem.NewAllocator(ctxRemote, &sourceMemAccount) batch := remoteAllocator.NewMemBatch(typs) batch.SetLength(coldata.BatchSize()) - runOutboxInbox(ctxRemote, cancelRemote, &outboxMemAccount, colexec.NewRepeatableBatchSource(remoteAllocator, batch), inboxes[i], streamID, outboxMetadataSources) + runOutboxInbox(ctxRemote, cancelRemote, &outboxMemAccount, colexecbase.NewRepeatableBatchSource(remoteAllocator, batch, typs), inboxes[i], streamID, outboxMetadataSources) } streamID++ } - var materializerInput colexec.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( - colexec.NewAllocator(ctxAnotherRemote, &inboxMemAccount), + colmem.NewAllocator(ctxAnotherRemote, &inboxMemAccount), typs, execinfrapb.StreamID(streamID), ) require.NoError(t, err) @@ -301,7 +302,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { flowCtx, 1, /* processorID */ materializerInput, - semtyps, + typs, &execinfrapb.PostProcessSpec{}, nil, /* output */ materializerMetadataSources, diff --git a/pkg/sql/colflow/vectorized_flow_space_test.go b/pkg/sql/colflow/vectorized_flow_space_test.go index ae1bb870906e..b511ff3e1fd4 100644 --- a/pkg/sql/colflow/vectorized_flow_space_test.go +++ b/pkg/sql/colflow/vectorized_flow_space_test.go @@ -16,10 +16,10 @@ import ( "math" "testing" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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 := []colexec.Operator{colexec.NewZeroOp(nil)} + inputs := []colexecbase.Operator{colexec.NewZeroOp(nil)} if len(tc.spec.Input) > 1 { inputs = append(inputs, colexec.NewZeroOp(nil)) } @@ -194,16 +194,15 @@ func TestVectorizeAllocatorSpaceError(t *testing.T) { }, } - batch := testAllocator.NewMemBatchWithSize( - []coltypes.T{coltypes.Int64}, 1, /* size */ - ) + typs := []types.T{*types.Int} + batch := testAllocator.NewMemBatchWithSize(typs, 1 /* size */) for _, tc := range testCases { 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 := []colexec.Operator{colexec.NewRepeatableBatchSource(testAllocator, batch)} + inputs := []colexecbase.Operator{colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs)} if len(tc.spec.Input) > 1 { - inputs = append(inputs, colexec.NewRepeatableBatchSource(testAllocator, batch)) + 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{} @@ -227,7 +226,7 @@ func TestVectorizeAllocatorSpaceError(t *testing.T) { Spec: tc.spec, Inputs: inputs, StreamingMemAccount: &acc, - FDSemaphore: colexec.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 @@ -235,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 = execerror.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 0ee10dfa583d..5cd8e5175e30 100644 --- a/pkg/sql/colflow/vectorized_flow_test.go +++ b/pkg/sql/colflow/vectorized_flow_test.go @@ -17,12 +17,13 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "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,14 +36,14 @@ import ( ) type callbackRemoteComponentCreator struct { - newOutboxFn func(*colexec.Allocator, colexec.Operator, []coltypes.T, []execinfrapb.MetadataSource) (*colrpc.Outbox, error) - newInboxFn func(allocator *colexec.Allocator, typs []coltypes.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 *colexec.Allocator, - input colexec.Operator, - typs []coltypes.T, + allocator *colmem.Allocator, + input colexecbase.Operator, + typs []types.T, metadataSources []execinfrapb.MetadataSource, toClose []colexec.IdempotentCloser, ) (*colrpc.Outbox, error) { @@ -50,7 +51,7 @@ func (c callbackRemoteComponentCreator) newOutbox( } func (c callbackRemoteComponentCreator) newInbox( - allocator *colexec.Allocator, typs []coltypes.T, streamID execinfrapb.StreamID, + allocator *colmem.Allocator, typs []types.T, streamID execinfrapb.StreamID, ) (*colrpc.Inbox, error) { return c.newInboxFn(allocator, typs, streamID) } @@ -182,13 +183,13 @@ func TestDrainOnlyInputDAG(t *testing.T) { }, } - inboxToNumInputTypes := make(map[*colrpc.Inbox][]coltypes.T) + inboxToNumInputTypes := make(map[*colrpc.Inbox][]types.T) outboxCreated := false componentCreator := callbackRemoteComponentCreator{ newOutboxFn: func( - allocator *colexec.Allocator, - op colexec.Operator, - typs []coltypes.T, + allocator *colmem.Allocator, + op colexecbase.Operator, + typs []types.T, sources []execinfrapb.MetadataSource, ) (*colrpc.Outbox, error) { require.False(t, outboxCreated) @@ -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 *colexec.Allocator, typs []coltypes.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: &colexec.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 1112cd0bb173..88c8282c9f2d 100644 --- a/pkg/sql/colflow/vectorized_panic_propagation_test.go +++ b/pkg/sql/colflow/vectorized_panic_propagation_test.go @@ -17,7 +17,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" @@ -70,8 +71,8 @@ func TestVectorizedInternalPanic(t *testing.T) { mat.Start(ctx) var meta *execinfrapb.ProducerMetadata - require.NotPanics(t, func() { _, meta = mat.Next() }, "VectorizedInternalPanic was not caught") - require.NotNil(t, meta.Err, "VectorizedInternalPanic was not propagated as metadata") + require.NotPanics(t, func() { _, meta = mat.Next() }, "InternalError was not caught") + require.NotNil(t, meta.Err, "InternalError was not propagated as metadata") } // TestNonVectorizedPanicPropagation verifies that materializers do not handle @@ -121,7 +122,7 @@ func TestNonVectorizedPanicPropagation(t *testing.T) { } // testVectorizedInternalPanicEmitter is an colexec.Operator that panics with -// execerror.VectorizedInternalPanic 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 { @@ -129,9 +130,9 @@ type testVectorizedInternalPanicEmitter struct { emitBatch bool } -var _ colexec.Operator = &testVectorizedInternalPanicEmitter{} +var _ colexecbase.Operator = &testVectorizedInternalPanicEmitter{} -func newTestVectorizedInternalPanicEmitter(input colexec.Operator) colexec.Operator { +func newTestVectorizedInternalPanicEmitter(input colexecbase.Operator) colexecbase.Operator { return &testVectorizedInternalPanicEmitter{ OneInputNode: colexec.NewOneInputNode(input), } @@ -146,7 +147,7 @@ func (e *testVectorizedInternalPanicEmitter) Init() { func (e *testVectorizedInternalPanicEmitter) Next(ctx context.Context) coldata.Batch { if !e.emitBatch { e.emitBatch = true - execerror.VectorizedInternalPanic("") + colexecerror.InternalError("") } e.emitBatch = false @@ -162,9 +163,9 @@ type testNonVectorizedPanicEmitter struct { emitBatch bool } -var _ colexec.Operator = &testVectorizedInternalPanicEmitter{} +var _ colexecbase.Operator = &testVectorizedInternalPanicEmitter{} -func newTestNonVectorizedPanicEmitter(input colexec.Operator) colexec.Operator { +func newTestNonVectorizedPanicEmitter(input colexecbase.Operator) colexecbase.Operator { return &testNonVectorizedPanicEmitter{ OneInputNode: colexec.NewOneInputNode(input), } diff --git a/pkg/sql/colexec/allocator.go b/pkg/sql/colmem/allocator.go similarity index 77% rename from pkg/sql/colexec/allocator.go rename to pkg/sql/colmem/allocator.go index e3243bcba480..c19c302410ef 100644 --- a/pkg/sql/colexec/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 colexec +package colmem import ( "context" @@ -18,7 +18,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coltypes" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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" "github.com/cockroachdb/errors" @@ -42,7 +44,7 @@ func getVecMemoryFootprint(vec coldata.Vec) int64 { if vec.Type() == coltypes.Bytes { return int64(vec.Bytes().Size()) } - return int64(estimateBatchSizeBytes([]coltypes.T{vec.Type()}, vec.Capacity())) + return int64(EstimateBatchSizeBytes([]coltypes.T{vec.Type()}, vec.Capacity())) } func getVecsMemoryFootprint(vecs []coldata.Vec) int64 { @@ -53,10 +55,10 @@ func getVecsMemoryFootprint(vecs []coldata.Vec) int64 { return size } -// getProportionalBatchMemSize returns the memory size of the batch that is +// GetProportionalBatchMemSize returns the memory size of the batch that is // proportional to given 'length'. This method returns the estimated memory // footprint *only* of the first 'length' tuples in 'b'. -func getProportionalBatchMemSize(b coldata.Batch, length int64) int64 { +func GetProportionalBatchMemSize(b coldata.Batch, length int64) int64 { usesSel := b.Selection() != nil b.SetSelection(true) selCapacity := cap(b.Selection()) @@ -81,27 +83,27 @@ func NewAllocator(ctx context.Context, acc *mon.BoundAccount) *Allocator { } // NewMemBatch allocates a new in-memory coldata.Batch. -func (a *Allocator) NewMemBatch(types []coltypes.T) coldata.Batch { - return a.NewMemBatchWithSize(types, coldata.BatchSize()) +func (a *Allocator) NewMemBatch(typs []types.T) coldata.Batch { + return a.NewMemBatchWithSize(typs, coldata.BatchSize()) } // NewMemBatchWithSize allocates a new in-memory coldata.Batch with the given // column size. -func (a *Allocator) NewMemBatchWithSize(types []coltypes.T, size int) coldata.Batch { - estimatedMemoryUsage := selVectorSize(size) + int64(estimateBatchSizeBytes(types, size)) +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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } - return coldata.NewMemBatchWithSize(types, size) + return coldata.NewMemBatchWithSize(typs, size) } // NewMemBatchNoCols creates a "skeleton" of new in-memory coldata.Batch. It // allocates memory for the selection vector but does *not* allocate any memory // for the column vectors - those will have to be added separately. -func (a *Allocator) NewMemBatchNoCols(types []coltypes.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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return coldata.NewMemBatchNoCols(types, size) } @@ -124,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } b.SetSelection(usesSel) } @@ -152,15 +154,15 @@ func (a *Allocator) ReleaseBatch(b coldata.Batch) { } // NewMemColumn returns a new coldata.Vec, initialized with a length. -func (a *Allocator) NewMemColumn(t coltypes.T, n int) coldata.Vec { - estimatedMemoryUsage := int64(estimateBatchSizeBytes([]coltypes.T{t}, n)) +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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } return coldata.NewMemColumn(t, n) } -// maybeAppendColumn might append a newly allocated coldata.Vec of the given +// MaybeAppendColumn might append a newly allocated coldata.Vec of the given // type to b at position colIdx. Behavior of the function depends on how colIdx // compares to the width of b: // 1. if colIdx < b.Width(), then we expect that correctly-typed vector is @@ -171,34 +173,35 @@ func (a *Allocator) NewMemColumn(t coltypes.T, n int) coldata.Vec { // indicates an error in setting up vector type enforcers during the planning // stage. // NOTE: b must be non-zero length batch. -func (a *Allocator) maybeAppendColumn(b coldata.Batch, t coltypes.T, colIdx int) { +func (a *Allocator) MaybeAppendColumn(b coldata.Batch, t *types.T, colIdx int) { if b.Length() == 0 { - execerror.VectorizedInternalPanic("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) if colIdx < width { - switch presentType := b.ColVec(colIdx).Type(); presentType { - case t: + switch presentPhysType := b.ColVec(colIdx).Type(); presentPhysType { + case desiredPhysType: // We already have the vector of the desired type in place. return default: // We have a vector with an unexpected type, so we panic. - execerror.VectorizedInternalPanic(errors.Errorf( + colexecerror.InternalError(errors.Errorf( "trying to add a column of %s type at index %d but %s vector already present", - t, colIdx, presentType, + t, colIdx, presentPhysType, )) } } else if colIdx > width { // We have a batch of unexpected width which indicates an error in the // planning stage. - execerror.VectorizedInternalPanic(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{t}, coldata.BatchSize())) + estimatedMemoryUsage := int64(EstimateBatchSizeBytes([]coltypes.T{desiredPhysType}, coldata.BatchSize())) if err := a.acc.Grow(a.ctx, estimatedMemoryUsage); err != nil { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } b.AppendCol(a.NewMemColumn(t, coldata.BatchSize())) } @@ -218,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 { - execerror.VectorizedInternalPanic(err) + colexecerror.InternalError(err) } } else { a.ReleaseMemory(-delta) @@ -250,16 +253,16 @@ const ( sizeOfDuration = int(unsafe.Sizeof(duration.Duration{})) ) -// sizeOfBatchSizeSelVector is the size (in bytes) of a selection vector of +// SizeOfBatchSizeSelVector is the size (in bytes) of a selection vector of // coldata.BatchSize() length. -var sizeOfBatchSizeSelVector = coldata.BatchSize() * sizeOfInt +var SizeOfBatchSizeSelVector = coldata.BatchSize() * sizeOfInt -// estimateBatchSizeBytes returns an estimated amount of bytes needed to +// EstimateBatchSizeBytes returns an estimated amount of bytes needed to // store a batch in memory that has column types vecTypes. // WARNING: This only is correct for fixed width types, and returns an // estimate for non fixed width coltypes. In future it might be possible to // remove the need for estimation by specifying batch sizes in terms of bytes. -func estimateBatchSizeBytes(vecTypes []coltypes.T, batchLength int) int { +func EstimateBatchSizeBytes(vecTypes []coltypes.T, batchLength int) int { // acc represents the number of bytes to represent a row in the batch. acc := 0 for _, t := range vecTypes { @@ -300,8 +303,23 @@ func estimateBatchSizeBytes(vecTypes []coltypes.T, batchLength int) int { case coltypes.Unhandled: // Placeholder coldata.Vecs of unknown types are allowed. default: - execerror.VectorizedInternalPanic(fmt.Sprintf("unhandled type %s", t)) + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", t)) } } return acc * batchLength } + +// EstimateBatchSizeBytesFromSQLTypes is the same as EstimateBatchSizeBytes +// except for taking in SQL types. All types will be first converted to their +// physical equivalents, and any unsupported types will be considered as taking +// up no space. +func EstimateBatchSizeBytesFromSQLTypes(typs []types.T, batchLength int) int { + // Note that we're ok if some types are converted to coltypes.Unhandled - + // we use physical types here only for the size estimation, and we're not + // responsible for making sure that all of the types are supported. + vecTypes := make([]coltypes.T, len(typs)) + for i := range vecTypes { + vecTypes[i] = typeconv.FromColumnType(&typs[i]) + } + return EstimateBatchSizeBytes(vecTypes, batchLength) +} diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 3a1c41cda4a0..4a77a31635d7 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -916,14 +916,14 @@ func (ex *connExecutor) beginTransactionTimestampsAndReadMode( historicalTimestamp *hlc.Timestamp, err error, ) { - now := ex.server.cfg.Clock.Now() + now := ex.server.cfg.Clock.PhysicalTime() if s.Modes.AsOf.Expr == nil { rwMode = ex.readWriteModeWithSessionDefault(s.Modes.ReadWriteMode) - return rwMode, now.GoTime(), nil, nil + return rwMode, now, nil, nil } ex.statsCollector.reset(&ex.server.sqlStats, ex.appStats, &ex.phaseTimes) p := &ex.planner - ex.resetPlanner(ctx, p, nil /* txn */, now.GoTime()) + ex.resetPlanner(ctx, p, nil /* txn */, now) ts, err := p.EvalAsOfTimestamp(s.Modes.AsOf) if err != nil { return 0, time.Time{}, nil, err diff --git a/pkg/sql/distsql/columnar_operators_test.go b/pkg/sql/distsql/columnar_operators_test.go index fba606c8997c..e4d0bdde0c49 100644 --- a/pkg/sql/distsql/columnar_operators_test.go +++ b/pkg/sql/distsql/columnar_operators_test.go @@ -20,9 +20,9 @@ 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/colexec" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" diff --git a/pkg/sql/distsql/columnar_utils_test.go b/pkg/sql/distsql/columnar_utils_test.go index ffd11d713aa7..5bb0c12c47bc 100644 --- a/pkg/sql/distsql/columnar_utils_test.go +++ b/pkg/sql/distsql/columnar_utils_test.go @@ -20,6 +20,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "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" @@ -97,8 +99,8 @@ func verifyColOperator(args verifyColOperatorArgs) error { acc := evalCtx.Mon.MakeBoundAccount() defer acc.Close(ctx) - testAllocator := colexec.NewAllocator(ctx, &acc) - columnarizers := make([]colexec.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 { @@ -113,7 +115,7 @@ func verifyColOperator(args verifyColOperatorArgs) error { StreamingMemAccount: &acc, ProcessorConstructor: rowexec.NewProcessor, DiskQueueCfg: colcontainer.DiskQueueCfg{FS: tempFS}, - FDSemaphore: colexec.NewTestingSemaphore(256), + FDSemaphore: colexecbase.NewTestingSemaphore(256), } var spilled bool if args.forceDiskSpill { diff --git a/pkg/sql/distsql/server.go b/pkg/sql/distsql/server.go index 7b324d59e8c9..602aad5b2380 100644 --- a/pkg/sql/distsql/server.go +++ b/pkg/sql/distsql/server.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" + "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/rowflow" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -248,14 +249,14 @@ func (ds *ServerImpl) setupFlow( return ctx, nil, err } - var be sessiondata.BytesEncodeFormat + var be lex.BytesEncodeFormat switch req.EvalContext.BytesEncodeFormat { case execinfrapb.BytesEncodeFormat_HEX: - be = sessiondata.BytesEncodeHex + be = lex.BytesEncodeHex case execinfrapb.BytesEncodeFormat_ESCAPE: - be = sessiondata.BytesEncodeEscape + be = lex.BytesEncodeEscape case execinfrapb.BytesEncodeFormat_BASE64: - be = sessiondata.BytesEncodeBase64 + be = lex.BytesEncodeBase64 default: return nil, nil, errors.AssertionFailedf("unknown byte encode format: %s", errors.Safe(req.EvalContext.BytesEncodeFormat)) diff --git a/pkg/sql/distsql/vectorized_panic_propagation_test.go b/pkg/sql/distsql/vectorized_panic_propagation_test.go index b155642c5abc..fc0e73fb4dcb 100644 --- a/pkg/sql/distsql/vectorized_panic_propagation_test.go +++ b/pkg/sql/distsql/vectorized_panic_propagation_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colflow" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -53,7 +54,7 @@ func TestNonVectorizedPanicDoesntHangServer(t *testing.T) { mat, err := colexec.NewMaterializer( &flowCtx, 0, /* processorID */ - &colexec.CallbackOperator{ + &colexecbase.CallbackOperator{ NextCb: func(ctx context.Context) coldata.Batch { panic("") }, diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 75c3ee88120c..80a7735ae865 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -42,6 +42,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec" "github.com/cockroachdb/cockroach/pkg/sql/distsql" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" @@ -1853,7 +1854,7 @@ func (m *sessionDataMutator) SetApplicationName(appName string) { m.paramStatusUpdater.AppendParamStatusUpdate("application_name", appName) } -func (m *sessionDataMutator) SetBytesEncodeFormat(val sessiondata.BytesEncodeFormat) { +func (m *sessionDataMutator) SetBytesEncodeFormat(val lex.BytesEncodeFormat) { m.data.DataConversion.BytesEncodeFormat = val } diff --git a/pkg/sql/execinfrapb/api.go b/pkg/sql/execinfrapb/api.go index c4fea9e89e85..0edb2497dd35 100644 --- a/pkg/sql/execinfrapb/api.go +++ b/pkg/sql/execinfrapb/api.go @@ -11,8 +11,8 @@ package execinfrapb import ( + "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/util/uuid" ) @@ -35,11 +35,11 @@ type DistSQLVersion uint32 func MakeEvalContext(evalCtx *tree.EvalContext) EvalContext { var be BytesEncodeFormat switch evalCtx.SessionData.DataConversion.BytesEncodeFormat { - case sessiondata.BytesEncodeHex: + case lex.BytesEncodeHex: be = BytesEncodeFormat_HEX - case sessiondata.BytesEncodeEscape: + case lex.BytesEncodeEscape: be = BytesEncodeFormat_ESCAPE - case sessiondata.BytesEncodeBase64: + case lex.BytesEncodeBase64: be = BytesEncodeFormat_BASE64 default: panic("unknown format") diff --git a/pkg/sql/explain_vec.go b/pkg/sql/explain_vec.go index 240ac179213d..2ad221887d6c 100644 --- a/pkg/sql/explain_vec.go +++ b/pkg/sql/explain_vec.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/colexec" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" + "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 := execerror.CatchVectorizedRuntimeError(func() { + if err := colexecerror.CatchVectorizedRuntimeError(func() { for _, op := range opChains { formatOpChain(op, node, verbose) } diff --git a/pkg/sql/lex/encode.go b/pkg/sql/lex/encode.go index 90c8aaf6d94c..5ae902a4c7db 100644 --- a/pkg/sql/lex/encode.go +++ b/pkg/sql/lex/encode.go @@ -24,11 +24,11 @@ import ( "encoding/base64" "encoding/hex" "fmt" + "strings" "unicode/utf8" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/util/stringencoding" "github.com/cockroachdb/errors" ) @@ -235,11 +235,9 @@ func EncodeSQLBytes(buf *bytes.Buffer, in string) { // If the skipHexPrefix argument is set, the hexadecimal encoding does not // prefix the output with "\x". This is suitable e.g. for the encode() // built-in. -func EncodeByteArrayToRawBytes( - data string, be sessiondata.BytesEncodeFormat, skipHexPrefix bool, -) string { +func EncodeByteArrayToRawBytes(data string, be BytesEncodeFormat, skipHexPrefix bool) string { switch be { - case sessiondata.BytesEncodeHex: + case BytesEncodeHex: head := 2 if skipHexPrefix { head = 0 @@ -252,7 +250,7 @@ func EncodeByteArrayToRawBytes( hex.Encode(res[head:], []byte(data)) return string(res) - case sessiondata.BytesEncodeEscape: + case BytesEncodeEscape: // PostgreSQL does not allow all the escapes formats recognized by // CockroachDB's scanner. It only recognizes octal and \\ for the // backslash itself. @@ -274,7 +272,7 @@ func EncodeByteArrayToRawBytes( } return string(res) - case sessiondata.BytesEncodeBase64: + case BytesEncodeBase64: return base64.StdEncoding.EncodeToString([]byte(data)) default: @@ -287,12 +285,12 @@ func EncodeByteArrayToRawBytes( // When using the Hex format, the caller is responsible for skipping the // "\x" prefix, if any. See DecodeRawBytesToByteArrayAuto() below for // an alternative. -func DecodeRawBytesToByteArray(data string, be sessiondata.BytesEncodeFormat) ([]byte, error) { +func DecodeRawBytesToByteArray(data string, be BytesEncodeFormat) ([]byte, error) { switch be { - case sessiondata.BytesEncodeHex: + case BytesEncodeHex: return hex.DecodeString(data) - case sessiondata.BytesEncodeEscape: + case BytesEncodeEscape: // PostgreSQL does not allow all the escapes formats recognized by // CockroachDB's scanner. It only recognizes octal and \\ for the // backslash itself. @@ -331,7 +329,7 @@ func DecodeRawBytesToByteArray(data string, be sessiondata.BytesEncodeFormat) ([ } return res, nil - case sessiondata.BytesEncodeBase64: + case BytesEncodeBase64: return base64.StdEncoding.DecodeString(data) default: @@ -344,7 +342,48 @@ func DecodeRawBytesToByteArray(data string, be sessiondata.BytesEncodeFormat) ([ // and escape. func DecodeRawBytesToByteArrayAuto(data []byte) ([]byte, error) { if len(data) >= 2 && data[0] == '\\' && (data[1] == 'x' || data[1] == 'X') { - return DecodeRawBytesToByteArray(string(data[2:]), sessiondata.BytesEncodeHex) + return DecodeRawBytesToByteArray(string(data[2:]), BytesEncodeHex) + } + return DecodeRawBytesToByteArray(string(data), BytesEncodeEscape) +} + +// BytesEncodeFormat controls which format to use for BYTES->STRING +// conversions. +type BytesEncodeFormat int + +const ( + // BytesEncodeHex uses the hex format: e'abc\n'::BYTES::STRING -> '\x61626312'. + // This is the default, for compatibility with PostgreSQL. + BytesEncodeHex BytesEncodeFormat = iota + // BytesEncodeEscape uses the escaped format: e'abc\n'::BYTES::STRING -> 'abc\012'. + BytesEncodeEscape + // BytesEncodeBase64 uses base64 encoding. + BytesEncodeBase64 +) + +func (f BytesEncodeFormat) String() string { + switch f { + case BytesEncodeHex: + return "hex" + case BytesEncodeEscape: + return "escape" + case BytesEncodeBase64: + return "base64" + default: + return fmt.Sprintf("invalid (%d)", f) + } +} + +// BytesEncodeFormatFromString converts a string into a BytesEncodeFormat. +func BytesEncodeFormatFromString(val string) (_ BytesEncodeFormat, ok bool) { + switch strings.ToUpper(val) { + case "HEX": + return BytesEncodeHex, true + case "ESCAPE": + return BytesEncodeEscape, true + case "BASE64": + return BytesEncodeBase64, true + default: + return -1, false } - return DecodeRawBytesToByteArray(string(data), sessiondata.BytesEncodeEscape) } diff --git a/pkg/sql/lex/encode_test.go b/pkg/sql/lex/encode_test.go index d4cefe550d62..e872c71e968c 100644 --- a/pkg/sql/lex/encode_test.go +++ b/pkg/sql/lex/encode_test.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/parser" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" ) func TestEncodeSQLBytes(t *testing.T) { @@ -127,14 +126,14 @@ func TestEncodeRestrictedSQLIdent(t *testing.T) { func TestByteArrayDecoding(t *testing.T) { const ( - fmtHex = sessiondata.BytesEncodeHex - fmtEsc = sessiondata.BytesEncodeEscape - fmtB64 = sessiondata.BytesEncodeBase64 + fmtHex = lex.BytesEncodeHex + fmtEsc = lex.BytesEncodeEscape + fmtB64 = lex.BytesEncodeBase64 ) testData := []struct { in string auto bool - inFmt sessiondata.BytesEncodeFormat + inFmt lex.BytesEncodeFormat out string err string }{ @@ -210,8 +209,8 @@ func TestByteArrayEncoding(t *testing.T) { for _, s := range testData { t.Run(s.in, func(t *testing.T) { - for _, format := range []sessiondata.BytesEncodeFormat{ - sessiondata.BytesEncodeHex, sessiondata.BytesEncodeEscape, sessiondata.BytesEncodeBase64} { + for _, format := range []lex.BytesEncodeFormat{ + lex.BytesEncodeHex, lex.BytesEncodeEscape, lex.BytesEncodeBase64} { t.Run(format.String(), func(t *testing.T) { enc := lex.EncodeByteArrayToRawBytes(s.in, format, false) @@ -220,7 +219,7 @@ func TestByteArrayEncoding(t *testing.T) { t.Fatalf("encoded %q, expected %q", enc, expEnc) } - if format == sessiondata.BytesEncodeHex { + if format == lex.BytesEncodeHex { // Check that the \x also can be skipped. enc2 := lex.EncodeByteArrayToRawBytes(s.in, format, true) if enc[2:] != enc2 { diff --git a/pkg/sql/opt/memo/memo_test.go b/pkg/sql/opt/memo/memo_test.go index 90246bb366cb..5e409a826c96 100644 --- a/pkg/sql/opt/memo/memo_test.go +++ b/pkg/sql/opt/memo/memo_test.go @@ -16,13 +16,13 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" opttestutils "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/opttester" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/testcat" "github.com/cockroachdb/cockroach/pkg/sql/opt/xform" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/datadriven" ) @@ -144,9 +144,9 @@ func TestMemoIsStale(t *testing.T) { notStale() // Stale bytes encode format. - evalCtx.SessionData.DataConversion.BytesEncodeFormat = sessiondata.BytesEncodeBase64 + evalCtx.SessionData.DataConversion.BytesEncodeFormat = lex.BytesEncodeBase64 stale() - evalCtx.SessionData.DataConversion.BytesEncodeFormat = sessiondata.BytesEncodeHex + evalCtx.SessionData.DataConversion.BytesEncodeFormat = lex.BytesEncodeHex notStale() // Stale extra float digits. diff --git a/pkg/sql/pgwire/conn_test.go b/pkg/sql/pgwire/conn_test.go index 8205d8c08ec7..fc0311611932 100644 --- a/pkg/sql/pgwire/conn_test.go +++ b/pkg/sql/pgwire/conn_test.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" @@ -343,7 +344,7 @@ func waitForClientConn(ln net.Listener) (*conn, error) { func makeTestingConvCfg() sessiondata.DataConversionConfig { return sessiondata.DataConversionConfig{ Location: time.UTC, - BytesEncodeFormat: sessiondata.BytesEncodeHex, + BytesEncodeFormat: lex.BytesEncodeHex, } } diff --git a/pkg/sql/pgwire/types_test.go b/pkg/sql/pgwire/types_test.go index 628990b3528f..77564200d07d 100644 --- a/pkg/sql/pgwire/types_test.go +++ b/pkg/sql/pgwire/types_test.go @@ -20,9 +20,9 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -192,8 +192,8 @@ func TestByteArrayRoundTrip(t *testing.T) { randValues = append(randValues, d) } - for _, be := range []sessiondata.BytesEncodeFormat{ - sessiondata.BytesEncodeHex, sessiondata.BytesEncodeEscape} { + for _, be := range []lex.BytesEncodeFormat{ + lex.BytesEncodeHex, lex.BytesEncodeEscape} { t.Run(be.String(), func(t *testing.T) { for i, d := range randValues { t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { diff --git a/pkg/sql/rowexec/stats.go b/pkg/sql/rowexec/stats.go index a2c9d4f03c33..5496d4308f59 100644 --- a/pkg/sql/rowexec/stats.go +++ b/pkg/sql/rowexec/stats.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/row" @@ -54,9 +53,7 @@ func (isc *inputStatCollector) Child(nth int, verbose bool) execinfra.OpNode { if nth == 0 { return isc.RowSource.(execinfra.OpNode) } - execerror.VectorizedInternalPanic(fmt.Sprintf("invalid index %d", nth)) - // This code is unreachable, but the compiler cannot infer that. - return nil + panic(fmt.Sprintf("invalid index %d", nth)) } // Next implements the RowSource interface. It calls Next on the embedded diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index ef50f1305df9..6e2960f441df 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -783,7 +783,7 @@ var builtins = map[string]builtinDefinition{ ReturnType: tree.FixedReturnType(types.String), Fn: func(evalCtx *tree.EvalContext, args tree.Datums) (_ tree.Datum, err error) { data, format := *args[0].(*tree.DBytes), string(tree.MustBeDString(args[1])) - be, ok := sessiondata.BytesEncodeFormatFromString(format) + be, ok := lex.BytesEncodeFormatFromString(format) if !ok { return nil, pgerror.New(pgcode.InvalidParameterValue, "only 'hex', 'escape', and 'base64' formats are supported for encode()") @@ -801,7 +801,7 @@ var builtins = map[string]builtinDefinition{ ReturnType: tree.FixedReturnType(types.Bytes), Fn: func(evalCtx *tree.EvalContext, args tree.Datums) (_ tree.Datum, err error) { data, format := string(tree.MustBeDString(args[0])), string(tree.MustBeDString(args[1])) - be, ok := sessiondata.BytesEncodeFormatFromString(format) + be, ok := lex.BytesEncodeFormatFromString(format) if !ok { return nil, pgerror.New(pgcode.InvalidParameterValue, "only 'hex', 'escape', and 'base64' formats are supported for decode()") diff --git a/pkg/sql/sem/tree/eval_test.go b/pkg/sql/sem/tree/eval_test.go index f751a2c62e26..04bd624b2bb1 100644 --- a/pkg/sql/sem/tree/eval_test.go +++ b/pkg/sql/sem/tree/eval_test.go @@ -23,7 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colexec" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/typeconv" + "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" @@ -195,8 +195,6 @@ func TestEval(t *testing.T) { // inputTyps has no relation to the actual expression result type. Used // for generating a batch. inputTyps := []types.T{*types.Int} - inputColTyps, err := typeconv.FromColumnTypes(inputTyps) - require.NoError(t, err) batchesReturned := 0 args := colexec.NewColOperatorArgs{ @@ -212,14 +210,14 @@ func TestEval(t *testing.T) { RenderExprs: []execinfrapb.Expression{{Expr: d.Input}}, }, }, - Inputs: []colexec.Operator{ - &colexec.CallbackOperator{ + Inputs: []colexecbase.Operator{ + &colexecbase.CallbackOperator{ NextCb: func(_ context.Context) coldata.Batch { if batchesReturned > 0 { return coldata.ZeroBatch } // It doesn't matter what types we create the input batch with. - batch := coldata.NewMemBatch(inputColTyps) + batch := coldata.NewMemBatch(inputTyps) batch.SetLength(1) batchesReturned++ return batch diff --git a/pkg/sql/sessiondata/session_data.go b/pkg/sql/sessiondata/session_data.go index 9aba76aed090..d5a6127b9472 100644 --- a/pkg/sql/sessiondata/session_data.go +++ b/pkg/sql/sessiondata/session_data.go @@ -15,6 +15,8 @@ import ( "net" "strings" "time" + + "github.com/cockroachdb/cockroach/pkg/sql/lex" ) // SessionData contains session parameters. They are all user-configurable. @@ -114,7 +116,7 @@ type DataConversionConfig struct { // BytesEncodeFormat indicates how to encode byte arrays when converting // to string. - BytesEncodeFormat BytesEncodeFormat + BytesEncodeFormat lex.BytesEncodeFormat // ExtraFloatDigits indicates the number of digits beyond the // standard number to use for float conversions. @@ -168,47 +170,6 @@ func (c *DataConversionConfig) Equals(other *DataConversionConfig) bool { return true } -// BytesEncodeFormat controls which format to use for BYTES->STRING -// conversions. -type BytesEncodeFormat int - -const ( - // BytesEncodeHex uses the hex format: e'abc\n'::BYTES::STRING -> '\x61626312'. - // This is the default, for compatibility with PostgreSQL. - BytesEncodeHex BytesEncodeFormat = iota - // BytesEncodeEscape uses the escaped format: e'abc\n'::BYTES::STRING -> 'abc\012'. - BytesEncodeEscape - // BytesEncodeBase64 uses base64 encoding. - BytesEncodeBase64 -) - -func (f BytesEncodeFormat) String() string { - switch f { - case BytesEncodeHex: - return "hex" - case BytesEncodeEscape: - return "escape" - case BytesEncodeBase64: - return "base64" - default: - return fmt.Sprintf("invalid (%d)", f) - } -} - -// BytesEncodeFormatFromString converts a string into a BytesEncodeFormat. -func BytesEncodeFormatFromString(val string) (_ BytesEncodeFormat, ok bool) { - switch strings.ToUpper(val) { - case "HEX": - return BytesEncodeHex, true - case "ESCAPE": - return BytesEncodeEscape, true - case "BASE64": - return BytesEncodeBase64, true - default: - return -1, false - } -} - // DistSQLExecMode controls if and when the Executor distributes queries. // Since 2.1, we run everything through the DistSQL infrastructure, // and these settings control whether to use a distributed plan, or use a plan diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index 0870a1a52c17..ce0f1386afdd 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/delegate" + "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" @@ -136,7 +137,7 @@ var varGen = map[string]sessionVar{ Set: func( _ context.Context, m *sessionDataMutator, s string, ) error { - mode, ok := sessiondata.BytesEncodeFormatFromString(s) + mode, ok := lex.BytesEncodeFormatFromString(s) if !ok { return newVarValueError(`bytea_output`, s, "hex", "escape", "base64") } @@ -146,7 +147,7 @@ var varGen = map[string]sessionVar{ Get: func(evalCtx *extendedEvalContext) string { return evalCtx.SessionData.DataConversion.BytesEncodeFormat.String() }, - GlobalDefault: func(sv *settings.Values) string { return sessiondata.BytesEncodeHex.String() }, + GlobalDefault: func(sv *settings.Values) string { return lex.BytesEncodeHex.String() }, }, // Supported for PG compatibility only. diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index cfcc27b765be..bc83e3c13d3d 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/colexec/execerror/error.go", + ":!sql/colexecbase/colexecerror/error.go", ":!util/protoutil/jsonpb_marshal.go", ":!util/protoutil/marshal.go", ":!util/protoutil/marshaler.go", @@ -1419,11 +1419,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/colexec/execerror/error.go file. - "sql/colexec", - "sql/colflow", - "sql/colcontainer", - ":!sql/colexec/execerror/error.go", + // sql/colexecbase/colexecerror/error.go file. + "sql/col*", + ":!sql/colexecbase/colexecerror/error.go", ":!sql/colexec/execpb/stats.pb.go", ":!sql/colflow/vectorized_panic_propagation_test.go", ) @@ -1436,7 +1434,7 @@ func TestLint(t *testing.T) { } if err := stream.ForEach(filter, func(s string) { - t.Errorf("\n%s <- forbidden; use either execerror.VectorizedInternalPanic() or execerror.NonVectorizedPanic() instead", s) + t.Errorf("\n%s <- forbidden; use either colexecerror.InternalError() or colexecerror.ExpectedError() instead", s) }); err != nil { t.Error(err) } @@ -1463,9 +1461,10 @@ 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/allocator.go", ":!sql/colexec/simple_project.go", ) if err != nil { @@ -1477,7 +1476,7 @@ func TestLint(t *testing.T) { } if err := stream.ForEach(filter, func(s string) { - t.Errorf("\n%s <- forbidden; use colexec.Allocator object instead", s) + t.Errorf("\n%s <- forbidden; use colmem.Allocator object instead", s) }); err != nil { t.Error(err) } @@ -1496,13 +1495,13 @@ func TestLint(t *testing.T) { "git", "grep", "-nE", - // We prohibit usage of Allocator.maybeAppendColumn outside of + // We prohibit usage of Allocator.MaybeAppendColumn outside of // vectorTypeEnforcer and batchSchemaPrefixEnforcer. - fmt.Sprintf(`(maybeAppendColumn)\(`), + fmt.Sprintf(`(MaybeAppendColumn)\(`), "--", - "sql/colexec", - ":!sql/colexec/allocator.go", + "sql/col*", ":!sql/colexec/operator.go", + ":!sql/colmem/allocator.go", ) if err != nil { t.Fatal(err) diff --git a/pkg/ui/src/components/core/colors.styl b/pkg/ui/src/components/core/colors.styl index b1de7adc922c..f9876f9348e9 100644 --- a/pkg/ui/src/components/core/colors.styl +++ b/pkg/ui/src/components/core/colors.styl @@ -31,6 +31,7 @@ $colors--neutral-7 = #394455 $colors--neutral-8 = #242a35 $colors--neutral-9 = #060c12 $colors--neutral-10 = #0c1628 +$colors--neutral-11 = #C4C4C4 // Primary and brand colors $colors--primary-blue-1 = #e0eefb diff --git a/pkg/ui/src/components/icon/spin.tsx b/pkg/ui/src/components/icon/spin.tsx new file mode 100644 index 000000000000..337f9e6b587f --- /dev/null +++ b/pkg/ui/src/components/icon/spin.tsx @@ -0,0 +1,24 @@ +// 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. + +import * as React from "react"; + +const SpinIcon = (props: React.SVGProps) => ( + + + +); + +export default SpinIcon; diff --git a/pkg/ui/src/views/databases/containers/databaseSummary/index.tsx b/pkg/ui/src/views/databases/containers/databaseSummary/index.tsx index 7785ada4a64d..6b58cd8f8518 100644 --- a/pkg/ui/src/views/databases/containers/databaseSummary/index.tsx +++ b/pkg/ui/src/views/databases/containers/databaseSummary/index.tsx @@ -14,7 +14,7 @@ import React from "react"; import * as protos from "src/js/protos"; import { AdminUIState } from "src/redux/state"; -import { refreshDatabaseDetails, refreshTableDetails, refreshTableStats, generateTableID} from "src/redux/apiReducers"; +import { refreshDatabaseDetails, refreshTableDetails, refreshTableStats, generateTableID, KeyedCachedDataReducerState} from "src/redux/apiReducers"; import { SortSetting } from "src/views/shared/components/sortabletable"; @@ -31,7 +31,7 @@ export interface DatabaseSummaryExplicitData { interface DatabaseSummaryConnectedData { sortSetting: SortSetting; tableInfos: TableInfo[]; - dbResponse: protos.cockroach.server.serverpb.DatabaseDetailsResponse; + dbResponse: KeyedCachedDataReducerState; grants: protos.cockroach.server.serverpb.DatabaseDetailsResponse.Grant[]; } diff --git a/pkg/ui/src/views/databases/containers/databaseTables/index.tsx b/pkg/ui/src/views/databases/containers/databaseTables/index.tsx index 036229dab3c4..76fb6d8b7176 100644 --- a/pkg/ui/src/views/databases/containers/databaseTables/index.tsx +++ b/pkg/ui/src/views/databases/containers/databaseTables/index.tsx @@ -10,7 +10,7 @@ import tableIcon from "!!raw-loader!assets/tableIcon.svg"; import _ from "lodash"; -import { SummaryCard } from "oss/src/views/shared/components/summaryCard"; +import { SummaryCard } from "src/views/shared/components/summaryCard"; import React from "react"; import { connect } from "react-redux"; import { Link } from "react-router-dom"; @@ -18,13 +18,13 @@ import { refreshDatabaseDetails, refreshTableDetails, refreshTableStats } from " import { LocalSetting } from "src/redux/localsettings"; import { AdminUIState } from "src/redux/state"; import { Bytes } from "src/util/format"; -import { trustIcon } from "src/util/trust"; import { databaseDetails, DatabaseSummaryBase, DatabaseSummaryExplicitData, grants, tableInfos as selectTableInfos } from "src/views/databases/containers/databaseSummary"; import { TableInfo } from "src/views/databases/data/tableInfo"; import { SortSetting } from "src/views/shared/components/sortabletable"; import { SortedTable } from "src/views/shared/components/sortedtable"; import { SummaryBar, SummaryHeadlineStat } from "src/views/shared/components/summaryBar"; import "./databaseTables.styl"; +import { trustIcon } from "src/util/trust"; const databaseTablesSortSetting = new LocalSetting( "databases/sort_setting/tables", (s) => s.localSettings, @@ -86,11 +86,10 @@ class DatabaseSummaryTables extends DatabaseSummaryBase { } render() { - const { tableInfos, sortSetting } = this.props; + const { tableInfos, dbResponse, sortSetting } = this.props; const dbID = this.props.name; - + const loading = dbResponse ? dbResponse.inFlight : true; const numTables = tableInfos && tableInfos.length || 0; - return (
@@ -100,47 +99,49 @@ class DatabaseSummaryTables extends DatabaseSummaryBase {
- { - (numTables === 0) ? : - this.props.setSort(setting)} - columns={[ - { - title: "Table Name", - cell: (tableInfo) => { - return ( -
- {tableInfo.name} -
- ); - }, - sort: (tableInfo) => tableInfo.name, - className: "expand-link", // don't pad the td element to allow the link to expand - }, - { - title: "Size", - cell: (tableInfo) => Bytes(tableInfo.physicalSize), - sort: (tableInfo) => tableInfo.physicalSize, - }, - { - title: "Ranges", - cell: (tableInfo) => tableInfo.rangeCount, - sort: (tableInfo) => tableInfo.rangeCount, - }, - { - title: "# of Columns", - cell: (tableInfo) => tableInfo.numColumns, - sort: (tableInfo) => tableInfo.numColumns, - }, - { - title: "# of Indices", - cell: (tableInfo) => tableInfo.numIndices, - sort: (tableInfo) => tableInfo.numIndices, + {!loading && numTables === 0 ? : ( + this.props.setSort(setting)} + loading={loading as boolean} + loadingLabel="Loading tables..." + columns={[ + { + title: "Table Name", + cell: (tableInfo) => { + return ( +
+ {tableInfo.name} +
+ ); }, - ]} /> - } + sort: (tableInfo) => tableInfo.name, + className: "expand-link", // don't pad the td element to allow the link to expand + }, + { + title: "Size", + cell: (tableInfo) => Bytes(tableInfo.physicalSize), + sort: (tableInfo) => tableInfo.physicalSize, + }, + { + title: "Ranges", + cell: (tableInfo) => tableInfo.rangeCount, + sort: (tableInfo) => tableInfo.rangeCount, + }, + { + title: "# of Columns", + cell: (tableInfo) => tableInfo.numColumns, + sort: (tableInfo) => tableInfo.numColumns, + }, + { + title: "# of Indices", + cell: (tableInfo) => tableInfo.numIndices, + sort: (tableInfo) => tableInfo.numIndices, + }, + ]} + /> + )}
@@ -170,7 +171,7 @@ class DatabaseSummaryTables extends DatabaseSummaryBase { const mapStateToProps = (state: AdminUIState, ownProps: DatabaseSummaryExplicitData) => ({ // RootState contains declaration for whole state tableInfos: selectTableInfos(state, ownProps.name), sortSetting: databaseTablesSortSetting.selector(state), - dbResponse: databaseDetails(state)[ownProps.name] && databaseDetails(state)[ownProps.name].data, + dbResponse: databaseDetails(state)[ownProps.name], grants: grants(state, ownProps.name), }); diff --git a/pkg/ui/src/views/shared/components/sortabletable/index.tsx b/pkg/ui/src/views/shared/components/sortabletable/index.tsx index cbbbda932f1c..e4fb7592bec0 100644 --- a/pkg/ui/src/views/shared/components/sortabletable/index.tsx +++ b/pkg/ui/src/views/shared/components/sortabletable/index.tsx @@ -19,6 +19,8 @@ import { DrawerComponent } from "../drawer"; import { trackTableSort } from "src/util/analytics"; import "./sortabletable.styl"; +import { Spin, Icon } from "antd"; +import SpinIcon from "oss/src/components/icon/spin"; /** * SortableColumn describes the contents a single column of a @@ -76,6 +78,8 @@ interface TableProps { drawer?: boolean; firstCellBordered?: boolean; renderNoResult?: React.ReactNode; + loading?: boolean; + loadingLabel?: string; } export interface ExpandableConfig { @@ -239,7 +243,7 @@ export class SortableTable extends React.Component { } render() { - const { sortSetting, columns, expandableConfig, drawer, firstCellBordered, count, renderNoResult, className } = this.props; + const { sortSetting, columns, expandableConfig, drawer, firstCellBordered, count, renderNoResult, className, loading, loadingLabel } = this.props; const { visible, drawerData } = this.state; return ( @@ -281,9 +285,15 @@ export class SortableTable extends React.Component { - {times(this.props.count, this.renderRow)} + {!loading && times(this.props.count, this.renderRow)} + {loading && ( +
+ } /> + {loadingLabel && {loadingLabel}} +
+ )} {drawer && ( {getHighlightedText(drawerData.statement, drawerData.search, true)} diff --git a/pkg/ui/src/views/shared/components/sortabletable/sortabletable.styl b/pkg/ui/src/views/shared/components/sortabletable/sortabletable.styl index 079d91af3c5f..646dec1911e3 100644 --- a/pkg/ui/src/views/shared/components/sortabletable/sortabletable.styl +++ b/pkg/ui/src/views/shared/components/sortabletable/sortabletable.styl @@ -109,3 +109,17 @@ color $colors--neutral-1 white-space pre-wrap margin-bottom 0 + +.table__loading + display flex + justify-content center + align-items center + margin-top 65px + &--spin + margin-right 11px + &--label + font-family $font-family--base + font-size $font-size--medium + line-height 22px + letter-spacing $letter-spacing--compact + color $colors--neutral-6 diff --git a/pkg/ui/src/views/shared/components/sortedtable/index.tsx b/pkg/ui/src/views/shared/components/sortedtable/index.tsx index ef324986fe0f..9a692db0e4d2 100644 --- a/pkg/ui/src/views/shared/components/sortedtable/index.tsx +++ b/pkg/ui/src/views/shared/components/sortedtable/index.tsx @@ -80,6 +80,8 @@ interface SortedTableProps { firstCellBordered?: boolean; renderNoResult?: React.ReactNode; pagination?: ISortedTablePagination; + loading?: boolean; + loadingLabel?: string; } interface SortedTableState { @@ -210,7 +212,7 @@ export class SortedTable extends React.Component, SortedT } render() { - const { data, sortSetting, onChangeSortSetting, firstCellBordered, renderNoResult } = this.props; + const { data, loading, sortSetting, onChangeSortSetting, firstCellBordered, renderNoResult, loadingLabel } = this.props; let expandableConfig: ExpandableConfig = null; if (this.props.expandableConfig) { expandableConfig = { @@ -219,22 +221,22 @@ export class SortedTable extends React.Component, SortedT onChangeExpansion: this.onChangeExpansion, }; } - if (data) { - return ( - - ); - } - return
No results.
; + + return ( + + ); } } diff --git a/pkg/util/protoutil/randnullability.go b/pkg/util/protoutil/randnullability.go index 3578484fdfcb..7428575a98d0 100644 --- a/pkg/util/protoutil/randnullability.go +++ b/pkg/util/protoutil/randnullability.go @@ -11,12 +11,10 @@ package protoutil import ( - "context" "fmt" "reflect" "strings" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "golang.org/x/sync/syncmap" @@ -87,7 +85,7 @@ func RandomZeroInsertingVisitor(v reflect.Value) { } actual, loaded := insertZero.LoadOrStore(key, flipCoin()) if !loaded { - log.Infof(context.Background(), "inserting null for (%v).%v: %t", typ, typ.Field(i).Name, actual) + fmt.Printf("inserting null for (%v).%v: %t", typ, typ.Field(i).Name, actual) } if b := actual.(bool); b { hookInsertZero(v, i) diff --git a/pkg/workload/bank/bank.go b/pkg/workload/bank/bank.go index 2148149e6198..19ce65ac3eea 100644 --- a/pkg/workload/bank/bank.go +++ b/pkg/workload/bank/bank.go @@ -17,7 +17,7 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/bufalloc" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/workload" @@ -123,10 +123,10 @@ func (b *bank) Hooks() workload.Hooks { } } -var bankColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Int64, - coltypes.Bytes, +var bankTypes = []types.T{ + *types.Int, + *types.Int, + *types.Bytes, } // Tables implements the Generator interface. @@ -144,7 +144,7 @@ func (b *bank) Tables() []workload.Table { if rowEnd > b.rows { rowEnd = b.rows } - cb.Reset(bankColTypes, rowEnd-rowBegin) + cb.Reset(bankTypes, rowEnd-rowBegin) idCol := cb.ColVec(0).Int64() balanceCol := cb.ColVec(1).Int64() payloadCol := cb.ColVec(2).Bytes() diff --git a/pkg/workload/bulkingest/bulkingest.go b/pkg/workload/bulkingest/bulkingest.go index 21b19c8ba65b..38b3368879d0 100644 --- a/pkg/workload/bulkingest/bulkingest.go +++ b/pkg/workload/bulkingest/bulkingest.go @@ -53,7 +53,7 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/bufalloc" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -129,11 +129,11 @@ func (w *bulkingest) Tables() []workload.Table { } schema += ")" - var bulkingestColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, - coltypes.Bytes, + var bulkingestTypes = []types.T{ + *types.Int, + *types.Int, + *types.Int, + *types.Bytes, } table := workload.Table{ @@ -149,7 +149,7 @@ func (w *bulkingest) Tables() []workload.Table { a = ab % w.aCount } - cb.Reset(bulkingestColTypes, w.cCount) + cb.Reset(bulkingestTypes, w.cCount) aCol := cb.ColVec(0).Int64() bCol := cb.ColVec(1).Int64() cCol := cb.ColVec(2).Int64() diff --git a/pkg/workload/dep_test.go b/pkg/workload/dep_test.go index 0e3180153793..1287ce849799 100644 --- a/pkg/workload/dep_test.go +++ b/pkg/workload/dep_test.go @@ -26,13 +26,26 @@ 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/lex`, + `github.com/cockroachdb/cockroach/pkg/sql/oidext`, + `github.com/cockroachdb/cockroach/pkg/sql/types`, `github.com/cockroachdb/cockroach/pkg/util/arith`, `github.com/cockroachdb/cockroach/pkg/util/bufalloc`, `github.com/cockroachdb/cockroach/pkg/util/duration`, `github.com/cockroachdb/cockroach/pkg/util/encoding/csv`, + `github.com/cockroachdb/cockroach/pkg/util/envutil`, + `github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented`, + `github.com/cockroachdb/cockroach/pkg/util/humanizeutil`, + `github.com/cockroachdb/cockroach/pkg/util/protoutil`, + `github.com/cockroachdb/cockroach/pkg/util/randutil`, `github.com/cockroachdb/cockroach/pkg/util/stacktrace`, + `github.com/cockroachdb/cockroach/pkg/util/stringencoding`, `github.com/cockroachdb/cockroach/pkg/util/syncutil`, `github.com/cockroachdb/cockroach/pkg/util/timeutil`, + `github.com/cockroachdb/cockroach/pkg/util/uint128`, + `github.com/cockroachdb/cockroach/pkg/util/uuid`, `github.com/cockroachdb/cockroach/pkg/workload/histogram`, // TODO(dan): These really shouldn't be used in util packages, but the // payoff of fixing it is not worth it right now. diff --git a/pkg/workload/examples/startrek.go b/pkg/workload/examples/startrek.go index 311c67aa36f0..0b3527531da7 100644 --- a/pkg/workload/examples/startrek.go +++ b/pkg/workload/examples/startrek.go @@ -11,7 +11,7 @@ package examples import ( - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/workload" ) @@ -48,7 +48,7 @@ func (startrek) Tables() []workload.Table { Schema: episodesSchema, InitialRows: workload.TypedTuples( len(startrekEpisodes), - episodesColTypes, + episodesTypes, func(rowIdx int) []interface{} { return startrekEpisodes[rowIdx] }, ), }, @@ -57,19 +57,19 @@ func (startrek) Tables() []workload.Table { Schema: quotesSchema, InitialRows: workload.TypedTuples( len(startrekQuotes), - quotesColTypes, + quotesTypes, func(rowIdx int) []interface{} { return startrekQuotes[rowIdx] }, ), }, } } -var episodesColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, - coltypes.Bytes, - coltypes.Float64, +var episodesTypes = []types.T{ + *types.Int, + *types.Int, + *types.Int, + *types.Bytes, + *types.Float, } // The data that follows was derived from the 'startrek' fortune cookie file. @@ -155,11 +155,11 @@ var startrekEpisodes = [...][]interface{}{ {79, 3, 24, `Turnabout Intruder`, 5928.5}, } -var quotesColTypes = []coltypes.T{ - coltypes.Bytes, - coltypes.Bytes, - coltypes.Float64, - coltypes.Int64, +var quotesTypes = []types.T{ + *types.Bytes, + *types.Bytes, + *types.Float, + *types.Int, } var startrekQuotes = [...][]interface{}{ diff --git a/pkg/workload/interleavedpartitioned/interleavedpartitioned.go b/pkg/workload/interleavedpartitioned/interleavedpartitioned.go index 71bc5e8f7ea9..afc64bf4351f 100644 --- a/pkg/workload/interleavedpartitioned/interleavedpartitioned.go +++ b/pkg/workload/interleavedpartitioned/interleavedpartitioned.go @@ -20,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach-go/crdb" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/bufalloc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -814,12 +814,12 @@ func (w *interleavedPartitioned) sessionsInitialRow(rowIdx int) []interface{} { } } -var childColTypes = []coltypes.T{ - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, +var childTypes = []types.T{ + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, } func (w *interleavedPartitioned) childInitialRowBatchFunc( @@ -831,7 +831,7 @@ func (w *interleavedPartitioned) childInitialRowBatchFunc( nowString := timeutil.Now().UTC().Format(time.RFC3339) rng := rand.New(rand.NewSource(int64(sessionRowIdx) + rngFactor)) - cb.Reset(childColTypes, nPerBatch) + cb.Reset(childTypes, nPerBatch) sessionIDCol := cb.ColVec(0).Bytes() idCol := cb.ColVec(1).Bytes() valueCol := cb.ColVec(2).Bytes() @@ -847,17 +847,17 @@ func (w *interleavedPartitioned) childInitialRowBatchFunc( } } -var deviceColTypes = []coltypes.T{ - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, +var deviceTypes = []types.T{ + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, } func (w *interleavedPartitioned) deviceInitialRowBatch( @@ -868,7 +868,7 @@ func (w *interleavedPartitioned) deviceInitialRowBatch( sessionID := randomSessionID(sessionRNG, `east`, w.initEastPercent) nowString := timeutil.Now().UTC().Format(time.RFC3339) - cb.Reset(deviceColTypes, w.devicesPerSession) + cb.Reset(deviceTypes, w.devicesPerSession) sessionIDCol := cb.ColVec(0).Bytes() idCol := cb.ColVec(1).Bytes() deviceIDCol := cb.ColVec(2).Bytes() @@ -893,11 +893,11 @@ func (w *interleavedPartitioned) deviceInitialRowBatch( } } -var queryColTypes = []coltypes.T{ - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, +var queryTypes = []types.T{ + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, } func (w *interleavedPartitioned) queryInitialRowBatch( @@ -908,7 +908,7 @@ func (w *interleavedPartitioned) queryInitialRowBatch( sessionID := randomSessionID(sessionRNG, `east`, w.initEastPercent) nowString := timeutil.Now().UTC().Format(time.RFC3339) - cb.Reset(queryColTypes, w.queriesPerSession) + cb.Reset(queryTypes, w.queriesPerSession) sessionIDCol := cb.ColVec(0).Bytes() idCol := cb.ColVec(1).Bytes() createdCol := cb.ColVec(2).Bytes() diff --git a/pkg/workload/ledger/generate.go b/pkg/workload/ledger/generate.go index e47a81180c05..9549f2eb95f0 100644 --- a/pkg/workload/ledger/generate.go +++ b/pkg/workload/ledger/generate.go @@ -17,7 +17,7 @@ import ( "math/rand" "strconv" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/uint128" "github.com/cockroachdb/cockroach/pkg/util/uuid" ) @@ -32,17 +32,17 @@ const ( cashMoneyType = "C" ) -var ledgerCustomerColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bool, - coltypes.Bool, - coltypes.Bytes, - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, +var ledgerCustomerTypes = []types.T{ + *types.Int, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bool, + *types.Bool, + *types.Bytes, + *types.Int, + *types.Int, + *types.Int, } func (w *ledger) ledgerCustomerInitialRow(rowIdx int) []interface{} { @@ -70,16 +70,16 @@ func (w *ledger) ledgerCustomerSplitRow(splitIdx int) []interface{} { } } -var ledgerTransactionColTypes = []coltypes.T{ - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Int64, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, +var ledgerTransactionColTypes = []types.T{ + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Int, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, } func (w *ledger) ledgerTransactionInitialRow(rowIdx int) []interface{} { diff --git a/pkg/workload/ledger/ledger.go b/pkg/workload/ledger/ledger.go index e47c74eb3205..0b2b2340212d 100644 --- a/pkg/workload/ledger/ledger.go +++ b/pkg/workload/ledger/ledger.go @@ -128,7 +128,7 @@ func (w *ledger) Tables() []workload.Table { Schema: ledgerCustomerSchema, InitialRows: workload.TypedTuples( w.customers, - ledgerCustomerColTypes, + ledgerCustomerTypes, w.ledgerCustomerInitialRow, ), Splits: workload.Tuples( diff --git a/pkg/workload/tpcc/generate.go b/pkg/workload/tpcc/generate.go index 1dc2d676052c..53d8c27d9de9 100644 --- a/pkg/workload/tpcc/generate.go +++ b/pkg/workload/tpcc/generate.go @@ -14,7 +14,7 @@ import ( "strconv" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/bufalloc" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/cockroach/pkg/workload" @@ -65,12 +65,12 @@ type generateLocals struct { uuidAlloc uuid.UUID } -var itemColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Int64, - coltypes.Bytes, - coltypes.Float64, - coltypes.Bytes, +var itemTypes = []types.T{ + *types.Int, + *types.Int, + *types.Bytes, + *types.Float, + *types.Bytes, } func (w *tpcc) tpccItemInitialRowBatch(rowIdx int, cb coldata.Batch, a *bufalloc.ByteAllocator) { @@ -81,7 +81,7 @@ func (w *tpcc) tpccItemInitialRowBatch(rowIdx int, cb coldata.Batch, a *bufalloc iID := rowIdx + 1 - cb.Reset(itemColTypes, 1) + cb.Reset(itemTypes, 1) cb.ColVec(0).Int64()[0] = int64(iID) cb.ColVec(1).Int64()[0] = randInt(l.rng.Rand, 1, 10000) // im_id: "Image ID associated to Item" cb.ColVec(2).Bytes().Set(0, randAStringInitialDataOnly(&l.rng, &ao, a, 14, 24)) // name @@ -102,16 +102,16 @@ func (w *tpcc) tpccItemStats() []workload.JSONStatistic { } } -var warehouseColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Float64, - coltypes.Float64, +var warehouseTypes = []types.T{ + *types.Int, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Float, + *types.Float, } func (w *tpcc) tpccWarehouseInitialRowBatch( @@ -125,7 +125,7 @@ func (w *tpcc) tpccWarehouseInitialRowBatch( wID := rowIdx // warehouse ids are 0-indexed. every other table is 1-indexed - cb.Reset(warehouseColTypes, 1) + cb.Reset(warehouseTypes, 1) cb.ColVec(0).Int64()[0] = int64(wID) cb.ColVec(1).Bytes().Set(0, []byte(strconv.FormatInt(randInt(l.rng.Rand, 6, 10), 10))) // name cb.ColVec(2).Bytes().Set(0, []byte(strconv.FormatInt(randInt(l.rng.Rand, 10, 20), 10))) // street_1 @@ -154,24 +154,24 @@ func (w *tpcc) tpccWarehouseStats() []workload.JSONStatistic { } } -var stockColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, - coltypes.Bytes, +var stockTypes = []types.T{ + *types.Int, + *types.Int, + *types.Int, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Int, + *types.Int, + *types.Int, + *types.Bytes, } func (w *tpcc) tpccStockInitialRowBatch(rowIdx int, cb coldata.Batch, a *bufalloc.ByteAllocator) { @@ -183,7 +183,7 @@ func (w *tpcc) tpccStockInitialRowBatch(rowIdx int, cb coldata.Batch, a *bufallo sID := (rowIdx % numStockPerWarehouse) + 1 wID := (rowIdx / numStockPerWarehouse) - cb.Reset(stockColTypes, 1) + cb.Reset(stockTypes, 1) cb.ColVec(0).Int64()[0] = int64(sID) cb.ColVec(1).Int64()[0] = int64(wID) cb.ColVec(2).Int64()[0] = randInt(l.rng.Rand, 10, 100) // quantity @@ -230,18 +230,18 @@ func (w *tpcc) tpccStockStats() []workload.JSONStatistic { } } -var districtColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Int64, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Float64, - coltypes.Float64, - coltypes.Int64, +var districtTypes = []types.T{ + *types.Int, + *types.Int, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Float, + *types.Float, + *types.Int, } func (w *tpcc) tpccDistrictInitialRowBatch( @@ -257,7 +257,7 @@ func (w *tpcc) tpccDistrictInitialRowBatch( dID := (rowIdx % numDistrictsPerWarehouse) + 1 wID := (rowIdx / numDistrictsPerWarehouse) - cb.Reset(districtColTypes, 1) + cb.Reset(districtTypes, 1) cb.ColVec(0).Int64()[0] = int64(dID) cb.ColVec(1).Int64()[0] = int64(wID) cb.ColVec(2).Bytes().Set(0, randAStringInitialDataOnly(&l.rng, &ao, a, 6, 10)) // name @@ -293,28 +293,28 @@ func (w *tpcc) tpccDistrictStats() []workload.JSONStatistic { } } -var customerColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Float64, - coltypes.Float64, - coltypes.Float64, - coltypes.Float64, - coltypes.Int64, - coltypes.Int64, - coltypes.Bytes, +var customerTypes = []types.T{ + *types.Int, + *types.Int, + *types.Int, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Float, + *types.Float, + *types.Float, + *types.Float, + *types.Int, + *types.Int, + *types.Bytes, } func (w *tpcc) tpccCustomerInitialRowBatch( @@ -347,7 +347,7 @@ func (w *tpcc) tpccCustomerInitialRowBatch( lastName = w.randCLast(l.rng.Rand, a) } - cb.Reset(customerColTypes, 1) + cb.Reset(customerTypes, 1) cb.ColVec(0).Int64()[0] = int64(cID) cb.ColVec(1).Int64()[0] = int64(dID) cb.ColVec(2).Int64()[0] = int64(wID) @@ -405,16 +405,16 @@ func (w *tpcc) tpccCustomerStats() []workload.JSONStatistic { } } -var historyColTypes = []coltypes.T{ - coltypes.Bytes, - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, - coltypes.Bytes, - coltypes.Float64, - coltypes.Bytes, +var historyTypes = []types.T{ + *types.Bytes, + *types.Int, + *types.Int, + *types.Int, + *types.Int, + *types.Int, + *types.Bytes, + *types.Float, + *types.Bytes, } func (w *tpcc) tpccHistoryInitialRowBatch(rowIdx int, cb coldata.Batch, a *bufalloc.ByteAllocator) { @@ -439,7 +439,7 @@ func (w *tpcc) tpccHistoryInitialRowBatch(rowIdx int, cb coldata.Batch, a *bufal dID := ((rowIdx / numCustomersPerDistrict) % numDistrictsPerWarehouse) + 1 wID := (rowIdx / numCustomersPerWarehouse) - cb.Reset(historyColTypes, 1) + cb.Reset(historyTypes, 1) cb.ColVec(0).Bytes().Set(0, rowID) cb.ColVec(1).Int64()[0] = int64(cID) cb.ColVec(2).Int64()[0] = int64(dID) @@ -468,15 +468,15 @@ func (w *tpcc) tpccHistoryStats() []workload.JSONStatistic { } } -var orderColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, - coltypes.Bytes, - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, +var orderTypes = []types.T{ + *types.Int, + *types.Int, + *types.Int, + *types.Int, + *types.Bytes, + *types.Int, + *types.Int, + *types.Int, } func (w *tpcc) tpccOrderInitialRowBatch(rowIdx int, cb coldata.Batch, a *bufalloc.ByteAllocator) { @@ -520,7 +520,7 @@ func (w *tpcc) tpccOrderInitialRowBatch(rowIdx int, cb coldata.Batch, a *bufallo carrierID = randInt(l.rng.Rand, 1, 10) } - cb.Reset(orderColTypes, 1) + cb.Reset(orderTypes, 1) cb.ColVec(0).Int64()[0] = int64(oID) cb.ColVec(1).Int64()[0] = int64(dID) cb.ColVec(2).Int64()[0] = int64(wID) @@ -555,10 +555,10 @@ func (w *tpcc) tpccOrderStats() []workload.JSONStatistic { } } -var newOrderColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, +var newOrderTypes = []types.T{ + *types.Int, + *types.Int, + *types.Int, } func (w *tpcc) tpccNewOrderInitialRowBatch( @@ -570,7 +570,7 @@ func (w *tpcc) tpccNewOrderInitialRowBatch( dID := ((rowIdx / numNewOrdersPerDistrict) % numDistrictsPerWarehouse) + 1 wID := (rowIdx / numNewOrdersPerWarehouse) - cb.Reset(newOrderColTypes, 1) + cb.Reset(newOrderTypes, 1) cb.ColVec(0).Int64()[0] = int64(oID) cb.ColVec(1).Int64()[0] = int64(dID) cb.ColVec(2).Int64()[0] = int64(wID) @@ -585,17 +585,17 @@ func (w *tpcc) tpccNewOrderStats() []workload.JSONStatistic { } } -var orderLineColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, - coltypes.Bytes, - coltypes.Int64, - coltypes.Float64, - coltypes.Bytes, +var orderLineTypes = []types.T{ + *types.Int, + *types.Int, + *types.Int, + *types.Int, + *types.Int, + *types.Int, + *types.Bytes, + *types.Int, + *types.Float, + *types.Bytes, } func (w *tpcc) tpccOrderLineInitialRowBatch( @@ -615,7 +615,7 @@ func (w *tpcc) tpccOrderLineInitialRowBatch( wID := (orderRowIdx / numOrdersPerWarehouse) ao := aCharsOffset(l.rng.Intn(len(aCharsAlphabet))) - cb.Reset(orderLineColTypes, numOrderLines) + cb.Reset(orderLineTypes, numOrderLines) olOIDCol := cb.ColVec(0).Int64() olDIDCol := cb.ColVec(1).Int64() olWIDCol := cb.ColVec(2).Int64() diff --git a/pkg/workload/tpch/generate.go b/pkg/workload/tpch/generate.go index 631fe872694b..ee41b7ef05b0 100644 --- a/pkg/workload/tpch/generate.go +++ b/pkg/workload/tpch/generate.go @@ -14,7 +14,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/bufalloc" "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate" "golang.org/x/exp/rand" @@ -52,10 +52,10 @@ var nations = [...]struct { {name: `UNITED STATES`, regionKey: 1}, } -var regionColTypes = []coltypes.T{ - coltypes.Int16, - coltypes.Bytes, - coltypes.Bytes, +var regionTypes = []types.T{ + *types.Int2, + *types.Bytes, + *types.Bytes, } func (w *tpch) tpchRegionInitialRowBatch( @@ -67,17 +67,17 @@ func (w *tpch) tpchRegionInitialRowBatch( rng.Seed(w.seed + uint64(batchIdx)) regionKey := batchIdx - cb.Reset(regionColTypes, 1) + cb.Reset(regionTypes, 1) cb.ColVec(0).Int16()[0] = int16(regionKey) // r_regionkey cb.ColVec(1).Bytes().Set(0, []byte(regionNames[regionKey])) // r_name cb.ColVec(2).Bytes().Set(0, w.textPool.randString(rng, 31, 115)) // r_comment } -var nationColTypes = []coltypes.T{ - coltypes.Int16, - coltypes.Bytes, - coltypes.Int16, - coltypes.Bytes, +var nationTypes = []types.T{ + *types.Int2, + *types.Bytes, + *types.Int2, + *types.Bytes, } func (w *tpch) tpchNationInitialRowBatch( @@ -90,21 +90,21 @@ func (w *tpch) tpchNationInitialRowBatch( nationKey := batchIdx nation := nations[nationKey] - cb.Reset(nationColTypes, 1) + cb.Reset(nationTypes, 1) cb.ColVec(0).Int16()[0] = int16(nationKey) // n_nationkey cb.ColVec(1).Bytes().Set(0, []byte(nation.name)) // n_name cb.ColVec(2).Int16()[0] = int16(nation.regionKey) // n_regionkey cb.ColVec(3).Bytes().Set(0, w.textPool.randString(rng, 31, 115)) // r_comment } -var supplierColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Int16, - coltypes.Bytes, - coltypes.Float64, - coltypes.Bytes, +var supplierTypes = []types.T{ + *types.Int, + *types.Bytes, + *types.Bytes, + *types.Int2, + *types.Bytes, + *types.Float, + *types.Bytes, } func (w *tpch) tpchSupplierInitialRowBatch( @@ -117,7 +117,7 @@ func (w *tpch) tpchSupplierInitialRowBatch( suppKey := int64(batchIdx) + 1 nationKey := int16(randInt(rng, 0, 24)) - cb.Reset(supplierColTypes, 1) + cb.Reset(supplierTypes, 1) cb.ColVec(0).Int64()[0] = suppKey // s_suppkey cb.ColVec(1).Bytes().Set(0, supplierName(a, suppKey)) // s_name cb.ColVec(2).Bytes().Set(0, randVString(rng, a, 10, 40)) // s_address @@ -128,16 +128,16 @@ func (w *tpch) tpchSupplierInitialRowBatch( cb.ColVec(6).Bytes().Set(0, w.textPool.randString(rng, 25, 100)) // s_comment } -var partColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Int16, - coltypes.Bytes, - coltypes.Float64, - coltypes.Bytes, +var partTypes = []types.T{ + *types.Int, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Bytes, + *types.Int2, + *types.Bytes, + *types.Float, + *types.Bytes, } func makeRetailPriceFromPartKey(partKey int) float32 { @@ -151,7 +151,7 @@ func (w *tpch) tpchPartInitialRowBatch(batchIdx int, cb coldata.Batch, a *bufall rng.Seed(w.seed + uint64(batchIdx)) partKey := batchIdx + 1 - cb.Reset(partColTypes, 1) + cb.Reset(partTypes, 1) // P_PARTKEY unique within [SF * 200,000]. cb.ColVec(0).Int64()[0] = int64(partKey) @@ -175,12 +175,12 @@ func (w *tpch) tpchPartInitialRowBatch(batchIdx int, cb coldata.Batch, a *bufall cb.ColVec(8).Bytes().Set(0, w.textPool.randString(rng, 5, 22)) } -var partSuppColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Int64, - coltypes.Int16, - coltypes.Float64, - coltypes.Bytes, +var partSuppTypes = []types.T{ + *types.Int, + *types.Int, + *types.Int2, + *types.Float, + *types.Bytes, } func (w *tpch) tpchPartSuppInitialRowBatch( @@ -192,7 +192,7 @@ func (w *tpch) tpchPartSuppInitialRowBatch( rng.Seed(w.seed + uint64(batchIdx)) partKey := batchIdx + 1 - cb.Reset(partSuppColTypes, numPartSuppPerPart) + cb.Reset(partSuppTypes, numPartSuppPerPart) // P_PARTKEY unique within [SF * 200,000]. partKeyCol := cb.ColVec(0).Int64() @@ -219,15 +219,15 @@ func (w *tpch) tpchPartSuppInitialRowBatch( } } -var customerColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Int16, - coltypes.Bytes, - coltypes.Float64, - coltypes.Bytes, - coltypes.Bytes, +var customerTypes = []types.T{ + *types.Int, + *types.Bytes, + *types.Bytes, + *types.Int2, + *types.Bytes, + *types.Float, + *types.Bytes, + *types.Bytes, } func (w *tpch) tpchCustomerInitialRowBatch( @@ -239,7 +239,7 @@ func (w *tpch) tpchCustomerInitialRowBatch( rng.Seed(w.seed + uint64(batchIdx)) custKey := int64(batchIdx) + 1 - cb.Reset(customerColTypes, 1) + cb.Reset(customerTypes, 1) // C_CUSTKEY unique within [SF * 150,000]. cb.ColVec(0).Int64()[0] = custKey @@ -322,16 +322,16 @@ type orderSharedRandomData struct { allF bool } -var ordersColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Int64, - coltypes.Bytes, - coltypes.Float64, - coltypes.Int64, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Int16, - coltypes.Bytes, +var ordersTypes = []types.T{ + *types.Int, + *types.Int, + *types.Bytes, + *types.Float, + *types.Int, + *types.Bytes, + *types.Bytes, + *types.Int2, + *types.Bytes, } func populateSharedData(rng *rand.Rand, seed uint64, sf int, data *orderSharedRandomData) { @@ -371,7 +371,7 @@ func (w *tpch) tpchOrdersInitialRowBatch( defer w.localsPool.Put(l) rng := l.rng - cb.Reset(ordersColTypes, numOrderPerCustomer) + cb.Reset(ordersTypes, numOrderPerCustomer) orderKeyCol := cb.ColVec(0).Int64() custKeyCol := cb.ColVec(1).Int64() @@ -424,23 +424,23 @@ func (w *tpch) tpchOrdersInitialRowBatch( } } -var lineItemColTypes = []coltypes.T{ - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, - coltypes.Int16, - coltypes.Float64, - coltypes.Float64, - coltypes.Float64, - coltypes.Float64, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Int64, - coltypes.Int64, - coltypes.Int64, - coltypes.Bytes, - coltypes.Bytes, - coltypes.Bytes, +var lineItemTypes = []types.T{ + *types.Int, + *types.Int, + *types.Int, + *types.Int2, + *types.Float, + *types.Float, + *types.Float, + *types.Float, + *types.Bytes, + *types.Bytes, + *types.Int, + *types.Int, + *types.Int, + *types.Bytes, + *types.Bytes, + *types.Bytes, } func (w *tpch) tpchLineItemInitialRowBatch( @@ -450,7 +450,7 @@ func (w *tpch) tpchLineItemInitialRowBatch( defer w.localsPool.Put(l) rng := l.rng - cb.Reset(lineItemColTypes, numOrderPerCustomer*7) + cb.Reset(lineItemTypes, numOrderPerCustomer*7) orderKeyCol := cb.ColVec(0).Int64() partKeyCol := cb.ColVec(1).Int64() diff --git a/pkg/workload/workload.go b/pkg/workload/workload.go index e06a3b0d9ede..3aa257038f21 100644 --- a/pkg/workload/workload.go +++ b/pkg/workload/workload.go @@ -26,6 +26,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/sql/types" "github.com/cockroachdb/cockroach/pkg/util/bufalloc" "github.com/cockroachdb/cockroach/pkg/workload/histogram" "github.com/pkg/errors" @@ -175,9 +177,9 @@ type BatchedTuples struct { // Tuples is like TypedTuples except that it tries to guess the type of each // datum. However, if the function ever returns nil for one of the datums, you -// need to use TypedTuples instead and specify the coltypes. +// need to use TypedTuples instead and specify the types. func Tuples(count int, fn func(int) []interface{}) BatchedTuples { - return TypedTuples(count, nil /* colTypes */, fn) + return TypedTuples(count, nil /* typs */, fn) } const ( @@ -187,12 +189,12 @@ const ( // TypedTuples returns a BatchedTuples where each batch has size 1. It's // intended to be easier to use than directly specifying a BatchedTuples, but -// the tradeoff is some bit of performance. If colTypes is nil, an attempt is +// the tradeoff is some bit of performance. If typs is nil, an attempt is // made to infer them. -func TypedTuples(count int, colTypes []coltypes.T, fn func(int) []interface{}) BatchedTuples { +func TypedTuples(count int, typs []types.T, fn func(int) []interface{}) BatchedTuples { // The FillBatch we create has to be concurrency safe, so we can't let it do - // the one-time initialization of colTypes without this protection. - var colTypesOnce sync.Once + // the one-time initialization of typs without this protection. + var typesOnce sync.Once t := BatchedTuples{ NumBatches: count, @@ -201,9 +203,9 @@ func TypedTuples(count int, colTypes []coltypes.T, fn func(int) []interface{}) B t.FillBatch = func(batchIdx int, cb coldata.Batch, _ *bufalloc.ByteAllocator) { row := fn(batchIdx) - colTypesOnce.Do(func() { - if colTypes == nil { - colTypes = make([]coltypes.T, len(row)) + typesOnce.Do(func() { + if typs == nil { + typs = make([]types.T, len(row)) for i, datum := range row { if datum == nil { panic(fmt.Sprintf( @@ -211,16 +213,20 @@ func TypedTuples(count int, colTypes []coltypes.T, fn func(int) []interface{}) B } else { switch datum.(type) { case time.Time: - colTypes[i] = coltypes.Bytes + typs[i] = *types.Bytes default: - colTypes[i] = coltypes.FromGoType(datum) + t, err := typeconv.UnsafeToSQLType(coltypes.FromGoType(datum)) + if err != nil { + panic(err) + } + typs[i] = *t } } } } }) - cb.Reset(colTypes, 1) + cb.Reset(typs, 1) for colIdx, col := range cb.ColVecs() { switch d := row[colIdx].(type) { case nil: diff --git a/pkg/workload/ycsb/ycsb.go b/pkg/workload/ycsb/ycsb.go index 7dbbc13d93a9..d4cd1f5235c2 100644 --- a/pkg/workload/ycsb/ycsb.go +++ b/pkg/workload/ycsb/ycsb.go @@ -24,7 +24,7 @@ import ( "sync/atomic" "github.com/cockroachdb/cockroach-go/crdb" - "github.com/cockroachdb/cockroach/pkg/col/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/workload" "github.com/cockroachdb/cockroach/pkg/workload/histogram" @@ -267,9 +267,9 @@ func preferColumnFamilies(workload string) bool { } } -var usertableColTypes = []coltypes.T{ - coltypes.Bytes, coltypes.Bytes, coltypes.Bytes, coltypes.Bytes, coltypes.Bytes, coltypes.Bytes, - coltypes.Bytes, coltypes.Bytes, coltypes.Bytes, coltypes.Bytes, coltypes.Bytes, +var usertableTypes = []types.T{ + *types.Bytes, *types.Bytes, *types.Bytes, *types.Bytes, *types.Bytes, *types.Bytes, + *types.Bytes, *types.Bytes, *types.Bytes, *types.Bytes, *types.Bytes, } // Tables implements the Generator interface. @@ -308,7 +308,7 @@ func (g *ycsb) Tables() []workload.Table { } usertable.InitialRows = workload.TypedTuples( g.insertCount, - usertableColTypes, + usertableTypes, usertableInitialRowsFn, ) }