From 5a6f53bfe55ea7575dbcb4f624f899a98d7ab544 Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Sat, 26 Jun 2021 04:33:06 -0700 Subject: [PATCH] colexec: implement vectorized index join This patch provides a vectorized implementation of the index join operator. Span generation is accomplished using two utility operators. `spanEncoder` operates on a single index key column and fills a `Bytes` column with the encoding of that column for each row. `spanAssembler` takes the output of each `spanEncoder` and generates spans, accounting for table/index prefixes and possibly splitting the spans over column families. Finally, the `ColIndexJoin` operator uses the generated spans to perform a lookup on the table's primary index, returns all batches resulting from the lookup, and repeats until the input is fully consumed. The `ColIndexJoin` operator queues up input rows until the memory footprint of the rows reaches a preset limit (default 4MB for parity with the row engine). This allows the cost of starting a scan to be amortized. Addresses #65905 Release note (sql change): The vectorized execution engine can now perform a scan over an index, and then join on the primary index to retrieve the required columns. --- Makefile | 2 + pkg/BUILD.bazel | 1 + pkg/col/coldata/bytes.go | 22 + pkg/keys/printer.go | 2 +- pkg/sql/colexec/colbuilder/execplan.go | 77 +- pkg/sql/colexec/colexecspan/BUILD.bazel | 83 + pkg/sql/colexec/colexecspan/dep_test.go | 31 + pkg/sql/colexec/colexecspan/main_test.go | 40 + .../colexec/colexecspan/span_assembler.eg.go | 361 ++++ .../colexecspan/span_assembler_test.go | 240 +++ .../colexecspan/span_assembler_tmpl.go | 344 +++ .../colexec/colexecspan/span_encoder.eg.go | 1904 +++++++++++++++++ .../colexec/colexecspan/span_encoder_tmpl.go | 195 ++ .../colexec/execgen/cmd/execgen/BUILD.bazel | 2 + .../execgen/cmd/execgen/span_assembler_gen.go | 44 + .../execgen/cmd/execgen/span_encoder_gen.go | 175 ++ pkg/sql/colexec/hash_aggregator.go | 2 +- pkg/sql/colexec/ordered_synchronizer.eg.go | 2 +- pkg/sql/colexec/ordered_synchronizer_tmpl.go | 2 +- pkg/sql/colfetcher/BUILD.bazel | 4 + pkg/sql/colfetcher/cfetcher.go | 52 +- pkg/sql/colfetcher/colbatch_scan.go | 128 +- pkg/sql/colfetcher/index_join.go | 526 +++++ pkg/sql/colmem/allocator.go | 70 +- .../testdata/logic_test/inverted_index | 18 +- .../logic_test/inverted_index_geospatial | 6 +- .../logictest/testdata/logic_test/tpch_vec | 20 +- pkg/testutils/lint/lint_test.go | 1 + 28 files changed, 4217 insertions(+), 137 deletions(-) create mode 100644 pkg/sql/colexec/colexecspan/BUILD.bazel create mode 100644 pkg/sql/colexec/colexecspan/dep_test.go create mode 100644 pkg/sql/colexec/colexecspan/main_test.go create mode 100644 pkg/sql/colexec/colexecspan/span_assembler.eg.go create mode 100644 pkg/sql/colexec/colexecspan/span_assembler_test.go create mode 100644 pkg/sql/colexec/colexecspan/span_assembler_tmpl.go create mode 100644 pkg/sql/colexec/colexecspan/span_encoder.eg.go create mode 100644 pkg/sql/colexec/colexecspan/span_encoder_tmpl.go create mode 100644 pkg/sql/colexec/execgen/cmd/execgen/span_assembler_gen.go create mode 100644 pkg/sql/colexec/execgen/cmd/execgen/span_encoder_gen.go create mode 100644 pkg/sql/colfetcher/index_join.go diff --git a/Makefile b/Makefile index 839be1e69c44..0408cf500eea 100644 --- a/Makefile +++ b/Makefile @@ -881,6 +881,8 @@ EXECGEN_TARGETS = \ pkg/sql/colexec/colexecsel/default_cmp_sel_ops.eg.go \ pkg/sql/colexec/colexecsel/selection_ops.eg.go \ pkg/sql/colexec/colexecsel/sel_like_ops.eg.go \ + pkg/sql/colexec/colexecspan/span_assembler.eg.go \ + pkg/sql/colexec/colexecspan/span_encoder.eg.go \ pkg/sql/colexec/colexecwindow/first_value.eg.go \ pkg/sql/colexec/colexecwindow/lag.eg.go \ pkg/sql/colexec/colexecwindow/last_value.eg.go \ diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 6e6075b27cd8..8dfad485037d 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -210,6 +210,7 @@ ALL_TESTS = [ "//pkg/sql/colexec/colexecjoin:colexecjoin_test", "//pkg/sql/colexec/colexecproj:colexecproj_test", "//pkg/sql/colexec/colexecsel:colexecsel_test", + "//pkg/sql/colexec/colexecspan:colexecspan_test", "//pkg/sql/colexec/colexectestutils:colexectestutils_test", "//pkg/sql/colexec/colexecutils:colexecutils_test", "//pkg/sql/colexec/colexecwindow:colexecwindow_test", diff --git a/pkg/col/coldata/bytes.go b/pkg/col/coldata/bytes.go index 7c06e9dba208..5ceb1a7ab04b 100644 --- a/pkg/col/coldata/bytes.go +++ b/pkg/col/coldata/bytes.go @@ -384,6 +384,9 @@ const FlatBytesOverhead = int64(unsafe.Sizeof(Bytes{})) // Size returns the total size of the receiver in bytes. func (b *Bytes) Size() int64 { + if b == nil { + return 0 + } return FlatBytesOverhead + int64(cap(b.data)) + int64(cap(b.offsets))*memsize.Int32 @@ -400,6 +403,16 @@ func (b *Bytes) ProportionalSize(n int64) int64 { return FlatBytesOverhead + int64(len(b.data[b.offsets[0]:b.offsets[n]])) + n*memsize.Int32 } +// ElemSize returns the size in bytes of the []byte elem at the given index. +// Panics if passed an invalid element. +func (b *Bytes) ElemSize(idx int) int64 { + if idx < 0 || idx >= b.Len() { + colexecerror.InternalError( + errors.AssertionFailedf("called ElemSize with invalid index: %d", idx)) + } + return int64(b.offsets[idx+1] - b.offsets[idx]) +} + // Abbreviated returns a uint64 slice where each uint64 represents the first // eight bytes of each []byte. It is used for byte comparison fast paths. // @@ -456,6 +469,15 @@ func (b *Bytes) Reset() { b.maxSetLength = 0 } +// ResetForAppend is similar to Reset, but it also resets the offsets slice so +// that future calls to AppendSlice or AppendVal will append starting from index +// zero. TODO(drewk): once Set is removed, this can just be Reset. +func (b *Bytes) ResetForAppend() { + b.Reset() + // The first offset indicates where the first element will start. + b.offsets = b.offsets[:1] +} + // Truncate truncates the underlying bytes to the given length. This allows Set // to be called at or beyond the given length. If the length of the bytes is // already less than or equal to the given length, Truncate is a no-op. diff --git a/pkg/keys/printer.go b/pkg/keys/printer.go index 4ee72289f997..1a345bc7eed4 100644 --- a/pkg/keys/printer.go +++ b/pkg/keys/printer.go @@ -706,7 +706,7 @@ func init() { // PrettyPrintRange pretty prints a compact representation of a key range. The // output is of the form: // commonPrefix{remainingStart-remainingEnd} -// If the end key is empty, the outut is of the form: +// If the end key is empty, the output is of the form: // start // It prints at most maxChars, truncating components as needed. See // TestPrettyPrintRange for some examples. diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 9d9b2b755a4f..794dc047c97a 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -178,6 +178,20 @@ func supportedNatively(spec *execinfrapb.ProcessorSpec) error { } return nil + case spec.Core.JoinReader != nil: + if spec.Core.JoinReader.LookupColumns != nil || !spec.Core.JoinReader.LookupExpr.Empty() { + return errLookupJoinUnsupported + } + for i := range spec.Core.JoinReader.Table.Indexes { + if spec.Core.JoinReader.Table.Indexes[i].IsInterleaved() { + // Interleaved indexes are going to be removed anyway, so there is no + // point in handling the extra complexity. Just let the row engine + // handle this. + return errInterleavedIndexJoin + } + } + return nil + case spec.Core.Filterer != nil: return nil @@ -249,6 +263,8 @@ var ( errSampleAggregatorWrap = errors.New("core.SampleAggregator is not supported (not an execinfra.RowSource)") errExperimentalWrappingProhibited = errors.New("wrapping for non-JoinReader and non-LocalPlanNode cores is prohibited in vectorize=experimental_always") errWrappedCast = errors.New("mismatched types in NewColOperator and unsupported casts") + errLookupJoinUnsupported = errors.New("lookup join reader is unsupported in vectorized") + errInterleavedIndexJoin = errors.New("vectorized join reader is unsupported for interleaved indexes") ) func canWrap(mode sessiondatapb.VectorizeExecMode, spec *execinfrapb.ProcessorSpec) error { @@ -764,23 +780,32 @@ func NewColOperator( if err != nil { return r, err } - result.Root = scanOp - if util.CrdbTestBuild { - result.Root = colexec.NewInvariantsChecker(result.Root) - } - result.KVReader = scanOp - result.MetadataSources = append(result.MetadataSources, result.Root.(colexecop.MetadataSource)) - result.Releasables = append(result.Releasables, scanOp) + result.finishScanPlanning(scanOp, scanOp.ResultTypes) - // We want to check for cancellation once per input batch, and - // wrapping only colBatchScan with a CancelChecker allows us to do - // just that. It's sufficient for most of the operators since they - // are extremely fast. However, some of the long-running operators - // (for example, sorter) are still responsible for doing the - // cancellation check on their own while performing long operations. - result.Root = colexecutils.NewCancelChecker(result.Root) - result.ColumnTypes = scanOp.ResultTypes - result.ToClose = append(result.ToClose, scanOp) + case core.JoinReader != nil: + if err := checkNumIn(inputs, 1); err != nil { + return r, err + } + if core.JoinReader.LookupColumns != nil || !core.JoinReader.LookupExpr.Empty() { + return r, errors.AssertionFailedf("lookup join reader is unsupported in vectorized") + } + // We have to create a separate account in order for the cFetcher to + // be able to precisely track the size of its output batch. This + // memory account is "streaming" in its nature, so we create an + // unlimited one. + cFetcherMemAcc := result.createUnlimitedMemAccount( + ctx, flowCtx, "cfetcher" /* opName */, spec.ProcessorID, + ) + semaCtx := flowCtx.TypeResolverFactory.NewSemaContext(evalCtx.Txn) + inputTypes := make([]*types.T, len(spec.Input[0].ColumnTypes)) + copy(inputTypes, spec.Input[0].ColumnTypes) + indexJoinOp, err := colfetcher.NewColIndexJoin( + ctx, streamingAllocator, colmem.NewAllocator(ctx, cFetcherMemAcc, factory), + flowCtx, evalCtx, semaCtx, inputs[0].Root, core.JoinReader, post, inputTypes) + if err != nil { + return r, err + } + result.finishScanPlanning(indexJoinOp, indexJoinOp.ResultTypes) case core.Filterer != nil: if err := checkNumIn(inputs, 1); err != nil { @@ -1752,6 +1777,26 @@ func (r opResult) finishBufferedWindowerArgs( } } +func (r opResult) finishScanPlanning(op colfetcher.ScanOperator, resultTypes []*types.T) { + r.Root = op + if util.CrdbTestBuild { + r.Root = colexec.NewInvariantsChecker(r.Root) + } + r.KVReader = op + r.MetadataSources = append(r.MetadataSources, r.Root.(colexecop.MetadataSource)) + r.Releasables = append(r.Releasables, op) + + // We want to check for cancellation once per input batch, and + // wrapping only colBatchScan with a CancelChecker allows us to do + // just that. It's sufficient for most of the operators since they + // are extremely fast. However, some of the long-running operators + // (for example, sorter) are still responsible for doing the + // cancellation check on their own while performing long operations. + r.Root = colexecutils.NewCancelChecker(r.Root) + r.ColumnTypes = resultTypes + r.ToClose = append(r.ToClose, op) +} + // planFilterExpr creates all operators to implement filter expression. func planFilterExpr( ctx context.Context, diff --git a/pkg/sql/colexec/colexecspan/BUILD.bazel b/pkg/sql/colexec/colexecspan/BUILD.bazel new file mode 100644 index 000000000000..a3ea026fe7e9 --- /dev/null +++ b/pkg/sql/colexec/colexecspan/BUILD.bazel @@ -0,0 +1,83 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") + +go_library( + name = "colexecspan", + srcs = [ + ":gen-exec", # keep + ], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecspan", # keep + visibility = ["//visibility:public"], + deps = [ + "//pkg/col/coldata", # keep + "//pkg/col/coldataext", # keep + "//pkg/col/typeconv", # keep + "//pkg/keys", # keep + "//pkg/roachpb:with-mocks", # keep + "//pkg/sql/catalog", # keep + "//pkg/sql/catalog/descpb", # keep + "//pkg/sql/colexecerror", # keep + "//pkg/sql/colmem", # keep + "//pkg/sql/execinfra", # keep + "//pkg/sql/rowenc", # keep + "//pkg/sql/sem/tree", # keep + "//pkg/sql/types", # keep + "//pkg/util", # keep + "//pkg/util/duration", # keep + "//pkg/util/encoding", # keep + "@com_github_cockroachdb_apd_v2//:apd", # keep + "@com_github_cockroachdb_errors//:errors", # keep + ], +) + +go_test( + name = "colexecspan_test", + srcs = [ + "dep_test.go", + "main_test.go", + "span_assembler_test.go", + ], + embed = [":colexecspan"], # keep + deps = [ + "//pkg/col/coldata", + "//pkg/col/coldataext", + "//pkg/col/coldatatestutils", + "//pkg/keys", + "//pkg/roachpb:with-mocks", + "//pkg/security", + "//pkg/settings/cluster", + "//pkg/sql/catalog", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/tabledesc", + "//pkg/sql/colconv", + "//pkg/sql/colexec/colexectestutils", + "//pkg/sql/colexecerror", + "//pkg/sql/colmem", + "//pkg/sql/execinfra", + "//pkg/sql/rowenc", + "//pkg/sql/sem/tree", + "//pkg/sql/span", + "//pkg/sql/types", + "//pkg/testutils/buildutil", + "//pkg/testutils/skip", + "//pkg/util", + "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/randutil", + ], +) + +# Map between target name and relevant template. +targets = [ + ("span_assembler.eg.go", "span_assembler_tmpl.go"), + ("span_encoder.eg.go", "span_encoder_tmpl.go"), +] + +# Define a file group for all the .eg.go targets. +eg_go_filegroup( + name = "gen-exec", + targets = targets, +) + +# Define gen rules for individual eg.go files. +gen_eg_go_rules(targets) diff --git a/pkg/sql/colexec/colexecspan/dep_test.go b/pkg/sql/colexec/colexecspan/dep_test.go new file mode 100644 index 000000000000..4d979668bbf1 --- /dev/null +++ b/pkg/sql/colexec/colexecspan/dep_test.go @@ -0,0 +1,31 @@ +// Copyright 2021 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 colexecspan + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" +) + +func TestNoLinkForbidden(t *testing.T) { + buildutil.VerifyNoImports(t, + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecspan", true, + []string{ + "github.com/cockroachdb/cockroach/pkg/sql/colexec", + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg", + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash", + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecjoin", + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj", + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecsel", + }, nil, + ) +} diff --git a/pkg/sql/colexec/colexecspan/main_test.go b/pkg/sql/colexec/colexecspan/main_test.go new file mode 100644 index 000000000000..ab947d46695b --- /dev/null +++ b/pkg/sql/colexec/colexecspan/main_test.go @@ -0,0 +1,40 @@ +// Copyright 2021 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 colexecspan + +import ( + "flag" + "fmt" + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func TestMain(m *testing.M) { + randutil.SeedForTests() + os.Exit(func() int { + flag.Parse() + if !skip.UnderBench() { + // (If we're running benchmarks, don't set a random batch size.) + randomBatchSize := colexectestutils.GenerateBatchSize() + fmt.Printf("coldata.BatchSize() is set to %d\n", randomBatchSize) + if err := coldata.SetBatchSizeForTests(randomBatchSize); err != nil { + colexecerror.InternalError(err) + } + } + return m.Run() + }()) +} diff --git a/pkg/sql/colexec/colexecspan/span_assembler.eg.go b/pkg/sql/colexec/colexecspan/span_assembler.eg.go new file mode 100644 index 000000000000..93f8fb04431c --- /dev/null +++ b/pkg/sql/colexec/colexecspan/span_assembler.eg.go @@ -0,0 +1,361 @@ +// Code generated by execgen; DO NOT EDIT. +// Copyright 2021 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 colexecspan + +import ( + "sync" + "unsafe" + + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/encoding" +) + +// NewColSpanAssembler returns a ColSpanAssembler operator that is able to +// generate lookup spans from input batches. +func NewColSpanAssembler( + codec keys.SQLCodec, + allocator *colmem.Allocator, + table catalog.TableDescriptor, + index catalog.Index, + inputTypes []*types.T, + neededCols util.FastIntSet, +) ColSpanAssembler { + base := spanAssemblerPool.Get().(*spanAssemblerBase) + base.colFamStartKeys, base.colFamEndKeys = getColFamilyEncodings(neededCols, table, index) + keyPrefix := rowenc.MakeIndexKeyPrefix(codec, table, index.GetID()) + base.scratchKey = append(base.scratchKey[:0], keyPrefix...) + base.prefixLength = len(keyPrefix) + base.allocator = allocator + + // Add span encoders to encode each primary key column as bytes. The + // ColSpanAssembler will later append these together to form valid spans. + for i := 0; i < index.NumKeyColumns(); i++ { + asc := index.GetKeyColumnDirection(i) == descpb.IndexDescriptor_ASC + base.spanEncoders = append(base.spanEncoders, newSpanEncoder(allocator, inputTypes[i], asc, i)) + } + if cap(base.spanCols) < len(base.spanEncoders) { + base.spanCols = make([]*coldata.Bytes, len(base.spanEncoders)) + } else { + base.spanCols = base.spanCols[:len(base.spanEncoders)] + } + + // Account for the memory currently in use. + usedMem := int64(cap(base.spans) * int(spanSize)) + base.allocator.AdjustMemoryUsage(usedMem) + + if len(base.colFamStartKeys) == 0 { + return &spanAssemblerNoColFamily{spanAssemblerBase: *base} + } + return &spanAssemblerWithColFamily{spanAssemblerBase: *base} +} + +var spanAssemblerPool = sync.Pool{ + New: func() interface{} { + return &spanAssemblerBase{} + }, +} + +// ColSpanAssembler is a utility operator that generates a series of spans from +// input batches which can be used to perform an index join. +type ColSpanAssembler interface { + execinfra.Releasable + + // ConsumeBatch generates lookup spans from input batches and stores them to + // later be returned by GetSpans. Spans are generated only for rows in the + // range [startIdx, endIdx). If startIdx >= endIdx, ConsumeBatch will perform + // no work. + ConsumeBatch(batch coldata.Batch, startIdx, endIdx int) + + // GetSpans returns the set of spans that have been generated so far. The + // returned Spans object is still owned by the SpanAssembler, so subsequent + // calls to GetSpans will invalidate the spans returned by previous calls. A + // caller that wishes to hold on to spans over the course of multiple calls + // should perform a shallow copy of the Spans. GetSpans will return an empty + // slice if it is called before ConsumeBatch. + GetSpans() roachpb.Spans + + // Close closes the ColSpanAssembler operator. + Close() +} + +// spanAssemblerBase extracts common fields between the SpanAssembler operators. +type spanAssemblerBase struct { + allocator *colmem.Allocator + + // keyBytes tracks the number of bytes that have been allocated for the span + // keys since the last call to GetSpans. It is reset each time GetSpans is + // called, since the SpanAssembler operator no longer owns the memory. + keyBytes int + + // spans is the list of spans that have been assembled so far. spans is owned + // and reset upon each call to GetSpans by the SpanAssembler operator. + spans roachpb.Spans + + // scratchKey is a scratch space used to append the key prefix and the key + // column encodings. It is reused for each span, and always contains at least + // the key prefix. + scratchKey roachpb.Key + + // prefixLength is the length in bytes of the key prefix. + prefixLength int + + // spanEncoders is an ordered list of utility operators that encode each key + // column in vectorized fashion. + spanEncoders []spanEncoder + + // spanCols is used to iterate through the input columns that contain the + // key encodings during span construction. + spanCols []*coldata.Bytes + + // colFamStartKeys and colFamEndKeys is the list of start and end key suffixes + // for the column families that should be scanned. The spans will be split to + // scan over each family individually. Note that it is not necessarily + // possible to break a span into family scans. + colFamStartKeys, colFamEndKeys []roachpb.Key +} + +type spanAssemblerNoColFamily struct { + spanAssemblerBase +} + +var _ ColSpanAssembler = &spanAssemblerNoColFamily{} + +// ConsumeBatch implements the ColSpanAssembler interface. +func (op *spanAssemblerNoColFamily) ConsumeBatch(batch coldata.Batch, startIdx, endIdx int) { + if startIdx >= endIdx { + return + } + + for i := range op.spanEncoders { + op.spanCols[i] = op.spanEncoders[i].next(batch, startIdx, endIdx) + } + + oldKeyBytes := op.keyBytes + oldSpansCap := cap(op.spans) + for i := 0; i < (endIdx - startIdx); i++ { + op.scratchKey = op.scratchKey[:op.prefixLength] + for j := range op.spanCols { + // The encoding for each primary key column has previously been + // calculated and stored in an input column. + op.scratchKey = append(op.scratchKey, op.spanCols[j].Get(i)...) + } + { + // The spans cannot be split into column family spans, so there will be + // exactly one span for each input row. + var span roachpb.Span + span.Key = make(roachpb.Key, 0, len(op.scratchKey)) + span.Key = append(span.Key, op.scratchKey...) + op.keyBytes += len(span.Key) + span.EndKey = make(roachpb.Key, 0, len(op.scratchKey)+1) + span.EndKey = append(span.EndKey, op.scratchKey...) + // TODO(drewk): change this to use PrefixEnd() when interleaved indexes are + // permanently removed. Keep it this way for now to allow testing + // against the row engine, even though the vectorized index joiner doesn't + // allow interleaved indexes. + span.EndKey = encoding.EncodeInterleavedSentinel(span.EndKey) + op.keyBytes += len(span.EndKey) + op.spans = append(op.spans, span) + } + } + + // Account for the memory allocated for the span slice and keys. + keyBytesMem := op.keyBytes - oldKeyBytes + spanSliceMem := (cap(op.spans) - oldSpansCap) * int(spanSize) + op.allocator.AdjustMemoryUsage(int64(spanSliceMem + keyBytesMem)) +} + +type spanAssemblerWithColFamily struct { + spanAssemblerBase +} + +var _ ColSpanAssembler = &spanAssemblerWithColFamily{} + +// ConsumeBatch implements the ColSpanAssembler interface. +func (op *spanAssemblerWithColFamily) ConsumeBatch(batch coldata.Batch, startIdx, endIdx int) { + if startIdx >= endIdx { + return + } + + for i := range op.spanEncoders { + op.spanCols[i] = op.spanEncoders[i].next(batch, startIdx, endIdx) + } + + oldKeyBytes := op.keyBytes + oldSpansCap := cap(op.spans) + for i := 0; i < (endIdx - startIdx); i++ { + op.scratchKey = op.scratchKey[:op.prefixLength] + for j := range op.spanCols { + // The encoding for each primary key column has previously been + // calculated and stored in an input column. + op.scratchKey = append(op.scratchKey, op.spanCols[j].Get(i)...) + } + { + // The span for each row can be split into a series of column family spans, + // which have the column family ID as a suffix. Individual column family + // spans can be served as Get requests, which are more efficient than Scan + // requests. + for j := range op.colFamStartKeys { + var span roachpb.Span + span.Key = make(roachpb.Key, 0, len(op.scratchKey)+len(op.colFamStartKeys[j])) + span.Key = append(span.Key, op.scratchKey...) + span.Key = append(span.Key, op.colFamStartKeys[j]...) + op.keyBytes += len(span.Key) + // The end key may be nil, in which case the span is a point lookup. + if len(op.colFamEndKeys[j]) > 0 { + span.EndKey = make(roachpb.Key, 0, len(op.scratchKey)+len(op.colFamEndKeys[j])) + span.EndKey = append(span.EndKey, op.scratchKey...) + span.EndKey = append(span.EndKey, op.colFamEndKeys[j]...) + op.keyBytes += len(span.EndKey) + } + op.spans = append(op.spans, span) + } + } + } + + // Account for the memory allocated for the span slice and keys. + keyBytesMem := op.keyBytes - oldKeyBytes + spanSliceMem := (cap(op.spans) - oldSpansCap) * int(spanSize) + op.allocator.AdjustMemoryUsage(int64(spanSliceMem + keyBytesMem)) +} + +const spanSize = unsafe.Sizeof(roachpb.Span{}) + +// GetSpans implements the ColSpanAssembler interface. +func (b *spanAssemblerBase) GetSpans() roachpb.Spans { + // Even though the memory allocated for the span keys probably can't be GC'd + // yet, we release now because the memory will be owned by the caller. + b.allocator.ReleaseMemory(int64(b.keyBytes)) + b.keyBytes = 0 + spans := b.spans + b.spans = b.spans[:0] + return spans +} + +// Close implements the ColSpanAssembler interface. +func (b *spanAssemblerBase) Close() { + for i := range b.spanEncoders { + b.spanEncoders[i].close() + } +} + +// Release implements the ColSpanAssembler interface. +// TODO(drewk): we account for the memory that is owned by the SpanAssembler +// operator, but release it once the SpanAssembler loses its references even +// though it can still be referenced elsewhere. The two cases are the spans +// slice (it is put into a pool) and the underlying bytes for the keys (they are +// referenced by the kv fetcher code). Ideally we would hand off memory +// accounting to whoever ends up owning the memory, instead of just no longer +// tracking it. +func (b *spanAssemblerBase) Release() { + for i := range b.spanCols { + // Release references to input columns. + b.spanCols[i] = nil + } + for i := range b.spanEncoders { + // Release references to input operators. + b.spanEncoders[i] = nil + } + b.spans = b.spans[:cap(b.spans)] + for i := range b.spans { + // Deeply reset all spans that were initialized during execution. + b.spans[i] = roachpb.Span{} + } + *b = spanAssemblerBase{ + spans: b.spans[:0], + spanEncoders: b.spanEncoders[:0], + spanCols: b.spanCols[:0], + scratchKey: b.scratchKey[:0], + } + spanAssemblerPool.Put(b) +} + +// execgen:inline +const _ = "template_constructSpans" + +// getColFamilyEncodings returns two lists of keys of the same length. Each pair +// of keys at the same index corresponds to the suffixes of the start and end +// keys of a span over a specific column family (or adjacent column families). +// If the returned lists are empty, the spans cannot be split into separate +// family spans. +func getColFamilyEncodings( + neededCols util.FastIntSet, table catalog.TableDescriptor, index catalog.Index, +) (startKeys, endKeys []roachpb.Key) { + familyIDs := rowenc.NeededColumnFamilyIDs(neededCols, table, index) + + if !canSplitSpans(len(familyIDs), table, index) { + return nil, nil + } + + for i, familyID := range familyIDs { + var key roachpb.Key + key = keys.MakeFamilyKey(key, uint32(familyID)) + if i > 0 && familyID-1 == familyIDs[i-1] && endKeys != nil { + // This column family is adjacent to the previous one. We can merge + // the two spans into one. + endKeys[len(endKeys)-1] = key.PrefixEnd() + } else { + startKeys = append(startKeys, key) + endKeys = append(endKeys, nil) + } + } + return startKeys, endKeys +} + +// canSplitSpans returns true if the spans that will be generated by the +// SpanAssembler operator can be split into spans over individual column +// families. For index joins, either all spans can be split or none can because +// the lookup columns are never nullable (null values prevent the index key from +// being fully knowable). +func canSplitSpans(numNeededFamilies int, table catalog.TableDescriptor, index catalog.Index) bool { + // We can only split a span into separate family specific point lookups if: + // * The table is not a special system table. (System tables claim to have + // column families, but actually do not, since they're written to with + // raw KV puts in a "legacy" way.) + if table.GetID() > 0 && table.GetID() < keys.MaxReservedDescID { + return false + } + + // * The index either has just 1 family (so we'll make a GetRequest) or we + // need fewer than every column family in the table (otherwise we'd just + // make a big ScanRequest). + numFamilies := len(table.GetFamilies()) + if numFamilies > 1 && numNeededFamilies == numFamilies { + return false + } + + // Other requirements that are always satisfied by index joins, and therefore + // do not need to be checked: + // * The index is unique. + // * The index is fully constrained. + // * If we're looking at a secondary index... + // * The index constraint must not contain null, since that would cause the + // index key to not be completely knowable. + // * The index cannot be inverted. + // * The index must store some columns. + // * The index is a new enough version. + // We've passed all the conditions, and should be able to safely split this + // span into multiple column-family-specific spans. + return true +} + +// execgen:inline +const _ = "inlined_constructSpans_true" + +// execgen:inline +const _ = "inlined_constructSpans_false" diff --git a/pkg/sql/colexec/colexecspan/span_assembler_test.go b/pkg/sql/colexec/colexecspan/span_assembler_test.go new file mode 100644 index 000000000000..59c0b8baf806 --- /dev/null +++ b/pkg/sql/colexec/colexecspan/span_assembler_test.go @@ -0,0 +1,240 @@ +// Copyright 2021 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 colexecspan + +import ( + "context" + "fmt" + "reflect" + "testing" + + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldataext" + "github.com/cockroachdb/cockroach/pkg/col/coldatatestutils" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/colconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/span" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func TestSpanAssembler(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + evalCtx := tree.MakeTestingEvalContext(st) + testMemMonitor := execinfra.NewTestMemMonitor(ctx, st) + defer testMemMonitor.Stop(ctx) + nTuples := 3 * coldata.BatchSize() + memAcc := testMemMonitor.MakeBoundAccount() + testMemAcc := &memAcc + testColumnFactory := coldataext.NewExtendedColumnFactory(&evalCtx) + testAllocator := colmem.NewAllocator(ctx, testMemAcc, testColumnFactory) + defer testMemAcc.Close(ctx) + rng, _ := randutil.NewPseudoRand() + typs := []*types.T{types.Int, types.Bytes, types.Decimal} + + for _, useColFamilies := range []bool{true, false} { + name := "WithColFamilies" + if !useColFamilies { + name = "NoColFamilies" + } + t.Run(name, func(t *testing.T) { + for _, sizeLimit := range []int{ + 1, // 1 byte + 1 << 10, // 1 KB + 1 << 20, // 1 MB + } { + t.Run(fmt.Sprintf("sizeLimit=%d", sizeLimit), func(t *testing.T) { + for _, useSel := range []bool{true, false} { + t.Run(fmt.Sprintf("sel=%v", useSel), func(t *testing.T) { + probOfOmittingRow := 0.0 + if useSel { + probOfOmittingRow = 0.3 + } + sel := coldatatestutils.RandomSel(rng, coldata.BatchSize(), probOfOmittingRow) + testTable := makeTable(useColFamilies) + neededColumns := util.MakeFastIntSet(1, 2, 3, 4) + + cols := make([]coldata.Vec, len(typs)) + for i, typ := range typs { + cols[i] = testAllocator.NewMemColumn(typ, nTuples) + } + for i := range typs { + coldatatestutils.RandomVec(coldatatestutils.RandomVecArgs{ + Rand: rng, + Vec: cols[i], + N: nTuples, + NullProbability: 0, // Primary key columns are non-null. + }) + } + source := colexectestutils.NewChunkingBatchSource(testAllocator, typs, cols, nTuples) + source.Init(ctx) + oracleSource := colexectestutils.NewChunkingBatchSource(testAllocator, typs, cols, nTuples) + oracleSource.Init(ctx) + converter := colconv.NewAllVecToDatumConverter(len(typs)) + + builder := span.MakeBuilder(&evalCtx, keys.TODOSQLCodec, testTable, testTable.GetPrimaryIndex()) + builder.SetNeededColumns(neededColumns) + + colBuilder := NewColSpanAssembler( + keys.TODOSQLCodec, testAllocator, testTable, + testTable.GetPrimaryIndex(), typs, neededColumns, + ) + defer func() { + colBuilder.Close() + colBuilder.Release() + }() + + var testSpans roachpb.Spans + for batch := source.Next(); ; batch = source.Next() { + if batch.Length() == 0 { + // Reached the end of the input. + testSpans = append(testSpans, colBuilder.GetSpans()...) + break + } + if useSel { + batch.SetSelection(true) + copy(batch.Selection(), sel) + batch.SetLength(len(sel)) + } + colBuilder.ConsumeBatch(batch, 0 /* startIdx */, batch.Length() /* endIdx */) + } + + var oracleSpans roachpb.Spans + for batch := oracleSource.Next(); batch.Length() > 0; batch = oracleSource.Next() { + batch.SetSelection(true) + copy(batch.Selection(), sel) + batch.SetLength(len(sel)) + converter.ConvertBatchAndDeselect(batch) + rows := make(rowenc.EncDatumRows, len(sel)) + for i := range sel { + // Note that sel contains all rows if useSel=false. + row := make(rowenc.EncDatumRow, len(typs)) + for j := range typs { + datum := converter.GetDatumColumn(j)[i] + row[j] = rowenc.DatumToEncDatum(typs[j], datum) + } + rows[i] = row + } + oracleSpans = append(oracleSpans, spanGeneratorOracle(t, builder, rows, len(typs))...) + } + + if len(oracleSpans) != len(testSpans) { + t.Fatalf("Expected %d spans, got %d.", len(oracleSpans), len(testSpans)) + } + for i := range oracleSpans { + oracleSpan := oracleSpans[i] + testSpan := testSpans[i] + if !reflect.DeepEqual(oracleSpan, testSpan) { + t.Fatalf("Span at index %d incorrect.\n\nExpected:\n%v\n\nFound:\n%v\n", + i, oracleSpan, testSpan) + } + } + }) + } + }) + } + }) + } +} + +// spanGeneratorOracle extracts the logic from joinreader_span_generator.go that +// pertains to index joins. +func spanGeneratorOracle( + t *testing.T, spanBuilder *span.Builder, rows []rowenc.EncDatumRow, lookupCols int, +) roachpb.Spans { + var spans roachpb.Spans + for _, inputRow := range rows { + generatedSpan, containsNull, err := spanBuilder.SpanFromEncDatums(inputRow, lookupCols) + if err != nil { + t.Fatal(err) + } + spans = spanBuilder.MaybeSplitSpanIntoSeparateFamilies( + spans, generatedSpan, lookupCols, containsNull) + } + return spans +} + +func makeTable(useColFamilies bool) catalog.TableDescriptor { + tableID := keys.MinNonPredefinedUserDescID + if !useColFamilies { + // We can prevent the span builder from splitting spans into separate column + // families by using a system table ID, since system tables do not have + // column families. + tableID = keys.SystemDatabaseID + } + + var testTableDesc = descpb.TableDescriptor{ + Name: "abcd", + ID: descpb.ID(tableID), + Privileges: descpb.NewDefaultPrivilegeDescriptor(security.AdminRoleName()), + Version: 1, + Columns: []descpb.ColumnDescriptor{ + {Name: "a", ID: 1, Type: types.Int}, + {Name: "b", ID: 2, Type: types.Bytes}, + {Name: "c", ID: 3, Type: types.Decimal}, + {Name: "d", ID: 4, Type: types.Int}, + }, + NextColumnID: 5, + Families: []descpb.ColumnFamilyDescriptor{ + {Name: "primary", ID: 0, ColumnNames: []string{"a", "b", "d"}, ColumnIDs: []descpb.ColumnID{1, 2, 4}}, + {Name: "secondary", ID: 1, ColumnNames: []string{"c"}, ColumnIDs: []descpb.ColumnID{3}}, + }, + NextFamilyID: 2, + PrimaryIndex: descpb.IndexDescriptor{ + Name: "primary", + ID: 1, + Unique: true, + KeyColumnNames: []string{"a", "b", "c"}, + KeyColumnDirections: []descpb.IndexDescriptor_Direction{descpb.IndexDescriptor_ASC, descpb.IndexDescriptor_ASC, descpb.IndexDescriptor_ASC}, + KeyColumnIDs: []descpb.ColumnID{1, 2, 3}, + }, + Indexes: []descpb.IndexDescriptor{ + { // Secondary index omits column 'd'. + Name: "secondary", + ID: 2, + Unique: true, + KeyColumnNames: []string{"c", "a", "b"}, + KeyColumnDirections: []descpb.IndexDescriptor_Direction{descpb.IndexDescriptor_ASC, descpb.IndexDescriptor_ASC, descpb.IndexDescriptor_ASC}, + KeyColumnIDs: []descpb.ColumnID{3, 1, 2}, + KeySuffixColumnIDs: []descpb.ColumnID{1, 2}, + }, + }, + NextIndexID: 3, + FormatVersion: descpb.FamilyFormatVersion, + NextMutationID: 1, + } + + ctx := context.Background() + b := tabledesc.NewBuilder(&testTableDesc) + err := b.RunPostDeserializationChanges(ctx, nil /* DescGetter */) + if err != nil { + log.Fatalf(ctx, "Error when building descriptor of system table %q: %s", testTableDesc.Name, err) + } + return b.BuildImmutableTable() +} diff --git a/pkg/sql/colexec/colexecspan/span_assembler_tmpl.go b/pkg/sql/colexec/colexecspan/span_assembler_tmpl.go new file mode 100644 index 000000000000..8f364cc479f1 --- /dev/null +++ b/pkg/sql/colexec/colexecspan/span_assembler_tmpl.go @@ -0,0 +1,344 @@ +// Copyright 2021 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. + +// {{/* +// +build execgen_template +// +// This file is the execgen template for span_assembler.eg.go. It's formatted in +// a special way, so it's both valid Go and a valid text/template input. This +// permits editing this file with editor support. +// +// */}} + +package colexecspan + +import ( + "sync" + "unsafe" + + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/encoding" +) + +// NewColSpanAssembler returns a ColSpanAssembler operator that is able to +// generate lookup spans from input batches. +func NewColSpanAssembler( + codec keys.SQLCodec, + allocator *colmem.Allocator, + table catalog.TableDescriptor, + index catalog.Index, + inputTypes []*types.T, + neededCols util.FastIntSet, +) ColSpanAssembler { + base := spanAssemblerPool.Get().(*spanAssemblerBase) + base.colFamStartKeys, base.colFamEndKeys = getColFamilyEncodings(neededCols, table, index) + keyPrefix := rowenc.MakeIndexKeyPrefix(codec, table, index.GetID()) + base.scratchKey = append(base.scratchKey[:0], keyPrefix...) + base.prefixLength = len(keyPrefix) + base.allocator = allocator + + // Add span encoders to encode each primary key column as bytes. The + // ColSpanAssembler will later append these together to form valid spans. + for i := 0; i < index.NumKeyColumns(); i++ { + asc := index.GetKeyColumnDirection(i) == descpb.IndexDescriptor_ASC + base.spanEncoders = append(base.spanEncoders, newSpanEncoder(allocator, inputTypes[i], asc, i)) + } + if cap(base.spanCols) < len(base.spanEncoders) { + base.spanCols = make([]*coldata.Bytes, len(base.spanEncoders)) + } else { + base.spanCols = base.spanCols[:len(base.spanEncoders)] + } + + // Account for the memory currently in use. + usedMem := int64(cap(base.spans) * int(spanSize)) + base.allocator.AdjustMemoryUsage(usedMem) + + if len(base.colFamStartKeys) == 0 { + return &spanAssemblerNoColFamily{spanAssemblerBase: *base} + } + return &spanAssemblerWithColFamily{spanAssemblerBase: *base} +} + +var spanAssemblerPool = sync.Pool{ + New: func() interface{} { + return &spanAssemblerBase{} + }, +} + +// ColSpanAssembler is a utility operator that generates a series of spans from +// input batches which can be used to perform an index join. +type ColSpanAssembler interface { + execinfra.Releasable + + // ConsumeBatch generates lookup spans from input batches and stores them to + // later be returned by GetSpans. Spans are generated only for rows in the + // range [startIdx, endIdx). If startIdx >= endIdx, ConsumeBatch will perform + // no work. + ConsumeBatch(batch coldata.Batch, startIdx, endIdx int) + + // GetSpans returns the set of spans that have been generated so far. The + // returned Spans object is still owned by the SpanAssembler, so subsequent + // calls to GetSpans will invalidate the spans returned by previous calls. A + // caller that wishes to hold on to spans over the course of multiple calls + // should perform a shallow copy of the Spans. GetSpans will return an empty + // slice if it is called before ConsumeBatch. + GetSpans() roachpb.Spans + + // Close closes the ColSpanAssembler operator. + Close() +} + +// spanAssemblerBase extracts common fields between the SpanAssembler operators. +type spanAssemblerBase struct { + allocator *colmem.Allocator + + // keyBytes tracks the number of bytes that have been allocated for the span + // keys since the last call to GetSpans. It is reset each time GetSpans is + // called, since the SpanAssembler operator no longer owns the memory. + keyBytes int + + // spans is the list of spans that have been assembled so far. spans is owned + // and reset upon each call to GetSpans by the SpanAssembler operator. + spans roachpb.Spans + + // scratchKey is a scratch space used to append the key prefix and the key + // column encodings. It is reused for each span, and always contains at least + // the key prefix. + scratchKey roachpb.Key + + // prefixLength is the length in bytes of the key prefix. + prefixLength int + + // spanEncoders is an ordered list of utility operators that encode each key + // column in vectorized fashion. + spanEncoders []spanEncoder + + // spanCols is used to iterate through the input columns that contain the + // key encodings during span construction. + spanCols []*coldata.Bytes + + // colFamStartKeys and colFamEndKeys is the list of start and end key suffixes + // for the column families that should be scanned. The spans will be split to + // scan over each family individually. Note that it is not necessarily + // possible to break a span into family scans. + colFamStartKeys, colFamEndKeys []roachpb.Key +} + +// {{range .}} + +type _OP_STRING struct { + spanAssemblerBase +} + +var _ ColSpanAssembler = &_OP_STRING{} + +// ConsumeBatch implements the ColSpanAssembler interface. +func (op *_OP_STRING) ConsumeBatch(batch coldata.Batch, startIdx, endIdx int) { + if startIdx >= endIdx { + return + } + + for i := range op.spanEncoders { + op.spanCols[i] = op.spanEncoders[i].next(batch, startIdx, endIdx) + } + + oldKeyBytes := op.keyBytes + oldSpansCap := cap(op.spans) + for i := 0; i < (endIdx - startIdx); i++ { + op.scratchKey = op.scratchKey[:op.prefixLength] + for j := range op.spanCols { + // The encoding for each primary key column has previously been + // calculated and stored in an input column. + op.scratchKey = append(op.scratchKey, op.spanCols[j].Get(i)...) + } + // {{if .WithColFamilies}} + constructSpans(true) + // {{else}} + constructSpans(false) + // {{end}} + } + + // Account for the memory allocated for the span slice and keys. + keyBytesMem := op.keyBytes - oldKeyBytes + spanSliceMem := (cap(op.spans) - oldSpansCap) * int(spanSize) + op.allocator.AdjustMemoryUsage(int64(spanSliceMem + keyBytesMem)) +} + +// {{end}} + +const spanSize = unsafe.Sizeof(roachpb.Span{}) + +// GetSpans implements the ColSpanAssembler interface. +func (b *spanAssemblerBase) GetSpans() roachpb.Spans { + // Even though the memory allocated for the span keys probably can't be GC'd + // yet, we release now because the memory will be owned by the caller. + b.allocator.ReleaseMemory(int64(b.keyBytes)) + b.keyBytes = 0 + spans := b.spans + b.spans = b.spans[:0] + return spans +} + +// Close implements the ColSpanAssembler interface. +func (b *spanAssemblerBase) Close() { + for i := range b.spanEncoders { + b.spanEncoders[i].close() + } +} + +// Release implements the ColSpanAssembler interface. +// TODO(drewk): we account for the memory that is owned by the SpanAssembler +// operator, but release it once the SpanAssembler loses its references even +// though it can still be referenced elsewhere. The two cases are the spans +// slice (it is put into a pool) and the underlying bytes for the keys (they are +// referenced by the kv fetcher code). Ideally we would hand off memory +// accounting to whoever ends up owning the memory, instead of just no longer +// tracking it. +func (b *spanAssemblerBase) Release() { + for i := range b.spanCols { + // Release references to input columns. + b.spanCols[i] = nil + } + for i := range b.spanEncoders { + // Release references to input operators. + b.spanEncoders[i] = nil + } + b.spans = b.spans[:cap(b.spans)] + for i := range b.spans { + // Deeply reset all spans that were initialized during execution. + b.spans[i] = roachpb.Span{} + } + *b = spanAssemblerBase{ + spans: b.spans[:0], + spanEncoders: b.spanEncoders[:0], + spanCols: b.spanCols[:0], + scratchKey: b.scratchKey[:0], + } + spanAssemblerPool.Put(b) +} + +// execgen:inline +// execgen:template +func constructSpans(hasFamilies bool) { + if hasFamilies { + // The span for each row can be split into a series of column family spans, + // which have the column family ID as a suffix. Individual column family + // spans can be served as Get requests, which are more efficient than Scan + // requests. + for j := range op.colFamStartKeys { + var span roachpb.Span + span.Key = make(roachpb.Key, 0, len(op.scratchKey)+len(op.colFamStartKeys[j])) + span.Key = append(span.Key, op.scratchKey...) + span.Key = append(span.Key, op.colFamStartKeys[j]...) + op.keyBytes += len(span.Key) + // The end key may be nil, in which case the span is a point lookup. + if len(op.colFamEndKeys[j]) > 0 { + span.EndKey = make(roachpb.Key, 0, len(op.scratchKey)+len(op.colFamEndKeys[j])) + span.EndKey = append(span.EndKey, op.scratchKey...) + span.EndKey = append(span.EndKey, op.colFamEndKeys[j]...) + op.keyBytes += len(span.EndKey) + } + op.spans = append(op.spans, span) + } + } else { + // The spans cannot be split into column family spans, so there will be + // exactly one span for each input row. + var span roachpb.Span + span.Key = make(roachpb.Key, 0, len(op.scratchKey)) + span.Key = append(span.Key, op.scratchKey...) + op.keyBytes += len(span.Key) + span.EndKey = make(roachpb.Key, 0, len(op.scratchKey)+1) + span.EndKey = append(span.EndKey, op.scratchKey...) + // TODO(drewk): change this to use PrefixEnd() when interleaved indexes are + // permanently removed. Keep it this way for now to allow testing + // against the row engine, even though the vectorized index joiner doesn't + // allow interleaved indexes. + span.EndKey = encoding.EncodeInterleavedSentinel(span.EndKey) + op.keyBytes += len(span.EndKey) + op.spans = append(op.spans, span) + } +} + +// getColFamilyEncodings returns two lists of keys of the same length. Each pair +// of keys at the same index corresponds to the suffixes of the start and end +// keys of a span over a specific column family (or adjacent column families). +// If the returned lists are empty, the spans cannot be split into separate +// family spans. +func getColFamilyEncodings( + neededCols util.FastIntSet, table catalog.TableDescriptor, index catalog.Index, +) (startKeys, endKeys []roachpb.Key) { + familyIDs := rowenc.NeededColumnFamilyIDs(neededCols, table, index) + + if !canSplitSpans(len(familyIDs), table, index) { + return nil, nil + } + + for i, familyID := range familyIDs { + var key roachpb.Key + key = keys.MakeFamilyKey(key, uint32(familyID)) + if i > 0 && familyID-1 == familyIDs[i-1] && endKeys != nil { + // This column family is adjacent to the previous one. We can merge + // the two spans into one. + endKeys[len(endKeys)-1] = key.PrefixEnd() + } else { + startKeys = append(startKeys, key) + endKeys = append(endKeys, nil) + } + } + return startKeys, endKeys +} + +// canSplitSpans returns true if the spans that will be generated by the +// SpanAssembler operator can be split into spans over individual column +// families. For index joins, either all spans can be split or none can because +// the lookup columns are never nullable (null values prevent the index key from +// being fully knowable). +func canSplitSpans(numNeededFamilies int, table catalog.TableDescriptor, index catalog.Index) bool { + // We can only split a span into separate family specific point lookups if: + // + // * The table is not a special system table. (System tables claim to have + // column families, but actually do not, since they're written to with + // raw KV puts in a "legacy" way.) + if table.GetID() > 0 && table.GetID() < keys.MaxReservedDescID { + return false + } + + // * The index either has just 1 family (so we'll make a GetRequest) or we + // need fewer than every column family in the table (otherwise we'd just + // make a big ScanRequest). + numFamilies := len(table.GetFamilies()) + if numFamilies > 1 && numNeededFamilies == numFamilies { + return false + } + + // Other requirements that are always satisfied by index joins, and therefore + // do not need to be checked: + // * The index is unique. + // * The index is fully constrained. + // * If we're looking at a secondary index... + // * The index constraint must not contain null, since that would cause the + // index key to not be completely knowable. + // * The index cannot be inverted. + // * The index must store some columns. + // * The index is a new enough version. + // + // We've passed all the conditions, and should be able to safely split this + // span into multiple column-family-specific spans. + return true +} diff --git a/pkg/sql/colexec/colexecspan/span_encoder.eg.go b/pkg/sql/colexec/colexecspan/span_encoder.eg.go new file mode 100644 index 000000000000..0f88dc9c8279 --- /dev/null +++ b/pkg/sql/colexec/colexecspan/span_encoder.eg.go @@ -0,0 +1,1904 @@ +// Code generated by execgen; DO NOT EDIT. +// Copyright 2021 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 colexecspan + +import ( + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/cockroachdb/errors" +) + +// Workaround for bazel auto-generated code. goimports does not automatically +// pick up the right packages when run within the bazel sandbox. +var ( + _ = rowenc.EncodeTableKey + _ tree.Datum +) + +// newSpanEncoder creates a new utility operator that, given input batches, +// generates the encoding for the given key column. It is used by SpanAssembler +// operators to generate spans for index joins and lookup joins. +func newSpanEncoder( + allocator *colmem.Allocator, typ *types.T, asc bool, encodeColIdx int, +) spanEncoder { + base := spanEncoderBase{ + allocator: allocator, + encodeColIdx: encodeColIdx, + } + switch asc { + case true: + switch typeconv.TypeFamilyToCanonicalTypeFamily(typ.Family()) { + case types.BoolFamily: + switch typ.Width() { + case -1: + default: + return &spanEncoderBoolAsc{spanEncoderBase: base} + } + case types.BytesFamily: + switch typ.Width() { + case -1: + default: + return &spanEncoderBytesAsc{spanEncoderBase: base} + } + case types.DecimalFamily: + switch typ.Width() { + case -1: + default: + return &spanEncoderDecimalAsc{spanEncoderBase: base} + } + case types.IntFamily: + switch typ.Width() { + case 16: + return &spanEncoderInt16Asc{spanEncoderBase: base} + case 32: + return &spanEncoderInt32Asc{spanEncoderBase: base} + case -1: + default: + return &spanEncoderInt64Asc{spanEncoderBase: base} + } + case types.FloatFamily: + switch typ.Width() { + case -1: + default: + return &spanEncoderFloat64Asc{spanEncoderBase: base} + } + case types.TimestampTZFamily: + switch typ.Width() { + case -1: + default: + return &spanEncoderTimestampAsc{spanEncoderBase: base} + } + case types.IntervalFamily: + switch typ.Width() { + case -1: + default: + return &spanEncoderIntervalAsc{spanEncoderBase: base} + } + case typeconv.DatumVecCanonicalTypeFamily: + switch typ.Width() { + case -1: + default: + return &spanEncoderDatumAsc{spanEncoderBase: base} + } + } + case false: + switch typeconv.TypeFamilyToCanonicalTypeFamily(typ.Family()) { + case types.BoolFamily: + switch typ.Width() { + case -1: + default: + return &spanEncoderBoolDesc{spanEncoderBase: base} + } + case types.BytesFamily: + switch typ.Width() { + case -1: + default: + return &spanEncoderBytesDesc{spanEncoderBase: base} + } + case types.DecimalFamily: + switch typ.Width() { + case -1: + default: + return &spanEncoderDecimalDesc{spanEncoderBase: base} + } + case types.IntFamily: + switch typ.Width() { + case 16: + return &spanEncoderInt16Desc{spanEncoderBase: base} + case 32: + return &spanEncoderInt32Desc{spanEncoderBase: base} + case -1: + default: + return &spanEncoderInt64Desc{spanEncoderBase: base} + } + case types.FloatFamily: + switch typ.Width() { + case -1: + default: + return &spanEncoderFloat64Desc{spanEncoderBase: base} + } + case types.TimestampTZFamily: + switch typ.Width() { + case -1: + default: + return &spanEncoderTimestampDesc{spanEncoderBase: base} + } + case types.IntervalFamily: + switch typ.Width() { + case -1: + default: + return &spanEncoderIntervalDesc{spanEncoderBase: base} + } + case typeconv.DatumVecCanonicalTypeFamily: + switch typ.Width() { + case -1: + default: + return &spanEncoderDatumDesc{spanEncoderBase: base} + } + } + } + colexecerror.InternalError(errors.AssertionFailedf("unsupported span encoder type %s", typ.Name())) + return nil +} + +type spanEncoder interface { + // next generates the encoding for the current key column for each row from + // the given batch in the range [startIdx, endIdx), then returns each row's + // encoding as a value in a Bytes column. The returned Bytes column is owned + // by the spanEncoder operator and should not be modified. Calling next + // invalidates previous calls to next. next assumes that startIdx and endIdx + // constitute a valid range of the given batch. + next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes + + close() +} + +type spanEncoderBase struct { + allocator *colmem.Allocator + encodeColIdx int + + // outputBytes contains the encoding for each row of the key column. It is + // reused between calls to next(). + outputBytes *coldata.Bytes + + // A scratch bytes slice used to hold each encoding before it is appended to + // the output column. It is reused to avoid allocating for every row. + scratch []byte +} + +type spanEncoderBoolAsc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderBoolAsc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderBoolAsc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Bool() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + val := col.Get(i) + + var x int64 + if val { + x = 1 + } else { + x = 0 + } + op.scratch = encoding.EncodeVarintAscending(op.scratch, x) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + + var x int64 + if val { + x = 1 + } else { + x = 0 + } + op.scratch = encoding.EncodeVarintAscending(op.scratch, x) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + //gcassert:bce + val := col.Get(i) + + var x int64 + if val { + x = 1 + } else { + x = 0 + } + op.scratch = encoding.EncodeVarintAscending(op.scratch, x) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + //gcassert:bce + val := col.Get(i) + + var x int64 + if val { + x = 1 + } else { + x = 0 + } + op.scratch = encoding.EncodeVarintAscending(op.scratch, x) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderBytesAsc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderBytesAsc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderBytesAsc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Bytes() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + val := col.Get(i) + op.scratch = encoding.EncodeBytesAscending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + op.scratch = encoding.EncodeBytesAscending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + val := col.Get(i) + op.scratch = encoding.EncodeBytesAscending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + op.scratch = encoding.EncodeBytesAscending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderDecimalAsc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderDecimalAsc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderDecimalAsc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Decimal() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + val := col.Get(i) + op.scratch = encoding.EncodeDecimalAscending(op.scratch, &val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + op.scratch = encoding.EncodeDecimalAscending(op.scratch, &val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeDecimalAscending(op.scratch, &val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeDecimalAscending(op.scratch, &val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderInt16Asc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderInt16Asc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderInt16Asc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Int16() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + val := col.Get(i) + op.scratch = encoding.EncodeVarintAscending(op.scratch, int64(val)) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + op.scratch = encoding.EncodeVarintAscending(op.scratch, int64(val)) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeVarintAscending(op.scratch, int64(val)) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeVarintAscending(op.scratch, int64(val)) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderInt32Asc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderInt32Asc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderInt32Asc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Int32() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + val := col.Get(i) + op.scratch = encoding.EncodeVarintAscending(op.scratch, int64(val)) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + op.scratch = encoding.EncodeVarintAscending(op.scratch, int64(val)) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeVarintAscending(op.scratch, int64(val)) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeVarintAscending(op.scratch, int64(val)) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderInt64Asc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderInt64Asc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderInt64Asc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Int64() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + val := col.Get(i) + op.scratch = encoding.EncodeVarintAscending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + op.scratch = encoding.EncodeVarintAscending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeVarintAscending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeVarintAscending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderFloat64Asc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderFloat64Asc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderFloat64Asc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Float64() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + val := col.Get(i) + op.scratch = encoding.EncodeFloatAscending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + op.scratch = encoding.EncodeFloatAscending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeFloatAscending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeFloatAscending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderTimestampAsc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderTimestampAsc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderTimestampAsc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Timestamp() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + val := col.Get(i) + op.scratch = encoding.EncodeTimeAscending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + op.scratch = encoding.EncodeTimeAscending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeTimeAscending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeTimeAscending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderIntervalAsc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderIntervalAsc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderIntervalAsc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Interval() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + val := col.Get(i) + + var err error + op.scratch, err = encoding.EncodeDurationAscending(op.scratch, val) + if err != nil { + colexecerror.ExpectedError(err) + } + + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + + var err error + op.scratch, err = encoding.EncodeDurationAscending(op.scratch, val) + if err != nil { + colexecerror.ExpectedError(err) + } + + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + //gcassert:bce + val := col.Get(i) + + var err error + op.scratch, err = encoding.EncodeDurationAscending(op.scratch, val) + if err != nil { + colexecerror.ExpectedError(err) + } + + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + //gcassert:bce + val := col.Get(i) + + var err error + op.scratch, err = encoding.EncodeDurationAscending(op.scratch, val) + if err != nil { + colexecerror.ExpectedError(err) + } + + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderDatumAsc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderDatumAsc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderDatumAsc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Datum() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + val := col.Get(i) + + var err error + op.scratch, err = rowenc.EncodeTableKey(op.scratch, val.(tree.Datum), encoding.Ascending) + if err != nil { + colexecerror.ExpectedError(err) + } + + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + + var err error + op.scratch, err = rowenc.EncodeTableKey(op.scratch, val.(tree.Datum), encoding.Ascending) + if err != nil { + colexecerror.ExpectedError(err) + } + + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + continue + } + val := col.Get(i) + + var err error + op.scratch, err = rowenc.EncodeTableKey(op.scratch, val.(tree.Datum), encoding.Ascending) + if err != nil { + colexecerror.ExpectedError(err) + } + + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + + var err error + op.scratch, err = rowenc.EncodeTableKey(op.scratch, val.(tree.Datum), encoding.Ascending) + if err != nil { + colexecerror.ExpectedError(err) + } + + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderBoolDesc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderBoolDesc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderBoolDesc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Bool() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + val := col.Get(i) + + var x int64 + if val { + x = 1 + } else { + x = 0 + } + op.scratch = encoding.EncodeVarintDescending(op.scratch, x) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + + var x int64 + if val { + x = 1 + } else { + x = 0 + } + op.scratch = encoding.EncodeVarintDescending(op.scratch, x) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + //gcassert:bce + val := col.Get(i) + + var x int64 + if val { + x = 1 + } else { + x = 0 + } + op.scratch = encoding.EncodeVarintDescending(op.scratch, x) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + //gcassert:bce + val := col.Get(i) + + var x int64 + if val { + x = 1 + } else { + x = 0 + } + op.scratch = encoding.EncodeVarintDescending(op.scratch, x) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderBytesDesc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderBytesDesc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderBytesDesc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Bytes() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + val := col.Get(i) + op.scratch = encoding.EncodeBytesDescending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + op.scratch = encoding.EncodeBytesDescending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + val := col.Get(i) + op.scratch = encoding.EncodeBytesDescending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + op.scratch = encoding.EncodeBytesDescending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderDecimalDesc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderDecimalDesc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderDecimalDesc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Decimal() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + val := col.Get(i) + op.scratch = encoding.EncodeDecimalDescending(op.scratch, &val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + op.scratch = encoding.EncodeDecimalDescending(op.scratch, &val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeDecimalDescending(op.scratch, &val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeDecimalDescending(op.scratch, &val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderInt16Desc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderInt16Desc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderInt16Desc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Int16() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + val := col.Get(i) + op.scratch = encoding.EncodeVarintDescending(op.scratch, int64(val)) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + op.scratch = encoding.EncodeVarintDescending(op.scratch, int64(val)) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeVarintDescending(op.scratch, int64(val)) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeVarintDescending(op.scratch, int64(val)) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderInt32Desc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderInt32Desc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderInt32Desc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Int32() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + val := col.Get(i) + op.scratch = encoding.EncodeVarintDescending(op.scratch, int64(val)) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + op.scratch = encoding.EncodeVarintDescending(op.scratch, int64(val)) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeVarintDescending(op.scratch, int64(val)) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeVarintDescending(op.scratch, int64(val)) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderInt64Desc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderInt64Desc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderInt64Desc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Int64() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + val := col.Get(i) + op.scratch = encoding.EncodeVarintDescending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + op.scratch = encoding.EncodeVarintDescending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeVarintDescending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeVarintDescending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderFloat64Desc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderFloat64Desc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderFloat64Desc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Float64() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + val := col.Get(i) + op.scratch = encoding.EncodeFloatDescending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + op.scratch = encoding.EncodeFloatDescending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeFloatDescending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeFloatDescending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderTimestampDesc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderTimestampDesc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderTimestampDesc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Timestamp() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + val := col.Get(i) + op.scratch = encoding.EncodeTimeDescending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + op.scratch = encoding.EncodeTimeDescending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeTimeDescending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + //gcassert:bce + val := col.Get(i) + op.scratch = encoding.EncodeTimeDescending(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderIntervalDesc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderIntervalDesc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderIntervalDesc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Interval() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + val := col.Get(i) + + var err error + op.scratch, err = encoding.EncodeDurationDescending(op.scratch, val) + if err != nil { + colexecerror.ExpectedError(err) + } + + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + + var err error + op.scratch, err = encoding.EncodeDurationDescending(op.scratch, val) + if err != nil { + colexecerror.ExpectedError(err) + } + + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + //gcassert:bce + val := col.Get(i) + + var err error + op.scratch, err = encoding.EncodeDurationDescending(op.scratch, val) + if err != nil { + colexecerror.ExpectedError(err) + } + + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + //gcassert:bce + val := col.Get(i) + + var err error + op.scratch, err = encoding.EncodeDurationDescending(op.scratch, val) + if err != nil { + colexecerror.ExpectedError(err) + } + + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +type spanEncoderDatumDesc struct { + spanEncoderBase +} + +var _ spanEncoder = &spanEncoderDatumDesc{} + +// next implements the spanEncoder interface. +func (op *spanEncoderDatumDesc) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.Datum() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + val := col.Get(i) + + var err error + op.scratch, err = rowenc.EncodeTableKey(op.scratch, val.(tree.Datum), encoding.Descending) + if err != nil { + colexecerror.ExpectedError(err) + } + + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for _, i := range sel { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + + var err error + op.scratch, err = rowenc.EncodeTableKey(op.scratch, val.(tree.Datum), encoding.Descending) + if err != nil { + colexecerror.ExpectedError(err) + } + + op.outputBytes.AppendVal(op.scratch) + } + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + if nulls.NullAt(i) { + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + continue + } + val := col.Get(i) + + var err error + op.scratch, err = rowenc.EncodeTableKey(op.scratch, val.(tree.Datum), encoding.Descending) + if err != nil { + colexecerror.ExpectedError(err) + } + + op.outputBytes.AppendVal(op.scratch) + } + } + } else { + for i := startIdx; i < endIdx; i++ { + { + op.scratch = op.scratch[:0] + val := col.Get(i) + + var err error + op.scratch, err = rowenc.EncodeTableKey(op.scratch, val.(tree.Datum), encoding.Descending) + if err != nil { + colexecerror.ExpectedError(err) + } + + op.outputBytes.AppendVal(op.scratch) + } + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +// close implements the spanEncoder interface. +func (b *spanEncoderBase) close() { + *b = spanEncoderBase{} +} + +// execgen:inline +const _ = "template_encodeSpan" + +// execgen:inline +const _ = "inlined_encodeSpan_true_true" + +// execgen:inline +const _ = "inlined_encodeSpan_true_false" + +// execgen:inline +const _ = "inlined_encodeSpan_false_true" + +// execgen:inline +const _ = "inlined_encodeSpan_false_false" diff --git a/pkg/sql/colexec/colexecspan/span_encoder_tmpl.go b/pkg/sql/colexec/colexecspan/span_encoder_tmpl.go new file mode 100644 index 000000000000..c1d668ce76b5 --- /dev/null +++ b/pkg/sql/colexec/colexecspan/span_encoder_tmpl.go @@ -0,0 +1,195 @@ +// Copyright 2021 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. + +// {{/* +// +build execgen_template +// +// This file is the execgen template for span_encoder.eg.go. It's formatted in a +// special way, so it's both valid Go and a valid text/template input. This +// permits editing this file with editor support. +// +// */}} + +package colexecspan + +import ( + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/cockroachdb/errors" +) + +// Workaround for bazel auto-generated code. goimports does not automatically +// pick up the right packages when run within the bazel sandbox. +var ( + _ = rowenc.EncodeTableKey + _ tree.Datum +) + +// {{/* + +// Declarations to make the template compile properly. These are template +// variables which are replaced during code generation. +const _CANONICAL_TYPE_FAMILY = types.UnknownFamily +const _TYPE_WIDTH = 0 +const _IS_ASC = true + +// _ASSIGN_SPAN_ENCODING is a template addition function for assigning the first +// input to the result of encoding the second input. +func _ASSIGN_SPAN_ENCODING(_, _ string) { + colexecerror.InternalError(errors.AssertionFailedf("")) +} + +// */}} + +// newSpanEncoder creates a new utility operator that, given input batches, +// generates the encoding for the given key column. It is used by SpanAssembler +// operators to generate spans for index joins and lookup joins. +func newSpanEncoder( + allocator *colmem.Allocator, typ *types.T, asc bool, encodeColIdx int, +) spanEncoder { + base := spanEncoderBase{ + allocator: allocator, + encodeColIdx: encodeColIdx, + } + switch asc { + // {{range .}} + case _IS_ASC: + switch typeconv.TypeFamilyToCanonicalTypeFamily(typ.Family()) { + // {{range .TypeFamilies}} + case _CANONICAL_TYPE_FAMILY: + switch typ.Width() { + // {{range .Overloads}} + case _TYPE_WIDTH: + return &_OP_STRING{spanEncoderBase: base} + // {{end}} + } + // {{end}} + } + // {{end}} + } + colexecerror.InternalError(errors.AssertionFailedf("unsupported span encoder type %s", typ.Name())) + return nil +} + +type spanEncoder interface { + // next generates the encoding for the current key column for each row from + // the given batch in the range [startIdx, endIdx), then returns each row's + // encoding as a value in a Bytes column. The returned Bytes column is owned + // by the spanEncoder operator and should not be modified. Calling next + // invalidates previous calls to next. next assumes that startIdx and endIdx + // constitute a valid range of the given batch. + next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes + + close() +} + +type spanEncoderBase struct { + allocator *colmem.Allocator + encodeColIdx int + + // outputBytes contains the encoding for each row of the key column. It is + // reused between calls to next(). + outputBytes *coldata.Bytes + + // A scratch bytes slice used to hold each encoding before it is appended to + // the output column. It is reused to avoid allocating for every row. + scratch []byte +} + +// {{range .}} +// {{range .TypeFamilies}} +// {{range .Overloads}} + +type _OP_STRING struct { + spanEncoderBase +} + +var _ spanEncoder = &_OP_STRING{} + +// next implements the spanEncoder interface. +func (op *_OP_STRING) next(batch coldata.Batch, startIdx, endIdx int) *coldata.Bytes { + oldBytesSize := op.outputBytes.Size() + if op.outputBytes == nil { + op.outputBytes = coldata.NewBytes(endIdx - startIdx) + } + op.outputBytes.ResetForAppend() + + vec := batch.ColVec(op.encodeColIdx) + col := vec.TemplateType() + + sel := batch.Selection() + if sel != nil { + sel = sel[startIdx:endIdx] + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for _, i := range sel { + encodeSpan(true, true) + } + } else { + for _, i := range sel { + encodeSpan(true, false) + } + } + } else { + _, _ = col.Get(startIdx), col.Get(endIdx-1) + if vec.Nulls().MaybeHasNulls() { + nulls := vec.Nulls() + for i := startIdx; i < endIdx; i++ { + encodeSpan(false, true) + } + } else { + for i := startIdx; i < endIdx; i++ { + encodeSpan(false, false) + } + } + } + + op.allocator.AdjustMemoryUsage(op.outputBytes.Size() - oldBytesSize) + return op.outputBytes +} + +// {{end}} +// {{end}} +// {{end}} + +// close implements the spanEncoder interface. +func (b *spanEncoderBase) close() { + *b = spanEncoderBase{} +} + +// execgen:inline +// execgen:template +func encodeSpan(hasSel bool, hasNulls bool) { + op.scratch = op.scratch[:0] + if hasNulls { + if nulls.NullAt(i) { + // {{if .Asc}} + op.outputBytes.AppendVal(encoding.EncodeNullAscending(op.scratch)) + // {{else}} + op.outputBytes.AppendVal(encoding.EncodeNullDescending(op.scratch)) + // {{end}} + continue + } + } + if !hasSel { + // {{if .Sliceable}} + //gcassert:bce + // {{end}} + } + val := col.Get(i) + _ASSIGN_SPAN_ENCODING(op.scratch, val) + op.outputBytes.AppendVal(op.scratch) +} diff --git a/pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel b/pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel index aad5e70db99a..3ce307c5f2e5 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel +++ b/pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel @@ -50,6 +50,8 @@ go_library( "select_in_gen.go", "selection_ops_gen.go", "sort_gen.go", + "span_assembler_gen.go", + "span_encoder_gen.go", "substring_gen.go", "sum_agg_gen.go", "values_differ_gen.go", diff --git a/pkg/sql/colexec/execgen/cmd/execgen/span_assembler_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/span_assembler_gen.go new file mode 100644 index 000000000000..a8e496358c8d --- /dev/null +++ b/pkg/sql/colexec/execgen/cmd/execgen/span_assembler_gen.go @@ -0,0 +1,44 @@ +// Copyright 2021 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 main + +import ( + "io" + "strings" + "text/template" +) + +type spanAssemblerTmplInfo struct { + WithColFamilies bool + String string +} + +const spanAssemblerTmpl = "pkg/sql/colexec/colexecspan/span_assembler_tmpl.go" + +func genSpanAssemblerOp(inputFileContents string, wr io.Writer) error { + s := strings.ReplaceAll(inputFileContents, "_OP_STRING", "{{.String}}") + + // Now, generate the op, from the template. + tmpl, err := template.New("span_assembler_op").Funcs(template.FuncMap{"buildDict": buildDict}).Parse(s) + if err != nil { + return err + } + + spanAssemblerTmplInfos := []spanAssemblerTmplInfo{ + {WithColFamilies: false, String: "spanAssemblerNoColFamily"}, + {WithColFamilies: true, String: "spanAssemblerWithColFamily"}, + } + return tmpl.Execute(wr, spanAssemblerTmplInfos) +} + +func init() { + registerGenerator(genSpanAssemblerOp, "span_assembler.eg.go", spanAssemblerTmpl) +} diff --git a/pkg/sql/colexec/execgen/cmd/execgen/span_encoder_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/span_encoder_gen.go new file mode 100644 index 000000000000..a164ccfbe5e7 --- /dev/null +++ b/pkg/sql/colexec/execgen/cmd/execgen/span_encoder_gen.go @@ -0,0 +1,175 @@ +// Copyright 2021 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 main + +import ( + "fmt" + "io" + "strings" + "text/template" + + "github.com/cockroachdb/cockroach/pkg/col/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/types" +) + +const spanEncoderTmpl = "pkg/sql/colexec/colexecspan/span_encoder_tmpl.go" + +func genSpanEncoder(inputFileContents string, wr io.Writer) error { + r := strings.NewReplacer( + "_OP_STRING", "{{.OpName}}", + "_IS_ASC", "{{.Asc}}", + "_CANONICAL_TYPE_FAMILY", "{{.TypeFamily}}", + "_TYPE_WIDTH", typeWidthReplacement, + "TemplateType", "{{.VecMethod}}", + ) + s := r.Replace(inputFileContents) + + assignAddRe := makeFunctionRegex("_ASSIGN_SPAN_ENCODING", 2) + s = assignAddRe.ReplaceAllString(s, makeTemplateFunctionCall("AssignSpanEncoding", 2)) + + tmpl, err := template.New("span_encoder").Funcs(template.FuncMap{"buildDict": buildDict}).Parse(s) + if err != nil { + return err + } + + var infos []spanEncoderDirectionInfo + for _, asc := range []bool{true, false} { + info := spanEncoderDirectionInfo{Asc: asc} + for _, family := range supportedCanonicalTypeFamilies { + if family == types.JsonFamily { + // We are currently unable to encode JSON as a table key. + continue + } + familyInfo := spanEncoderTypeFamilyInfo{TypeFamily: toString(family)} + for _, width := range supportedWidthsByCanonicalTypeFamily[family] { + overload := spanEncoderTmplInfo{ + Asc: asc, + Sliceable: sliceable(family), + Width: width, + OpName: getSpanEncoderOpName(asc, family, width), + VecMethod: toVecMethod(family, width), + TypeFamily: family, + } + familyInfo.Overloads = append(familyInfo.Overloads, overload) + } + info.TypeFamilies = append(info.TypeFamilies, familyInfo) + } + infos = append(infos, info) + } + return tmpl.Execute(wr, infos) +} + +func init() { + registerGenerator(genSpanEncoder, "span_encoder.eg.go", spanEncoderTmpl) +} + +type spanEncoderDirectionInfo struct { + Asc bool + TypeFamilies []spanEncoderTypeFamilyInfo +} + +type spanEncoderTypeFamilyInfo struct { + TypeFamily string + Overloads []spanEncoderTmplInfo +} + +type spanEncoderTmplInfo struct { + Asc bool + Sliceable bool + Width int32 + OpName string + VecMethod string + TypeFamily types.Family +} + +func (info spanEncoderTmplInfo) AssignSpanEncoding(appendTo, valToEncode string) string { + assignEncoding := func(funcName, val string) string { + ascString := "Ascending" + if !info.Asc { + ascString = "Descending" + } + return fmt.Sprintf("%[1]s = encoding.Encode%[2]s%[3]s(%[1]s, %[4]s)", + appendTo, funcName, ascString, val) + } + + switch info.TypeFamily { + case types.IntFamily: + funcName := "Varint" + if info.Width == 16 || info.Width == 32 { + // We need to cast the input to an int64. + valToEncode = "int64(" + valToEncode + ")" + } + return assignEncoding(funcName, valToEncode) + case types.BoolFamily: + funcName := "Varint" + prefix := fmt.Sprintf(` + var x int64 + if %s { + x = 1 + } else { + x = 0 + } + `, valToEncode) + valToEncode = "x" + return prefix + assignEncoding(funcName, valToEncode) + case types.FloatFamily: + funcName := "Float" + return assignEncoding(funcName, valToEncode) + case types.DecimalFamily: + funcName := "Decimal" + valToEncode = "&" + valToEncode + return assignEncoding(funcName, valToEncode) + case types.BytesFamily: + funcName := "Bytes" + return assignEncoding(funcName, valToEncode) + case types.TimestampTZFamily: + funcName := "Time" + return assignEncoding(funcName, valToEncode) + case types.IntervalFamily: + funcName := "DurationAscending" + if !info.Asc { + funcName = "DurationDescending" + } + return fmt.Sprintf(` + var err error + %[1]s, err = encoding.Encode%[2]s(%[1]s, %[3]s) + if err != nil { + colexecerror.ExpectedError(err) + } + `, appendTo, funcName, valToEncode) + case typeconv.DatumVecCanonicalTypeFamily: + dir := "encoding.Ascending" + if !info.Asc { + dir = "encoding.Descending" + } + valToEncode += ".(tree.Datum)" + return fmt.Sprintf(` + var err error + %[1]s, err = rowenc.EncodeTableKey(%[1]s, %[2]s, %[3]s) + if err != nil { + colexecerror.ExpectedError(err) + } + `, appendTo, valToEncode, dir) + } + return fmt.Sprintf("unsupported type: %s", info.TypeFamily.Name()) +} + +var _ = spanEncoderTmplInfo{}.AssignSpanEncoding + +func getSpanEncoderOpName(asc bool, family types.Family, width int32) string { + opName := "spanEncoder" + toVecMethod(family, width) + if asc { + opName += "Asc" + } else { + opName += "Desc" + } + return opName +} diff --git a/pkg/sql/colexec/hash_aggregator.go b/pkg/sql/colexec/hash_aggregator.go index bfdf41ce5284..6cca21e2bdcd 100644 --- a/pkg/sql/colexec/hash_aggregator.go +++ b/pkg/sql/colexec/hash_aggregator.go @@ -546,7 +546,7 @@ func (op *hashAggregator) Close() error { if !op.CloserHelper.Close() { return nil } - op.accountingHelper.Close() + op.accountingHelper.Release() var retErr error if op.inputTrackingState.tuples != nil { retErr = op.inputTrackingState.tuples.Close(op.EnsureCtx()) diff --git a/pkg/sql/colexec/ordered_synchronizer.eg.go b/pkg/sql/colexec/ordered_synchronizer.eg.go index 3ce136db5acb..fb011362b88b 100644 --- a/pkg/sql/colexec/ordered_synchronizer.eg.go +++ b/pkg/sql/colexec/ordered_synchronizer.eg.go @@ -414,7 +414,7 @@ func (o *OrderedSynchronizer) DrainMeta() []execinfrapb.ProducerMetadata { } func (o *OrderedSynchronizer) Close() error { - o.accountingHelper.Close() + o.accountingHelper.Release() for _, input := range o.inputs { input.ToClose.CloseAndLogOnErr(o.EnsureCtx(), "ordered synchronizer") } diff --git a/pkg/sql/colexec/ordered_synchronizer_tmpl.go b/pkg/sql/colexec/ordered_synchronizer_tmpl.go index cd67994f9f05..9f091bdef3c5 100644 --- a/pkg/sql/colexec/ordered_synchronizer_tmpl.go +++ b/pkg/sql/colexec/ordered_synchronizer_tmpl.go @@ -288,7 +288,7 @@ func (o *OrderedSynchronizer) DrainMeta() []execinfrapb.ProducerMetadata { } func (o *OrderedSynchronizer) Close() error { - o.accountingHelper.Close() + o.accountingHelper.Release() for _, input := range o.inputs { input.ToClose.CloseAndLogOnErr(o.EnsureCtx(), "ordered synchronizer") } diff --git a/pkg/sql/colfetcher/BUILD.bazel b/pkg/sql/colfetcher/BUILD.bazel index 78d54888fdac..0928f0ecceec 100644 --- a/pkg/sql/colfetcher/BUILD.bazel +++ b/pkg/sql/colfetcher/BUILD.bazel @@ -6,12 +6,14 @@ go_library( srcs = [ "cfetcher.go", "colbatch_scan.go", + "index_join.go", ":gen-fetcherstate-stringer", # keep ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/colfetcher", visibility = ["//visibility:public"], deps = [ "//pkg/col/coldata", + "//pkg/col/typeconv", "//pkg/keys", "//pkg/kv", "//pkg/roachpb:with-mocks", @@ -21,11 +23,13 @@ go_library( "//pkg/sql/catalog/tabledesc", "//pkg/sql/colconv", "//pkg/sql/colencoding", + "//pkg/sql/colexec/colexecspan", "//pkg/sql/colexecerror", "//pkg/sql/colexecop", "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", + "//pkg/sql/memsize", "//pkg/sql/row", "//pkg/sql/rowenc", "//pkg/sql/scrub", diff --git a/pkg/sql/colfetcher/cfetcher.go b/pkg/sql/colfetcher/cfetcher.go index e1d1b839ca9f..464ce669ff5b 100644 --- a/pkg/sql/colfetcher/cfetcher.go +++ b/pkg/sql/colfetcher/cfetcher.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/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" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/scrub" @@ -795,6 +796,10 @@ const ( // Turn this on to enable super verbose logging of the fetcher state machine. const debugState = false +func (rf *cFetcher) setEstimatedRowCount(estimatedRowCount uint64) { + rf.estimatedRowCount = estimatedRowCount +} + // setNextKV sets the next KV to process to the input KV. needsCopy, if true, // causes the input kv to be deep copied. needsCopy should be set to true if // the input KV is pointing to the last KV of a batch, so that the batch can @@ -1159,7 +1164,6 @@ func (rf *cFetcher) NextBatch(ctx context.Context) (coldata.Batch, error) { case stateEmitLastBatch: rf.machine.state[0] = stateFinished rf.finalizeBatch() - rf.accountingHelper.Close() return rf.machine.batch, nil case stateFinished: @@ -1597,6 +1601,7 @@ var cFetcherPool = sync.Pool{ } func (rf *cFetcher) Release() { + rf.accountingHelper.Release() rf.table.Release() *rf = cFetcher{ // The types are small objects, so we don't bother deeply resetting this @@ -1605,3 +1610,48 @@ func (rf *cFetcher) Release() { } cFetcherPool.Put(rf) } + +type cFetcherArgs struct { + visibility execinfrapb.ScanVisibility + lockingStrength descpb.ScanLockingStrength + lockingWaitPolicy descpb.ScanLockingWaitPolicy + hasSystemColumns bool + reverse bool + memoryLimit int64 + estimatedRowCount uint64 +} + +// initCFetcher extracts common logic for operators in the colfetcher package +// that need to use cFetcher operators. +func initCFetcher( + flowCtx *execinfra.FlowCtx, + allocator *colmem.Allocator, + desc catalog.TableDescriptor, + index catalog.Index, + neededCols util.FastIntSet, + colIdxMap catalog.TableColMap, + virtualCol catalog.Column, + args cFetcherArgs, +) (*cFetcher, error) { + fetcher := cFetcherPool.Get().(*cFetcher) + fetcher.setEstimatedRowCount(args.estimatedRowCount) + + tableArgs := row.FetcherTableArgs{ + Desc: desc, + Index: index, + ColIdxMap: colIdxMap, + IsSecondaryIndex: !index.Primary(), + ValNeededForCol: neededCols, + } + + tableArgs.InitCols(desc, args.visibility, args.hasSystemColumns, virtualCol) + + if err := fetcher.Init( + flowCtx.Codec(), allocator, args.memoryLimit, args.reverse, args.lockingStrength, + args.lockingWaitPolicy, flowCtx.EvalCtx.SessionData.LockTimeout, tableArgs, + ); err != nil { + return nil, err + } + + return fetcher, nil +} diff --git a/pkg/sql/colfetcher/colbatch_scan.go b/pkg/sql/colfetcher/colbatch_scan.go index 3b253bea1715..57cd290ce2fa 100644 --- a/pkg/sql/colfetcher/colbatch_scan.go +++ b/pkg/sql/colfetcher/colbatch_scan.go @@ -24,7 +24,6 @@ import ( "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" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" @@ -41,8 +40,9 @@ import ( // should get rid off table readers entirely. We will have to be careful about // propagating the metadata though. -// ColBatchScan is the exec.Operator implementation of TableReader. It reads a table -// from kv, presenting it as coldata.Batches via the exec.Operator interface. +// 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 { colexecop.ZeroInputNode colexecop.InitHelper @@ -67,10 +67,15 @@ type ColBatchScan struct { ResultTypes []*types.T } -var _ colexecop.KVReader = &ColBatchScan{} -var _ execinfra.Releasable = &ColBatchScan{} -var _ colexecop.Closer = &ColBatchScan{} -var _ colexecop.Operator = &ColBatchScan{} +// ScanOperator combines common interfaces between operators that perform KV +// scans, such as ColBatchScan and ColIndexJoin. +type ScanOperator interface { + colexecop.KVReader + execinfra.Releasable + colexecop.ClosableOperator +} + +var _ ScanOperator = &ColBatchScan{} // Init initializes a ColBatchScan. func (s *ColBatchScan) Init(ctx context.Context) { @@ -187,27 +192,9 @@ func NewColBatchScan( // retrieving the hydrated immutable from cache. table := spec.BuildTableDescriptor() virtualColumn := tabledesc.FindVirtualColumn(table, spec.VirtualColumn) - cols := table.PublicColumns() - if spec.Visibility == execinfra.ScanVisibilityPublicAndNotPublic { - cols = table.DeletableColumns() - } - columnIdxMap := catalog.ColumnIDToOrdinalMap(cols) - typs := catalog.ColumnTypesWithVirtualCol(cols, virtualColumn) - - // Add all requested system columns to the output. - if spec.HasSystemColumns { - for _, sysCol := range table.SystemColumns() { - typs = append(typs, sysCol.GetType()) - columnIdxMap.Set(sysCol.GetID(), columnIdxMap.Len()) - } - } - - // Before we can safely use types from the table descriptor, we need to - // make sure they are hydrated. In row execution engine it is done during - // the processor initialization, but neither ColBatchScan nor cFetcher are - // processors, so we need to do the hydration ourselves. - resolver := flowCtx.TypeResolverFactory.NewTypeResolver(evalCtx.Txn) - if err := resolver.HydrateTypeSlice(ctx, typs); err != nil { + typs, columnIdxMap, err := retrieveTypsAndColOrds( + ctx, flowCtx, evalCtx, table, virtualColumn, spec.Visibility, spec.HasSystemColumns) + if err != nil { return nil, err } @@ -216,11 +203,20 @@ func NewColBatchScan( neededColumns.Add(int(neededColumn)) } - fetcher := cFetcherPool.Get().(*cFetcher) - fetcher.estimatedRowCount = estimatedRowCount - if _, _, err := initCRowFetcher( - flowCtx, allocator, fetcher, table, columnIdxMap, neededColumns, spec, spec.HasSystemColumns, - ); err != nil { + fetcher, err := initCFetcher( + flowCtx, allocator, table, table.ActiveIndexes()[spec.IndexIdx], + neededColumns, columnIdxMap, virtualColumn, + cFetcherArgs{ + visibility: spec.Visibility, + lockingStrength: spec.LockingStrength, + lockingWaitPolicy: spec.LockingWaitPolicy, + hasSystemColumns: spec.HasSystemColumns, + reverse: spec.Reverse, + memoryLimit: execinfra.GetWorkMemLimit(flowCtx), + estimatedRowCount: estimatedRowCount, + }, + ) + if err != nil { return nil, err } @@ -244,49 +240,43 @@ func NewColBatchScan( return s, nil } -// initCRowFetcher initializes a row.cFetcher. See initRowFetcher. -func initCRowFetcher( +// retrieveTypsAndColOrds extracts logic that retrieves a slice with the column +// types and a map between column IDs and ordinal positions for the columns from +// the given table. +func retrieveTypsAndColOrds( + ctx context.Context, flowCtx *execinfra.FlowCtx, - allocator *colmem.Allocator, - fetcher *cFetcher, - desc catalog.TableDescriptor, - colIdxMap catalog.TableColMap, - valNeededForCol util.FastIntSet, - spec *execinfrapb.TableReaderSpec, - withSystemColumns bool, -) (index catalog.Index, isSecondaryIndex bool, err error) { - indexIdx := int(spec.IndexIdx) - if indexIdx >= len(desc.ActiveIndexes()) { - return nil, false, errors.Errorf("invalid indexIdx %d", indexIdx) + evalCtx *tree.EvalContext, + table catalog.TableDescriptor, + virtualCol catalog.Column, + visibility execinfrapb.ScanVisibility, + hasSystemColumns bool, +) ([]*types.T, catalog.TableColMap, error) { + cols := table.PublicColumns() + if visibility == execinfra.ScanVisibilityPublicAndNotPublic { + cols = table.DeletableColumns() } - index = desc.ActiveIndexes()[indexIdx] - isSecondaryIndex = !index.Primary() + columnIdxMap := catalog.ColumnIDToOrdinalMap(cols) + typs := catalog.ColumnTypesWithVirtualCol(cols, virtualCol) - tableArgs := row.FetcherTableArgs{ - Desc: desc, - Index: index, - ColIdxMap: colIdxMap, - IsSecondaryIndex: isSecondaryIndex, - ValNeededForCol: valNeededForCol, + // Add all requested system columns to the output. + if hasSystemColumns { + for _, sysCol := range table.SystemColumns() { + typs = append(typs, sysCol.GetType()) + columnIdxMap.Set(sysCol.GetID(), columnIdxMap.Len()) + } } - virtualColumn := tabledesc.FindVirtualColumn(desc, spec.VirtualColumn) - tableArgs.InitCols(desc, spec.Visibility, withSystemColumns, virtualColumn) - - if err := fetcher.Init( - flowCtx.Codec(), - allocator, - execinfra.GetWorkMemLimit(flowCtx), - spec.Reverse, - spec.LockingStrength, - spec.LockingWaitPolicy, - flowCtx.EvalCtx.SessionData.LockTimeout, - tableArgs, - ); err != nil { - return nil, false, err + // Before we can safely use types from the table descriptor, we need to + // make sure they are hydrated. In row execution engine it is done during + // the processor initialization, but neither ColBatchScan nor cFetcher are + // processors, so we need to do the hydration ourselves. + resolver := flowCtx.TypeResolverFactory.NewTypeResolver(evalCtx.Txn) + if err := resolver.HydrateTypeSlice(ctx, typs); err != nil { + return nil, catalog.TableColMap{}, err } - return index, isSecondaryIndex, nil + return typs, columnIdxMap, nil } // Release implements the execinfra.Releasable interface. diff --git a/pkg/sql/colfetcher/index_join.go b/pkg/sql/colfetcher/index_join.go new file mode 100644 index 000000000000..a331dab2548f --- /dev/null +++ b/pkg/sql/colfetcher/index_join.go @@ -0,0 +1,526 @@ +// Copyright 2021 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 colfetcher + +import ( + "context" + "sort" + "time" + + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/typeconv" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecspan" + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/memsize" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/errors" +) + +// ColIndexJoin operators are used to execute index joins (lookup joins that +// scan the primary index and discard input rows). +type ColIndexJoin struct { + colexecop.InitHelper + colexecop.OneInputNode + + state indexJoinState + + // spanAssembler is used to construct the lookup spans for each input batch. + spanAssembler colexecspan.ColSpanAssembler + + // batch keeps track of the input batch currently being processed; if we only + // generate spans for a portion of the batch on one iteration, we need to keep + // a reference to it for the next iteration. + batch coldata.Batch + + // startIdx keeps track of the index into the current input batch from which + // the next set of spans should start to be generated. This is necessary + // because the size of input rows from which spans are generated is limited, + // and may not correspond to batch boundaries. + startIdx int + + mem struct { + // inputBatchSize tracks the size of the rows that have been used to + // generate spans so far. This is used to prevent memory usage from growing + // too large. + inputBatchSize int64 + + // currentBatchSize tracks the size of the current input batch. This + // provides a shortcut when the entire batch fits in the memory limit. + currentBatchSize int64 + + // constRowSize tracks the portion of the size of each row that remains + // constant between rows - for example, an int64 column will add 8 bytes to + // this field. + constRowSize int64 + + // Fields that deal with variable-size types. + hasVarSizeCols bool + varSizeVecIdxs util.FastIntSet + byteLikeCols []*coldata.Bytes + decimalCols []coldata.Decimals + datumCols []coldata.DatumVec + } + + flowCtx *execinfra.FlowCtx + rf *cFetcher + + // tracingSpan is created when the stats should be collected for the query + // execution, and it will be finished when closing the operator. + tracingSpan *tracing.Span + mu struct { + syncutil.Mutex + // rowsRead contains the number of total rows this ColIndexJoin has + // returned so far. + rowsRead int64 + } + // ResultTypes is the slice of resulting column types from this operator. + // It should be used rather than the slice of column types from the scanned + // table because the scan might synthesize additional implicit system columns. + ResultTypes []*types.T + + // maintainOrdering is true when the index join is required to maintain its + // input ordering, in which case the ordering of the spans cannot be changed. + maintainOrdering bool +} + +var _ colexecop.KVReader = &ColIndexJoin{} +var _ execinfra.Releasable = &ColIndexJoin{} +var _ colexecop.ClosableOperator = &ColIndexJoin{} + +// Init initializes a ColIndexJoin. +func (s *ColIndexJoin) Init(ctx context.Context) { + if !s.InitHelper.Init(ctx) { + return + } + // If tracing is enabled, we need to start a child span so that the only + // contention events present in the recording would be because of this + // cFetcher. Note that ProcessorSpan method itself will check whether + // tracing is enabled. + s.Ctx, s.tracingSpan = execinfra.ProcessorSpan(s.Ctx, "colindexjoin") + s.Input.Init(s.Ctx) +} + +type indexJoinState uint8 + +const ( + indexJoinConstructingSpans indexJoinState = iota + indexJoinScanning + indexJoinDone +) + +// Next is part of the Operator interface. +func (s *ColIndexJoin) Next() coldata.Batch { + for { + switch s.state { + case indexJoinConstructingSpans: + var rowCount int + var spans roachpb.Spans + s.mem.inputBatchSize = 0 + for s.next() { + // Because index joins discard input rows, we do not have to maintain a + // reference to input tuples after span generation. So, we can discard + // the input batch reference on each iteration. + endIdx := s.findEndIndex(len(spans) > 0) + rowCount += endIdx - s.startIdx + s.spanAssembler.ConsumeBatch(s.batch, s.startIdx, endIdx) + s.startIdx = endIdx + if endIdx < s.batch.Length() { + // Reached the memory limit. + break + } + } + spans = s.spanAssembler.GetSpans() + if len(spans) == 0 { + // No lookups left to perform. + s.state = indexJoinDone + continue + } + + if !s.maintainOrdering { + // Sort the spans when !maintainOrdering. This allows lower layers to + // optimize iteration over the data. Note that the looked up rows are + // output unchanged, in the retrieval order, so it is not safe to do + // this when maintainOrdering is true (the ordering to be maintained + // may be different than the ordering in the index). + sort.Sort(spans) + } + + // Index joins will always return exactly one output row per input row. + s.rf.setEstimatedRowCount(uint64(rowCount)) + if err := s.rf.StartScan( + s.flowCtx.Txn, spans, false /* limitBatches */, 0 /* limitHint */, s.flowCtx.TraceKV, + s.flowCtx.EvalCtx.TestingKnobs.ForceProductionBatchSizes, + ); err != nil { + colexecerror.InternalError(err) + } + s.state = indexJoinScanning + case indexJoinScanning: + batch, err := s.rf.NextBatch(s.Ctx) + if err != nil { + colexecerror.InternalError(err) + } + if batch.Selection() != nil { + colexecerror.InternalError( + errors.AssertionFailedf("unexpected selection vector on the batch coming from CFetcher")) + } + n := batch.Length() + if n == 0 { + s.state = indexJoinConstructingSpans + continue + } + s.mu.Lock() + s.mu.rowsRead += int64(n) + s.mu.Unlock() + return batch + case indexJoinDone: + return coldata.ZeroBatch + } + } +} + +// inputBatchSizeLimit is a batch size limit for the number of input rows that +// will be used to form lookup spans for each scan. This is used as a proxy for +// result batch size in order to prevent OOMs, because index joins do not limit +// result batches. TODO(drewk): once the Streamer work is finished, the fetcher +// logic will be able to control result size without sacrificing parallelism, so +// we can remove this limit. +const inputBatchSizeLimit = 4 << 20 /* 4 MB */ + +// findEndIndex returns an index endIdx into s.batch such that generating spans +// for rows in the interval [s.startIdx, endIdx) will get as close to the memory +// limit as possible without exceeding it, subject to the length of the batch. +// If no spans have been generated so far, the interval will include at least +// one row to ensure that progress is made. If no more spans should be generated +// for the current iteration, endIdx == s.startIdx. +func (s *ColIndexJoin) findEndIndex(hasSpans bool) (endIdx int) { + n := s.batch.Length() + if n == 0 || s.startIdx >= n || s.mem.inputBatchSize >= inputBatchSizeLimit { + // No more spans should be generated. + return s.startIdx + } + if s.mem.inputBatchSize+s.mem.currentBatchSize <= inputBatchSizeLimit { + // The entire batch fits within the memory limit. + s.mem.inputBatchSize += s.mem.currentBatchSize + return n + } + for endIdx = s.startIdx; endIdx < n; endIdx++ { + s.mem.inputBatchSize += s.getRowSize(endIdx) + if s.mem.inputBatchSize > inputBatchSizeLimit { + // The current row (but not the previous) brings us to or over the memory + // limit, so use it as the exclusive end index. + break + } + if s.mem.inputBatchSize == inputBatchSizeLimit { + // The current row exactly meets the memory limit. Increment idx in order + // to make it exclusive. + endIdx++ + break + } + } + if !hasSpans && endIdx == s.startIdx { + // We must generate spans for at least one row in order to make progress. + return s.startIdx + 1 + } + return endIdx +} + +// getRowSize calculates the size of the row stored at index i in the current +// batch. Note that it accounts only for the size of the data itself, and +// ignores extra overhead such as selection vectors or byte offsets. +func (s *ColIndexJoin) getRowSize(idx int) int64 { + rowSize := s.mem.constRowSize + if s.mem.hasVarSizeCols { + for i := range s.mem.byteLikeCols { + rowSize += adjustMemEstimate(s.mem.byteLikeCols[i].ElemSize(idx)) + } + for i := range s.mem.decimalCols { + rowSize += adjustMemEstimate(int64(tree.SizeOfDecimal(&s.mem.decimalCols[i][idx]))) + } + for i := range s.mem.datumCols { + memEstimate := int64(s.mem.datumCols[i].Get(idx).(tree.Datum).Size()) + memsize.DatumOverhead + rowSize += adjustMemEstimate(memEstimate) + } + } + return rowSize +} + +// getBatchSize calculates the size of the entire current batch. Note that it +// accounts only for the size of the data itself, and ignores extra overhead +// such as selection vectors or byte offsets. getBatchSize is not exactly +// equivalent to calling getRowSize for every row, but it is not necessary for +// the accounting to be exact, anyway. +func (s *ColIndexJoin) getBatchSize() int64 { + n := s.batch.Length() + batchSize := colmem.GetBatchMemSize(s.batch) + batchSize += int64(n*s.batch.Width()) * memEstimateAdditive + batchSize += int64(n) * int64(rowenc.EncDatumRowOverhead) + return batchSize +} + +// next pulls the next input batch (if the current one is entirely finished) +// and performs initial processing of the batch. This includes performing +// interface conversions up front and retrieving the overall memory footprint of +// the data. next returns false once the input is finished, and otherwise true. +func (s *ColIndexJoin) next() bool { + if s.batch == nil || s.startIdx >= s.batch.Length() { + // The current batch is finished. + s.startIdx = 0 + s.batch = s.Input.Next() + if s.batch.Length() == 0 { + return false + } + s.mem.currentBatchSize = s.getBatchSize() + } + if !s.mem.hasVarSizeCols { + return true + } + s.mem.byteLikeCols = s.mem.byteLikeCols[:0] + s.mem.decimalCols = s.mem.decimalCols[:0] + s.mem.datumCols = s.mem.datumCols[:0] + for i, ok := s.mem.varSizeVecIdxs.Next(0); ok; i, ok = s.mem.varSizeVecIdxs.Next(i + 1) { + vec := s.batch.ColVec(i) + switch vec.CanonicalTypeFamily() { + case types.BytesFamily: + s.mem.byteLikeCols = append(s.mem.byteLikeCols, vec.Bytes()) + case types.JsonFamily: + s.mem.byteLikeCols = append(s.mem.byteLikeCols, &vec.JSON().Bytes) + case types.DecimalFamily: + s.mem.decimalCols = append(s.mem.decimalCols, vec.Decimal()) + case typeconv.DatumVecCanonicalTypeFamily: + s.mem.datumCols = append(s.mem.datumCols, vec.Datum()) + } + } + return true +} + +// DrainMeta is part of the colexecop.MetadataSource interface. +func (s *ColIndexJoin) DrainMeta() []execinfrapb.ProducerMetadata { + var trailingMeta []execinfrapb.ProducerMetadata + if tfs := execinfra.GetLeafTxnFinalState(s.Ctx, s.flowCtx.Txn); tfs != nil { + trailingMeta = append(trailingMeta, execinfrapb.ProducerMetadata{LeafTxnFinalState: tfs}) + } + meta := execinfrapb.GetProducerMeta() + meta.Metrics = execinfrapb.GetMetricsMeta() + meta.Metrics.BytesRead = s.GetBytesRead() + meta.Metrics.RowsRead = s.GetRowsRead() + trailingMeta = append(trailingMeta, *meta) + if trace := execinfra.GetTraceData(s.Ctx); trace != nil { + trailingMeta = append(trailingMeta, execinfrapb.ProducerMetadata{TraceData: trace}) + } + return trailingMeta +} + +// GetBytesRead is part of the colexecop.KVReader interface. +func (s *ColIndexJoin) GetBytesRead() int64 { + s.mu.Lock() + defer s.mu.Unlock() + // Note that if Init() was never called, s.rf.fetcher will remain nil, and + // GetBytesRead() will return 0. We are also holding the mutex, so a + // concurrent call to Init() will have to wait, and the fetcher will remain + // uninitialized until we return. + return s.rf.fetcher.GetBytesRead() +} + +// GetRowsRead is part of the colexecop.KVReader interface. +func (s *ColIndexJoin) GetRowsRead() int64 { + s.mu.Lock() + defer s.mu.Unlock() + return s.mu.rowsRead +} + +// GetCumulativeContentionTime is part of the colexecop.KVReader interface. +func (s *ColIndexJoin) GetCumulativeContentionTime() time.Duration { + return execinfra.GetCumulativeContentionTime(s.Ctx) +} + +// NewColIndexJoin creates a new ColIndexJoin operator. +func NewColIndexJoin( + ctx context.Context, + allocator *colmem.Allocator, + fetcherAllocator *colmem.Allocator, + flowCtx *execinfra.FlowCtx, + evalCtx *tree.EvalContext, + semaCtx *tree.SemaContext, + input colexecop.Operator, + spec *execinfrapb.JoinReaderSpec, + post *execinfrapb.PostProcessSpec, + inputTypes []*types.T, +) (*ColIndexJoin, error) { + // NB: we hit this with a zero NodeID (but !ok) with multi-tenancy. + if nodeID, ok := flowCtx.NodeID.OptionalNodeID(); nodeID == 0 && ok { + return nil, errors.Errorf("attempting to create a ColIndexJoin with uninitialized NodeID") + } + if !spec.LookupExpr.Empty() { + return nil, errors.AssertionFailedf("non-empty lookup expressions are not supported for index joins") + } + if !spec.RemoteLookupExpr.Empty() { + return nil, errors.AssertionFailedf("non-empty remote lookup expressions are not supported for index joins") + } + if !spec.OnExpr.Empty() { + return nil, errors.AssertionFailedf("non-empty ON expressions are not supported for index joins") + } + + // TODO(ajwerner): The need to construct an immutable here + // indicates that we're probably doing this wrong. Instead we should be + // just setting the ID and Version in the spec or something like that and + // retrieving the hydrated immutable from cache. + table := spec.BuildTableDescriptor() + + cols := table.PublicColumns() + if spec.Visibility == execinfra.ScanVisibilityPublicAndNotPublic { + cols = table.DeletableColumns() + } + columnIdxMap := catalog.ColumnIDToOrdinalMap(cols) + typs := catalog.ColumnTypesWithVirtualCol(cols, nil /* virtualCol */) + + // Add all requested system columns to the output. + if spec.HasSystemColumns { + for _, sysCol := range table.SystemColumns() { + typs = append(typs, sysCol.GetType()) + columnIdxMap.Set(sysCol.GetID(), columnIdxMap.Len()) + } + } + + // Before we can safely use types from the table descriptor, we need to + // make sure they are hydrated. In row execution engine it is done during + // the processor initialization, but neither ColIndexJoin nor cFetcher are + // processors, so we need to do the hydration ourselves. + resolver := flowCtx.TypeResolverFactory.NewTypeResolver(evalCtx.Txn) + if err := resolver.HydrateTypeSlice(ctx, typs); err != nil { + return nil, err + } + + indexIdx := int(spec.IndexIdx) + if indexIdx >= len(table.ActiveIndexes()) { + return nil, errors.Errorf("invalid indexIdx %d", indexIdx) + } + index := table.ActiveIndexes()[indexIdx] + + // Retrieve the set of columns that the index join needs to fetch. + var neededColumns util.FastIntSet + if post.OutputColumns != nil { + for _, neededColumn := range post.OutputColumns { + neededColumns.Add(int(neededColumn)) + } + } else { + proc := &execinfra.ProcOutputHelper{} + if err := proc.Init(post, typs, semaCtx, evalCtx); err != nil { + colexecerror.InternalError(err) + } + neededColumns = proc.NeededColumns() + } + + fetcher, err := initCFetcher( + flowCtx, fetcherAllocator, table, index, neededColumns, columnIdxMap, nil, /* virtualColumn */ + cFetcherArgs{ + visibility: spec.Visibility, + lockingStrength: spec.LockingStrength, + lockingWaitPolicy: spec.LockingWaitPolicy, + hasSystemColumns: spec.HasSystemColumns, + memoryLimit: execinfra.GetWorkMemLimit(flowCtx), + }, + ) + if err != nil { + return nil, err + } + + spanAssembler := colexecspan.NewColSpanAssembler( + flowCtx.Codec(), allocator, table, index, inputTypes, neededColumns) + + op := &ColIndexJoin{ + OneInputNode: colexecop.NewOneInputNode(input), + flowCtx: flowCtx, + rf: fetcher, + spanAssembler: spanAssembler, + ResultTypes: typs, + maintainOrdering: spec.MaintainOrdering, + } + op.prepareMemLimit(inputTypes) + + return op, nil +} + +// prepareMemLimit sets up the fields used to limit lookup batch size. +func (s *ColIndexJoin) prepareMemLimit(inputTypes []*types.T) { + // Add the EncDatum overhead to ensure parity with row engine size limits. + s.mem.constRowSize = int64(rowenc.EncDatumRowOverhead) + for i, t := range inputTypes { + switch typeconv.TypeFamilyToCanonicalTypeFamily(t.Family()) { + case + types.BoolFamily, + types.IntFamily, + types.FloatFamily, + types.TimestampTZFamily, + types.IntervalFamily: + s.mem.constRowSize += adjustMemEstimate(colmem.GetFixedSizeTypeSize(t)) + case + types.DecimalFamily, + types.BytesFamily, + types.JsonFamily, + typeconv.DatumVecCanonicalTypeFamily: + s.mem.varSizeVecIdxs.Add(i) + s.mem.hasVarSizeCols = true + default: + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", t)) + } + } + s.mem.hasVarSizeCols = !s.mem.varSizeVecIdxs.Empty() +} + +var ( + // memEstimateAdditive is an additive correction that simulates the overhead + // of the EncDatum struct the row engine uses to store values. + memEstimateAdditive = int64(rowenc.EncDatumOverhead) + + // memEstimateMultiplier is a multiplicative correction that simulates the + // overhead of the encoded bytes field of EncDatum objects. It is somewhat + // arbitrary, but the size of the 'encoded' field should not greatly exceed + // the size of the decoded value, so the result should not be too far off. + memEstimateMultiplier = int64(2) +) + +// adjustMemEstimate attempts to adjust the given estimate for the size of a +// single data value to reflect what the size would be in the row engine. This +// is necessary in order to achieve similar batch sizes to the row-wise index +// joiner. Until the Streamer work is finished, increasing batch size could +// increase cluster instability. +func adjustMemEstimate(estimate int64) int64 { + return estimate*memEstimateMultiplier + memEstimateAdditive +} + +// Release implements the execinfra.Releasable interface. +func (s *ColIndexJoin) Release() { + s.rf.Release() + s.spanAssembler.Release() + *s = ColIndexJoin{} +} + +// Close implements the colexecop.Closer interface. +func (s *ColIndexJoin) Close() error { + if s.tracingSpan != nil { + s.tracingSpan.Finish() + s.tracingSpan = nil + } + s.spanAssembler.Close() + s.batch = nil + return nil +} diff --git a/pkg/sql/colmem/allocator.go b/pkg/sql/colmem/allocator.go index cced4c0d9839..34b6ceda7c01 100644 --- a/pkg/sql/colmem/allocator.go +++ b/pkg/sql/colmem/allocator.go @@ -415,40 +415,16 @@ func EstimateBatchSizeBytes(vecTypes []*types.T, batchLength int) int64 { numUUIDVectors := 0 for _, t := range vecTypes { switch typeconv.TypeFamilyToCanonicalTypeFamily(t.Family()) { - case types.BoolFamily: - acc += memsize.Bool case types.BytesFamily: if t.Family() == types.UuidFamily { numUUIDVectors++ } else { numBytesVectors++ } - case types.IntFamily: - switch t.Width() { - case 16: - acc += memsize.Int16 - case 32: - acc += memsize.Int32 - default: - acc += memsize.Int64 - } - case types.FloatFamily: - acc += memsize.Float64 case types.DecimalFamily: // Similar to byte arrays, we can't tell how much space is used // to hold the arbitrary precision decimal objects. acc += decimalEstimate - case types.TimestampTZFamily: - // time.Time consists of two 64 bit integers and a pointer to - // time.Location. We will only account for this 3 bytes without paying - // attention to the full time.Location struct. The reason is that it is - // likely that time.Location's are cached and are shared among all the - // timestamps, so if we were to include that in the estimation, we would - // significantly overestimate. - // TODO(yuzefovich): figure out whether the caching does take place. - acc += memsize.Time - case types.IntervalFamily: - acc += memsize.Duration case types.JsonFamily: numBytesVectors++ case typeconv.DatumVecCanonicalTypeFamily: @@ -461,6 +437,14 @@ func EstimateBatchSizeBytes(vecTypes []*types.T, batchLength int) int64 { // Note: keep the calculation here in line with datumVec.Size. implementationSize, _ := tree.DatumTypeSize(t) acc += int64(implementationSize) + memsize.DatumOverhead + case + types.BoolFamily, + types.IntFamily, + types.FloatFamily, + types.TimestampTZFamily, + types.IntervalFamily: + // Types that have a statically known size. + acc += GetFixedSizeTypeSize(t) default: colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", t)) } @@ -483,6 +467,40 @@ func EstimateBatchSizeBytes(vecTypes []*types.T, batchLength int) int64 { return acc*int64(batchLength) + bytesVectorsSize } +// GetFixedSizeTypeSize returns the size of a type that is not variable in size; +// e.g. its size is known statically. +func GetFixedSizeTypeSize(t *types.T) (size int64) { + switch typeconv.TypeFamilyToCanonicalTypeFamily(t.Family()) { + case types.BoolFamily: + size = memsize.Bool + case types.IntFamily: + switch t.Width() { + case 16: + size = memsize.Int16 + case 32: + size = memsize.Int32 + default: + size = memsize.Int64 + } + case types.FloatFamily: + size = memsize.Float64 + case types.TimestampTZFamily: + // time.Time consists of two 64 bit integers and a pointer to + // time.Location. We will only account for this 3 bytes without paying + // attention to the full time.Location struct. The reason is that it is + // likely that time.Location's are cached and are shared among all the + // timestamps, so if we were to include that in the estimation, we would + // significantly overestimate. + // TODO(yuzefovich): figure out whether the caching does take place. + size = memsize.Time + case types.IntervalFamily: + size = memsize.Duration + default: + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", t)) + } + return size +} + // SetAccountingHelper is a utility struct that should be used by callers that // only perform "set" operations on the coldata.Batch (i.e. neither copies nor // appends). It encapsulates the logic for performing the memory accounting for @@ -660,8 +678,8 @@ func (h *SetAccountingHelper) AccountForSet(rowIdx int) { } } -// Close releases all of the resources so that they could be garbage collected. +// Release releases all of the resources so that they can be garbage collected. // It should be called once the caller is done with batch manipulation. -func (h *SetAccountingHelper) Close() { +func (h *SetAccountingHelper) Release() { *h = SetAccountingHelper{} } diff --git a/pkg/sql/logictest/testdata/logic_test/inverted_index b/pkg/sql/logictest/testdata/logic_test/inverted_index index d8bf517650cf..db876a1d0f05 100644 --- a/pkg/sql/logictest/testdata/logic_test/inverted_index +++ b/pkg/sql/logictest/testdata/logic_test/inverted_index @@ -1237,13 +1237,13 @@ SELECT numbers FROM cb@n WHERE numbers <@ ARRAY[]::INT[] ---- {} -query T +query T rowsort SELECT numbers FROM cb@n WHERE numbers <@ ARRAY[1] ---- {} {1} -query T +query T rowsort SELECT numbers FROM cb@n WHERE numbers <@ ARRAY[0,1,2] ---- {} @@ -1253,14 +1253,14 @@ SELECT numbers FROM cb@n WHERE numbers <@ ARRAY[0,1,2] {0,1,2} {1,2,1} -query T +query T rowsort SELECT numbers FROM cb@n WHERE numbers <@ ARRAY[1,2,3] ---- {} {1} {1,2,1} -query T +query T rowsort SELECT numbers FROM cb@n WHERE numbers <@ ARRAY[0,1,NULL] ---- {} @@ -1278,7 +1278,7 @@ SELECT words FROM cb@w WHERE words <@ ARRAY[]::STRING[] ---- {} -query T +query T rowsort SELECT words FROM cb@w WHERE words <@ ARRAY['']::STRING[] ---- {} @@ -1290,14 +1290,14 @@ SELECT words FROM cb@w WHERE words <@ ARRAY[NULL]::STRING[] {} -query T +query T rowsort SELECT words FROM cb@w WHERE words <@ ARRAY['cat'] ---- {} {cat} -query T +query T rowsort SELECT words FROM cb@w WHERE words <@ ARRAY['cat', 'mouse'] ---- {} @@ -1305,7 +1305,7 @@ SELECT words FROM cb@w WHERE words <@ ARRAY['cat', 'mouse'] {mouse} {cat,mouse} -query T +query T rowsort SELECT words FROM cb@w WHERE words <@ ARRAY['cat', 'mouse', NULL] ---- {} @@ -1313,7 +1313,7 @@ SELECT words FROM cb@w WHERE words <@ ARRAY['cat', 'mouse', NULL] {mouse} {cat,mouse} -query T +query T rowsort SELECT words FROM cb@w WHERE words <@ ARRAY[NULL, 'rat'] ---- {} diff --git a/pkg/sql/logictest/testdata/logic_test/inverted_index_geospatial b/pkg/sql/logictest/testdata/logic_test/inverted_index_geospatial index 66d5bb175b4a..7bab56f6bc7f 100644 --- a/pkg/sql/logictest/testdata/logic_test/inverted_index_geospatial +++ b/pkg/sql/logictest/testdata/logic_test/inverted_index_geospatial @@ -78,7 +78,7 @@ regions: table: geo_table@geom_index spans: 31 spans · -Diagram: https://cockroachdb.github.io/distsqlplan/decode.html#eJzMleFO40YQx7_3KUbzBVBdZdc2adiqUgqYNm0hyIlaoTpCiz0NFrbX7G7ujFAe617gnuxkGzhCFEPu7sPlg6OZyX-z8_vPru_R3GYocBL8HRxN4QZOwvEpzEldWnmVEfz7RxAGYOxlWljShmJrdncm4ej4V7d_wAe_nI9HZ9NdnzHGPGi-mLe3I8Tvwfg0mIYXTr1Wvgfj8DgI4fACbtDBQiV0JnMyKP5DjjMHS61iMkbpOnXf_GCUVCiYg2lRLmydnjkYK00o7tGmNiMUOK33GJJMSPcYOpiQlWnWLPvUwrDewGVaJFShg5NSFkZAL8LDKKr-T6Ko4iyKKvbaA3_aVsMjBFkk4DFQ9pq0QQf_-gdsmpMA9vHDQxyrwlJhU1WslbR6b0CTTAT4bebqztJjynPhEB3MZQU55UrfgcwyFUtLiQDW1K6kja_JgFrYcmEF1IyaRR8TPs6WDrbRA2Nj5ZxQ8KXzdh9GxTvSlpKTNLOkSff4qhmP9aAqNagChlyAqZ0AY6W2oiHr_bwfRcxlUcTYaw8EKpJtZbUha46MawzDer9Ngw3zFncbGyuzbNUZqiherBvWZURdM7cZWMpLSFJzAwsj5_Rmn9yNPrnb-PSnSouH4-JuOi6lTnOp7z6jcYbuCzpuF50vHnF3fcR5v8GzAfhXUPO2ofY01d4qszYvXt6OjDNe34Muc_v9A_b8c9T_jQ983gYDNuAD3w98viOeX5hDd2_jXHaS__aY_G0wTZS2pHv-KqQh__F7OzL723QVkilVYWilq00rs-XMQUrm1L7GjFromM61ipu_acNxo2sSCRnbVnkbjIq2VG_wuZh3it1usdsp9rrFXqfY7xb7neL9F-LZ8odPAQAA__8NeMBk +Diagram: https://cockroachdb.github.io/distsqlplan/decode.html#eJzMVeFO40YQ_t-nGM0fQHWVXdukYatKKWDatIUgJ2qF6ggt9jRY2F6zu7kzQnmse4F7spNt4AhRDDn-XH44mm_yTWa-b3Z9j-Y2Q4GT4O_gaAo3cBKOT2FO6tLKq4zg3z-CMABjL9PCkjYUW7O7MwlHx7-6_QM--OV8PDqb7vqMMeZB88W8vR0hfg_Gp8E0vHDqWvkejMPjIITDC7hBBwuV0JnMyaD4DznOHCy1iskYpWvovvnBKKlQMAfTolzYGp45GCtNKO7RpjYjFDitewxJJqR7DB1MyMo0a8o-jTCsG7hMi4QqdHBSysII6EV4GEXV_0kUVZxFUcVee-BP23J4hCCLBDwGyl6TNujgX_-ATXMSwD5_eohjVVgqbKqKtZRWHw1okokAv0Wu7iw9Qp4Lh-hgLivIKVf6DmSWqVhaSgSwJnclbXxNBtTClgsroNaoKfoI-DhbOthGDxobK-eEgi-dt_swKj6QtpScpJklTbrHV814zAdVqUEVMOQCTO0EGCu1FY2y3s_7UcRcFkWMvfZAoCLZllYbsubIuJZhWPfbDNho3srdxsbKLFt1hiqKF-uGdRlR58xtBpbyEpLU3MDCyDm92Sd3o0_uNj79qdLi4bi4m45LqdNc6ruv0jhD9z2b665vLu-_f3M3K-Jto8jTxnqrerS4eHnzMc54fce5zO33D9jzz1H_Nz7weRsM2IAPfD_w-Y54fhkO3b2NO-d-w869QyZ_G5kmSlvSPX9VpCH_8Xs7DvvbTBWSKVVhaGWqTZXZcuYgJXNqX1FGLXRM51rFzd-04bjhNUBCxrZZ3gajok3VDT4n806y2012O8leN9nrJPvdZL-TvP-CPFv-8CUAAP__Wwmyxw== statement ok DROP TABLE geo_table @@ -164,7 +164,7 @@ regions: table: geo_table@geom_index spans: 31 spans · -Diagram: https://cockroachdb.github.io/distsqlplan/decode.html#eJzMleFO40YQx7_3KUbzBVBdZdc2adiqUgqYNm0hyIlaoTpCiz0NFrbX7G7uglAe617gnuy0NnAEhO8iOOnyYaOZ8X-98_vvrm_RXBcocBL9HR1M4QqO4vExzEmdW3lREPz7RxRHYOx5XlnShlJrtrcm8ejwV7-_xwe_nI5HJ9PtkDHGAmj-WLCzJcTv0fg4msZnnpur3IFxfBjFsH8GV-hhpTI6kSUZFP8hx5mHtVYpGaO0S902D4yyJQrmYV7VC-vSMw9TpQnFLdrcFoQCp26NMcmMdI-hhxlZmRfNtA8tDN0CzvMqoyV6OKllZQT0EtxPkuX_WZIsWeAG9oUBf9pUwxMEWWUQMFD2krRBD__6B2xekgD28cNdnKrKUmVzVT0rafXegCaZCfDbzMWNpfsU78M-eljKJZRUKn0DsihUKi1lAlhTu5A2vSQDamHrhRXgGDWT3id8nK08bKM7xsbKOaHgK-_rfRhV70hbyo7ywpIm3ePrZtzXo2WtQVUw5AKMcwKMldqKhmzw826SMEeWOYCdAwJV2aYyZ8gzR8YOw9Ctt2mwYd7ibmNjZVGsO0NLShfPDesywtXMdQGWyhqy3FzBwsg5vYFP_iY-_any6u64-C8dl1rnpdQ3n9F4Q38TOt9gi78A_BXUgk2oPezqYJ1ZmxdPb0fGGXf3oM_8fn-PPf4d9H_jg5C3wYAN-CAMo5BviccX5tDfedN9-QpM4SaYJkpb0r1wHdKQ__i9HZndTbqKydSqMrTW1Uszs9XMQ8rm1H7GjFrolE61SpvXtOG40TWJjIxtq7wNRlVbcgt8LOadYr9b7HeKg25x0CkOu8Vhp3j3iXi2-uFTAAAA__8evcBm +Diagram: https://cockroachdb.github.io/distsqlplan/decode.html#eJzUVeFO40YQ_t-nGM0fQHWVXdukYatKKWDatIUgJ2qFzhFa7LlgYXvN7uYuCOWx7gXuyU5rA0dA-C46ftzlx0bzjb_xzPftrm_RXBcocBL9Gx1M4QqO4vExzEmdW3lREPz_VxRHYOx5XlnShlJrtrcm8ejwd7-_xwe_nY5HJ9PtkDHGAmj-WLCzJcSf0fg4msZnnqtV7sA4Poxi2D-DK_SwUhmdyJIMijfIceZhrVVKxijtoNvmgVG2RME8zKt6YR088zBVmlDcos1tQShw6nqMSWakeww9zMjKvGjKPowwdA2c51VGS_RwUsvKCOgluJ8ky7dZkixZ4Bb2hQV_2ZTDEwRZZRAwUPaStEEP__kPbF6SAPbxw12cqspSZXNVPUtp9d6AJpkJ8Fvk4sbSPcT7sI8elnIJJZVK34AsCpVKS5kA1uQupE0vyYBa2HphBTiNmqL3gI-zlYdtdKexsXJOKPjK-3ofRtU70payo7ywpEn3-LoZ9_loWWtQFQy5AOOcAGOltqJRNvh1N0mYU5Y5ATsXBKqyTWnOkGeOjJ0MQ9dvM2CjeSt3Gxsri2LdGVpSunhuWJcRLmeuC7BU1pDl5goWRs7pFXzyN_Hpb5VXd8fFf-m41Dovpb75LI039H-onRtsosjDjg3W9Whx8fTmY5xxd8f5zO_399jj30H_Dz4IeRsM2IAPwjAK-ZZ4fBkO_Z1X3XPfIFO4iUwTpS3pXrgu0pD__L0dh91NporJ1KoytDbVS5XZauYhZXNqP1FGLXRKp1qlzWvacNzwGiAjY9ssb4NR1aZcg4_JvJPsd5P9TnLQTQ46yWE3Oewk7z4hz1Y_fQoAAP__bAayyQ== # Also works when creating an index. statement ok @@ -226,4 +226,4 @@ regions: table: geo_table@geom_index spans: 31 spans · -Diagram: https://cockroachdb.github.io/distsqlplan/decode.html#eJzMleFO40YQx7_3KUbzBVBdZdc2adiqUgqYNm0hyIlaoTpCiz0NFrbX7G7uglAe617gnuy0NnAEhO8iOOnyYaOZ8X-98_vvrm_RXBcocBL9HR1M4QqO4vExzEmdW3lREPz7RxRHYOx5XlnShlJrtrcm8ejwV7-_xwe_nI5HJ9PtkDHGAmj-WLCzJcTv0fg4msZnnpur3IFxfBjFsH8GV-hhpTI6kSUZFP8hx5mHtVYpGaO0S902D4yyJQrmYV7VC-vSMw9TpQnFLdrcFoQCp26NMcmMdI-hhxlZmRfNtA8tDN0CzvMqoyV6OKllZQT0EtxPkuX_WZIsWeAG9oUBf9pUwxMEWWUQMFD2krRBD__6B2xekgD28cNdnKrKUmVzVT0rafXegCaZCfDbzMWNpfsU78M-eljKJZRUKn0DsihUKi1lAlhTu5A2vSQDamHrhRXgGDWT3id8nK08bKM7xsbKOaHgK-_rfRhV70hbyo7ywpIm3ePrZtzXo2WtQVUw5AKMcwKMldqKhmzw826SMEeWOYCdAwJV2aYyZ8gzR8YOw9Ctt2mwYd7ibmNjZVGsO0NLShfPDesywtXMdQGWyhqy3FzBwsg5vYFP_iY-_any6u64-C8dl1rnpdQ3n9F4Q38TOt9gi78A_BXUgk2oPezqYJ1ZmxdPb0fGGXf3oM_8fn-PPf4d9H_jg5C3wYAN-CAMo5BviccX5tDfedN9-QpM4SaYJkpb0r1wHdKQ__i9HZndTbqKydSqMrTW1Uszs9XMQ8rm1H7GjFrolE61SpvXtOG40TWJjIxtq7wNRlVbcgt8LOadYr9b7HeKg25x0CkOu8Vhp3j3iXi2-uFTAAAA__8evcBm +Diagram: https://cockroachdb.github.io/distsqlplan/decode.html#eJzUVeFO40YQ_t-nGM0fQHWVXdukYatKKWDatIUgJ2qFzhFa7LlgYXvN7uYuCOWx7gXuyU5rA0dA-C46ftzlx0bzjb_xzPftrm_RXBcocBL9Gx1M4QqO4vExzEmdW3lREPz_VxRHYOx5XlnShlJrtrcm8ejwd7-_xwe_nY5HJ9PtkDHGAmj-WLCzJcSf0fg4msZnnqtV7sA4Poxi2D-DK_SwUhmdyJIMijfIceZhrVVKxijtoNvmgVG2RME8zKt6YR088zBVmlDcos1tQShw6nqMSWakeww9zMjKvGjKPowwdA2c51VGS_RwUsvKCOgluJ8ky7dZkixZ4Bb2hQV_2ZTDEwRZZRAwUPaStEEP__kPbF6SAPbxw12cqspSZXNVPUtp9d6AJpkJ8Fvk4sbSPcT7sI8elnIJJZVK34AsCpVKS5kA1uQupE0vyYBa2HphBTiNmqL3gI-zlYdtdKexsXJOKPjK-3ofRtU70payo7ywpEn3-LoZ9_loWWtQFQy5AOOcAGOltqJRNvh1N0mYU5Y5ATsXBKqyTWnOkGeOjJ0MQ9dvM2CjeSt3Gxsri2LdGVpSunhuWJcRLmeuC7BU1pDl5goWRs7pFXzyN_Hpb5VXd8fFf-m41Dovpb75LI039H-onRtsosjDjg3W9Whx8fTmY5xxd8f5zO_399jj30H_Dz4IeRsM2IAPwjAK-ZZ4fBkO_Z1X3XPfIFO4iUwTpS3pXrgu0pD__L0dh91NporJ1KoytDbVS5XZauYhZXNqP1FGLXRKp1qlzWvacNzwGiAjY9ssb4NR1aZcg4_JvJPsd5P9TnLQTQ46yWE3Oewk7z4hz1Y_fQoAAP__bAayyQ== diff --git a/pkg/sql/logictest/testdata/logic_test/tpch_vec b/pkg/sql/logictest/testdata/logic_test/tpch_vec index 4926b8da573d..aa537c1ada4b 100644 --- a/pkg/sql/logictest/testdata/logic_test/tpch_vec +++ b/pkg/sql/logictest/testdata/logic_test/tpch_vec @@ -598,7 +598,7 @@ EXPLAIN (VEC) SELECT o_orderpriority, count(*) AS order_count FROM orders WHERE └ *colexec.sortOp └ *colexec.hashAggregator └ *rowexec.joinReader - └ *rowexec.joinReader + └ *colfetcher.ColIndexJoin └ *colfetcher.ColBatchScan # Query 5 @@ -614,7 +614,7 @@ EXPLAIN (VEC) SELECT n_name, sum(l_extendedprice * (1 - l_discount)) AS revenue └ *colexecjoin.hashJoiner ├ *rowexec.joinReader │ └ *colexecjoin.hashJoiner - │ ├ *rowexec.joinReader + │ ├ *colfetcher.ColIndexJoin │ │ └ *colfetcher.ColBatchScan │ └ *rowexec.joinReader │ └ *colexecjoin.hashJoiner @@ -634,7 +634,7 @@ EXPLAIN (VEC) SELECT sum(l_extendedprice * l_discount) AS revenue FROM lineitem └ *colexecsel.selLTFloat64Float64ConstOp └ *colexecsel.selLEFloat64Float64ConstOp └ *colexecsel.selGEFloat64Float64ConstOp - └ *rowexec.joinReader + └ *colfetcher.ColIndexJoin └ *colfetcher.ColBatchScan # Query 7 @@ -748,7 +748,7 @@ EXPLAIN (VEC) SELECT c_custkey, c_name, sum(l_extendedprice * (1 - l_discount)) ├ *rowexec.joinReader │ └ *colexecjoin.hashJoiner │ ├ *colfetcher.ColBatchScan - │ └ *rowexec.joinReader + │ └ *colfetcher.ColIndexJoin │ └ *colfetcher.ColBatchScan └ *colfetcher.ColBatchScan @@ -781,7 +781,7 @@ EXPLAIN (VEC) SELECT l_shipmode, sum(CASE WHEN o_orderpriority = '1-URGENT' or o └ *colexecsel.selLTInt64Int64Op └ *colexecsel.selLTInt64Int64Op └ *colexec.selectInOpBytes - └ *rowexec.joinReader + └ *colfetcher.ColIndexJoin └ *colfetcher.ColBatchScan # Query 13 @@ -813,7 +813,7 @@ EXPLAIN (VEC) SELECT 100.00 * sum(CASE WHEN p_type LIKE 'PROMO%' THEN l_extended ├ *colexec.bufferOp │ └ *colexecjoin.hashJoiner │ ├ *colfetcher.ColBatchScan - │ └ *rowexec.joinReader + │ └ *colfetcher.ColIndexJoin │ └ *colfetcher.ColBatchScan ├ *colexecproj.projMultFloat64Float64Op │ └ *colexecproj.projMinusFloat64ConstFloat64Op @@ -838,8 +838,10 @@ EXPLAIN (VEC) SELECT s_suppkey, s_name, s_address, s_phone, total_revenue FROM s └ *colexecbase.castOpNullAny └ *colexecbase.constNullOp └ *colexec.hashAggregator - └ *rowexec.joinReader - └ *colfetcher.ColBatchScan + └ *colexecproj.projMultFloat64Float64Op + └ *colexecproj.projMinusFloat64ConstFloat64Op + └ *colfetcher.ColIndexJoin + └ *colfetcher.ColBatchScan statement ok DROP VIEW revenue0 @@ -968,7 +970,7 @@ EXPLAIN (VEC) SELECT s_name, s_address FROM supplier, nation WHERE s_suppkey IN └ *colexecproj.projMultFloat64Float64ConstOp └ *colexec.hashAggregator └ *colexecjoin.hashJoiner - ├ *rowexec.joinReader + ├ *colfetcher.ColIndexJoin │ └ *colfetcher.ColBatchScan └ *colfetcher.ColBatchScan diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index e201a5b9fafb..4b6f0b874bca 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1956,6 +1956,7 @@ func TestLint(t *testing.T) { "../../sql/colexec/colexecjoin", "../../sql/colexec/colexecproj", "../../sql/colexec/colexecsel", + "../../sql/colexec/colexecspan", "../../sql/colexec/colexecwindow", "../../sql/colfetcher", "../../sql/row",