From 296ed1a87ffeea65f7a74f633fe1c195557d9fec Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 6 Nov 2020 16:03:38 -1000 Subject: [PATCH 1/4] colmem: fix appending column to the windowed batches We use `MaybeAppendColumn` to enforce that a vector of the desired type is at the correct position. However, with the introduction of the concept of capacity to the batches we now also need to make sure that the vector has the desired capacity. In vast majority of cases we are reallocating the whole batch because of the dynamic size, so it shouldn't be an issue, still this commit fixes that theoretical problem. Additionally, this commit fixes a problem of appending a vector to the windowed batches - such batches are instantiated with 0 capacity, and previously when we were appending a vector to it, we would always allocate the vector of 0 capacity. Release note: None --- pkg/sql/colmem/BUILD.bazel | 18 +++++++- pkg/sql/colmem/allocator.go | 34 ++++++++++---- pkg/sql/colmem/allocator_test.go | 79 ++++++++++++++++++++++++++++++++ pkg/testutils/lint/lint_test.go | 1 + 4 files changed, 123 insertions(+), 9 deletions(-) create mode 100644 pkg/sql/colmem/allocator_test.go diff --git a/pkg/sql/colmem/BUILD.bazel b/pkg/sql/colmem/BUILD.bazel index 80733542fabc..65b35b449e96 100644 --- a/pkg/sql/colmem/BUILD.bazel +++ b/pkg/sql/colmem/BUILD.bazel @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "colmem", @@ -17,3 +17,19 @@ go_library( "//vendor/github.com/cockroachdb/errors", ], ) + +go_test( + name = "colmem_test", + srcs = ["allocator_test.go"], + deps = [ + ":colmem", + "//pkg/col/coldata", + "//pkg/col/coldataext", + "//pkg/settings/cluster", + "//pkg/sql/colexecbase/colexecerror", + "//pkg/sql/execinfra", + "//pkg/sql/sem/tree", + "//pkg/sql/types", + "//vendor/github.com/stretchr/testify/require", + ], +) diff --git a/pkg/sql/colmem/allocator.go b/pkg/sql/colmem/allocator.go index 53cb437281de..ee7bddfc295e 100644 --- a/pkg/sql/colmem/allocator.go +++ b/pkg/sql/colmem/allocator.go @@ -242,11 +242,35 @@ func (a *Allocator) MaybeAppendColumn(b coldata.Batch, t *types.T, colIdx int) { colexecerror.InternalError(errors.AssertionFailedf("trying to add a column to zero length batch")) } width := b.Width() + desiredCapacity := b.Capacity() + if desiredCapacity == 0 { + // In some cases (like when we have a windowed batch), the capacity + // might be set to zero, yet we want to make sure that the vectors have + // enough space to accommodate the length of the batch. + desiredCapacity = b.Length() + } if colIdx < width { presentVec := b.ColVec(colIdx) presentType := presentVec.Type() + if presentType.Family() == types.UnknownFamily { + // We already have an unknown vector in place. If this is expected, + // then it will not be accessed and we're good; if this is not + // expected, then an error will occur later. + return + } if presentType.Identical(t) { // We already have the vector of the desired type in place. + if presentVec.Capacity() < desiredCapacity { + // Unfortunately, the present vector is not of sufficient + // capacity, so we need to replace it. + oldMemUsage := getVecMemoryFootprint(presentVec) + newEstimatedMemoryUsage := int64(EstimateBatchSizeBytes([]*types.T{t}, desiredCapacity)) + if err := a.acc.Grow(a.ctx, newEstimatedMemoryUsage-oldMemUsage); err != nil { + colexecerror.InternalError(err) + } + b.ReplaceCol(a.NewMemColumn(t, desiredCapacity), colIdx) + return + } if presentVec.CanonicalTypeFamily() == types.BytesFamily { // Flat bytes vector needs to be reset before the vector can be // reused. @@ -254,12 +278,6 @@ func (a *Allocator) MaybeAppendColumn(b coldata.Batch, t *types.T, colIdx int) { } return } - if presentType.Family() == types.UnknownFamily { - // We already have an unknown vector in place. If this is expected, - // then it will not be accessed and we're good; if this is not - // expected, then an error will occur later. - return - } // We have a vector with an unexpected type, so we panic. colexecerror.InternalError(errors.AssertionFailedf( "trying to add a column of %s type at index %d but %s vector already present", @@ -273,11 +291,11 @@ func (a *Allocator) MaybeAppendColumn(b coldata.Batch, t *types.T, colIdx int) { t, colIdx, width, )) } - estimatedMemoryUsage := int64(EstimateBatchSizeBytes([]*types.T{t}, b.Capacity())) + estimatedMemoryUsage := int64(EstimateBatchSizeBytes([]*types.T{t}, desiredCapacity)) if err := a.acc.Grow(a.ctx, estimatedMemoryUsage); err != nil { colexecerror.InternalError(err) } - b.AppendCol(a.NewMemColumn(t, b.Capacity())) + b.AppendCol(a.NewMemColumn(t, desiredCapacity)) } // PerformOperation executes 'operation' (that somehow modifies 'destVecs') and diff --git a/pkg/sql/colmem/allocator_test.go b/pkg/sql/colmem/allocator_test.go new file mode 100644 index 000000000000..5001f3d17f08 --- /dev/null +++ b/pkg/sql/colmem/allocator_test.go @@ -0,0 +1,79 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package colmem_test + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldataext" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/stretchr/testify/require" +) + +func TestMaybeAppendColumn(t *testing.T) { + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + testMemMonitor := execinfra.NewTestMemMonitor(ctx, st) + defer testMemMonitor.Stop(ctx) + memAcc := testMemMonitor.MakeBoundAccount() + defer memAcc.Close(ctx) + evalCtx := tree.MakeTestingEvalContext(st) + testColumnFactory := coldataext.NewExtendedColumnFactory(&evalCtx) + testAllocator := colmem.NewAllocator(ctx, &memAcc, testColumnFactory) + + t.Run("VectorAlreadyPresent", func(t *testing.T) { + b := testAllocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + b.SetLength(coldata.BatchSize()) + colIdx := 0 + + // We expect an error to occur because of a type mismatch. + err := colexecerror.CatchVectorizedRuntimeError(func() { + testAllocator.MaybeAppendColumn(b, types.Float, colIdx) + }) + require.NotNil(t, err) + + // We expect that the old vector is reallocated because the present one + // is made to be of insufficient capacity. + b.ReplaceCol(testAllocator.NewMemColumn(types.Int, 1 /* capacity */), colIdx) + testAllocator.MaybeAppendColumn(b, types.Int, colIdx) + require.Equal(t, coldata.BatchSize(), b.ColVec(colIdx).Capacity()) + + // We expect that Bytes vector is reset when it is being reused (if it + // isn't, a panic will occur when we try to set at the same positions). + bytesColIdx := 1 + testAllocator.MaybeAppendColumn(b, types.Bytes, bytesColIdx) + b.ColVec(bytesColIdx).Bytes().Set(0, []byte{0}) + b.ColVec(bytesColIdx).Bytes().Set(1, []byte{1}) + testAllocator.MaybeAppendColumn(b, types.Bytes, bytesColIdx) + b.ColVec(bytesColIdx).Bytes().Set(0, []byte{0}) + b.ColVec(bytesColIdx).Bytes().Set(1, []byte{1}) + }) + + t.Run("WindowedBatchZeroCapacity", func(t *testing.T) { + b := testAllocator.NewMemBatchWithFixedCapacity([]*types.T{}, 0 /* capacity */) + b.SetLength(coldata.BatchSize()) + colIdx := 0 + + // We expect that although the batch is of zero capacity, the newly + // appended vectors are allocated of coldata.BatchSize() capacity. + testAllocator.MaybeAppendColumn(b, types.Int, colIdx) + require.Equal(t, 1, b.Width()) + require.Equal(t, coldata.BatchSize(), b.ColVec(colIdx).Length()) + _ = b.ColVec(colIdx).Int64()[0] + }) +} diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 8432e765f221..34677cfece08 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1765,6 +1765,7 @@ func TestLint(t *testing.T) { "sql/col*", ":!sql/colexec/operator.go", ":!sql/colmem/allocator.go", + ":!sql/colmem/allocator_test.go", ) if err != nil { t.Fatal(err) From 130caeb8d316094fc9d3f9c3e21895a23b8b467c Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Sat, 21 Nov 2020 10:33:23 -1000 Subject: [PATCH 2/4] colexec: extract hash-based partitioner base from external hash joiner This commit extracts the logic of the Grace hash join algorithm into a hash-based partitioner that can be reused by the unordered distinct and the hash aggregator to add support for disk-spilling. The idea is that at planning time we can provide different in-memory "main" and disk-backed "fallback" strategies. If a partition fits under the memory limit, then the former is used; however, if recursive partitioning isn't successful in reducing the size, that partition is assigned to be handled by the latter. In case of the external hash joiner, the in-memory hash joiner is the "main" operator whereas the external sort and the merge joiner form the "fallback" operator. Release note: None --- pkg/sql/colexec/BUILD.bazel | 1 + pkg/sql/colexec/colbuilder/execplan.go | 69 +- pkg/sql/colexec/external_hash_joiner.go | 704 ++---------------- pkg/sql/colexec/external_sort.go | 12 +- pkg/sql/colexec/external_sort_test.go | 8 +- pkg/sql/colexec/hash_based_partitioner.go | 639 ++++++++++++++++ pkg/sql/colexec/hashjoiner.go | 2 +- .../testdata/logic_test/dist_vectorize | 10 +- 8 files changed, 761 insertions(+), 684 deletions(-) create mode 100644 pkg/sql/colexec/hash_based_partitioner.go diff --git a/pkg/sql/colexec/BUILD.bazel b/pkg/sql/colexec/BUILD.bazel index 35c9fc9d69a8..5e0fefc46883 100644 --- a/pkg/sql/colexec/BUILD.bazel +++ b/pkg/sql/colexec/BUILD.bazel @@ -21,6 +21,7 @@ go_library( "fn_op.go", "hash.go", "hash_aggregator.go", + "hash_based_partitioner.go", "hash_utils.go", "hashjoiner.go", "hashtable.go", diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 60cbae80c12e..cdc976350831 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -490,7 +490,6 @@ func (r opResult) createDiskBackedSort( maxNumberPartitions = args.TestingKnobs.NumForcedRepartitions } es := colexec.NewExternalSorter( - ctx, unlimitedAllocator, standaloneMemAccount, input, inputTypes, ordering, @@ -508,6 +507,43 @@ func (r opResult) createDiskBackedSort( ), nil } +// makeDistBackedSorterConstructors creates a DiskBackedSorterConstructor that +// can be used by the hash-based partitioner. +// NOTE: unless DelegateFDAcquisitions testing knob is set to true, it is up to +// the caller to acquire the necessary file descriptors up front. +func (r opResult) makeDiskBackedSorterConstructor( + ctx context.Context, + flowCtx *execinfra.FlowCtx, + args *colexec.NewColOperatorArgs, + monitorNamePrefix string, + factory coldata.ColumnFactory, +) colexec.DiskBackedSorterConstructor { + return func(input colexecbase.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column, maxNumberPartitions int) colexecbase.Operator { + if maxNumberPartitions < colexec.ExternalSorterMinPartitions { + colexecerror.InternalError(errors.AssertionFailedf( + "external sorter is attempted to be created with %d partitions, minimum %d required", + maxNumberPartitions, colexec.ExternalSorterMinPartitions, + )) + } + sortArgs := *args + if !args.TestingKnobs.DelegateFDAcquisitions { + // Set the FDSemaphore to nil. This indicates that no FDs should be + // acquired. The hash-based partitioner will do this up front. + sortArgs.FDSemaphore = nil + } + sorter, err := r.createDiskBackedSort( + ctx, flowCtx, &sortArgs, input, inputTypes, + execinfrapb.Ordering{Columns: orderingCols}, + 0 /* matchLen */, maxNumberPartitions, args.Spec.ProcessorID, + &execinfrapb.PostProcessSpec{}, monitorNamePrefix, factory, + ) + if err != nil { + colexecerror.InternalError(err) + } + return sorter + } +} + // createAndWrapRowSource takes a processor spec, creating the row source and // wrapping it using wrapRowSources. Note that the post process spec is included // in the processor creation, so make sure to clear it if it will be inspected @@ -910,34 +946,13 @@ func NewColOperator( unlimitedAllocator := colmem.NewAllocator( ctx, result.createBufferingUnlimitedMemAccount(ctx, flowCtx, monitorNamePrefix), factory, ) - // Make a copy of the DiskQueueCfg and set defaults for the hash - // joiner. The cache mode is chosen to automatically close the cache - // belonging to partitions at a parent level when repartitioning. - diskQueueCfg := args.DiskQueueCfg - diskQueueCfg.CacheMode = colcontainer.DiskQueueCacheModeClearAndReuseCache - diskQueueCfg.SetDefaultBufferSizeBytesForCacheMode() ehj := colexec.NewExternalHashJoiner( - unlimitedAllocator, hjSpec, + unlimitedAllocator, + flowCtx, + args, + hjSpec, inputOne, inputTwo, - execinfra.GetWorkMemLimit(flowCtx.Cfg), - diskQueueCfg, - args.FDSemaphore, - func(input colexecbase.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column, maxNumberPartitions int) (colexecbase.Operator, error) { - sortArgs := *args - if !args.TestingKnobs.DelegateFDAcquisitions { - // Set the FDSemaphore to nil. This indicates that no FDs - // should be acquired. The external hash joiner will do this - // up front. - sortArgs.FDSemaphore = nil - } - return result.createDiskBackedSort( - ctx, flowCtx, &sortArgs, input, inputTypes, - execinfrapb.Ordering{Columns: orderingCols}, - 0 /* matchLen */, maxNumberPartitions, spec.ProcessorID, - &execinfrapb.PostProcessSpec{}, monitorNamePrefix+"-", factory) - }, - args.TestingKnobs.NumForcedRepartitions, - args.TestingKnobs.DelegateFDAcquisitions, + result.makeDiskBackedSorterConstructor(ctx, flowCtx, args, monitorNamePrefix, factory), diskAccount, ) result.ToClose = append(result.ToClose, ehj.(colexecbase.Closer)) diff --git a/pkg/sql/colexec/external_hash_joiner.go b/pkg/sql/colexec/external_hash_joiner.go index 8f43e8335f20..8e8138901528 100644 --- a/pkg/sql/colexec/external_hash_joiner.go +++ b/pkg/sql/colexec/external_hash_joiner.go @@ -11,114 +11,36 @@ package colexec import ( - "context" - "math" - "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" - "github.com/cockroachdb/errors" "github.com/marusama/semaphore" ) -// externalHashJoinerState indicates the current state of the external hash -// joiner. -type externalHashJoinerState int - -const ( - // externalHJInitialPartitioning indicates that the operator is currently - // reading batches from both inputs and distributing tuples to different - // partitions based on the hash values. Once both inputs are exhausted, the - // external hash joiner transitions to externalHJJoinNewPartition state. - externalHJInitialPartitioning externalHashJoinerState = iota - // externalHJRecursivePartitioning indicates that the operator is recursively - // partitioning one of the existing partitions (that is too big to join at - // once). It will do so using a different hash function and will spill newly - // created partitions to disk. We also keep track whether repartitioning - // reduces the size of the partitions in question - if we see that the newly - // created largest partition is about the same in size as the "parent" - // partition (the percentage difference is less than - // externalHJRecursivePartitioningSizeDecreaseThreshold), it is likely that - // the partition consists of the tuples not distinct on the equality columns, - // so we fall back to using a combination of sort and merge join to process - // such partition. After repartitioning, the operator transitions to - // externalHJJoinNewPartition state. - externalHJRecursivePartitioning - // externalHJJoinNewPartition indicates that the operator should choose a - // partition index and join the corresponding partitions from both sides - // using the in-memory hash joiner. We will only join the partitions if the - // right side partition fits into memory (because in-memory hash joiner will - // fully buffer the right side but will process left side in the streaming - // fashion). If there are no partition indices that the operator can join, it - // transitions into externalHJRecursivePartitioning state. If there are no - // partition indices to join using in-memory hash joiner, but there are - // indices to join using sort + merge join strategy, the operator transitions - // to externalHJSortMergeNewPartition state. If there are no partition - // indices left at all to join, the operator transitions to - // externalHJFinished state. - externalHJJoinNewPartition - // externalHJJoining indicates that the operator is currently joining tuples - // from the corresponding partitions from both sides. An in-memory hash join - // operator is used to perform the join. Once the in-memory operator returns - // a zero-length batch (indicating that full output for the current - // partitions has been emitted), the external hash joiner transitions to - // externalHJJoinNewPartition state. - externalHJJoining - // externalHJSortMergeNewPartition indicates that the operator should choose - // a partition index to join using sort + merge join strategy. If there are - // no partition indices for this strategy left, the operator transitions to - // externalHJFinished state. - externalHJSortMergeNewPartition - // externalHJSortMergeJoining indicates that the operator is currently - // joining tuples from the corresponding partitions from both sides using - // (disk-backed) sort + merge join strategy. Once the in-memory merge joiner - // returns a zero-length batch (indicating that full output for the current - // partitions has been emitted), the external hash joiner transitions to - // externalHJSortMergeNewPartition state. - externalHJSortMergeJoining - // externalHJFinished indicates that the external hash joiner has emitted all - // tuples already and only zero-length batch will be emitted from now on. - externalHJFinished -) - const ( - // externalHJRecursivePartitioningSizeDecreaseThreshold determines by how - // much the newly-created partitions in the recursive partitioning stage - // should be smaller than the "parent" partition in order to consider the - // repartitioning "successful". If this threshold is not met, then this newly - // created partition will be added to sort + merge join list (which, in a - // sense, serves as the base case for "recursion"). - externalHJRecursivePartitioningSizeDecreaseThreshold = 0.05 - // externalHJDiskQueuesMemFraction determines the fraction of the available - // RAM that is allocated for the in-memory cache of disk queues. - externalHJDiskQueuesMemFraction = 0.5 // We need at least two buckets per side to make progress. However, the // minimum number of partitions necessary are the partitions in use during a // fallback to sort and merge join. We'll be using the minimum necessary per // input + 2 (1 for each spilling queue that the merge joiner uses). For // clarity this is what happens: // - The 2 partitions that need to be sorted + merged will use an FD each: 2 - // FDs. Meanwhile, each sorter will use up to externalSorterMinPartitions to + // FDs. Meanwhile, each sorter will use up to ExternalSorterMinPartitions to // sort and partition this input. At this stage 2 + 2 * - // externalSorterMinPartitions FDs are used. + // ExternalSorterMinPartitions FDs are used. // - Once the inputs (the hash joiner partitions) are finished, both FDs will // be released. The merge joiner will now be in use, which uses two // spillingQueues with 1 FD each for a total of 2. Since each sorter will - // use externalSorterMinPartitions, the FDs used at this stage are 2 + - // (2 * externalSorterMinPartitions) as well. Note that as soon as the + // use ExternalSorterMinPartitions, the FDs used at this stage are 2 + + // (2 * ExternalSorterMinPartitions) as well. Note that as soon as the // sorter emits its first batch, it must be the case that the input to it // has returned a zero batch, and thus the FD has been closed. sortMergeNonSortMinFDsOpen = 2 - externalHJMinPartitions = sortMergeNonSortMinFDsOpen + (externalSorterMinPartitions * 2) - // externalHJMinimalMaxRightPartitionSize determines the minimum value for - // maxRightPartitionSizeToJoin variable of the external hash joiner. - externalHJMinimalMaxRightPartitionSize = 64 << 10 /* 64 KiB */ + externalHJMinPartitions = sortMergeNonSortMinFDsOpen + (ExternalSorterMinPartitions * 2) ) // externalHashJoiner is an operator that performs Grace hash join algorithm @@ -153,579 +75,79 @@ const ( // hash function, spilled to disk, and so on. If repartitioning doesn't reduce // size of the partitions sufficiently, then such partitions will be handled // using the combination of disk-backed sort and merge join operators. -type externalHashJoiner struct { - twoInputNode - NonExplainable - closerHelper - - state externalHashJoinerState - unlimitedAllocator *colmem.Allocator - spec HashJoinerSpec - diskQueueCfg colcontainer.DiskQueueCfg - - // fdState is used to acquire file descriptors up front. - fdState struct { - fdSemaphore semaphore.Semaphore - acquiredFDs int - } - - // Partitioning phase variables. - leftPartitioner colcontainer.PartitionedQueue - rightPartitioner colcontainer.PartitionedQueue - tupleDistributor *tupleHashDistributor - // maxNumberActivePartitions determines the maximum number of active - // partitions that the operator is allowed to have. This number is computed - // semi-dynamically and will influence the choice of numBuckets value. - maxNumberActivePartitions int - // numBuckets is the number of buckets that a partition is divided into. - numBuckets int - // partitionsToJoinUsingInMemHash is a map from partitionIdx to a utility - // struct. This map contains all partition indices that need to be joined - // using the in-memory hash joiner. If the partition is too big, it will be - // tried to be repartitioned; if during repartitioning the size doesn't - // decrease enough, it will be added to partitionsToJoinUsingSortMerge. - partitionsToJoinUsingInMemHash map[int]*externalHJPartitionInfo - // partitionsToJoinUsingSortMerge contains all partition indices that need to - // be joined using sort + merge join strategy. Partition indices will be - // added into this map if recursive partitioning doesn't seem to make - // progress on partition' size reduction. - partitionsToJoinUsingSortMerge []int - // partitionIdxOffset stores the first "available" partition index to use. - // During the partitioning step, all tuples will go into one of the buckets - // in [partitionIdxOffset, partitionIdxOffset + numBuckets) range. - partitionIdxOffset int - // numRepartitions tracks the number of times the external hash joiner had to - // recursively repartition another partition because the latter was too big - // to join. - numRepartitions int - // scratch and recursiveScratch are helper structs. - scratch, recursiveScratch struct { - // Input sources can have different schemas, so when distributing tuples - // (i.e. copying them into scratch batch to be spilled) we might need two - // different batches. - leftBatch, rightBatch coldata.Batch - } - - // Join phase variables. - leftJoinerInput, rightJoinerInput *partitionerToOperator - inMemHashJoiner *hashJoiner - // diskBackedSortMerge is a side chain of disk-backed sorters that feed into - // disk-backed merge joiner which the external hash joiner can fall back to. - diskBackedSortMerge ResettableOperator - - memState struct { - // maxRightPartitionSizeToJoin indicates the maximum memory size of a - // partition on the right side that we're ok with joining without having to - // repartition it. We pay attention only to the right side because in-memory - // hash joiner will buffer the whole right input before processing the left - // input in a "streaming" fashion. - maxRightPartitionSizeToJoin int64 - } - - testingKnobs struct { - // numForcedRepartitions is a number of times that the external hash joiner - // is forced to recursively repartition (even if it is otherwise not - // needed) before it proceeds to actual join partitions. - numForcedRepartitions int - // delegateFDAcquisitions, if true, means that a test wants to force the - // PartitionedDiskQueues to track the number of file descriptors the hash - // joiner will open/close. This disables the default behavior of acquiring - // all file descriptors up front in Next. - delegateFDAcquisitions bool - } -} - -var _ closableOperator = &externalHashJoiner{} - -type externalHJPartitionInfo struct { - rightMemSize int64 - rightParentMemSize int64 -} - -type joinSide int - -const ( - leftSide joinSide = iota - rightSide -) // NewExternalHashJoiner returns a disk-backed hash joiner. // - unlimitedAllocator must have been created with a memory account derived // from an unlimited memory monitor. It will be used by several internal // components of the external hash joiner which is responsible for making sure // that the components stay within the memory limit. -// - numForcedRepartitions is a number of times that the external hash joiner -// is forced to recursively repartition (even if it is otherwise not needed). -// This should be non-zero only in tests. -// - delegateFDAcquisitions specifies whether the external hash joiner should -// let the partitioned disk queues acquire file descriptors instead of acquiring -// them up front in Next. Should be true only in tests. func NewExternalHashJoiner( unlimitedAllocator *colmem.Allocator, + flowCtx *execinfra.FlowCtx, + args *NewColOperatorArgs, spec HashJoinerSpec, leftInput, rightInput colexecbase.Operator, - memoryLimit int64, - diskQueueCfg colcontainer.DiskQueueCfg, - fdSemaphore semaphore.Semaphore, - createReusableDiskBackedSorter func(input colexecbase.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column, maxNumberPartitions int) (colexecbase.Operator, error), - numForcedRepartitions int, - delegateFDAcquisitions bool, + createDiskBackedSorter DiskBackedSorterConstructor, diskAcc *mon.BoundAccount, ) colexecbase.Operator { - if diskQueueCfg.CacheMode != colcontainer.DiskQueueCacheModeClearAndReuseCache { - colexecerror.InternalError(errors.Errorf("external hash joiner instantiated with suboptimal disk queue cache mode: %d", diskQueueCfg.CacheMode)) - } - partitionedDiskQueueSemaphore := fdSemaphore - if !delegateFDAcquisitions { - // To avoid deadlocks with other disk queues, we manually attempt to acquire - // the maximum number of descriptors all at once in Next. Passing in a nil - // semaphore indicates that the caller will do the acquiring. - partitionedDiskQueueSemaphore = nil - } - leftPartitioner := colcontainer.NewPartitionedDiskQueue( - spec.left.sourceTypes, diskQueueCfg, partitionedDiskQueueSemaphore, colcontainer.PartitionerStrategyDefault, diskAcc, - ) - leftJoinerInput := newPartitionerToOperator( - unlimitedAllocator, spec.left.sourceTypes, leftPartitioner, 0, /* partitionIdx */ - ) - rightPartitioner := colcontainer.NewPartitionedDiskQueue( - spec.right.sourceTypes, diskQueueCfg, partitionedDiskQueueSemaphore, colcontainer.PartitionerStrategyDefault, diskAcc, - ) - rightJoinerInput := newPartitionerToOperator( - unlimitedAllocator, spec.right.sourceTypes, rightPartitioner, 0, /* partitionIdx */ - ) - // With the default limit of 256 file descriptors, this results in 16 - // partitions. This is a hard maximum of partitions that will be used by the - // external hash joiner. Below we check whether we have enough RAM to support - // the caches of this number of partitions. - // TODO(yuzefovich): this number should be tuned. - maxNumberActivePartitions := fdSemaphore.GetLimit() / 16 - if diskQueueCfg.BufferSizeBytes > 0 { - diskQueuesTotalMemLimit := int(float64(memoryLimit) * externalHJDiskQueuesMemFraction) - numDiskQueuesThatFit := diskQueuesTotalMemLimit / diskQueueCfg.BufferSizeBytes - if numDiskQueuesThatFit < maxNumberActivePartitions { - maxNumberActivePartitions = numDiskQueuesThatFit - } - } - if maxNumberActivePartitions < externalHJMinPartitions { - maxNumberActivePartitions = externalHJMinPartitions - } - diskQueuesMemUsed := maxNumberActivePartitions * diskQueueCfg.BufferSizeBytes - makeOrderingCols := func(eqCols []uint32) []execinfrapb.Ordering_Column { - res := make([]execinfrapb.Ordering_Column, len(eqCols)) - for i, colIdx := range eqCols { - res[i].ColIdx = colIdx - } - return res - } - // We need to allocate 2 FDs for reading the partitions (reused by the merge - // joiner) that we need to join using sort + merge join strategy, and all - // others are divided between the two inputs. - externalSorterMaxNumberPartitions := (maxNumberActivePartitions - sortMergeNonSortMinFDsOpen) / 2 - if externalSorterMaxNumberPartitions < externalSorterMinPartitions { - // This code gets a maximum number of partitions based on the semaphore - // limit. In tests, this limit is set artificially low to catch any - // violations of the limit, resulting in possibly computing a low number of - // partitions for the sorter, which we overwrite here. - externalSorterMaxNumberPartitions = externalSorterMinPartitions - } - leftOrdering := makeOrderingCols(spec.left.eqCols) - leftPartitionSorter, err := createReusableDiskBackedSorter( - leftJoinerInput, spec.left.sourceTypes, leftOrdering, externalSorterMaxNumberPartitions, - ) - if err != nil { - colexecerror.InternalError(err) - } - rightOrdering := makeOrderingCols(spec.right.eqCols) - rightPartitionSorter, err := createReusableDiskBackedSorter( - rightJoinerInput, spec.right.sourceTypes, rightOrdering, externalSorterMaxNumberPartitions, - ) - if err != nil { - colexecerror.InternalError(err) - } - diskBackedSortMerge, err := NewMergeJoinOp( - unlimitedAllocator, memoryLimit, diskQueueCfg, - partitionedDiskQueueSemaphore, spec.joinType, leftPartitionSorter, rightPartitionSorter, - spec.left.sourceTypes, spec.right.sourceTypes, leftOrdering, rightOrdering, - diskAcc, - ) - if err != nil { - colexecerror.InternalError(err) - } - ehj := &externalHashJoiner{ - twoInputNode: newTwoInputNode(leftInput, rightInput), - unlimitedAllocator: unlimitedAllocator, - spec: spec, - diskQueueCfg: diskQueueCfg, - leftPartitioner: leftPartitioner, - rightPartitioner: rightPartitioner, - maxNumberActivePartitions: maxNumberActivePartitions, - // In the initial partitioning state we will use half of available - // partitions to write the partitioned input from the left side and another - // half for the right side. - // TODO(yuzefovich): figure out whether we should care about - // hj.numBuckets being a power of two (finalizeHash step is faster if so). - numBuckets: maxNumberActivePartitions / 2, - leftJoinerInput: leftJoinerInput, - rightJoinerInput: rightJoinerInput, - // Note that the external hash joiner is responsible for making sure - // that partitions to join using in-memory hash joiner fit under the - // limit, so we use the same unlimited allocator for both - // buildSideAllocator and outputUnlimitedAllocator arguments. - inMemHashJoiner: NewHashJoiner( - unlimitedAllocator, unlimitedAllocator, spec, leftJoinerInput, rightJoinerInput, + inMemMainOpConstructor := func(partitionedInputs []*partitionerToOperator) ResettableOperator { + // Note that the hash-based partitioner will make sure that partitions + // to join using in-memory hash joiner fit under the limit, so we use + // the same unlimited allocator for both buildSideAllocator and + // outputUnlimitedAllocator arguments. + return NewHashJoiner( + unlimitedAllocator, unlimitedAllocator, spec, partitionedInputs[0], partitionedInputs[1], // We start with relatively large initial number of buckets since we // expect each partition to be of significant size. uint64(coldata.BatchSize()), - ).(*hashJoiner), - diskBackedSortMerge: diskBackedSortMerge, - } - ehj.fdState.fdSemaphore = fdSemaphore - // To simplify the accounting, we will assume that the in-memory hash - // joiner's memory usage is equal to the size of the right partition to be - // joined (which will be fully buffered). This is an underestimate because a - // single batch from the left partition will be read at a time as well as an - // output batch will be used, but that shouldn't matter in the grand scheme - // of things. - ehj.memState.maxRightPartitionSizeToJoin = memoryLimit - int64(diskQueuesMemUsed) - if ehj.memState.maxRightPartitionSizeToJoin < externalHJMinimalMaxRightPartitionSize { - ehj.memState.maxRightPartitionSizeToJoin = externalHJMinimalMaxRightPartitionSize - } - ehj.testingKnobs.numForcedRepartitions = numForcedRepartitions - ehj.testingKnobs.delegateFDAcquisitions = delegateFDAcquisitions - return ehj -} - -func (hj *externalHashJoiner) Init() { - hj.inputOne.Init() - hj.inputTwo.Init() - hj.partitionsToJoinUsingInMemHash = make(map[int]*externalHJPartitionInfo) - // If we are initializing the external hash joiner, it means that we had to - // fallback from the in-memory hash joiner since the inputs had more tuples - // that could fit into the memory, and, therefore, it makes sense to - // instantiate the batches with maximum capacity. - hj.scratch.leftBatch = hj.unlimitedAllocator.NewMemBatchWithFixedCapacity(hj.spec.left.sourceTypes, coldata.BatchSize()) - hj.recursiveScratch.leftBatch = hj.unlimitedAllocator.NewMemBatchWithFixedCapacity(hj.spec.left.sourceTypes, coldata.BatchSize()) - sameSourcesSchema := len(hj.spec.left.sourceTypes) == len(hj.spec.right.sourceTypes) - for i, leftType := range hj.spec.left.sourceTypes { - if i < len(hj.spec.right.sourceTypes) && !leftType.Identical(hj.spec.right.sourceTypes[i]) { - sameSourcesSchema = false - } - } - if sameSourcesSchema { - // The schemas of both sources are the same, so we can reuse the left - // scratch batch. - hj.scratch.rightBatch = hj.scratch.leftBatch - hj.recursiveScratch.rightBatch = hj.recursiveScratch.leftBatch - } else { - hj.scratch.rightBatch = hj.unlimitedAllocator.NewMemBatchWithFixedCapacity(hj.spec.right.sourceTypes, coldata.BatchSize()) - hj.recursiveScratch.rightBatch = hj.unlimitedAllocator.NewMemBatchWithFixedCapacity(hj.spec.right.sourceTypes, coldata.BatchSize()) - } - // In the join phase, hash join operator will use the default init hash - // value, so in order to use a "different" hash function in the partitioning - // phase we use a different init hash value. - hj.tupleDistributor = newTupleHashDistributor( - defaultInitHashValue+1, hj.numBuckets, - ) - hj.state = externalHJInitialPartitioning -} - -func (hj *externalHashJoiner) partitionBatch( - ctx context.Context, batch coldata.Batch, side joinSide, parentMemSize int64, -) { - batchLen := batch.Length() - if batchLen == 0 { - return - } - scratchBatch := hj.scratch.leftBatch - eqCols := hj.spec.left.eqCols - partitioner := hj.leftPartitioner - if side == rightSide { - scratchBatch = hj.scratch.rightBatch - eqCols = hj.spec.right.eqCols - partitioner = hj.rightPartitioner - } - selections := hj.tupleDistributor.distribute(ctx, batch, eqCols) - for idx, sel := range selections { - partitionIdx := hj.partitionIdxOffset + idx - if len(sel) > 0 { - scratchBatch.ResetInternalBatch() - // The partitioner expects the batches without a selection vector, so we - // need to copy the tuples according to the selection vector into a - // scratch batch. - colVecs := scratchBatch.ColVecs() - hj.unlimitedAllocator.PerformOperation(colVecs, func() { - for i, colvec := range colVecs { - colvec.Copy(coldata.CopySliceArgs{ - SliceArgs: coldata.SliceArgs{ - Src: batch.ColVec(i), - Sel: sel, - SrcEndIdx: len(sel), - }, - }) - } - scratchBatch.SetLength(len(sel)) - }) - if err := partitioner.Enqueue(ctx, partitionIdx, scratchBatch); err != nil { - colexecerror.InternalError(err) - } - partitionInfo, ok := hj.partitionsToJoinUsingInMemHash[partitionIdx] - if !ok { - partitionInfo = &externalHJPartitionInfo{} - hj.partitionsToJoinUsingInMemHash[partitionIdx] = partitionInfo - } - if side == rightSide { - partitionInfo.rightParentMemSize = parentMemSize - // We cannot use allocator's methods directly because those - // look at the capacities of the vectors, and in our case only - // first len(sel) tuples belong to the "current" batch. - partitionInfo.rightMemSize += colmem.GetProportionalBatchMemSize(scratchBatch, int64(len(sel))) + ) + } + diskBackedFallbackOpConstructor := func( + partitionedInputs []*partitionerToOperator, + maxNumberActivePartitions int, + fdSemaphore semaphore.Semaphore, + ) ResettableOperator { + makeOrderingCols := func(eqCols []uint32) []execinfrapb.Ordering_Column { + res := make([]execinfrapb.Ordering_Column, len(eqCols)) + for i, colIdx := range eqCols { + res[i].ColIdx = colIdx } + return res } - } -} - -func (hj *externalHashJoiner) Next(ctx context.Context) coldata.Batch { -StateChanged: - for { - switch hj.state { - case externalHJInitialPartitioning: - leftBatch := hj.inputOne.Next(ctx) - rightBatch := hj.inputTwo.Next(ctx) - if leftBatch.Length() == 0 && rightBatch.Length() == 0 { - // Both inputs have been partitioned and spilled, so we transition to - // "joining" phase. Close all the open write file descriptors. - // - // TODO(yuzefovich): this will also clear the cache once the new PR is - // in. This means we will reallocate a cache whenever reading from the - // partitions. What I think we might want to do is not close the - // partitions here. Instead, we move on to joining, which will switch - // all of these reserved file descriptors to read in the best case (no - // repartitioning) and reuse the cache. Only if we need to repartition - // should we CloseAllOpenWriteFileDescriptors of both sides. It might - // also be more efficient to Dequeue from the partitions you'll read - // from before doing that to exempt them from releasing their FDs to - // the semaphore. - if err := hj.leftPartitioner.CloseAllOpenWriteFileDescriptors(ctx); err != nil { - colexecerror.InternalError(err) - } - if err := hj.rightPartitioner.CloseAllOpenWriteFileDescriptors(ctx); err != nil { - colexecerror.InternalError(err) - } - hj.inMemHashJoiner.Init() - hj.partitionIdxOffset += hj.numBuckets - hj.state = externalHJJoinNewPartition - continue - } - if !hj.testingKnobs.delegateFDAcquisitions && hj.fdState.acquiredFDs == 0 { - toAcquire := hj.maxNumberActivePartitions - if err := hj.fdState.fdSemaphore.Acquire(ctx, toAcquire); err != nil { - colexecerror.InternalError(err) - } - hj.fdState.acquiredFDs = toAcquire - } - hj.partitionBatch(ctx, leftBatch, leftSide, math.MaxInt64) - hj.partitionBatch(ctx, rightBatch, rightSide, math.MaxInt64) - - case externalHJRecursivePartitioning: - hj.numRepartitions++ - if log.V(2) && hj.numRepartitions%10 == 0 { - log.Infof(ctx, - "external hash joiner is performing %d'th repartition", hj.numRepartitions, - ) - } - // In order to use a different hash function when repartitioning, we need - // to increase the seed value of the tuple distributor. - hj.tupleDistributor.initHashValue++ - // We're actively will be using hj.numBuckets + 1 partitions (because - // we're repartitioning one side at a time), so we can set hj.numBuckets - // higher than in the initial partitioning step. - // TODO(yuzefovich): figure out whether we should care about - // hj.numBuckets being a power of two (finalizeHash step is faster if so). - hj.numBuckets = hj.maxNumberActivePartitions - 1 - hj.tupleDistributor.resetNumOutputs(hj.numBuckets) - for parentPartitionIdx, parentPartitionInfo := range hj.partitionsToJoinUsingInMemHash { - for _, side := range []joinSide{leftSide, rightSide} { - batch := hj.recursiveScratch.leftBatch - partitioner := hj.leftPartitioner - memSize := int64(math.MaxInt64) - if side == rightSide { - batch = hj.recursiveScratch.rightBatch - partitioner = hj.rightPartitioner - memSize = parentPartitionInfo.rightMemSize - } - for { - if err := partitioner.Dequeue(ctx, parentPartitionIdx, batch); err != nil { - colexecerror.InternalError(err) - } - if batch.Length() == 0 { - break - } - hj.partitionBatch(ctx, batch, side, memSize) - } - // We're done reading from this partition, and it will never be read - // from again, so we can close it. - if err := partitioner.CloseInactiveReadPartitions(ctx); err != nil { - colexecerror.InternalError(err) - } - // We're done writing to the newly created partitions. - // TODO(yuzefovich): we should not release the descriptors here. The - // invariant should be: we're entering - // externalHJRecursivePartitioning, at that stage we have at most - // numBuckets*2 file descriptors open. At the top of the state - // transition, close all open write file descriptors, which should - // reduce the open descriptors to 0. Now we open the two read' - // partitions for 2 file descriptors and whatever number of write - // partitions we want. This'll allow us to remove the call to - // CloseAllOpen... in the first state as well. - if err := partitioner.CloseAllOpenWriteFileDescriptors(ctx); err != nil { - colexecerror.InternalError(err) - } - } - for idx := 0; idx < hj.numBuckets; idx++ { - newPartitionIdx := hj.partitionIdxOffset + idx - if partitionInfo, ok := hj.partitionsToJoinUsingInMemHash[newPartitionIdx]; ok { - before, after := partitionInfo.rightParentMemSize, partitionInfo.rightMemSize - if before > 0 { - sizeDecrease := 1.0 - float64(after)/float64(before) - if sizeDecrease < externalHJRecursivePartitioningSizeDecreaseThreshold { - // We will need to join this partition using sort + merge - // join strategy. - hj.partitionsToJoinUsingSortMerge = append(hj.partitionsToJoinUsingSortMerge, newPartitionIdx) - delete(hj.partitionsToJoinUsingInMemHash, newPartitionIdx) - } - } - } - } - // We have successfully repartitioned the partitions with index - // 'parentPartitionIdx' on both sides, so we delete that index from the - // map and proceed on joining the newly created partitions. - delete(hj.partitionsToJoinUsingInMemHash, parentPartitionIdx) - hj.partitionIdxOffset += hj.numBuckets - hj.state = externalHJJoinNewPartition - continue StateChanged - } - - case externalHJJoinNewPartition: - if hj.testingKnobs.numForcedRepartitions > 0 && len(hj.partitionsToJoinUsingInMemHash) > 0 { - hj.testingKnobs.numForcedRepartitions-- - hj.state = externalHJRecursivePartitioning - continue - } - // Find next partition that we can join without having to recursively - // repartition. - for partitionIdx, partitionInfo := range hj.partitionsToJoinUsingInMemHash { - if partitionInfo.rightMemSize <= hj.memState.maxRightPartitionSizeToJoin { - // Update the inputs to in-memory hash joiner and reset the latter. - hj.leftJoinerInput.partitionIdx = partitionIdx - hj.rightJoinerInput.partitionIdx = partitionIdx - hj.inMemHashJoiner.reset(ctx) - delete(hj.partitionsToJoinUsingInMemHash, partitionIdx) - hj.state = externalHJJoining - continue StateChanged - } - } - if len(hj.partitionsToJoinUsingInMemHash) == 0 { - // All partitions to join using the hash joiner have been processed. - if len(hj.partitionsToJoinUsingSortMerge) > 0 { - // But there are still some partitions to join using sort + merge - // join strategy. - hj.diskBackedSortMerge.Init() - if log.V(2) { - log.Infof(ctx, - "external hash joiner will join %d partitions using sort + merge join", - len(hj.partitionsToJoinUsingSortMerge), - ) - } - hj.state = externalHJSortMergeNewPartition - continue - } - // All partitions have been processed, so we transition to finished - // state. - hj.state = externalHJFinished - continue - } - // We have partitions that we cannot join without recursively - // repartitioning first, so we transition to the corresponding state. - hj.state = externalHJRecursivePartitioning - continue - - case externalHJJoining: - b := hj.inMemHashJoiner.Next(ctx) - if b.Length() == 0 { - // We're done joining these partitions, so we close them and transition - // to joining new ones. - if err := hj.leftPartitioner.CloseInactiveReadPartitions(ctx); err != nil { - colexecerror.InternalError(err) - } - if err := hj.rightPartitioner.CloseInactiveReadPartitions(ctx); err != nil { - colexecerror.InternalError(err) - } - hj.state = externalHJJoinNewPartition - continue - } - return b - - case externalHJSortMergeNewPartition: - if len(hj.partitionsToJoinUsingSortMerge) == 0 { - // All partitions have been processed, so we transition to finished - // state. - hj.state = externalHJFinished - continue - } - partitionIdx := hj.partitionsToJoinUsingSortMerge[0] - hj.partitionsToJoinUsingSortMerge = hj.partitionsToJoinUsingSortMerge[1:] - // Update the inputs to sort + merge joiner and reset that chain. - hj.leftJoinerInput.partitionIdx = partitionIdx - hj.rightJoinerInput.partitionIdx = partitionIdx - hj.diskBackedSortMerge.reset(ctx) - hj.state = externalHJSortMergeJoining - continue - - case externalHJSortMergeJoining: - b := hj.diskBackedSortMerge.Next(ctx) - if b.Length() == 0 { - // We're done joining these partitions, so we close them and transition - // to joining new ones. - if err := hj.leftPartitioner.CloseInactiveReadPartitions(ctx); err != nil { - colexecerror.InternalError(err) - } - if err := hj.rightPartitioner.CloseInactiveReadPartitions(ctx); err != nil { - colexecerror.InternalError(err) - } - hj.state = externalHJSortMergeNewPartition - continue - } - return b - - case externalHJFinished: - if err := hj.Close(ctx); err != nil { - colexecerror.InternalError(err) - } - return coldata.ZeroBatch - default: - colexecerror.InternalError(errors.AssertionFailedf("unexpected externalHashJoinerState %d", hj.state)) + // We need to allocate 2 FDs for reading the partitions (reused by the merge + // joiner) that we need to join using sort + merge join strategy, and all + // others are divided between the two inputs. + externalSorterMaxNumberPartitions := (maxNumberActivePartitions - sortMergeNonSortMinFDsOpen) / 2 + leftOrdering := makeOrderingCols(spec.left.eqCols) + leftPartitionSorter := createDiskBackedSorter( + partitionedInputs[0], spec.left.sourceTypes, leftOrdering, externalSorterMaxNumberPartitions, + ) + rightOrdering := makeOrderingCols(spec.right.eqCols) + rightPartitionSorter := createDiskBackedSorter( + partitionedInputs[1], spec.right.sourceTypes, rightOrdering, externalSorterMaxNumberPartitions, + ) + diskBackedSortMerge, err := NewMergeJoinOp( + unlimitedAllocator, execinfra.GetWorkMemLimit(flowCtx.Cfg), args.DiskQueueCfg, + fdSemaphore, spec.joinType, leftPartitionSorter, rightPartitionSorter, + spec.left.sourceTypes, spec.right.sourceTypes, leftOrdering, rightOrdering, + diskAcc, + ) + if err != nil { + colexecerror.InternalError(err) } - } -} - -func (hj *externalHashJoiner) Close(ctx context.Context) error { - if !hj.close() { - return nil - } - var retErr error - if err := hj.leftPartitioner.Close(ctx); err != nil { - retErr = err - } - if err := hj.rightPartitioner.Close(ctx); err != nil && retErr == nil { - retErr = err - } - if err := hj.diskBackedSortMerge.(colexecbase.Closer).Close(ctx); err != nil && retErr == nil { - retErr = err - } - if !hj.testingKnobs.delegateFDAcquisitions && hj.fdState.acquiredFDs > 0 { - hj.fdState.fdSemaphore.Release(hj.fdState.acquiredFDs) - hj.fdState.acquiredFDs = 0 - } - return retErr + return diskBackedSortMerge + } + return newHashBasedPartitioner( + unlimitedAllocator, + flowCtx, + args, + "external hash joiner", /* name */ + []colexecbase.Operator{leftInput, rightInput}, + [][]*types.T{spec.left.sourceTypes, spec.right.sourceTypes}, + [][]uint32{spec.left.eqCols, spec.right.eqCols}, + inMemMainOpConstructor, + diskBackedFallbackOpConstructor, + diskAcc, + externalHJMinPartitions, + ) } diff --git a/pkg/sql/colexec/external_sort.go b/pkg/sql/colexec/external_sort.go index c136baff3612..0c3cfe933df8 100644 --- a/pkg/sql/colexec/external_sort.go +++ b/pkg/sql/colexec/external_sort.go @@ -60,9 +60,10 @@ const ( externalSorterFinished ) -// In order to make progress when merging we have to merge at least two -// partitions into a new third one. -const externalSorterMinPartitions = 3 +// ExternalSorterMinPartitions determines the minimum number of file descriptors +// that the external sorter needs to have in order to make progress (when +// merging, we have to merge at least two partitions into a new third one). +const ExternalSorterMinPartitions = 3 // externalSorter is an Operator that performs external merge sort. It works in // two stages: @@ -164,7 +165,6 @@ var _ closableOperator = &externalSorter{} // the partitioned disk queue acquire file descriptors instead of acquiring // them up front in Next. This should only be true in tests. func NewExternalSorter( - ctx context.Context, unlimitedAllocator *colmem.Allocator, standaloneMemAccount *mon.BoundAccount, input colexecbase.Operator, @@ -187,8 +187,8 @@ func NewExternalSorter( // TODO(asubiotto): this number should be tuned. maxNumberPartitions = fdSemaphore.GetLimit() / 16 } - if maxNumberPartitions < externalSorterMinPartitions { - maxNumberPartitions = externalSorterMinPartitions + if maxNumberPartitions < ExternalSorterMinPartitions { + maxNumberPartitions = ExternalSorterMinPartitions } // Each disk queue will use up to BufferSizeBytes of RAM, so we reduce the // memoryLimit of the partitions to sort in memory by those cache sizes. To be diff --git a/pkg/sql/colexec/external_sort_test.go b/pkg/sql/colexec/external_sort_test.go index 51913cfdcfe1..fe2191208b72 100644 --- a/pkg/sql/colexec/external_sort_test.go +++ b/pkg/sql/colexec/external_sort_test.go @@ -80,10 +80,10 @@ func TestExternalSort(t *testing.T) { tc.expected, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { - // A sorter should never exceed externalSorterMinPartitions, even + // A sorter should never exceed ExternalSorterMinPartitions, even // during repartitioning. A panic will happen if a sorter requests // more than this number of file descriptors. - sem := colexecbase.NewTestingSemaphore(externalSorterMinPartitions) + sem := colexecbase.NewTestingSemaphore(ExternalSorterMinPartitions) // If a limit is satisfied before the sorter is drained of all its // tuples, the sorter will not close its partitioner. During a // flow this will happen in a downstream materializer/outbox, @@ -166,7 +166,7 @@ func TestExternalSortRandomized(t *testing.T) { // limit. With a maximum number of partitions of 2 this will result in // repartitioning twice. To make this a total amount of memory, we also need // to add the cache sizes of the queues. - partitionSize := int64(memoryToSort/4) + int64(externalSorterMinPartitions*queueCfg.BufferSizeBytes) + partitionSize := int64(memoryToSort/4) + int64(ExternalSorterMinPartitions*queueCfg.BufferSizeBytes) for _, tk := range []execinfra.TestingKnobs{{ForceDiskSpill: true}, {MemoryLimitBytes: partitionSize}} { flowCtx.Cfg.TestingKnobs = tk for nCols := 1; nCols <= maxCols; nCols++ { @@ -198,7 +198,7 @@ func TestExternalSortRandomized(t *testing.T) { expected, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { - sem := colexecbase.NewTestingSemaphore(externalSorterMinPartitions) + sem := colexecbase.NewTestingSemaphore(ExternalSorterMinPartitions) semsToCheck = append(semsToCheck, sem) sorter, newAccounts, newMonitors, closers, err := createDiskBackedSorter( ctx, flowCtx, input, typs[:nCols], ordCols, diff --git a/pkg/sql/colexec/hash_based_partitioner.go b/pkg/sql/colexec/hash_based_partitioner.go new file mode 100644 index 000000000000..d34b34ba0b3d --- /dev/null +++ b/pkg/sql/colexec/hash_based_partitioner.go @@ -0,0 +1,639 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package colexec + +import ( + "context" + "math" + + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/mon" + "github.com/cockroachdb/errors" + "github.com/marusama/semaphore" +) + +// hashBasedPartitionerState indicates the current state of the hash-based +// partitioner. +type hashBasedPartitionerState int + +const ( + // hbpInitialPartitioning indicates that the operator is currently reading + // batches from its inputs and distributing tuples to different partitions + // based on the hash values. Once all inputs are exhausted, the hash-based + // partitioner transitions to hbpProcessNewPartitionUsingMain state. + hbpInitialPartitioning hashBasedPartitionerState = iota + // hbpRecursivePartitioning indicates that the operator is recursively + // partitioning one of the existing partitions (that is too big to process + // using the "main" strategy at once). It will do so using a different hash + // function and will spill newly created partitions to disk. We also keep + // track whether repartitioning reduces the size of the partitions in + // question - if we see that the newly created largest partition is about + // the same in size as the "parent" partition (the percentage difference is + // less than hbpRecursivePartitioningSizeDecreaseThreshold), it is likely + // that the partition consists of the tuples not distinct on the equality + // columns, so we fall back to processing strategy that is provided as the + // "fallback". After repartitioning, the operator transitions to + // hbpProcessNewPartitionUsingMain state. + hbpRecursivePartitioning + // hbpProcessNewPartitionUsingMain indicates that the operator should choose + // a partition index and process the corresponding partitions from all + // inputs using the "main" operator. We will only process the partitions if + // the partition fits into memory. If there are no partition indices that + // the operator can process, it transitions into hbpRecursivePartitioning + // state. If there are no partition indices to process using the main + // operator, but there are indices to process using the "fallback" strategy, + // the operator transitions to hbpProcessNewPartitionUsingFallback state. If + // there are no partition indices left at all to process, the operator + // transitions to hbpFinished state. + hbpProcessNewPartitionUsingMain + // hbpProcessingUsingMain indicates that the operator is currently + // processing tuples from the corresponding partitions from all inputs using + // the "main" operator. Once the "main" operator returns a zero-length batch + // (indicating that full output for the current partitions has been + // emitted), the hash-based partitioner transitions to + // hbpProcessNewPartitionUsingMain state. + hbpProcessingUsingMain + // hbpProcessNewPartitionUsingFallback indicates that the operator should + // choose a partition index to process using the "fallback" strategy. If + // there are no partition indices for this strategy left, the operator + // transitions to hbpFinished state. + hbpProcessNewPartitionUsingFallback + // hbpProcessingUsingFallback indicates that the operator is currently + // processing tuples from the corresponding partitions from all inputs using + // (disk-backed) "fallback" strategy. Once the "fallback" returns a + // zero-length batch (indicating that full output for the current partitions + // has been emitted), the hash-based partitioner transitions to + // hbpProcessNewPartitionUsingFallback state. + hbpProcessingUsingFallback + // hbpFinished indicates that the hash-based partitioner has emitted all + // tuples already and only zero-length batch will be emitted from now on. + hbpFinished +) + +const ( + // hbpRecursivePartitioningSizeDecreaseThreshold determines by how much the + // newly-created partitions in the recursive partitioning stage should be + // smaller than the "parent" partition in order to consider the + // repartitioning "successful". If this threshold is not met, then the newly + // created partition will be processed by the "fallback" disk-backed + // operator. + hbpRecursivePartitioningSizeDecreaseThreshold = 0.05 + // hbpDiskQueuesMemFraction determines the fraction of the available RAM + // that is allocated for the in-memory cache of disk queues. + hbpDiskQueuesMemFraction = 0.5 + // hbpMinimalMaxPartitionSizeForMain determines the minimum value for the + // partition size that can be processed using the "main" in-memory strategy. + hbpMinimalMaxPartitionSizeForMain = 64 << 10 /* 64 KiB */ +) + +// hashBasedPartitioner is an operator that extracts the logic of Grace hash +// join (see the comment in external_hash_joiner.go for more details) to be +// reused for other operations. +// +// It works in two phases: partitioning and processing. Every partition is +// processed either using the "main" in-memory strategy (preferable) or the +// "fallback" disk-backed strategy (when the recursive repartitioning doesn't +// seem to make progress in reducing the size of the partitions). +type hashBasedPartitioner struct { + NonExplainable + closerHelper + + unlimitedAllocator *colmem.Allocator + name string + state hashBasedPartitionerState + inputs []colexecbase.Operator + inputTypes [][]*types.T + hashCols [][]uint32 + inMemMainOp ResettableOperator + diskBackedFallbackOp ResettableOperator + maxPartitionSizeToProcessUsingMain int64 + // fdState is used to acquire file descriptors up front. + fdState struct { + fdSemaphore semaphore.Semaphore + acquiredFDs int + } + + partitioners []*colcontainer.PartitionedDiskQueue + partitionedInputs []*partitionerToOperator + tupleDistributor *tupleHashDistributor + // maxNumberActivePartitions determines the maximum number of active + // partitions that the operator is allowed to have. This number is computed + // semi-dynamically and will influence the choice of numBuckets value. + maxNumberActivePartitions int + // numBuckets is the number of buckets that a partition is divided into. + numBuckets int + // partitionsToProcessUsingMain is a map from partitionIdx to a utility + // struct. This map contains all partition indices that need to be processed + // using the in-memory "main" operator. If the partition is too big, it will + // be tried to be repartitioned; if during repartitioning the size doesn't + // decrease enough, it will be added to partitionsToProcessUsingFallback. + partitionsToProcessUsingMain map[int]*hbpPartitionInfo + // partitionsToProcessUsingFallback contains all partition indices that need + // to be processed using the "fallback" strategy. Partition indices will be + // added into this map if recursive partitioning doesn't seem to make + // progress on partition' size reduction. + partitionsToProcessUsingFallback []int + // partitionIdxOffset stores the first "available" partition index to use. + // During the partitioning step, all tuples will go into one of the buckets + // in [partitionIdxOffset, partitionIdxOffset + numBuckets) range. + partitionIdxOffset int + // numRepartitions tracks the number of times the hash-based partitioner had + // to recursively repartition another partition because the latter was too + // big to process using the "main" operator. + numRepartitions int + // scratch and recursiveScratch are helper structs. + scratch, recursiveScratch struct { + // Input sources can have different schemas, so when distributing tuples + // (i.e. copying them into scratch batch to be spilled) we might need + // two different batches. + batches []coldata.Batch + } + + testingKnobs struct { + // numForcedRepartitions is a number of times that the hash-based + // partitioner is forced to recursively repartition (even if it is + // otherwise not needed) before it proceeds to actually processing the + // partitions. + numForcedRepartitions int + // delegateFDAcquisitions, if true, means that a test wants to force the + // PartitionedDiskQueues to track the number of file descriptors the + // hash-based partitioner will open/close. This disables the default + // behavior of acquiring all file descriptors up front in Next. + delegateFDAcquisitions bool + } +} + +var _ closableOperator = &hashBasedPartitioner{} + +// hbpPartitionInfo is a helper struct that tracks the memory usage of a +// partition. Note that if the hash-based partitioner has two inputs, we take +// the right partition size for this since that is the input we're buffering +// from. +type hbpPartitionInfo struct { + memSize int64 + parentMemSize int64 +} + +// DiskBackedSorterConstructor is used by the external operators to instantiate +// a disk-backed sorter used in the fallback strategies. +type DiskBackedSorterConstructor func(input colexecbase.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column, maxNumberPartitions int) colexecbase.Operator + +// newHashBasedPartitioner returns a disk-backed operator that utilizes +// partitioning by hash approach to divide up the input set into separate +// partitions which are then processed using the "main" in-memory operator if +// they fit under the memory limit. If a partition is too big, it is attempted +// to be recursively repartitioned; if that is not successful, the partition in +// question is handled by the "fallback" disk-backed operator. +func newHashBasedPartitioner( + unlimitedAllocator *colmem.Allocator, + flowCtx *execinfra.FlowCtx, + args *NewColOperatorArgs, + name string, + inputs []colexecbase.Operator, + inputTypes [][]*types.T, + hashCols [][]uint32, + inMemMainOpConstructor func([]*partitionerToOperator) ResettableOperator, + diskBackedFallbackOpConstructor func( + partitionedInputs []*partitionerToOperator, + maxNumberActivePartitions int, + fdSemaphore semaphore.Semaphore, + ) ResettableOperator, + diskAcc *mon.BoundAccount, + numRequiredActivePartitions int, +) *hashBasedPartitioner { + // Make a copy of the DiskQueueCfg and set defaults for the partitioning + // operators. The cache mode is chosen to automatically close the cache + // belonging to partitions at a parent level when repartitioning. + diskQueueCfg := args.DiskQueueCfg + diskQueueCfg.CacheMode = colcontainer.DiskQueueCacheModeClearAndReuseCache + diskQueueCfg.SetDefaultBufferSizeBytesForCacheMode() + partitionedDiskQueueSemaphore := args.FDSemaphore + if !args.TestingKnobs.DelegateFDAcquisitions { + // To avoid deadlocks with other disk queues, we manually attempt to + // acquire the maximum number of descriptors all at once in Next. + // Passing in a nil semaphore indicates that the caller will do the + // acquiring. + partitionedDiskQueueSemaphore = nil + } + numInputs := len(inputs) + partitioners := make([]*colcontainer.PartitionedDiskQueue, numInputs) + partitionedInputs := make([]*partitionerToOperator, numInputs) + for i := range inputs { + partitioners[i] = colcontainer.NewPartitionedDiskQueue( + inputTypes[i], diskQueueCfg, partitionedDiskQueueSemaphore, colcontainer.PartitionerStrategyDefault, diskAcc, + ) + partitionedInputs[i] = newPartitionerToOperator( + unlimitedAllocator, inputTypes[i], partitioners[i], 0, /* partitionIdx */ + ) + } + maxNumberActivePartitions := calculateMaxNumberActivePartitions(flowCtx, args, numRequiredActivePartitions) + diskQueuesMemUsed := maxNumberActivePartitions * diskQueueCfg.BufferSizeBytes + maxPartitionSizeToProcessUsingMain := execinfra.GetWorkMemLimit(flowCtx.Cfg) - int64(diskQueuesMemUsed) + if maxPartitionSizeToProcessUsingMain < hbpMinimalMaxPartitionSizeForMain { + maxPartitionSizeToProcessUsingMain = hbpMinimalMaxPartitionSizeForMain + } + op := &hashBasedPartitioner{ + unlimitedAllocator: unlimitedAllocator, + name: name, + inputs: inputs, + inputTypes: inputTypes, + hashCols: hashCols, + inMemMainOp: inMemMainOpConstructor(partitionedInputs), + diskBackedFallbackOp: diskBackedFallbackOpConstructor( + partitionedInputs, maxNumberActivePartitions, partitionedDiskQueueSemaphore, + ), + maxPartitionSizeToProcessUsingMain: maxPartitionSizeToProcessUsingMain, + partitioners: partitioners, + partitionedInputs: partitionedInputs, + maxNumberActivePartitions: maxNumberActivePartitions, + // In the initial partitioning state we will use all available + // partitions fairly among all inputs. + // TODO(yuzefovich): figure out whether we should care about + // op.numBuckets being a power of two (finalizeHash step is faster if + // so). + numBuckets: maxNumberActivePartitions / numInputs, + } + op.fdState.fdSemaphore = args.FDSemaphore + op.testingKnobs.numForcedRepartitions = args.TestingKnobs.NumForcedRepartitions + op.testingKnobs.delegateFDAcquisitions = args.TestingKnobs.DelegateFDAcquisitions + return op +} + +func calculateMaxNumberActivePartitions( + flowCtx *execinfra.FlowCtx, args *NewColOperatorArgs, numRequiredActivePartitions int, +) int { + // With the default limit of 256 file descriptors, this results in 16 + // partitions. This is a hard maximum of partitions that will be used by the + // hash-based partitioner. Below we check whether we have enough RAM to + // support the caches of this number of partitions. + // TODO(yuzefovich): this number should be tuned. + maxNumberActivePartitions := args.FDSemaphore.GetLimit() / 16 + memoryLimit := execinfra.GetWorkMemLimit(flowCtx.Cfg) + if args.DiskQueueCfg.BufferSizeBytes > 0 { + diskQueuesTotalMemLimit := int(float64(memoryLimit) * hbpDiskQueuesMemFraction) + numDiskQueuesThatFit := diskQueuesTotalMemLimit / args.DiskQueueCfg.BufferSizeBytes + if numDiskQueuesThatFit < maxNumberActivePartitions { + maxNumberActivePartitions = numDiskQueuesThatFit + } + } + if maxNumberActivePartitions < numRequiredActivePartitions { + maxNumberActivePartitions = numRequiredActivePartitions + } + return maxNumberActivePartitions +} + +func (op *hashBasedPartitioner) Init() { + for i := range op.inputs { + op.inputs[i].Init() + } + op.partitionsToProcessUsingMain = make(map[int]*hbpPartitionInfo) + // If we are initializing the hash-based partitioner, it means that we had + // to fallback from the in-memory one since the inputs had more tuples that + // could fit into the memory, and, therefore, it makes sense to instantiate + // the batches with maximum capacity. + op.scratch.batches = append(op.scratch.batches, op.unlimitedAllocator.NewMemBatchWithFixedCapacity(op.inputTypes[0], coldata.BatchSize())) + op.recursiveScratch.batches = append(op.recursiveScratch.batches, op.unlimitedAllocator.NewMemBatchWithFixedCapacity(op.inputTypes[0], coldata.BatchSize())) + if len(op.inputs) == 2 { + sameSourcesSchema := len(op.inputTypes[0]) == len(op.inputTypes[1]) + for i := 0; sameSourcesSchema && i < len(op.inputTypes[0]); i++ { + if !op.inputTypes[0][i].Identical(op.inputTypes[1][i]) { + sameSourcesSchema = false + } + } + if sameSourcesSchema { + // The schemas of both sources are the same, so we can reuse the + // first scratch batch. + op.scratch.batches = append(op.scratch.batches, op.scratch.batches[0]) + op.recursiveScratch.batches = append(op.recursiveScratch.batches, op.recursiveScratch.batches[0]) + } else { + op.scratch.batches = append(op.scratch.batches, op.unlimitedAllocator.NewMemBatchWithFixedCapacity(op.inputTypes[1], coldata.BatchSize())) + op.recursiveScratch.batches = append(op.recursiveScratch.batches, op.unlimitedAllocator.NewMemBatchWithFixedCapacity(op.inputTypes[1], coldata.BatchSize())) + } + } + // In the processing phase, the in-memory operator will use the default init + // hash value, so in order to use a "different" hash function in the + // partitioning phase we use a different init hash value. + op.tupleDistributor = newTupleHashDistributor( + defaultInitHashValue+1, op.numBuckets, + ) + op.state = hbpInitialPartitioning +} + +func (op *hashBasedPartitioner) partitionBatch( + ctx context.Context, batch coldata.Batch, inputIdx int, parentMemSize int64, +) { + batchLen := batch.Length() + if batchLen == 0 { + return + } + scratchBatch := op.scratch.batches[inputIdx] + selections := op.tupleDistributor.distribute(ctx, batch, op.hashCols[inputIdx]) + for idx, sel := range selections { + partitionIdx := op.partitionIdxOffset + idx + if len(sel) > 0 { + scratchBatch.ResetInternalBatch() + // The partitioner expects the batches without a selection vector, + // so we need to copy the tuples according to the selection vector + // into a scratch batch. + colVecs := scratchBatch.ColVecs() + op.unlimitedAllocator.PerformOperation(colVecs, func() { + for i, colvec := range colVecs { + colvec.Copy(coldata.CopySliceArgs{ + SliceArgs: coldata.SliceArgs{ + Src: batch.ColVec(i), + Sel: sel, + SrcEndIdx: len(sel), + }, + }) + } + scratchBatch.SetLength(len(sel)) + }) + if err := op.partitioners[inputIdx].Enqueue(ctx, partitionIdx, scratchBatch); err != nil { + colexecerror.InternalError(err) + } + partitionInfo, ok := op.partitionsToProcessUsingMain[partitionIdx] + if !ok { + partitionInfo = &hbpPartitionInfo{} + op.partitionsToProcessUsingMain[partitionIdx] = partitionInfo + } + if inputIdx == len(op.inputs)-1 { + partitionInfo.parentMemSize = parentMemSize + // We cannot use allocator's methods directly because those look + // at the capacities of the vectors, and in our case only first + // len(sel) tuples belong to the "current" batch. + partitionInfo.memSize += colmem.GetProportionalBatchMemSize(scratchBatch, int64(len(sel))) + } + } + } +} + +func (op *hashBasedPartitioner) Next(ctx context.Context) coldata.Batch { + var batches [2]coldata.Batch +StateChanged: + for { + switch op.state { + case hbpInitialPartitioning: + allZero := true + for i := range op.inputs { + batches[i] = op.inputs[i].Next(ctx) + allZero = allZero && batches[i].Length() == 0 + } + if allZero { + // All inputs have been partitioned and spilled, so we + // transition to processing phase. Close all the open write file + // descriptors. + // + // TODO(yuzefovich): this will also clear the cache once the new + // PR is in. This means we will reallocate a cache whenever + // reading from the partitions. What I think we might want to do + // is not close the partitions here. Instead, we move on to + // joining, which will switch all of these reserved file + // descriptors to read in the best case (no repartitioning) and + // reuse the cache. Only if we need to repartition should we + // CloseAllOpenWriteFileDescriptors of both sides. It might also + // be more efficient to Dequeue from the partitions you'll read + // from before doing that to exempt them from releasing their + // FDs to the semaphore. + for i := range op.inputs { + if err := op.partitioners[i].CloseAllOpenWriteFileDescriptors(ctx); err != nil { + colexecerror.InternalError(err) + } + } + op.inMemMainOp.Init() + op.partitionIdxOffset += op.numBuckets + op.state = hbpProcessNewPartitionUsingMain + continue + } + if !op.testingKnobs.delegateFDAcquisitions && op.fdState.acquiredFDs == 0 { + toAcquire := op.maxNumberActivePartitions + if err := op.fdState.fdSemaphore.Acquire(ctx, toAcquire); err != nil { + colexecerror.InternalError(err) + } + op.fdState.acquiredFDs = toAcquire + } + for i := range op.inputs { + op.partitionBatch(ctx, batches[i], i, math.MaxInt64) + } + + case hbpRecursivePartitioning: + op.numRepartitions++ + if log.V(2) && op.numRepartitions%10 == 0 { + log.Infof(ctx, + "%s is performing %d'th repartition", op.name, op.numRepartitions, + ) + } + // In order to use a different hash function when repartitioning, we + // need to increase the seed value of the tuple distributor. + op.tupleDistributor.initHashValue++ + // We're actively will be using op.numBuckets + 1 partitions + // (because we're repartitioning one side at a time), so we can set + // op.numBuckets higher than in the initial partitioning step. + // TODO(yuzefovich): figure out whether we should care about + // op.numBuckets being a power of two (finalizeHash step is faster + // if so). + op.numBuckets = op.maxNumberActivePartitions - 1 + op.tupleDistributor.resetNumOutputs(op.numBuckets) + for parentPartitionIdx, parentPartitionInfo := range op.partitionsToProcessUsingMain { + for i := range op.inputs { + batch := op.recursiveScratch.batches[i] + partitioner := op.partitioners[i] + for { + if err := partitioner.Dequeue(ctx, parentPartitionIdx, batch); err != nil { + colexecerror.InternalError(err) + } + if batch.Length() == 0 { + break + } + op.partitionBatch(ctx, batch, i, parentPartitionInfo.memSize) + } + // We're done reading from this partition, and it will never + // be read from again, so we can close it. + if err := partitioner.CloseInactiveReadPartitions(ctx); err != nil { + colexecerror.InternalError(err) + } + // We're done writing to the newly created partitions. + // TODO(yuzefovich): we should not release the descriptors + // here. The invariant should be: we're entering + // hbpRecursivePartitioning, at that stage we have at + // most numBuckets*2 file descriptors open. At the top of + // the state transition, close all open write file + // descriptors, which should reduce the open descriptors to + // 0. Now we open the two read' partitions for 2 file + // descriptors and whatever number of write partitions we + // want. This will allow us to remove the call to + // CloseAllOpen... in the first state as well. + if err := partitioner.CloseAllOpenWriteFileDescriptors(ctx); err != nil { + colexecerror.InternalError(err) + } + } + for idx := 0; idx < op.numBuckets; idx++ { + newPartitionIdx := op.partitionIdxOffset + idx + if partitionInfo, ok := op.partitionsToProcessUsingMain[newPartitionIdx]; ok { + before, after := partitionInfo.parentMemSize, partitionInfo.memSize + if before > 0 { + sizeDecrease := 1.0 - float64(after)/float64(before) + if sizeDecrease < hbpRecursivePartitioningSizeDecreaseThreshold { + // We will need to process this partition using + // the "fallback" strategy. + op.partitionsToProcessUsingFallback = append(op.partitionsToProcessUsingFallback, newPartitionIdx) + delete(op.partitionsToProcessUsingMain, newPartitionIdx) + } + } + } + } + // We have successfully repartitioned the partitions with index + // 'parentPartitionIdx' from all inputs, so we delete that index + // from the map and proceed on processing the newly created + // partitions. + delete(op.partitionsToProcessUsingMain, parentPartitionIdx) + op.partitionIdxOffset += op.numBuckets + op.state = hbpProcessNewPartitionUsingMain + continue StateChanged + } + + case hbpProcessNewPartitionUsingMain: + if op.testingKnobs.numForcedRepartitions > 0 && len(op.partitionsToProcessUsingMain) > 0 { + op.testingKnobs.numForcedRepartitions-- + op.state = hbpRecursivePartitioning + continue + } + // Find next partition that we can process without having to + // recursively repartition. + for partitionIdx, partitionInfo := range op.partitionsToProcessUsingMain { + if partitionInfo.memSize <= op.maxPartitionSizeToProcessUsingMain { + for i := range op.partitionedInputs { + op.partitionedInputs[i].partitionIdx = partitionIdx + } + op.inMemMainOp.reset(ctx) + delete(op.partitionsToProcessUsingMain, partitionIdx) + op.state = hbpProcessingUsingMain + continue StateChanged + } + } + if len(op.partitionsToProcessUsingMain) == 0 { + // All partitions to process using the in-memory operator have + // been exhausted. + if len(op.partitionsToProcessUsingFallback) > 0 { + // But there are still some partitions to process using the + // "fallback" strategy. + op.diskBackedFallbackOp.Init() + if log.V(2) { + log.Infof(ctx, + `%s will process %d partitions using the "fallback" strategy`, + op.name, len(op.partitionsToProcessUsingFallback), + ) + } + op.state = hbpProcessNewPartitionUsingFallback + continue + } + // All partitions have been processed, so we transition to + // finished state. + op.state = hbpFinished + continue + } + // We have partitions that we cannot process without recursively + // repartitioning first, so we transition to the corresponding state. + op.state = hbpRecursivePartitioning + continue + + case hbpProcessingUsingMain: + b := op.inMemMainOp.Next(ctx) + if b.Length() == 0 { + // We're done processing these partitions, so we close them and + // transition to processing new ones. + for i := range op.inputs { + if err := op.partitioners[i].CloseInactiveReadPartitions(ctx); err != nil { + colexecerror.InternalError(err) + } + } + op.state = hbpProcessNewPartitionUsingMain + continue + } + return b + + case hbpProcessNewPartitionUsingFallback: + if len(op.partitionsToProcessUsingFallback) == 0 { + // All partitions have been processed, so we transition to + // finished state. + op.state = hbpFinished + continue + } + partitionIdx := op.partitionsToProcessUsingFallback[0] + op.partitionsToProcessUsingFallback = op.partitionsToProcessUsingFallback[1:] + for i := range op.partitionedInputs { + op.partitionedInputs[i].partitionIdx = partitionIdx + } + op.diskBackedFallbackOp.reset(ctx) + op.state = hbpProcessingUsingFallback + continue + + case hbpProcessingUsingFallback: + b := op.diskBackedFallbackOp.Next(ctx) + if b.Length() == 0 { + // We're done processing these partitions, so we close them and + // transition to processing new ones. + for i := range op.inputs { + if err := op.partitioners[i].CloseInactiveReadPartitions(ctx); err != nil { + colexecerror.InternalError(err) + } + } + op.state = hbpProcessNewPartitionUsingFallback + continue + } + return b + + case hbpFinished: + if err := op.Close(ctx); err != nil { + colexecerror.InternalError(err) + } + return coldata.ZeroBatch + + default: + colexecerror.InternalError(errors.AssertionFailedf("unexpected hashBasedPartitionerState %d", op.state)) + } + } +} + +func (op *hashBasedPartitioner) Close(ctx context.Context) error { + if !op.close() { + return nil + } + var retErr error + for i := range op.inputs { + if err := op.partitioners[i].Close(ctx); err != nil { + retErr = err + } + } + if err := op.diskBackedFallbackOp.(colexecbase.Closer).Close(ctx); err != nil && retErr == nil { + retErr = err + } + if !op.testingKnobs.delegateFDAcquisitions && op.fdState.acquiredFDs > 0 { + op.fdState.fdSemaphore.Release(op.fdState.acquiredFDs) + op.fdState.acquiredFDs = 0 + } + return retErr +} + +func (op *hashBasedPartitioner) ChildCount(_ bool) int { + return len(op.inputs) +} + +func (op *hashBasedPartitioner) Child(nth int, _ bool) execinfra.OpNode { + return op.inputs[nth] +} diff --git a/pkg/sql/colexec/hashjoiner.go b/pkg/sql/colexec/hashjoiner.go index 9b0d9154aa0a..8d0b2cfdb444 100644 --- a/pkg/sql/colexec/hashjoiner.go +++ b/pkg/sql/colexec/hashjoiner.go @@ -775,7 +775,7 @@ func NewHashJoiner( spec HashJoinerSpec, leftSource, rightSource colexecbase.Operator, initialNumBuckets uint64, -) colexecbase.Operator { +) ResettableOperator { var outputTypes []*types.T if spec.joinType.ShouldIncludeLeftColsInOutput() { outputTypes = append(outputTypes, spec.left.sourceTypes...) diff --git a/pkg/sql/logictest/testdata/logic_test/dist_vectorize b/pkg/sql/logictest/testdata/logic_test/dist_vectorize index c7c00c2959fd..242b10d16203 100644 --- a/pkg/sql/logictest/testdata/logic_test/dist_vectorize +++ b/pkg/sql/logictest/testdata/logic_test/dist_vectorize @@ -167,7 +167,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ │ └ *colrpc.Inbox │ │ ├ *colexec.ParallelUnorderedSynchronizer │ │ ├ *colexec.ParallelUnorderedSynchronizer -│ │ └ *colexec.externalHashJoiner +│ │ └ *colexec.hashBasedPartitioner │ │ ├ *colexec.bufferExportingOperator │ │ └ *colexec.bufferExportingOperator │ ├ *colrpc.Inbox @@ -201,7 +201,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ └ *colrpc.Inbox │ ├ *colexec.ParallelUnorderedSynchronizer │ ├ *colexec.ParallelUnorderedSynchronizer -│ └ *colexec.externalHashJoiner +│ └ *colexec.hashBasedPartitioner │ ├ *colexec.bufferExportingOperator │ └ *colexec.bufferExportingOperator ├ Node 3 @@ -231,7 +231,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ └ *colrpc.Inbox │ ├ *colexec.ParallelUnorderedSynchronizer │ ├ *colexec.ParallelUnorderedSynchronizer -│ └ *colexec.externalHashJoiner +│ └ *colexec.hashBasedPartitioner │ ├ *colexec.bufferExportingOperator │ └ *colexec.bufferExportingOperator ├ Node 4 @@ -261,7 +261,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ └ *colrpc.Inbox │ ├ *colexec.ParallelUnorderedSynchronizer │ ├ *colexec.ParallelUnorderedSynchronizer -│ └ *colexec.externalHashJoiner +│ └ *colexec.hashBasedPartitioner │ ├ *colexec.bufferExportingOperator │ └ *colexec.bufferExportingOperator └ Node 5 @@ -291,7 +291,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ └ *colfetcher.ColBatchScan ├ *colexec.ParallelUnorderedSynchronizer ├ *colexec.ParallelUnorderedSynchronizer - └ *colexec.externalHashJoiner + └ *colexec.hashBasedPartitioner ├ *colexec.bufferExportingOperator └ *colexec.bufferExportingOperator From ba4c43fba3c7eb8dc04ca5ac22ac0839752631c6 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 7 Dec 2020 22:11:42 -0800 Subject: [PATCH 3/4] colexec: fix test harness in some cases when comparing sets This commit fixes our test harness when comparing expected and actual tuples as sets. We use a sort to speed things up, and previously in some cases we could have different ordering for essentially the same sets of tuples because in the actual ones we have tree.Datums and in the expected ones we have strings. Release note: None --- pkg/sql/colexec/utils_test.go | 52 ++++++++++++++++++++++++++++------- 1 file changed, 42 insertions(+), 10 deletions(-) diff --git a/pkg/sql/colexec/utils_test.go b/pkg/sql/colexec/utils_test.go index 14b90b3993bc..3e33c41a1ad2 100644 --- a/pkg/sql/colexec/utils_test.go +++ b/pkg/sql/colexec/utils_test.go @@ -71,7 +71,7 @@ func (t tuple) String() string { return sb.String() } -func (t tuple) less(other tuple, evalCtx *tree.EvalContext) bool { +func (t tuple) less(other tuple, evalCtx *tree.EvalContext, tupleFromOtherSet tuple) bool { for i := range t { // If either side is nil, we short circuit the comparison. For nil, we // define: nil < {any_none_nil} @@ -137,7 +137,22 @@ func (t tuple) less(other tuple, evalCtx *tree.EvalContext) bool { case "bool": return lhsVal.Bool() == false && rhsVal.Bool() == true case "string": - return lhsVal.String() < rhsVal.String() + lString, rString := lhsVal.String(), rhsVal.String() + if tupleFromOtherSet != nil && len(tupleFromOtherSet) > i { + if d, ok := tupleFromOtherSet[i].(tree.Datum); ok { + // The tuple from the other set has a datum value, so we + // will convert the string to datum. See the comment on + // tuples.sort for more details. + d1 := stringToDatum(lString, d.ResolvedType(), evalCtx) + d2 := stringToDatum(rString, d.ResolvedType(), evalCtx) + cmp := d1.Compare(evalCtx, d2) + if cmp == 0 { + continue + } + return cmp < 0 + } + } + return lString < rString default: colexecerror.InternalError(errors.AssertionFailedf("Unhandled comparison type: %s", typ)) } @@ -178,8 +193,15 @@ func (t tuples) String() string { return sb.String() } -// sort returns a copy of sorted tuples. -func (t tuples) sort(evalCtx *tree.EvalContext) tuples { +// sort returns a copy of sorted tuples. tupleFromOtherSet is any tuple that +// comes from other tuples and is used to determine the desired types. +// +// Currently, this function is only used in order to speed up the comparison of +// the expected tuple set with the actual one, and it is possible that we have +// tree.Datum in the latter but strings in the former. In order to use the same +// ordering when sorting the strings, we need to peek into the actual tuple to +// determine whether we want to convert the string to datum before comparison. +func (t tuples) sort(evalCtx *tree.EvalContext, tupleFromOtherSet tuple) tuples { b := make(tuples, len(t)) for i := range b { b[i] = make(tuple, len(t[i])) @@ -188,7 +210,7 @@ func (t tuples) sort(evalCtx *tree.EvalContext) tuples { sort.SliceStable(b, func(i, j int) bool { lhs := b[i] rhs := b[j] - return lhs.less(rhs, evalCtx) + return lhs.less(rhs, evalCtx, tupleFromOtherSet) }) return b } @@ -1147,8 +1169,15 @@ func assertTuplesSetsEqual(expected tuples, actual tuples, evalCtx *tree.EvalCon if len(expected) != len(actual) { return makeError(expected, actual) } - actual = actual.sort(evalCtx) - expected = expected.sort(evalCtx) + var tupleFromOtherSet tuple + if len(expected) > 0 { + tupleFromOtherSet = expected[0] + } + actual = actual.sort(evalCtx, tupleFromOtherSet) + if len(actual) > 0 { + tupleFromOtherSet = actual[0] + } + expected = expected.sort(evalCtx, tupleFromOtherSet) return assertTuplesOrderedEqual(expected, actual, evalCtx) } @@ -1412,10 +1441,13 @@ func (c *chunkingBatchSource) Next(context.Context) coldata.Batch { if lastIdx > c.len { lastIdx = c.len } - for i, vec := range c.batch.ColVecs() { - vec.SetCol(c.cols[i].Window(c.curIdx, lastIdx).Col()) + for i := range c.typs { + // Note that new vectors could be appended to the batch, but we are not + // responsible for updating those, so we iterate only up to len(c.typs) + // as per out initialization. + c.batch.ColVec(i).SetCol(c.cols[i].Window(c.curIdx, lastIdx).Col()) nullsSlice := c.cols[i].Nulls().Slice(c.curIdx, lastIdx) - vec.SetNulls(&nullsSlice) + c.batch.ColVec(i).SetNulls(&nullsSlice) } c.batch.SetLength(lastIdx - c.curIdx) c.curIdx = lastIdx From 0ca7f6c61f5a3262911ddd655ade3b82d0111392 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 7 Dec 2020 22:13:28 -0800 Subject: [PATCH 4/4] colexec: implement disk-spilling for unordered distinct This commit introduces an external distinct operator that reuses the hash-based partitioner that works according to "partitioning by hash" scheme when dividing the input into separate partitions. The "main" strategy for the new operator is already existing in-memory unordered distinct whereas the "fallback" strategy is the external sort followed by the ordered distinct. The benchmarks have shown that using such approach is significantly faster in most cases when comparing against the external sort + ordered distinct approach (i.e. against making the fallback strategy the main one). One important detail of the fallback strategy is that we need to make sure that we keep the very first tuple from the input among all tuples that are identical on the distinct columns. If we naively use a sort plus ordered distinct, we might break that, so in order to go around it, we plan an ordinality operator and include the ordinality column as the last one in the ordering, we then project out that temporary column when feeding into the ordered distinct. Another important detail is that distinct is expected to maintain the ordering of the output stream (to be the same as the ordering of the input stream) when the output ordering is specified. Previously, this was achieved for "free" in both the vectorized and the row-by-row engines; however, with the partitioning by hash approach we don't have that anymore. Therefore, a new field was added to DistinctSpec that specified the desired output ordering, and the field is now used to optionally plan an external sort on top of the external distinct. The benchmarks have shown that the performance overhead of having such sort is relatively small. Release note: None --- pkg/sql/colexec/BUILD.bazel | 2 + pkg/sql/colexec/bool_vec_to_sel.go | 8 +- pkg/sql/colexec/colbuilder/execplan.go | 48 +- pkg/sql/colexec/distinct.eg.go | 4 +- pkg/sql/colexec/distinct_test.go | 426 ++++++------ pkg/sql/colexec/distinct_tmpl.go | 4 +- pkg/sql/colexec/external_distinct.go | 107 +++ pkg/sql/colexec/external_distinct_test.go | 223 ++++++ pkg/sql/colexec/hash_based_partitioner.go | 9 +- pkg/sql/colexec/ordered_aggregator.go | 9 +- pkg/sql/colexec/unordered_distinct.go | 37 +- pkg/sql/distsql/columnar_operators_test.go | 6 + pkg/sql/distsql_physical_planner.go | 93 ++- pkg/sql/distsql_spec_exec_factory.go | 10 +- pkg/sql/exec_factory_util.go | 9 + pkg/sql/execinfrapb/processors_sql.pb.go | 632 ++++++++++-------- pkg/sql/execinfrapb/processors_sql.proto | 4 + .../testdata/logic_test/explain_analyze_plans | 2 +- pkg/sql/opt/exec/factory.opt | 5 +- 19 files changed, 1057 insertions(+), 581 deletions(-) create mode 100644 pkg/sql/colexec/external_distinct.go create mode 100644 pkg/sql/colexec/external_distinct_test.go diff --git a/pkg/sql/colexec/BUILD.bazel b/pkg/sql/colexec/BUILD.bazel index 5e0fefc46883..c82c5de10bb4 100644 --- a/pkg/sql/colexec/BUILD.bazel +++ b/pkg/sql/colexec/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "deselector.go", "disk_spiller.go", "expr.go", + "external_distinct.go", "external_hash_joiner.go", "external_sort.go", "fn_op.go", @@ -115,6 +116,7 @@ go_test( "dep_test.go", "deselector_test.go", "distinct_test.go", + "external_distinct_test.go", "external_hash_joiner_test.go", "external_sort_test.go", "hash_utils_test.go", diff --git a/pkg/sql/colexec/bool_vec_to_sel.go b/pkg/sql/colexec/bool_vec_to_sel.go index c2fb2ab05fcf..d1c5c924b38e 100644 --- a/pkg/sql/colexec/bool_vec_to_sel.go +++ b/pkg/sql/colexec/bool_vec_to_sel.go @@ -48,7 +48,7 @@ type boolVecToSelOp struct { outputCol []bool } -var _ colexecbase.Operator = &boolVecToSelOp{} +var _ ResettableOperator = &boolVecToSelOp{} func (p *boolVecToSelOp) Next(ctx context.Context) coldata.Batch { // Loop until we have non-zero amount of output to return, or our input's been @@ -107,6 +107,12 @@ func (p *boolVecToSelOp) Init() { p.input.Init() } +func (p *boolVecToSelOp) reset(ctx context.Context) { + if r, ok := p.input.(resetter); ok { + r.reset(ctx) + } +} + func boolVecToSel64(vec []bool, sel []int) []int { l := len(vec) for i := 0; i < l; i++ { diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index cdc976350831..7523b4a721ee 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -860,24 +860,44 @@ func NewColOperator( if len(core.Distinct.OrderedColumns) == len(core.Distinct.DistinctColumns) { result.Op, err = colexec.NewOrderedDistinct(inputs[0], core.Distinct.OrderedColumns, result.ColumnTypes) } else { - distinctMemAccount := streamingMemAccount - if !useStreamingMemAccountForBuffering { - // Create an unlimited mem account explicitly even though there is no - // disk spilling because the memory usage of an unordered distinct - // operator is proportional to the number of distinct tuples, not the - // number of input tuples. - // The row execution engine also gives an unlimited amount (that still - // needs to be approved by the upstream monitor, so not really - // "unlimited") amount of memory to the unordered distinct operator. - distinctMemAccount = result.createBufferingUnlimitedMemAccount(ctx, flowCtx, "distinct") - } + // We have separate unit tests that instantiate in-memory + // distinct operators, so we don't need to look at + // args.TestingKnobs.DiskSpillingDisabled and always instantiate + // a disk-backed one here. + distinctMemMonitorName := fmt.Sprintf("distinct-%d", spec.ProcessorID) + distinctMemAccount := result.createMemAccountForSpillStrategy( + ctx, flowCtx, distinctMemMonitorName, + ) // TODO(yuzefovich): we have an implementation of partially ordered // distinct, and we should plan it when we have non-empty ordered // columns and we think that the probability of distinct tuples in the // input is about 0.01 or less. - result.Op = colexec.NewUnorderedDistinct( - colmem.NewAllocator(ctx, distinctMemAccount, factory), inputs[0], - core.Distinct.DistinctColumns, result.ColumnTypes, + allocator := colmem.NewAllocator(ctx, distinctMemAccount, factory) + inMemoryUnorderedDistinct := colexec.NewUnorderedDistinct( + allocator, inputs[0], core.Distinct.DistinctColumns, result.ColumnTypes, + ) + diskAccount := result.createDiskAccount(ctx, flowCtx, distinctMemMonitorName) + result.Op = colexec.NewOneInputDiskSpiller( + inputs[0], inMemoryUnorderedDistinct.(colexecbase.BufferingInMemoryOperator), + distinctMemMonitorName, + func(input colexecbase.Operator) colexecbase.Operator { + monitorNamePrefix := "external-distinct" + unlimitedAllocator := colmem.NewAllocator( + ctx, result.createBufferingUnlimitedMemAccount(ctx, flowCtx, monitorNamePrefix), factory, + ) + ed := colexec.NewExternalDistinct( + unlimitedAllocator, + flowCtx, + args, + input, + result.ColumnTypes, + result.makeDiskBackedSorterConstructor(ctx, flowCtx, args, monitorNamePrefix, factory), + diskAccount, + ) + result.ToClose = append(result.ToClose, ed.(colexecbase.Closer)) + return ed + }, + args.TestingKnobs.SpillingCallbackFn, ) } diff --git a/pkg/sql/colexec/distinct.eg.go b/pkg/sql/colexec/distinct.eg.go index 3efcfadc3b32..60bc1bc3dd05 100644 --- a/pkg/sql/colexec/distinct.eg.go +++ b/pkg/sql/colexec/distinct.eg.go @@ -41,7 +41,7 @@ var ( // last distinct operator in that chain as well as its output column. func OrderedDistinctColsToOperators( input colexecbase.Operator, distinctCols []uint32, typs []*types.T, -) (colexecbase.Operator, []bool, error) { +) (ResettableOperator, []bool, error) { distinctCol := make([]bool, coldata.BatchSize()) // zero the boolean column on every iteration. input = fnOp{ @@ -78,7 +78,7 @@ var _ ResettableOperator = &distinctChainOps{} // input columns with the given types. func NewOrderedDistinct( input colexecbase.Operator, distinctCols []uint32, typs []*types.T, -) (colexecbase.Operator, error) { +) (ResettableOperator, error) { op, outputCol, err := OrderedDistinctColsToOperators(input, distinctCols, typs) if err != nil { return nil, err diff --git a/pkg/sql/colexec/distinct_test.go b/pkg/sql/colexec/distinct_test.go index cddaa54cc411..1dddf6cf8052 100644 --- a/pkg/sql/colexec/distinct_test.go +++ b/pkg/sql/colexec/distinct_test.go @@ -25,166 +25,168 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/randutil" ) -func TestDistinct(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - rng, _ := randutil.NewPseudoRand() - tcs := []struct { - distinctCols []uint32 - typs []*types.T - tuples []tuple - expected []tuple - isOrderedOnDistinctCols bool - }{ - { - distinctCols: []uint32{0, 1, 2}, - typs: []*types.T{types.Float, types.Int, types.String, types.Int}, - tuples: tuples{ - {nil, nil, nil, nil}, - {nil, nil, nil, nil}, - {nil, nil, "30", nil}, - {1.0, 2, "30", 4}, - {1.0, 2, "30", 4}, - {2.0, 2, "30", 4}, - {2.0, 3, "30", 4}, - {2.0, 3, "40", 4}, - {2.0, 3, "40", 4}, - }, - expected: tuples{ - {nil, nil, nil, nil}, - {nil, nil, "30", nil}, - {1.0, 2, "30", 4}, - {2.0, 2, "30", 4}, - {2.0, 3, "30", 4}, - {2.0, 3, "40", 4}, - }, - isOrderedOnDistinctCols: true, +type distinctTestCase struct { + distinctCols []uint32 + typs []*types.T + tuples []tuple + expected []tuple + isOrderedOnDistinctCols bool +} + +var distinctTestCases = []distinctTestCase{ + { + distinctCols: []uint32{0, 1, 2}, + typs: []*types.T{types.Float, types.Int, types.String, types.Int}, + tuples: tuples{ + {nil, nil, nil, nil}, + {nil, nil, nil, nil}, + {nil, nil, "30", nil}, + {1.0, 2, "30", 4}, + {1.0, 2, "30", 4}, + {2.0, 2, "30", 4}, + {2.0, 3, "30", 4}, + {2.0, 3, "40", 4}, + {2.0, 3, "40", 4}, }, - { - distinctCols: []uint32{1, 0, 2}, - typs: []*types.T{types.Float, types.Int, types.Bytes, types.Int}, - tuples: tuples{ - {nil, nil, nil, nil}, - {nil, nil, nil, nil}, - {nil, nil, "30", nil}, - {1.0, 2, "30", 4}, - {1.0, 2, "30", 4}, - {2.0, 2, "30", 4}, - {2.0, 3, "30", 4}, - {2.0, 3, "40", 4}, - {2.0, 3, "40", 4}, - }, - expected: tuples{ - {nil, nil, nil, nil}, - {nil, nil, "30", nil}, - {1.0, 2, "30", 4}, - {2.0, 2, "30", 4}, - {2.0, 3, "30", 4}, - {2.0, 3, "40", 4}, - }, - isOrderedOnDistinctCols: true, + expected: tuples{ + {nil, nil, nil, nil}, + {nil, nil, "30", nil}, + {1.0, 2, "30", 4}, + {2.0, 2, "30", 4}, + {2.0, 3, "30", 4}, + {2.0, 3, "40", 4}, }, - { - distinctCols: []uint32{0, 1, 2}, - typs: []*types.T{types.Float, types.Int, types.String, types.Int}, - tuples: tuples{ - {1.0, 2, "30", 4}, - {1.0, 2, "30", 4}, - {nil, nil, nil, nil}, - {nil, nil, nil, nil}, - {2.0, 2, "30", 4}, - {2.0, 3, "30", 4}, - {nil, nil, "30", nil}, - {2.0, 3, "40", 4}, - {2.0, 3, "40", 4}, - }, - expected: tuples{ - {1.0, 2, "30", 4}, - {nil, nil, nil, nil}, - {2.0, 2, "30", 4}, - {2.0, 3, "30", 4}, - {nil, nil, "30", nil}, - {2.0, 3, "40", 4}, - }, + isOrderedOnDistinctCols: true, + }, + { + distinctCols: []uint32{1, 0, 2}, + typs: []*types.T{types.Float, types.Int, types.Bytes, types.Int}, + tuples: tuples{ + {nil, nil, nil, nil}, + {nil, nil, nil, nil}, + {nil, nil, "30", nil}, + {1.0, 2, "30", 4}, + {1.0, 2, "30", 4}, + {2.0, 2, "30", 4}, + {2.0, 3, "30", 4}, + {2.0, 3, "40", 4}, + {2.0, 3, "40", 4}, }, - { - distinctCols: []uint32{0}, - typs: []*types.T{types.Int, types.Bytes}, - tuples: tuples{ - {1, "a"}, - {2, "b"}, - {3, "c"}, - {nil, "d"}, - {5, "e"}, - {6, "f"}, - {1, "1"}, - {2, "2"}, - {3, "3"}, - }, - expected: tuples{ - {1, "a"}, - {2, "b"}, - {3, "c"}, - {nil, "d"}, - {5, "e"}, - {6, "f"}, - }, + expected: tuples{ + {nil, nil, nil, nil}, + {nil, nil, "30", nil}, + {1.0, 2, "30", 4}, + {2.0, 2, "30", 4}, + {2.0, 3, "30", 4}, + {2.0, 3, "40", 4}, }, - { - // This is to test hashTable deduplication with various batch size - // boundaries and ensure it always emits the first tuple it encountered. - distinctCols: []uint32{0}, - typs: []*types.T{types.Int, types.String}, - tuples: tuples{ - {1, "1"}, - {1, "2"}, - {1, "3"}, - {1, "4"}, - {1, "5"}, - {2, "6"}, - {2, "7"}, - {2, "8"}, - {2, "9"}, - {2, "10"}, - {0, "11"}, - {0, "12"}, - {0, "13"}, - {1, "14"}, - {1, "15"}, - {1, "16"}, - }, - expected: tuples{ - {1, "1"}, - {2, "6"}, - {0, "11"}, - }, + isOrderedOnDistinctCols: true, + }, + { + distinctCols: []uint32{0, 1, 2}, + typs: []*types.T{types.Float, types.Int, types.String, types.Int}, + tuples: tuples{ + {1.0, 2, "30", 4}, + {1.0, 2, "30", 4}, + {nil, nil, nil, nil}, + {nil, nil, nil, nil}, + {2.0, 2, "30", 4}, + {2.0, 3, "30", 4}, + {nil, nil, "30", nil}, + {2.0, 3, "40", 4}, + {2.0, 3, "40", 4}, }, - { - distinctCols: []uint32{0}, - typs: []*types.T{types.Jsonb, types.String}, - tuples: tuples{ - {`'{"id": 1}'`, "a"}, - {`'{"id": 2}'`, "b"}, - {`'{"id": 3}'`, "c"}, - {`'{"id": 1}'`, "1"}, - {`'{"id": null}'`, "d"}, - {`'{"id": 2}'`, "2"}, - {`'{"id": 5}'`, "e"}, - {`'{"id": 6}'`, "f"}, - {`'{"id": 3}'`, "3"}, - }, - expected: tuples{ - {`'{"id": 1}'`, "a"}, - {`'{"id": 2}'`, "b"}, - {`'{"id": 3}'`, "c"}, - {`'{"id": null}'`, "d"}, - {`'{"id": 5}'`, "e"}, - {`'{"id": 6}'`, "f"}, - }, + expected: tuples{ + {1.0, 2, "30", 4}, + {nil, nil, nil, nil}, + {2.0, 2, "30", 4}, + {2.0, 3, "30", 4}, + {nil, nil, "30", nil}, + {2.0, 3, "40", 4}, }, - } + }, + { + distinctCols: []uint32{0}, + typs: []*types.T{types.Int, types.Bytes}, + tuples: tuples{ + {1, "a"}, + {2, "b"}, + {3, "c"}, + {nil, "d"}, + {5, "e"}, + {6, "f"}, + {1, "1"}, + {2, "2"}, + {3, "3"}, + }, + expected: tuples{ + {1, "a"}, + {2, "b"}, + {3, "c"}, + {nil, "d"}, + {5, "e"}, + {6, "f"}, + }, + }, + { + // This is to test hashTable deduplication with various batch size + // boundaries and ensure it always emits the first tuple it encountered. + distinctCols: []uint32{0}, + typs: []*types.T{types.Int, types.String}, + tuples: tuples{ + {1, "1"}, + {1, "2"}, + {1, "3"}, + {1, "4"}, + {1, "5"}, + {2, "6"}, + {2, "7"}, + {2, "8"}, + {2, "9"}, + {2, "10"}, + {0, "11"}, + {0, "12"}, + {0, "13"}, + {1, "14"}, + {1, "15"}, + {1, "16"}, + }, + expected: tuples{ + {1, "1"}, + {2, "6"}, + {0, "11"}, + }, + }, + { + distinctCols: []uint32{0}, + typs: []*types.T{types.Jsonb, types.String}, + tuples: tuples{ + {`'{"id": 1}'`, "a"}, + {`'{"id": 2}'`, "b"}, + {`'{"id": 3}'`, "c"}, + {`'{"id": 1}'`, "1"}, + {`'{"id": null}'`, "d"}, + {`'{"id": 2}'`, "2"}, + {`'{"id": 5}'`, "e"}, + {`'{"id": 6}'`, "f"}, + {`'{"id": 3}'`, "3"}, + }, + expected: tuples{ + {`'{"id": 1}'`, "a"}, + {`'{"id": 2}'`, "b"}, + {`'{"id": 3}'`, "c"}, + {`'{"id": null}'`, "d"}, + {`'{"id": 5}'`, "e"}, + {`'{"id": 6}'`, "f"}, + }, + }, +} - for _, tc := range tcs { +func TestDistinct(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + rng, _ := randutil.NewPseudoRand() + for _, tc := range distinctTestCases { log.Infof(context.Background(), "unordered") runTestsWithTyps(t, []tuples{tc.tuples}, [][]*types.T{tc.typs}, tc.expected, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { @@ -202,8 +204,7 @@ func TestDistinct(t *testing.T) { runTestsWithTyps(t, []tuples{tc.tuples}, [][]*types.T{tc.typs}, tc.expected, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { return newPartiallyOrderedDistinct( - testAllocator, input[0], tc.distinctCols, - orderedCols, tc.typs, + testAllocator, input[0], tc.distinctCols, orderedCols, tc.typs, ) }) } @@ -216,30 +217,29 @@ func TestDistinct(t *testing.T) { } } -func BenchmarkDistinct(b *testing.B) { +// runDistinctBenchmarks runs the benchmarks of a distinct operator variant on +// multiple configurations. +func runDistinctBenchmarks( + ctx context.Context, + b *testing.B, + distinctConstructor func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecbase.Operator, error), + getNumOrderedCols func(nCols int) int, + namePrefix string, + isExternal bool, +) { rng, _ := randutil.NewPseudoRand() - ctx := context.Background() - - distinctConstructors := []func(*colmem.Allocator, colexecbase.Operator, []uint32, int, []*types.T) (colexecbase.Operator, error){ - func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecbase.Operator, error) { - return NewUnorderedDistinct(allocator, input, distinctCols, typs), nil - }, - func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecbase.Operator, error) { - return newPartiallyOrderedDistinct(allocator, input, distinctCols, distinctCols[:numOrderedCols], typs) - }, - func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecbase.Operator, error) { - return NewOrderedDistinct(input, distinctCols, typs) - }, - } - distinctNames := []string{"Unordered", "PartiallyOrdered", "Ordered"} - orderedColsFraction := []float64{0, 0.5, 1.0} + nullsOptions := []bool{false, true} nRowsOptions := []int{1, 64, 4 * coldata.BatchSize(), 256 * coldata.BatchSize()} nColsOptions := []int{2, 4} + if isExternal { + nullsOptions = []bool{false} + nRowsOptions = []int{coldata.BatchSize(), 64 * coldata.BatchSize(), 4096 * coldata.BatchSize()} + } if testing.Short() { nRowsOptions = []int{coldata.BatchSize()} nColsOptions = []int{2} } - for _, hasNulls := range []bool{false, true} { + for _, hasNulls := range nullsOptions { for _, newTupleProbability := range []float64{0.001, 0.1} { for _, nRows := range nRowsOptions { for _, nCols := range nColsOptions { @@ -250,6 +250,7 @@ func BenchmarkDistinct(b *testing.B) { cols[i] = testAllocator.NewMemColumn(typs[i], nRows) } distinctCols := []uint32{0, 1, 2, 3}[:nCols] + numOrderedCols := getNumOrderedCols(nCols) // We have the following equation: // newTupleProbability = 1 - (1 - newValueProbability) ^ nCols, // so applying some manipulations we get: @@ -268,34 +269,65 @@ func BenchmarkDistinct(b *testing.B) { cols[i].Nulls().SetNull(0) } } - for distinctIdx, distinctConstructor := range distinctConstructors { - numOrderedCols := int(float64(nCols) * orderedColsFraction[distinctIdx]) - b.Run( - fmt.Sprintf("%s/hasNulls=%v/newTupleProbability=%.3f/rows=%d/cols=%d/ordCols=%d", - distinctNames[distinctIdx], hasNulls, newTupleProbability, - nRows, nCols, numOrderedCols, - ), - func(b *testing.B) { - b.SetBytes(int64(8 * nRows * nCols)) - b.ResetTimer() - for n := 0; n < b.N; n++ { - // Note that the source will be ordered on all nCols so that the - // number of distinct tuples doesn't vary between different - // distinct operator variations. - source := newChunkingBatchSource(typs, cols, nRows) - distinct, err := distinctConstructor(testAllocator, source, distinctCols, numOrderedCols, typs) - if err != nil { - b.Fatal(err) - } - distinct.Init() - for b := distinct.Next(ctx); b.Length() > 0; b = distinct.Next(ctx) { - } - } - b.StopTimer() - }) + nullsPrefix := "" + if len(nullsOptions) > 1 { + nullsPrefix = fmt.Sprintf("/hasNulls=%t", hasNulls) } + b.Run( + fmt.Sprintf("%s%s/newTupleProbability=%.3f/rows=%d/cols=%d/ordCols=%d", + namePrefix, nullsPrefix, newTupleProbability, + nRows, nCols, numOrderedCols, + ), + func(b *testing.B) { + b.SetBytes(int64(8 * nRows * nCols)) + b.ResetTimer() + for n := 0; n < b.N; n++ { + // Note that the source will be ordered on all nCols so that the + // number of distinct tuples doesn't vary between different + // distinct operator variations. + source := newChunkingBatchSource(typs, cols, nRows) + distinct, err := distinctConstructor(testAllocator, source, distinctCols, numOrderedCols, typs) + if err != nil { + b.Fatal(err) + } + distinct.Init() + for b := distinct.Next(ctx); b.Length() > 0; b = distinct.Next(ctx) { + } + } + b.StopTimer() + }) } } } } } + +func BenchmarkDistinct(b *testing.B) { + ctx := context.Background() + + distinctConstructors := []func(*colmem.Allocator, colexecbase.Operator, []uint32, int, []*types.T) (colexecbase.Operator, error){ + func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecbase.Operator, error) { + return NewUnorderedDistinct(allocator, input, distinctCols, typs), nil + }, + func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecbase.Operator, error) { + return newPartiallyOrderedDistinct(allocator, input, distinctCols, distinctCols[:numOrderedCols], typs) + }, + func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecbase.Operator, error) { + return NewOrderedDistinct(input, distinctCols, typs) + }, + } + distinctNames := []string{"Unordered", "PartiallyOrdered", "Ordered"} + orderedColsFraction := []float64{0, 0.5, 1.0} + for distinctIdx, distinctConstructor := range distinctConstructors { + runDistinctBenchmarks( + ctx, + b, + distinctConstructor, + func(nCols int) int { + return int(float64(nCols) * orderedColsFraction[distinctIdx]) + }, + distinctNames[distinctIdx], + false, /* isExternal */ + ) + } +} diff --git a/pkg/sql/colexec/distinct_tmpl.go b/pkg/sql/colexec/distinct_tmpl.go index 64dc7cf15347..ac2ff5e1bb90 100644 --- a/pkg/sql/colexec/distinct_tmpl.go +++ b/pkg/sql/colexec/distinct_tmpl.go @@ -49,7 +49,7 @@ var ( // last distinct operator in that chain as well as its output column. func OrderedDistinctColsToOperators( input colexecbase.Operator, distinctCols []uint32, typs []*types.T, -) (colexecbase.Operator, []bool, error) { +) (ResettableOperator, []bool, error) { distinctCol := make([]bool, coldata.BatchSize()) // zero the boolean column on every iteration. input = fnOp{ @@ -86,7 +86,7 @@ var _ ResettableOperator = &distinctChainOps{} // input columns with the given types. func NewOrderedDistinct( input colexecbase.Operator, distinctCols []uint32, typs []*types.T, -) (colexecbase.Operator, error) { +) (ResettableOperator, error) { op, outputCol, err := OrderedDistinctColsToOperators(input, distinctCols, typs) if err != nil { return nil, err diff --git a/pkg/sql/colexec/external_distinct.go b/pkg/sql/colexec/external_distinct.go new file mode 100644 index 000000000000..352cb1c121db --- /dev/null +++ b/pkg/sql/colexec/external_distinct.go @@ -0,0 +1,107 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package colexec + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/mon" + "github.com/marusama/semaphore" +) + +// NewExternalDistinct returns a new disk-backed unordered distinct operator. It +// uses the in-memory unordered distinct as the "main" strategy for the external +// operator and the external sort + ordered distinct as the "fallback". +func NewExternalDistinct( + unlimitedAllocator *colmem.Allocator, + flowCtx *execinfra.FlowCtx, + args *NewColOperatorArgs, + input colexecbase.Operator, + inputTypes []*types.T, + createDiskBackedSorter DiskBackedSorterConstructor, + diskAcc *mon.BoundAccount, +) colexecbase.Operator { + distinctSpec := args.Spec.Core.Distinct + distinctCols := distinctSpec.DistinctColumns + inMemMainOpConstructor := func(partitionedInputs []*partitionerToOperator) ResettableOperator { + // Note that the hash-based partitioner will make sure that partitions + // to process using the in-memory unordered distinct fit under the + // limit, so we use an unlimited allocator. + // TODO(yuzefovich): it might be worth increasing the number of buckets. + return NewUnorderedDistinct( + unlimitedAllocator, partitionedInputs[0], distinctCols, inputTypes, + ) + } + diskBackedFallbackOpConstructor := func( + partitionedInputs []*partitionerToOperator, + maxNumberActivePartitions int, + _ semaphore.Semaphore, + ) ResettableOperator { + // The distinct operator *must* keep the first tuple from the input + // among all that are identical on distinctCols. In order to guarantee + // such behavior in the fallback, we append an ordinality column to + // every tuple before the disk-backed sorter and include that column in + // the desired ordering. We then project out that temporary column + // before feeding the tuples into the ordered distinct. + ordinalityOp := NewOrdinalityOp(unlimitedAllocator, partitionedInputs[0], len(inputTypes)) + orderingCols := make([]execinfrapb.Ordering_Column, len(distinctCols)+1) + for i := range distinctCols { + orderingCols[i].ColIdx = distinctCols[i] + } + orderingCols[len(distinctCols)].ColIdx = uint32(len(inputTypes)) + sortTypes := make([]*types.T, 0, len(inputTypes)+1) + sortTypes = append(sortTypes, inputTypes...) + sortTypes = append(sortTypes, types.Int) + diskBackedSorter := createDiskBackedSorter(ordinalityOp, sortTypes, orderingCols, maxNumberActivePartitions) + projection := make([]uint32, len(inputTypes)) + for i := range projection { + projection[i] = uint32(i) + } + diskBackedWithoutOrdinality := NewSimpleProjectOp(diskBackedSorter, len(sortTypes), projection) + diskBackedFallbackOp, err := NewOrderedDistinct(diskBackedWithoutOrdinality, distinctCols, inputTypes) + if err != nil { + colexecerror.InternalError(err) + } + return diskBackedFallbackOp + } + numRequiredActivePartitions := ExternalSorterMinPartitions + ed := newHashBasedPartitioner( + unlimitedAllocator, + flowCtx, + args, + "external unordered distinct", /* name */ + []colexecbase.Operator{input}, + [][]*types.T{inputTypes}, + [][]uint32{distinctCols}, + inMemMainOpConstructor, + diskBackedFallbackOpConstructor, + diskAcc, + numRequiredActivePartitions, + ) + // The last thing we need to do is making sure that the output has the + // desired ordering if any is required. Note that since the input is assumed + // to be already ordered according to the desired ordering, for the + // in-memory unordered distinct we get it for "free" since it doesn't change + // the ordering of tuples. However, that is not that the case with the + // hash-based partitioner, so we might need to plan an external sort on top + // of it. + outputOrdering := args.Spec.Core.Distinct.OutputOrdering + if len(outputOrdering.Columns) == 0 { + // No particular output ordering is required. + return ed + } + maxNumberActivePartitions := calculateMaxNumberActivePartitions(flowCtx, args, numRequiredActivePartitions) + return createDiskBackedSorter(ed, inputTypes, outputOrdering.Columns, maxNumberActivePartitions) +} diff --git a/pkg/sql/colexec/external_distinct_test.go b/pkg/sql/colexec/external_distinct_test.go new file mode 100644 index 000000000000..280e78609601 --- /dev/null +++ b/pkg/sql/colexec/external_distinct_test.go @@ -0,0 +1,223 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package colexec + +import ( + "context" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/testutils/colcontainerutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/mon" + "github.com/marusama/semaphore" + "github.com/stretchr/testify/require" +) + +func TestExternalDistinct(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + evalCtx := tree.MakeTestingEvalContext(st) + defer evalCtx.Stop(ctx) + flowCtx := &execinfra.FlowCtx{ + EvalCtx: &evalCtx, + Cfg: &execinfra.ServerConfig{ + Settings: st, + DiskMonitor: testDiskMonitor, + }, + } + + queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(t, true /* inMem */) + defer cleanup() + + var ( + accounts []*mon.BoundAccount + monitors []*mon.BytesMonitor + ) + // Test the case in which the default memory is used as well as the case in + // which the joiner spills to disk. + for _, spillForced := range []bool{false, true} { + flowCtx.Cfg.TestingKnobs.ForceDiskSpill = spillForced + for tcIdx, tc := range distinctTestCases { + log.Infof(context.Background(), "spillForced=%t/%d", spillForced, tcIdx) + var semsToCheck []semaphore.Semaphore + runTestsWithTyps( + t, + []tuples{tc.tuples}, + [][]*types.T{tc.typs}, + tc.expected, + // We're using an unordered verifier because the in-memory + // unordered distinct is free to change the order of the tuples + // when exporting them into an external distinct. + unorderedVerifier, + func(input []colexecbase.Operator) (colexecbase.Operator, error) { + // A sorter should never exceed ExternalSorterMinPartitions, even + // during repartitioning. A panic will happen if a sorter requests + // more than this number of file descriptors. + sem := colexecbase.NewTestingSemaphore(ExternalSorterMinPartitions) + semsToCheck = append(semsToCheck, sem) + var outputOrdering execinfrapb.Ordering + if tc.isOrderedOnDistinctCols { + outputOrdering = convertDistinctColsToOrdering(tc.distinctCols) + } + distinct, newAccounts, newMonitors, closers, err := createExternalDistinct( + ctx, flowCtx, input, tc.typs, tc.distinctCols, outputOrdering, queueCfg, sem, + ) + // Check that the external distinct and the disk-backed sort + // were added as Closers. + numExpectedClosers := 2 + if len(outputOrdering.Columns) > 0 { + // The final disk-backed sort must also be added as a + // Closer. + numExpectedClosers++ + } + require.Equal(t, numExpectedClosers, len(closers)) + accounts = append(accounts, newAccounts...) + monitors = append(monitors, newMonitors...) + return distinct, err + }, + ) + for i, sem := range semsToCheck { + require.Equal(t, 0, sem.GetCount(), "sem still reports open FDs at index %d", i) + } + } + } + for _, acc := range accounts { + acc.Close(ctx) + } + for _, mon := range monitors { + mon.Stop(ctx) + } +} + +func convertDistinctColsToOrdering(distinctCols []uint32) execinfrapb.Ordering { + var ordering execinfrapb.Ordering + for _, colIdx := range distinctCols { + ordering.Columns = append(ordering.Columns, execinfrapb.Ordering_Column{ColIdx: colIdx}) + } + return ordering +} + +func BenchmarkExternalDistinct(b *testing.B) { + defer leaktest.AfterTest(b)() + defer log.Scope(b).Close(b) + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + evalCtx := tree.MakeTestingEvalContext(st) + defer evalCtx.Stop(ctx) + flowCtx := &execinfra.FlowCtx{ + EvalCtx: &evalCtx, + Cfg: &execinfra.ServerConfig{ + Settings: st, + DiskMonitor: testDiskMonitor, + }, + } + var ( + memAccounts []*mon.BoundAccount + memMonitors []*mon.BytesMonitor + ) + + queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(b, false /* inMem */) + defer cleanup() + + for _, spillForced := range []bool{false, true} { + for _, maintainOrdering := range []bool{false, true} { + if !spillForced && maintainOrdering { + // The in-memory unordered distinct maintains the input ordering + // by design, so it's not an interesting case to test it with + // both options for 'maintainOrdering' parameter, and we skip + // one. + continue + } + flowCtx.Cfg.TestingKnobs.ForceDiskSpill = spillForced + name := fmt.Sprintf("spilled=%t/ordering=%t", spillForced, maintainOrdering) + runDistinctBenchmarks( + ctx, + b, + func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecbase.Operator, error) { + var outputOrdering execinfrapb.Ordering + if maintainOrdering { + outputOrdering = convertDistinctColsToOrdering(distinctCols) + } + op, accs, mons, _, err := createExternalDistinct( + ctx, flowCtx, []colexecbase.Operator{input}, typs, + distinctCols, outputOrdering, queueCfg, &colexecbase.TestingSemaphore{}, + ) + memAccounts = append(memAccounts, accs...) + memMonitors = append(memMonitors, mons...) + return op, err + }, + func(nCols int) int { + return 0 + }, + name, + true, /* isExternal */ + ) + } + } + for _, account := range memAccounts { + account.Close(ctx) + } + for _, monitor := range memMonitors { + monitor.Stop(ctx) + } +} + +// createExternalDistinct is a helper function that instantiates a disk-backed +// distinct operator. It returns an operator and an error as well as memory +// monitors and memory accounts that will need to be closed once the caller is +// done with the operator. +func createExternalDistinct( + ctx context.Context, + flowCtx *execinfra.FlowCtx, + input []colexecbase.Operator, + typs []*types.T, + distinctCols []uint32, + outputOrdering execinfrapb.Ordering, + diskQueueCfg colcontainer.DiskQueueCfg, + testingSemaphore semaphore.Semaphore, +) (colexecbase.Operator, []*mon.BoundAccount, []*mon.BytesMonitor, []colexecbase.Closer, error) { + distinctSpec := &execinfrapb.DistinctSpec{ + DistinctColumns: distinctCols, + OutputOrdering: outputOrdering, + } + spec := &execinfrapb.ProcessorSpec{ + Input: []execinfrapb.InputSyncSpec{{ColumnTypes: typs}}, + Core: execinfrapb.ProcessorCoreUnion{ + Distinct: distinctSpec, + }, + Post: execinfrapb.PostProcessSpec{}, + ResultTypes: typs, + } + args := &NewColOperatorArgs{ + Spec: spec, + Inputs: input, + StreamingMemAccount: testMemAcc, + DiskQueueCfg: diskQueueCfg, + FDSemaphore: testingSemaphore, + } + // External sorter relies on different memory accounts to + // understand when to start a new partition, so we will not use + // the streaming memory account. + result, err := TestNewColOperator(ctx, flowCtx, args) + return result.Op, result.OpAccounts, result.OpMonitors, result.ToClose, err +} diff --git a/pkg/sql/colexec/hash_based_partitioner.go b/pkg/sql/colexec/hash_based_partitioner.go index d34b34ba0b3d..75740d49a0cb 100644 --- a/pkg/sql/colexec/hash_based_partitioner.go +++ b/pkg/sql/colexec/hash_based_partitioner.go @@ -620,8 +620,13 @@ func (op *hashBasedPartitioner) Close(ctx context.Context) error { retErr = err } } - if err := op.diskBackedFallbackOp.(colexecbase.Closer).Close(ctx); err != nil && retErr == nil { - retErr = err + // Note that it is ok if the disk-backed fallback operator is not a Closer - + // it will still be closed appropriately because we accumulate all closers + // in NewColOperatorResult. + if c, ok := op.diskBackedFallbackOp.(colexecbase.Closer); ok { + if err := c.Close(ctx); err != nil { + retErr = err + } } if !op.testingKnobs.delegateFDAcquisitions && op.fdState.acquiredFDs > 0 { op.fdState.fdSemaphore.Release(op.fdState.acquiredFDs) diff --git a/pkg/sql/colexec/ordered_aggregator.go b/pkg/sql/colexec/ordered_aggregator.go index 3c33fe894798..517ee39b2027 100644 --- a/pkg/sql/colexec/ordered_aggregator.go +++ b/pkg/sql/colexec/ordered_aggregator.go @@ -154,7 +154,12 @@ func NewOrderedAggregator( return nil, errors.AssertionFailedf("filtering ordered aggregation is not supported") } } - op, groupCol, err := OrderedDistinctColsToOperators(input, spec.GroupCols, inputTypes) + var ( + op colexecbase.Operator + groupCol []bool + err error + ) + op, groupCol, err = OrderedDistinctColsToOperators(input, spec.GroupCols, inputTypes) if err != nil { return nil, err } @@ -165,6 +170,8 @@ func NewOrderedAggregator( // value in groupCol. In order to satisfy that requirement we plan a // oneShotOp that explicitly sets groupCol for the very first tuple it // sees to 'false' and then deletes itself from the operator tree. + // TODO(yuzefovich): this is pretty annoying, consider refactoring the + // contract and removing this. op = &oneShotOp{ OneInputNode: NewOneInputNode(op), fn: func(batch coldata.Batch) { diff --git a/pkg/sql/colexec/unordered_distinct.go b/pkg/sql/colexec/unordered_distinct.go index 855b50cb5d9e..ea44d8e9afd8 100644 --- a/pkg/sql/colexec/unordered_distinct.go +++ b/pkg/sql/colexec/unordered_distinct.go @@ -21,11 +21,9 @@ import ( // NewUnorderedDistinct creates an unordered distinct on the given distinct // columns. -// numHashBuckets determines the number of buckets that the hash table is -// created with. func NewUnorderedDistinct( allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, typs []*types.T, -) colexecbase.Operator { +) ResettableOperator { // These numbers were chosen after running the micro-benchmarks. const hashTableLoadFactor = 2.0 const hashTableNumBuckets = 128 @@ -53,10 +51,12 @@ func NewUnorderedDistinct( type unorderedDistinct struct { OneInputNode - ht *hashTable + ht *hashTable + lastInputBatch coldata.Batch } -var _ colexecbase.Operator = &unorderedDistinct{} +var _ colexecbase.BufferingInMemoryOperator = &unorderedDistinct{} +var _ ResettableOperator = &unorderedDistinct{} func (op *unorderedDistinct) Init() { op.input.Init() @@ -64,21 +64,38 @@ func (op *unorderedDistinct) Init() { func (op *unorderedDistinct) Next(ctx context.Context) coldata.Batch { for { - batch := op.input.Next(ctx) - if batch.Length() == 0 { + op.lastInputBatch = op.input.Next(ctx) + if op.lastInputBatch.Length() == 0 { return coldata.ZeroBatch } - op.ht.distinctBuild(ctx, batch) - if batch.Length() > 0 { + // Note that distinctBuild might panic with a memory budget exceeded + // error, in which case no tuples from the last input batch are output. + // In such scenario, we don't know at which point of distinctBuild that + // happened, but it doesn't matter - we will export the last input batch + // when falling back to disk. + op.ht.distinctBuild(ctx, op.lastInputBatch) + if op.lastInputBatch.Length() > 0 { // We've just appended some distinct tuples to the hash table, so we // will emit all of them as the output. Note that the selection // vector on batch is set in such a manner that only the distinct // tuples are selected, so we can just emit batch directly. - return batch + return op.lastInputBatch } } } +func (op *unorderedDistinct) ExportBuffered(colexecbase.Operator) coldata.Batch { + // We have output all the distinct tuples except for the ones that are part + // of the last input batch, so we only need to export that batch, and then + // we're done exporting. + if op.lastInputBatch != nil { + batch := op.lastInputBatch + op.lastInputBatch = nil + return batch + } + return coldata.ZeroBatch +} + // reset resets the unorderedDistinct. func (op *unorderedDistinct) reset(ctx context.Context) { if r, ok := op.input.(resetter); ok { diff --git a/pkg/sql/distsql/columnar_operators_test.go b/pkg/sql/distsql/columnar_operators_test.go index eccc42feb17f..9e3e8d86b355 100644 --- a/pkg/sql/distsql/columnar_operators_test.go +++ b/pkg/sql/distsql/columnar_operators_test.go @@ -386,9 +386,15 @@ func TestDistinctAgainstProcessor(t *testing.T) { return cmp < 0 }) + var outputOrdering execinfrapb.Ordering + if rng.Float64() < 0.5 { + outputOrdering = execinfrapb.Ordering{Columns: ordCols} + } + spec := &execinfrapb.DistinctSpec{ DistinctColumns: distinctCols, OrderedColumns: orderedCols, + OutputOrdering: outputOrdering, } pspec := &execinfrapb.ProcessorSpec{ Input: []execinfrapb.InputSyncSpec{{ColumnTypes: inputTypes}}, diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 67b0683377de..f2fe48dd66f0 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -1515,10 +1515,13 @@ func (dsp *DistSQLPlanner) planAggregators( sort.Slice(orderedColumns, func(i, j int) bool { return orderedColumns[i] < orderedColumns[j] }) sort.Slice(distinctColumns, func(i, j int) bool { return distinctColumns[i] < distinctColumns[j] }) distinctSpec := execinfrapb.ProcessorCoreUnion{ - Distinct: &execinfrapb.DistinctSpec{ - OrderedColumns: orderedColumns, - DistinctColumns: distinctColumns, - }, + Distinct: dsp.createDistinctSpec( + distinctColumns, + orderedColumns, + false, /* nullsAreDistinct */ + "", /* errorOnDup */ + p.MergeOrdering, + ), } // Add distinct processors local to each existing current result // processor. @@ -2396,10 +2399,11 @@ func (dsp *DistSQLPlanner) createPlanForInvertedFilter( if !distributable { return nil, errors.Errorf("expected distributable inverted filterer") } + reqOrdering := execinfrapb.Ordering{} // Instantiate one inverted filterer for every stream. plan.AddNoGroupingStage( execinfrapb.ProcessorCoreUnion{InvertedFilterer: invertedFiltererSpec}, - execinfrapb.PostProcessSpec{}, plan.GetResultTypes(), execinfrapb.Ordering{}, + execinfrapb.PostProcessSpec{}, plan.GetResultTypes(), reqOrdering, ) // De-duplicate the PKs. Note that the inverted filterer output includes // the inverted column always set to NULL, so we exclude it from the @@ -2414,7 +2418,13 @@ func (dsp *DistSQLPlanner) createPlanForInvertedFilter( plan.AddSingleGroupStage( dsp.gatewayNodeID, execinfrapb.ProcessorCoreUnion{ - Distinct: &execinfrapb.DistinctSpec{DistinctColumns: distinctColumns}, + Distinct: dsp.createDistinctSpec( + distinctColumns, + []uint32{}, /* orderedColumns */ + false, /* nullsAreDistinct */ + "", /* errorOnDup */ + reqOrdering, + ), }, execinfrapb.PostProcessSpec{}, plan.GetResultTypes(), @@ -2972,42 +2982,19 @@ func (dsp *DistSQLPlanner) createPlanForZero( return dsp.createValuesPlan(planCtx, spec, types) } -func createDistinctSpec( - distinctOnColIdxs util.FastIntSet, - columnsInOrder util.FastIntSet, +func (dsp *DistSQLPlanner) createDistinctSpec( + distinctColumns []uint32, + orderedColumns []uint32, nullsAreDistinct bool, errorOnDup string, - cols []int, + outputOrdering execinfrapb.Ordering, ) *execinfrapb.DistinctSpec { - var orderedColumns []uint32 - if !columnsInOrder.Empty() { - orderedColumns = make([]uint32, 0, columnsInOrder.Len()) - for i, ok := columnsInOrder.Next(0); ok; i, ok = columnsInOrder.Next(i + 1) { - orderedColumns = append(orderedColumns, uint32(cols[i])) - } - } - - var distinctColumns []uint32 - if !distinctOnColIdxs.Empty() { - for planCol, streamCol := range cols { - if streamCol != -1 && distinctOnColIdxs.Contains(planCol) { - distinctColumns = append(distinctColumns, uint32(streamCol)) - } - } - } else { - // If no distinct columns were specified, run distinct on the entire row. - for _, streamCol := range cols { - if streamCol != -1 { - distinctColumns = append(distinctColumns, uint32(streamCol)) - } - } - } - return &execinfrapb.DistinctSpec{ OrderedColumns: orderedColumns, DistinctColumns: distinctColumns, NullsAreDistinct: nullsAreDistinct, ErrorOnDup: errorOnDup, + OutputOrdering: outputOrdering, } } @@ -3018,26 +3005,23 @@ func (dsp *DistSQLPlanner) createPlanForDistinct( if err != nil { return nil, err } - spec := createDistinctSpec( - n.distinctOnColIdxs, - n.columnsInOrder, + spec := dsp.createDistinctSpec( + convertFastIntSetToUint32Slice(n.distinctOnColIdxs), + convertFastIntSetToUint32Slice(n.columnsInOrder), n.nullsAreDistinct, n.errorOnDup, - plan.PlanToStreamColMap, + dsp.convertOrdering(n.reqOrdering, plan.PlanToStreamColMap), ) - dsp.addDistinctProcessors(plan, spec, n.reqOrdering) + dsp.addDistinctProcessors(plan, spec) return plan, nil } func (dsp *DistSQLPlanner) addDistinctProcessors( - plan *PhysicalPlan, spec *execinfrapb.DistinctSpec, reqOrdering ReqOrdering, + plan *PhysicalPlan, spec *execinfrapb.DistinctSpec, ) { distinctSpec := execinfrapb.ProcessorCoreUnion{ Distinct: spec, } - defer func() { - plan.SetMergeOrdering(dsp.convertOrdering(reqOrdering, plan.PlanToStreamColMap)) - }() // Add distinct processors local to each existing current result processor. plan.AddNoGroupingStage(distinctSpec, execinfrapb.PostProcessSpec{}, plan.GetResultTypes(), plan.MergeOrdering) @@ -3051,6 +3035,7 @@ func (dsp *DistSQLPlanner) addDistinctProcessors( distinctSpec.Distinct.DistinctColumns, plan.GetResultTypes(), plan.GetResultTypes(), plan.MergeOrdering, plan.ResultRouters, ) + plan.SetMergeOrdering(spec.OutputOrdering) } func (dsp *DistSQLPlanner) createPlanForOrdinality( @@ -3252,15 +3237,17 @@ func (dsp *DistSQLPlanner) createPlanForSetOp( for i, ord := range distinctOrds[side].Columns { sortCols[i] = ord.ColIdx } - distinctSpec := &distinctSpecs[side] - distinctSpec.Distinct = &execinfrapb.DistinctSpec{ - DistinctColumns: streamCols, - OrderedColumns: sortCols, - } + distinctSpecs[side].Distinct = dsp.createDistinctSpec( + streamCols, + sortCols, + false, /* nullsAreDistinct */ + "", /* errorOnDup */ + distinctOrds[side], + ) if !dsp.isOnlyOnGateway(plan) { // TODO(solon): We could skip this stage if there is a strong key on // the result columns. - plan.AddNoGroupingStage(*distinctSpec, execinfrapb.PostProcessSpec{}, plan.GetResultTypes(), distinctOrds[side]) + plan.AddNoGroupingStage(distinctSpecs[side], execinfrapb.PostProcessSpec{}, plan.GetResultTypes(), distinctOrds[side]) plan.AddProjection(streamCols) } } @@ -3312,7 +3299,13 @@ func (dsp *DistSQLPlanner) createPlanForSetOp( // OrderingColumns field in DistinctSpec once the unused columns // are projected out. distinctSpec := execinfrapb.ProcessorCoreUnion{ - Distinct: &execinfrapb.DistinctSpec{DistinctColumns: streamCols}, + Distinct: dsp.createDistinctSpec( + streamCols, + []uint32{}, /* orderedColumns */ + false, /* nullsAreDistinct */ + "", /* errorOnDup */ + mergeOrdering, + ), } p.AddSingleGroupStage(dsp.gatewayNodeID, distinctSpec, execinfrapb.PostProcessSpec{}, resultTypes) } else { diff --git a/pkg/sql/distsql_spec_exec_factory.go b/pkg/sql/distsql_spec_exec_factory.go index 7b359044e538..a658d9cac548 100644 --- a/pkg/sql/distsql_spec_exec_factory.go +++ b/pkg/sql/distsql_spec_exec_factory.go @@ -571,14 +571,14 @@ func (e *distSQLSpecExecFactory) ConstructDistinct( errorOnDup string, ) (exec.Node, error) { physPlan, plan := getPhysPlan(input) - spec := createDistinctSpec( - distinctCols, - orderedCols, + spec := e.dsp.createDistinctSpec( + convertFastIntSetToUint32Slice(distinctCols), + convertFastIntSetToUint32Slice(orderedCols), nullsAreDistinct, errorOnDup, - physPlan.PlanToStreamColMap, + e.dsp.convertOrdering(ReqOrdering(reqOrdering), physPlan.PlanToStreamColMap), ) - e.dsp.addDistinctProcessors(physPlan, spec, ReqOrdering(reqOrdering)) + e.dsp.addDistinctProcessors(physPlan, spec) // Since addition of distinct processors doesn't change any properties of // the physical plan, we don't need to update any of those. return plan, nil diff --git a/pkg/sql/exec_factory_util.go b/pkg/sql/exec_factory_util.go index 98a60b70555b..8c5d8109b9c9 100644 --- a/pkg/sql/exec_factory_util.go +++ b/pkg/sql/exec_factory_util.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/errors" ) @@ -315,3 +316,11 @@ func constructOpaque(metadata opt.OpaqueMetadata) (planNode, error) { } return o.plan, nil } + +func convertFastIntSetToUint32Slice(colIdxs util.FastIntSet) []uint32 { + cols := make([]uint32, 0, colIdxs.Len()) + for i, ok := colIdxs.Next(0); ok; i, ok = colIdxs.Next(i + 1) { + cols = append(cols, uint32(i)) + } + return cols +} diff --git a/pkg/sql/execinfrapb/processors_sql.pb.go b/pkg/sql/execinfrapb/processors_sql.pb.go index ab612013aab8..f8d46ea27787 100644 --- a/pkg/sql/execinfrapb/processors_sql.pb.go +++ b/pkg/sql/execinfrapb/processors_sql.pb.go @@ -64,7 +64,7 @@ func (x *ScanVisibility) UnmarshalJSON(data []byte) error { return nil } func (ScanVisibility) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{0} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{0} } // These mirror the aggregate functions supported by sql/parser. See @@ -232,7 +232,7 @@ func (x *AggregatorSpec_Func) UnmarshalJSON(data []byte) error { return nil } func (AggregatorSpec_Func) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{13, 0} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{13, 0} } type AggregatorSpec_Type int32 @@ -278,7 +278,7 @@ func (x *AggregatorSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (AggregatorSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{13, 1} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{13, 1} } type WindowerSpec_WindowFunc int32 @@ -342,7 +342,7 @@ func (x *WindowerSpec_WindowFunc) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_WindowFunc) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{15, 0} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{15, 0} } // Mode indicates which mode of framing is used. @@ -386,7 +386,7 @@ func (x *WindowerSpec_Frame_Mode) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_Mode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{15, 1, 0} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{15, 1, 0} } // BoundType indicates which type of boundary is used. @@ -433,7 +433,7 @@ func (x *WindowerSpec_Frame_BoundType) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_BoundType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{15, 1, 1} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{15, 1, 1} } // Exclusion specifies the type of frame exclusion. @@ -476,7 +476,7 @@ func (x *WindowerSpec_Frame_Exclusion) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_Exclusion) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{15, 1, 2} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{15, 1, 2} } // ValuesCoreSpec is the core of a processor that has no inputs and generates @@ -496,7 +496,7 @@ func (m *ValuesCoreSpec) Reset() { *m = ValuesCoreSpec{} } func (m *ValuesCoreSpec) String() string { return proto.CompactTextString(m) } func (*ValuesCoreSpec) ProtoMessage() {} func (*ValuesCoreSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{0} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{0} } func (m *ValuesCoreSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -606,7 +606,7 @@ func (m *TableReaderSpec) Reset() { *m = TableReaderSpec{} } func (m *TableReaderSpec) String() string { return proto.CompactTextString(m) } func (*TableReaderSpec) ProtoMessage() {} func (*TableReaderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{1} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{1} } func (m *TableReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -643,7 +643,7 @@ func (m *FiltererSpec) Reset() { *m = FiltererSpec{} } func (m *FiltererSpec) String() string { return proto.CompactTextString(m) } func (*FiltererSpec) ProtoMessage() {} func (*FiltererSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{2} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{2} } func (m *FiltererSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -698,7 +698,7 @@ func (m *IndexSkipTableReaderSpec) Reset() { *m = IndexSkipTableReaderSp func (m *IndexSkipTableReaderSpec) String() string { return proto.CompactTextString(m) } func (*IndexSkipTableReaderSpec) ProtoMessage() {} func (*IndexSkipTableReaderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{3} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{3} } func (m *IndexSkipTableReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -869,7 +869,7 @@ func (m *JoinReaderSpec) Reset() { *m = JoinReaderSpec{} } func (m *JoinReaderSpec) String() string { return proto.CompactTextString(m) } func (*JoinReaderSpec) ProtoMessage() {} func (*JoinReaderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{4} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{4} } func (m *JoinReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -913,7 +913,7 @@ func (m *SorterSpec) Reset() { *m = SorterSpec{} } func (m *SorterSpec) String() string { return proto.CompactTextString(m) } func (*SorterSpec) ProtoMessage() {} func (*SorterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{5} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{5} } func (m *SorterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -969,13 +969,17 @@ type DistinctSpec struct { // UPSERT and INSERT..ON CONFLICT statements, both of which prohibit the same // row from being changed twice. ErrorOnDup string `protobuf:"bytes,4,opt,name=error_on_dup,json=errorOnDup" json:"error_on_dup"` + // OutputOrdering specifies the required ordering of the output produced by + // the distinct. The input to the processor *must* already be ordered + // according to it. + OutputOrdering Ordering `protobuf:"bytes,5,opt,name=output_ordering,json=outputOrdering" json:"output_ordering"` } func (m *DistinctSpec) Reset() { *m = DistinctSpec{} } func (m *DistinctSpec) String() string { return proto.CompactTextString(m) } func (*DistinctSpec) ProtoMessage() {} func (*DistinctSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{6} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{6} } func (m *DistinctSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1010,7 +1014,7 @@ func (m *OrdinalitySpec) Reset() { *m = OrdinalitySpec{} } func (m *OrdinalitySpec) String() string { return proto.CompactTextString(m) } func (*OrdinalitySpec) ProtoMessage() {} func (*OrdinalitySpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{7} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{7} } func (m *OrdinalitySpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1068,7 +1072,7 @@ func (m *ZigzagJoinerSpec) Reset() { *m = ZigzagJoinerSpec{} } func (m *ZigzagJoinerSpec) String() string { return proto.CompactTextString(m) } func (*ZigzagJoinerSpec) ProtoMessage() {} func (*ZigzagJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{8} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{8} } func (m *ZigzagJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1148,7 +1152,7 @@ func (m *MergeJoinerSpec) Reset() { *m = MergeJoinerSpec{} } func (m *MergeJoinerSpec) String() string { return proto.CompactTextString(m) } func (*MergeJoinerSpec) ProtoMessage() {} func (*MergeJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{9} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{9} } func (m *MergeJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1221,7 +1225,7 @@ func (m *HashJoinerSpec) Reset() { *m = HashJoinerSpec{} } func (m *HashJoinerSpec) String() string { return proto.CompactTextString(m) } func (*HashJoinerSpec) ProtoMessage() {} func (*HashJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{10} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{10} } func (m *HashJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1337,7 +1341,7 @@ func (m *InvertedJoinerSpec) Reset() { *m = InvertedJoinerSpec{} } func (m *InvertedJoinerSpec) String() string { return proto.CompactTextString(m) } func (*InvertedJoinerSpec) ProtoMessage() {} func (*InvertedJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{11} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{11} } func (m *InvertedJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1398,7 +1402,7 @@ func (m *InvertedFiltererSpec) Reset() { *m = InvertedFiltererSpec{} } func (m *InvertedFiltererSpec) String() string { return proto.CompactTextString(m) } func (*InvertedFiltererSpec) ProtoMessage() {} func (*InvertedFiltererSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{12} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{12} } func (m *InvertedFiltererSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1436,7 +1440,7 @@ func (m *InvertedFiltererSpec_PreFiltererSpec) Reset() { *m = InvertedFi func (m *InvertedFiltererSpec_PreFiltererSpec) String() string { return proto.CompactTextString(m) } func (*InvertedFiltererSpec_PreFiltererSpec) ProtoMessage() {} func (*InvertedFiltererSpec_PreFiltererSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{12, 0} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{12, 0} } func (m *InvertedFiltererSpec_PreFiltererSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1482,7 +1486,7 @@ func (m *AggregatorSpec) Reset() { *m = AggregatorSpec{} } func (m *AggregatorSpec) String() string { return proto.CompactTextString(m) } func (*AggregatorSpec) ProtoMessage() {} func (*AggregatorSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{13} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{13} } func (m *AggregatorSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1533,7 +1537,7 @@ func (m *AggregatorSpec_Aggregation) Reset() { *m = AggregatorSpec_Aggre func (m *AggregatorSpec_Aggregation) String() string { return proto.CompactTextString(m) } func (*AggregatorSpec_Aggregation) ProtoMessage() {} func (*AggregatorSpec_Aggregation) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{13, 0} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{13, 0} } func (m *AggregatorSpec_Aggregation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1573,7 +1577,7 @@ func (m *ProjectSetSpec) Reset() { *m = ProjectSetSpec{} } func (m *ProjectSetSpec) String() string { return proto.CompactTextString(m) } func (*ProjectSetSpec) ProtoMessage() {} func (*ProjectSetSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{14} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{14} } func (m *ProjectSetSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1615,7 +1619,7 @@ func (m *WindowerSpec) Reset() { *m = WindowerSpec{} } func (m *WindowerSpec) String() string { return proto.CompactTextString(m) } func (*WindowerSpec) ProtoMessage() {} func (*WindowerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{15} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{15} } func (m *WindowerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1651,7 +1655,7 @@ func (m *WindowerSpec_Func) Reset() { *m = WindowerSpec_Func{} } func (m *WindowerSpec_Func) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Func) ProtoMessage() {} func (*WindowerSpec_Func) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{15, 0} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{15, 0} } func (m *WindowerSpec_Func) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1687,7 +1691,7 @@ func (m *WindowerSpec_Frame) Reset() { *m = WindowerSpec_Frame{} } func (m *WindowerSpec_Frame) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame) ProtoMessage() {} func (*WindowerSpec_Frame) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{15, 1} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{15, 1} } func (m *WindowerSpec_Frame) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1728,7 +1732,7 @@ func (m *WindowerSpec_Frame_Bound) Reset() { *m = WindowerSpec_Frame_Bou func (m *WindowerSpec_Frame_Bound) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame_Bound) ProtoMessage() {} func (*WindowerSpec_Frame_Bound) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{15, 1, 0} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{15, 1, 0} } func (m *WindowerSpec_Frame_Bound) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1764,7 +1768,7 @@ func (m *WindowerSpec_Frame_Bounds) Reset() { *m = WindowerSpec_Frame_Bo func (m *WindowerSpec_Frame_Bounds) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame_Bounds) ProtoMessage() {} func (*WindowerSpec_Frame_Bounds) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{15, 1, 1} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{15, 1, 1} } func (m *WindowerSpec_Frame_Bounds) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1814,7 +1818,7 @@ func (m *WindowerSpec_WindowFn) Reset() { *m = WindowerSpec_WindowFn{} } func (m *WindowerSpec_WindowFn) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_WindowFn) ProtoMessage() {} func (*WindowerSpec_WindowFn) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_fcf41d6067fc431e, []int{15, 2} + return fileDescriptor_processors_sql_6df5e05de401c165, []int{15, 2} } func (m *WindowerSpec_WindowFn) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2267,6 +2271,14 @@ func (m *DistinctSpec) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(len(m.ErrorOnDup))) i += copy(dAtA[i:], m.ErrorOnDup) + dAtA[i] = 0x2a + i++ + i = encodeVarintProcessorsSql(dAtA, i, uint64(m.OutputOrdering.Size())) + n9, err := m.OutputOrdering.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n9 return i, nil } @@ -2328,30 +2340,30 @@ func (m *ZigzagJoinerSpec) MarshalTo(dAtA []byte) (int, error) { } } if len(m.IndexOrdinals) > 0 { - dAtA10 := make([]byte, len(m.IndexOrdinals)*10) - var j9 int + dAtA11 := make([]byte, len(m.IndexOrdinals)*10) + var j10 int for _, num := range m.IndexOrdinals { for num >= 1<<7 { - dAtA10[j9] = uint8(uint64(num)&0x7f | 0x80) + dAtA11[j10] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j9++ + j10++ } - dAtA10[j9] = uint8(num) - j9++ + dAtA11[j10] = uint8(num) + j10++ } dAtA[i] = 0x1a i++ - i = encodeVarintProcessorsSql(dAtA, i, uint64(j9)) - i += copy(dAtA[i:], dAtA10[:j9]) + i = encodeVarintProcessorsSql(dAtA, i, uint64(j10)) + i += copy(dAtA[i:], dAtA11[:j10]) } dAtA[i] = 0x22 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.OnExpr.Size())) - n11, err := m.OnExpr.MarshalTo(dAtA[i:]) + n12, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n11 + i += n12 if len(m.FixedValues) > 0 { for _, msg := range m.FixedValues { dAtA[i] = 0x2a @@ -2388,27 +2400,27 @@ func (m *MergeJoinerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.LeftOrdering.Size())) - n12, err := m.LeftOrdering.MarshalTo(dAtA[i:]) + n13, err := m.LeftOrdering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n12 + i += n13 dAtA[i] = 0x12 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.RightOrdering.Size())) - n13, err := m.RightOrdering.MarshalTo(dAtA[i:]) + n14, err := m.RightOrdering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n13 + i += n14 dAtA[i] = 0x2a i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.OnExpr.Size())) - n14, err := m.OnExpr.MarshalTo(dAtA[i:]) + n15, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n14 + i += n15 dAtA[i] = 0x30 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Type)) @@ -2455,47 +2467,47 @@ func (m *HashJoinerSpec) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.LeftEqColumns) > 0 { - dAtA16 := make([]byte, len(m.LeftEqColumns)*10) - var j15 int + dAtA17 := make([]byte, len(m.LeftEqColumns)*10) + var j16 int for _, num := range m.LeftEqColumns { for num >= 1<<7 { - dAtA16[j15] = uint8(uint64(num)&0x7f | 0x80) + dAtA17[j16] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j15++ + j16++ } - dAtA16[j15] = uint8(num) - j15++ + dAtA17[j16] = uint8(num) + j16++ } dAtA[i] = 0xa i++ - i = encodeVarintProcessorsSql(dAtA, i, uint64(j15)) - i += copy(dAtA[i:], dAtA16[:j15]) + i = encodeVarintProcessorsSql(dAtA, i, uint64(j16)) + i += copy(dAtA[i:], dAtA17[:j16]) } if len(m.RightEqColumns) > 0 { - dAtA18 := make([]byte, len(m.RightEqColumns)*10) - var j17 int + dAtA19 := make([]byte, len(m.RightEqColumns)*10) + var j18 int for _, num := range m.RightEqColumns { for num >= 1<<7 { - dAtA18[j17] = uint8(uint64(num)&0x7f | 0x80) + dAtA19[j18] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j17++ + j18++ } - dAtA18[j17] = uint8(num) - j17++ + dAtA19[j18] = uint8(num) + j18++ } dAtA[i] = 0x12 i++ - i = encodeVarintProcessorsSql(dAtA, i, uint64(j17)) - i += copy(dAtA[i:], dAtA18[:j17]) + i = encodeVarintProcessorsSql(dAtA, i, uint64(j18)) + i += copy(dAtA[i:], dAtA19[:j18]) } dAtA[i] = 0x2a i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.OnExpr.Size())) - n19, err := m.OnExpr.MarshalTo(dAtA[i:]) + n20, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n19 + i += n20 dAtA[i] = 0x30 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Type)) @@ -2536,30 +2548,30 @@ func (m *InvertedJoinerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Table.Size())) - n20, err := m.Table.MarshalTo(dAtA[i:]) + n21, err := m.Table.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n20 + i += n21 dAtA[i] = 0x10 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.IndexIdx)) dAtA[i] = 0x22 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.InvertedExpr.Size())) - n21, err := m.InvertedExpr.MarshalTo(dAtA[i:]) + n22, err := m.InvertedExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n21 + i += n22 dAtA[i] = 0x2a i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.OnExpr.Size())) - n22, err := m.OnExpr.MarshalTo(dAtA[i:]) + n23, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n22 + i += n23 dAtA[i] = 0x30 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Type)) @@ -2603,20 +2615,20 @@ func (m *InvertedFiltererSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.InvertedExpr.Size())) - n23, err := m.InvertedExpr.MarshalTo(dAtA[i:]) + n24, err := m.InvertedExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n23 + i += n24 if m.PreFiltererSpec != nil { dAtA[i] = 0x32 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.PreFiltererSpec.Size())) - n24, err := m.PreFiltererSpec.MarshalTo(dAtA[i:]) + n25, err := m.PreFiltererSpec.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n24 + i += n25 } return i, nil } @@ -2639,20 +2651,20 @@ func (m *InvertedFiltererSpec_PreFiltererSpec) MarshalTo(dAtA []byte) (int, erro dAtA[i] = 0xa i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Expression.Size())) - n25, err := m.Expression.MarshalTo(dAtA[i:]) + n26, err := m.Expression.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n25 + i += n26 if m.Type != nil { dAtA[i] = 0x12 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Type.Size())) - n26, err := m.Type.MarshalTo(dAtA[i:]) + n27, err := m.Type.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n26 + i += n27 } return i, nil } @@ -2673,21 +2685,21 @@ func (m *AggregatorSpec) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.GroupCols) > 0 { - dAtA28 := make([]byte, len(m.GroupCols)*10) - var j27 int + dAtA29 := make([]byte, len(m.GroupCols)*10) + var j28 int for _, num := range m.GroupCols { for num >= 1<<7 { - dAtA28[j27] = uint8(uint64(num)&0x7f | 0x80) + dAtA29[j28] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j27++ + j28++ } - dAtA28[j27] = uint8(num) - j27++ + dAtA29[j28] = uint8(num) + j28++ } dAtA[i] = 0x12 i++ - i = encodeVarintProcessorsSql(dAtA, i, uint64(j27)) - i += copy(dAtA[i:], dAtA28[:j27]) + i = encodeVarintProcessorsSql(dAtA, i, uint64(j28)) + i += copy(dAtA[i:], dAtA29[:j28]) } if len(m.Aggregations) > 0 { for _, msg := range m.Aggregations { @@ -2702,21 +2714,21 @@ func (m *AggregatorSpec) MarshalTo(dAtA []byte) (int, error) { } } if len(m.OrderedGroupCols) > 0 { - dAtA30 := make([]byte, len(m.OrderedGroupCols)*10) - var j29 int + dAtA31 := make([]byte, len(m.OrderedGroupCols)*10) + var j30 int for _, num := range m.OrderedGroupCols { for num >= 1<<7 { - dAtA30[j29] = uint8(uint64(num)&0x7f | 0x80) + dAtA31[j30] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j29++ + j30++ } - dAtA30[j29] = uint8(num) - j29++ + dAtA31[j30] = uint8(num) + j30++ } dAtA[i] = 0x22 i++ - i = encodeVarintProcessorsSql(dAtA, i, uint64(j29)) - i += copy(dAtA[i:], dAtA30[:j29]) + i = encodeVarintProcessorsSql(dAtA, i, uint64(j30)) + i += copy(dAtA[i:], dAtA31[:j30]) } dAtA[i] = 0x28 i++ @@ -2912,11 +2924,11 @@ func (m *WindowerSpec_Frame) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Bounds.Size())) - n31, err := m.Bounds.MarshalTo(dAtA[i:]) + n32, err := m.Bounds.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n31 + i += n32 dAtA[i] = 0x18 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Exclusion)) @@ -2953,11 +2965,11 @@ func (m *WindowerSpec_Frame_Bound) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.OffsetType.Size())) - n32, err := m.OffsetType.MarshalTo(dAtA[i:]) + n33, err := m.OffsetType.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n32 + i += n33 return i, nil } @@ -2979,20 +2991,20 @@ func (m *WindowerSpec_Frame_Bounds) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Start.Size())) - n33, err := m.Start.MarshalTo(dAtA[i:]) + n34, err := m.Start.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n33 + i += n34 if m.End != nil { dAtA[i] = 0x12 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.End.Size())) - n34, err := m.End.MarshalTo(dAtA[i:]) + n35, err := m.End.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n34 + i += n35 } return i, nil } @@ -3015,28 +3027,28 @@ func (m *WindowerSpec_WindowFn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Func.Size())) - n35, err := m.Func.MarshalTo(dAtA[i:]) + n36, err := m.Func.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n35 + i += n36 dAtA[i] = 0x22 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Ordering.Size())) - n36, err := m.Ordering.MarshalTo(dAtA[i:]) + n37, err := m.Ordering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n36 + i += n37 if m.Frame != nil { dAtA[i] = 0x2a i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Frame.Size())) - n37, err := m.Frame.MarshalTo(dAtA[i:]) + n38, err := m.Frame.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n37 + i += n38 } dAtA[i] = 0x30 i++ @@ -3211,6 +3223,8 @@ func (m *DistinctSpec) Size() (n int) { n += 2 l = len(m.ErrorOnDup) n += 1 + l + sovProcessorsSql(uint64(l)) + l = m.OutputOrdering.Size() + n += 1 + l + sovProcessorsSql(uint64(l)) return n } @@ -5071,6 +5085,36 @@ func (m *DistinctSpec) Unmarshal(dAtA []byte) error { } m.ErrorOnDup = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OutputOrdering", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProcessorsSql + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.OutputOrdering.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -8002,190 +8046,190 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors_sql.proto", fileDescriptor_processors_sql_fcf41d6067fc431e) -} - -var fileDescriptor_processors_sql_fcf41d6067fc431e = []byte{ - // 2881 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x1a, 0x4d, 0x6c, 0xdb, 0xd6, - 0xd9, 0xd4, 0x8f, 0x2d, 0x7d, 0xfa, 0xf1, 0xcb, 0x4b, 0xda, 0xa8, 0x6e, 0xe7, 0x38, 0x4a, 0x9a, - 0x38, 0x69, 0x6a, 0xaf, 0xde, 0xd0, 0xa1, 0xdd, 0x30, 0x8c, 0x96, 0x28, 0x45, 0x8e, 0x4c, 0x2a, - 0x94, 0xe4, 0x38, 0x2d, 0xd0, 0x07, 0x5a, 0x7a, 0x96, 0xd9, 0x50, 0xa4, 0x4c, 0x52, 0xb1, 0xdd, - 0xd3, 0xb0, 0xcb, 0x4e, 0x03, 0x06, 0xec, 0xb2, 0xd3, 0x50, 0x0c, 0xd8, 0x65, 0xe7, 0x9d, 0x77, - 0xda, 0x21, 0xc7, 0x9e, 0xb6, 0x9e, 0x8a, 0x35, 0xbd, 0xed, 0xba, 0xc3, 0xb0, 0xdb, 0xf0, 0x1e, - 0x1f, 0x69, 0xca, 0xb0, 0xb2, 0x28, 0x4d, 0x9b, 0x4b, 0xa0, 0xef, 0xf7, 0x7d, 0xff, 0xef, 0x7b, - 0x74, 0xe0, 0xba, 0x77, 0x68, 0xad, 0xd3, 0x63, 0xda, 0x33, 0xed, 0x7d, 0xd7, 0x18, 0xed, 0xad, - 0x8f, 0x5c, 0xa7, 0x47, 0x3d, 0xcf, 0x71, 0x3d, 0xe2, 0x1d, 0x5a, 0x6b, 0x23, 0xd7, 0xf1, 0x1d, - 0x5c, 0xea, 0x39, 0xbd, 0x47, 0xae, 0x63, 0xf4, 0x0e, 0xd6, 0x18, 0xb2, 0x6f, 0x7a, 0xbe, 0x77, - 0x68, 0xb9, 0x63, 0x7b, 0xe9, 0x1a, 0x93, 0xef, 0x19, 0xbe, 0x61, 0x39, 0x83, 0xf5, 0x3e, 0xf5, - 0x7a, 0xa3, 0xbd, 0x75, 0xcf, 0x77, 0xc7, 0x3d, 0x7f, 0xec, 0xd2, 0x7e, 0x20, 0xbe, 0x54, 0x3e, - 0x87, 0xe9, 0x53, 0xc7, 0xb4, 0x89, 0x7f, 0x32, 0xa2, 0x82, 0x67, 0xe5, 0x1c, 0x1e, 0xcb, 0xe9, - 0x3d, 0x32, 0xed, 0x81, 0xe0, 0x78, 0x8d, 0x71, 0x30, 0x09, 0x2f, 0xf8, 0x57, 0xa0, 0x97, 0xce, - 0x7a, 0xd0, 0x37, 0x7c, 0x43, 0xd0, 0xde, 0x7e, 0x86, 0x77, 0x7b, 0x86, 0x17, 0x9e, 0x7d, 0x9b, - 0xb1, 0x39, 0x23, 0x7f, 0xdd, 0xb4, 0x1f, 0x53, 0xd7, 0xa7, 0x7d, 0x7a, 0x3c, 0x72, 0xd7, 0xbd, - 0x91, 0x61, 0x13, 0xf6, 0x8b, 0x7a, 0x9e, 0xe9, 0xd8, 0x82, 0xf7, 0xd2, 0xc0, 0x19, 0x38, 0xfc, - 0xe7, 0x3a, 0xfb, 0x15, 0x60, 0xcb, 0xbf, 0x93, 0xa0, 0xb8, 0x63, 0x58, 0x63, 0xea, 0x55, 0x1c, - 0x97, 0xb6, 0x47, 0xb4, 0x87, 0x2b, 0xb0, 0xd0, 0x73, 0xac, 0xf1, 0xd0, 0xf6, 0x4a, 0xd2, 0x4a, - 0x72, 0x35, 0xb7, 0x71, 0x6d, 0x6d, 0x5a, 0x14, 0xd7, 0xaa, 0x86, 0x3f, 0x1e, 0x36, 0xec, 0x7d, - 0x67, 0x33, 0xf5, 0xe4, 0xab, 0x2b, 0x73, 0x7a, 0x28, 0x89, 0xdf, 0x84, 0xac, 0x6b, 0x1c, 0x91, - 0xbd, 0x13, 0x9f, 0x7a, 0xa5, 0xc4, 0x4a, 0x72, 0x35, 0xaf, 0x67, 0x5c, 0xe3, 0x68, 0x93, 0xc1, - 0xf8, 0x0a, 0x64, 0xec, 0xf1, 0x90, 0xb8, 0xce, 0x91, 0x57, 0x4a, 0xae, 0x48, 0xab, 0xa9, 0x50, - 0xda, 0x1e, 0x0f, 0x75, 0xe7, 0xc8, 0x2b, 0xff, 0x6a, 0x1e, 0x16, 0x3b, 0xc6, 0x9e, 0x45, 0x75, - 0x6a, 0xf4, 0xa9, 0xcb, 0xcd, 0xda, 0x84, 0xb4, 0xcf, 0x50, 0x25, 0x69, 0x45, 0x5a, 0xcd, 0x6d, - 0xdc, 0x38, 0x63, 0x94, 0x77, 0x68, 0xf1, 0xc0, 0x70, 0xb1, 0x2a, 0xf5, 0x7a, 0xae, 0x39, 0xf2, - 0x1d, 0x57, 0x68, 0x0e, 0x44, 0xf1, 0x55, 0xc8, 0x9a, 0x76, 0x9f, 0x1e, 0x13, 0xb3, 0x7f, 0x5c, - 0x4a, 0xac, 0x48, 0xab, 0x05, 0x41, 0xcf, 0x70, 0x74, 0xa3, 0x7f, 0x8c, 0x97, 0x61, 0xc1, 0xa5, - 0x8f, 0xa9, 0xeb, 0x51, 0x6e, 0x5a, 0x26, 0x34, 0x4d, 0x20, 0xb1, 0x02, 0x69, 0x16, 0x5f, 0xaf, - 0x94, 0xe2, 0xb1, 0xb9, 0x35, 0x3d, 0x36, 0x13, 0x0e, 0x18, 0x76, 0x68, 0x09, 0x97, 0xc6, 0xd7, - 0x00, 0x2c, 0x73, 0x68, 0xfa, 0xe4, 0xc0, 0xb4, 0xfd, 0x52, 0x7a, 0x45, 0x5a, 0x4d, 0x0a, 0x86, - 0x2c, 0xc7, 0xdf, 0x35, 0x6d, 0x9f, 0xc5, 0xc9, 0xf4, 0x48, 0xef, 0x80, 0xf6, 0x1e, 0x95, 0xe6, - 0xe3, 0xc6, 0x98, 0x5e, 0x85, 0x21, 0xb1, 0x0a, 0xf0, 0xd8, 0xf4, 0xcc, 0x3d, 0xd3, 0x32, 0xfd, - 0x93, 0xd2, 0xc2, 0x8a, 0xb4, 0x5a, 0xdc, 0x58, 0x9d, 0x6e, 0x51, 0xbb, 0x67, 0xd8, 0x3b, 0x11, - 0xbf, 0x50, 0x16, 0xd3, 0x80, 0x7f, 0x0a, 0x97, 0x87, 0xc6, 0x31, 0xf1, 0xcd, 0x21, 0xf5, 0x7c, - 0x63, 0x38, 0x22, 0xc6, 0x80, 0x12, 0xdb, 0xb0, 0x1d, 0xaf, 0x94, 0x8d, 0xe5, 0xe9, 0xd2, 0xd0, - 0x38, 0xee, 0x84, 0x3c, 0xf2, 0x80, 0xaa, 0x8c, 0x03, 0x7f, 0x0c, 0x48, 0xd4, 0x3d, 0xf1, 0x7c, - 0x97, 0xda, 0x03, 0xff, 0xa0, 0x04, 0xdc, 0xa4, 0xdb, 0x53, 0x72, 0xc5, 0xec, 0x69, 0x06, 0x22, - 0x6d, 0x21, 0x21, 0x4e, 0x58, 0xb4, 0x26, 0xd1, 0x78, 0x0f, 0x2e, 0x86, 0xca, 0x8f, 0x0c, 0xd3, - 0x27, 0x23, 0xc7, 0x32, 0x7b, 0x27, 0xa5, 0x1c, 0xd7, 0x7f, 0xe7, 0xff, 0xeb, 0x7f, 0x60, 0x98, - 0x7e, 0x8b, 0xcb, 0x88, 0x13, 0x2e, 0x58, 0x67, 0x09, 0xf8, 0x06, 0xe4, 0x46, 0x86, 0x6b, 0x58, - 0x16, 0xb5, 0xcc, 0xcf, 0x68, 0x29, 0x1f, 0x8b, 0x78, 0x9c, 0x80, 0x37, 0x00, 0x1f, 0x18, 0x1e, - 0xf1, 0x4e, 0x3c, 0x9f, 0x0e, 0x49, 0xd8, 0x2b, 0xc5, 0x18, 0x3b, 0x3a, 0x30, 0xbc, 0x36, 0x27, - 0x57, 0x44, 0x3f, 0xbc, 0x0d, 0x45, 0x9b, 0xd2, 0x3e, 0xed, 0x47, 0xfc, 0x8b, 0x2b, 0xc9, 0xd5, - 0x82, 0x5e, 0x08, 0xb0, 0x82, 0x6d, 0x2b, 0x95, 0xc9, 0xa0, 0xec, 0x56, 0x2a, 0x53, 0x40, 0xc5, - 0xb2, 0x0e, 0xf9, 0x9a, 0x69, 0xf9, 0xd4, 0x8d, 0x1a, 0x60, 0x7e, 0x9f, 0xc3, 0xa2, 0x03, 0xae, - 0x4f, 0x4f, 0xb4, 0x12, 0x35, 0xbf, 0x30, 0x48, 0x48, 0x96, 0xff, 0x9b, 0x84, 0x52, 0x83, 0x95, - 0x7a, 0xfb, 0x91, 0x39, 0x7a, 0x45, 0x1d, 0x16, 0x75, 0x50, 0xf2, 0x5b, 0x75, 0xd0, 0x64, 0xed, - 0xa7, 0xbe, 0x75, 0xed, 0xc7, 0x1a, 0x3f, 0x7d, 0x5e, 0xe3, 0x9f, 0x57, 0xde, 0xf3, 0xdf, 0x71, - 0x79, 0x2f, 0xbc, 0xc4, 0xf2, 0x2e, 0xff, 0x72, 0x01, 0x8a, 0x5b, 0x8e, 0x69, 0x7f, 0xff, 0x19, - 0xbf, 0x05, 0x45, 0xcb, 0x71, 0x1e, 0x8d, 0x47, 0x51, 0xf1, 0xb3, 0xd4, 0x17, 0x36, 0x13, 0x48, - 0xd2, 0x0b, 0x01, 0x25, 0xec, 0x93, 0x0a, 0x2c, 0x38, 0xc1, 0xe5, 0xc5, 0x53, 0x3a, 0x63, 0x95, - 0x3b, 0x36, 0xc3, 0xe1, 0x0f, 0x20, 0xc5, 0x2e, 0x5a, 0x91, 0x9e, 0x2b, 0x53, 0xbc, 0x62, 0xb1, - 0xe8, 0x9c, 0x8c, 0xa8, 0x10, 0xe6, 0x22, 0x2f, 0x7d, 0xa2, 0x7e, 0x00, 0xaf, 0x4f, 0xba, 0x4e, - 0x0c, 0x97, 0x92, 0x47, 0xf4, 0xa4, 0x94, 0x89, 0x15, 0xd9, 0xc5, 0x89, 0x20, 0xc8, 0x2e, 0xbd, - 0x47, 0x4f, 0xce, 0x2d, 0xb8, 0xec, 0x77, 0x5c, 0x70, 0xf0, 0x32, 0xe7, 0xe9, 0x7b, 0x70, 0x61, - 0x68, 0x98, 0xb6, 0x6f, 0x98, 0x36, 0x71, 0xdc, 0x3e, 0x75, 0x4d, 0x7b, 0xc0, 0x27, 0x76, 0x34, - 0x26, 0x43, 0xb2, 0x26, 0xa8, 0x53, 0x46, 0x6b, 0xe1, 0x99, 0xa3, 0xb5, 0x0a, 0x6f, 0x59, 0x74, - 0xdf, 0x27, 0x7c, 0x31, 0x3b, 0x32, 0xfd, 0x03, 0x32, 0x32, 0x4c, 0x97, 0xf6, 0x39, 0x82, 0xba, - 0x13, 0x83, 0xb9, 0xc4, 0x38, 0x59, 0xe2, 0x1f, 0x98, 0xfe, 0x41, 0x8b, 0xb3, 0x6d, 0x71, 0x2e, - 0xfc, 0x00, 0x6e, 0x38, 0x63, 0x7f, 0x34, 0xf6, 0xc9, 0xc0, 0x75, 0x78, 0xba, 0x6c, 0xdf, 0xb4, - 0xc7, 0x86, 0x6f, 0x3a, 0x36, 0xd9, 0x77, 0x5c, 0xc2, 0xcf, 0x70, 0x9d, 0xa3, 0xd2, 0x62, 0x4c, - 0xdf, 0xd5, 0x40, 0xa6, 0xce, 0x44, 0x2a, 0x31, 0x89, 0x9a, 0xe3, 0x36, 0xe9, 0xbe, 0xaf, 0x3b, - 0x47, 0x5b, 0xa9, 0x4c, 0x1a, 0xcd, 0x6f, 0xa5, 0x32, 0x79, 0x54, 0x60, 0xdb, 0x16, 0xb4, 0x1d, - 0xd7, 0x17, 0xed, 0x77, 0x1f, 0x16, 0xc5, 0x99, 0x51, 0x78, 0x82, 0x46, 0x2c, 0x4f, 0xaf, 0xb8, - 0x30, 0x54, 0xc2, 0x80, 0x62, 0xa0, 0x20, 0x1e, 0xc0, 0x50, 0x17, 0x19, 0x1a, 0x7e, 0xef, 0x80, - 0x58, 0xd4, 0x9e, 0x68, 0x4b, 0x14, 0xd2, 0xb7, 0x19, 0xb9, 0x49, 0xed, 0xf2, 0x5f, 0x25, 0xc8, - 0x57, 0x4d, 0xcf, 0x37, 0xed, 0x9e, 0xcf, 0xed, 0xba, 0x09, 0x8b, 0x9c, 0x29, 0x76, 0x5b, 0x49, - 0xfc, 0xb6, 0x2a, 0x0a, 0x74, 0x18, 0xfa, 0x5b, 0x80, 0xfa, 0x42, 0x30, 0xe2, 0x4c, 0x70, 0xce, - 0xc5, 0x10, 0x1f, 0xb2, 0x6e, 0x00, 0xb6, 0xc7, 0x96, 0x15, 0xd4, 0x7f, 0x48, 0x9c, 0x58, 0xb1, - 0x10, 0xa7, 0xcb, 0x2e, 0x0d, 0x6d, 0xc1, 0x37, 0x20, 0x4f, 0x5d, 0xd7, 0x71, 0x89, 0x63, 0x93, - 0xfe, 0x78, 0xc4, 0x27, 0x42, 0x36, 0x6c, 0x32, 0x4e, 0xd1, 0xec, 0xea, 0x78, 0x54, 0x46, 0x50, - 0xd4, 0xdc, 0xbe, 0x69, 0x1b, 0xac, 0xe5, 0x98, 0x07, 0xe5, 0xdf, 0x27, 0x01, 0x7d, 0x64, 0x0e, - 0x3e, 0x33, 0x06, 0x41, 0x7a, 0xb9, 0x5b, 0x55, 0x98, 0xe7, 0x23, 0x2b, 0xdc, 0x6b, 0x67, 0x1b, - 0x77, 0x42, 0x16, 0xd7, 0x00, 0xe8, 0xe1, 0x84, 0xb7, 0xb9, 0x8d, 0xab, 0xd3, 0xf3, 0x25, 0xfc, - 0x0f, 0x97, 0x3b, 0x7a, 0x78, 0x1a, 0xbb, 0x62, 0x30, 0x37, 0x9d, 0xc0, 0xf4, 0x89, 0xa1, 0xc8, - 0x29, 0xc2, 0xa7, 0x97, 0x34, 0x14, 0xef, 0x41, 0x7e, 0xdf, 0x3c, 0xa6, 0x7d, 0xf2, 0x98, 0xaf, - 0xfb, 0xa5, 0x34, 0xb7, 0xfc, 0x19, 0xb3, 0x6d, 0xf2, 0x59, 0xa0, 0xe7, 0xb8, 0x74, 0x80, 0xfc, - 0x16, 0x13, 0xb6, 0xfc, 0xf7, 0x24, 0x2c, 0x6e, 0x53, 0x77, 0x40, 0x63, 0x99, 0xd9, 0x86, 0x02, - 0x6f, 0xaf, 0x17, 0x6e, 0x83, 0x3c, 0x13, 0x8f, 0x9a, 0x40, 0x83, 0xa2, 0x6b, 0x0e, 0x0e, 0x62, - 0xfa, 0x12, 0x33, 0xea, 0x2b, 0x70, 0xf9, 0x48, 0x61, 0x2c, 0x01, 0xe9, 0x57, 0x71, 0x2b, 0xdd, - 0x82, 0x02, 0x6b, 0x0e, 0x42, 0x0f, 0xc7, 0x46, 0x74, 0x31, 0x85, 0x7d, 0x93, 0x67, 0x24, 0x45, - 0x50, 0xf0, 0x87, 0x70, 0x99, 0x87, 0xf2, 0xb4, 0x46, 0xa7, 0xdc, 0x38, 0x74, 0xdf, 0x57, 0x0e, - 0x27, 0x6f, 0x9c, 0x9f, 0x41, 0x29, 0x88, 0xdb, 0x39, 0xc2, 0xd9, 0x98, 0xf0, 0x25, 0xce, 0x75, - 0x46, 0xba, 0xfc, 0xaf, 0x04, 0x14, 0xef, 0x1a, 0xde, 0x41, 0x2c, 0xaf, 0xb7, 0x61, 0xf1, 0x8c, - 0x31, 0xc1, 0x20, 0x11, 0x37, 0x7f, 0xdc, 0x04, 0x7c, 0x07, 0xd0, 0xd9, 0xc3, 0x83, 0x59, 0xc2, - 0x99, 0x8b, 0x93, 0x47, 0xbe, 0xf2, 0x8c, 0xbc, 0xb2, 0x30, 0x6f, 0xa5, 0x32, 0x0b, 0x28, 0x53, - 0xfe, 0x2a, 0x09, 0xb8, 0x21, 0x1e, 0xfd, 0xb1, 0x80, 0x7f, 0x4f, 0x0b, 0x9d, 0x06, 0x85, 0xf0, - 0x8b, 0xc3, 0x8b, 0x8e, 0xa5, 0x7c, 0xa8, 0x80, 0x67, 0xe2, 0x55, 0xa7, 0xf3, 0xdc, 0x55, 0x65, - 0xe1, 0x99, 0xab, 0xca, 0xf3, 0x2f, 0x0c, 0x99, 0x59, 0x17, 0x86, 0x24, 0x4a, 0x95, 0xff, 0x98, - 0x84, 0x4b, 0x61, 0x82, 0x27, 0x9e, 0x81, 0x6b, 0x80, 0xa2, 0xd8, 0xf7, 0x1c, 0x8b, 0x67, 0x49, - 0x8a, 0x65, 0xa9, 0x18, 0x52, 0x2b, 0x8e, 0xc5, 0x72, 0xf5, 0xc9, 0xd9, 0x5c, 0x05, 0xb3, 0xf0, - 0x47, 0x67, 0xc2, 0xe3, 0x8c, 0xfc, 0xb5, 0xf8, 0x57, 0xa4, 0x35, 0xf6, 0xe2, 0x3a, 0x0d, 0x75, - 0xcb, 0x75, 0x7c, 0xe7, 0xdc, 0xd4, 0x7d, 0x0a, 0x17, 0x46, 0x2e, 0x25, 0xfb, 0xc2, 0x46, 0xe2, - 0x8d, 0x68, 0x8f, 0xa7, 0x20, 0xb7, 0xf1, 0xf3, 0xe9, 0x49, 0x3c, 0xcf, 0xb5, 0xb5, 0x96, 0x4b, - 0xe3, 0xb0, 0xbe, 0x38, 0x9a, 0x44, 0x2c, 0xfd, 0x46, 0x82, 0xc5, 0x33, 0x4c, 0x78, 0x0b, 0xe0, - 0xf4, 0x5b, 0xd7, 0x0b, 0x3c, 0x8d, 0x63, 0xd2, 0x78, 0x4d, 0x54, 0x50, 0x10, 0xa2, 0xa5, 0xb3, - 0x15, 0x44, 0x87, 0x6b, 0xc1, 0x07, 0xbc, 0x8e, 0xb8, 0xcb, 0xfe, 0x9d, 0x85, 0xa2, 0x3c, 0x18, - 0xb8, 0x74, 0x60, 0xf8, 0x4e, 0x60, 0xce, 0x55, 0x80, 0xb0, 0x1e, 0xac, 0xf8, 0x00, 0xcb, 0x0e, - 0x82, 0x94, 0x5b, 0x1e, 0xfe, 0x04, 0xf2, 0x86, 0x10, 0x32, 0x9d, 0xe8, 0x1d, 0xfc, 0xe3, 0xe9, - 0x36, 0x4f, 0x1e, 0x11, 0x81, 0xb1, 0x66, 0x8a, 0xeb, 0xc3, 0x3f, 0x14, 0x3b, 0x20, 0xed, 0x93, - 0x98, 0x29, 0xa9, 0xc8, 0x14, 0x24, 0xa8, 0xf5, 0xc8, 0xa2, 0xba, 0xf0, 0x3b, 0xcd, 0x3b, 0xe7, - 0xdd, 0xe7, 0xb6, 0xe4, 0x6c, 0x1f, 0x2d, 0xfd, 0x3a, 0x01, 0xb9, 0x98, 0x79, 0x4c, 0xf1, 0xfe, - 0xd8, 0xee, 0xf1, 0xb4, 0xcc, 0xa2, 0xb8, 0x36, 0xb6, 0x7b, 0xa1, 0x62, 0xa6, 0x00, 0xaf, 0x40, - 0x26, 0x5a, 0x1a, 0x13, 0xb1, 0x7e, 0x8a, 0xb0, 0xf8, 0x3a, 0x14, 0x83, 0x1a, 0x8c, 0xba, 0x82, - 0x0d, 0xa5, 0x82, 0x9e, 0x0f, 0xb0, 0xa2, 0x1b, 0x2e, 0xf3, 0x8f, 0x9b, 0x9c, 0x9c, 0xe6, 0x8b, - 0xea, 0x7c, 0x2f, 0x20, 0xdc, 0x85, 0xac, 0xe1, 0x0e, 0xc6, 0x43, 0x6a, 0xfb, 0x5e, 0x69, 0x9e, - 0x67, 0x64, 0x96, 0x2a, 0x3a, 0x15, 0x16, 0xfd, 0xfb, 0xe7, 0x34, 0xa4, 0x98, 0x17, 0x18, 0x41, - 0x5e, 0x56, 0x1f, 0x12, 0x55, 0xeb, 0x10, 0xb5, 0xdb, 0x6c, 0xa2, 0x39, 0xbc, 0x00, 0x49, 0x79, - 0xa7, 0x8e, 0x24, 0x9c, 0x87, 0xcc, 0xa6, 0xa6, 0x35, 0x89, 0xac, 0x56, 0x51, 0x02, 0xe7, 0x60, - 0x81, 0x43, 0x9a, 0x8e, 0x92, 0xb8, 0x08, 0x50, 0xd1, 0xd4, 0x8a, 0xdc, 0x21, 0x72, 0xbd, 0x8e, - 0x52, 0x38, 0x0b, 0xe9, 0x8a, 0xd6, 0x55, 0x3b, 0x28, 0xcd, 0xc4, 0xb7, 0xe5, 0x5d, 0xb4, 0xc0, - 0x7f, 0x34, 0x54, 0x94, 0xc1, 0x00, 0xf3, 0xed, 0x4e, 0xb5, 0xaa, 0xec, 0xa0, 0x2c, 0x43, 0xb6, - 0xbb, 0xdb, 0x08, 0x98, 0xba, 0x76, 0x77, 0x9b, 0x34, 0xd4, 0x0e, 0xca, 0xb1, 0x93, 0x76, 0x64, - 0xbd, 0x21, 0xab, 0x15, 0x05, 0xe5, 0x19, 0x69, 0x57, 0xd3, 0xb9, 0xe6, 0x42, 0x70, 0x52, 0x57, - 0xed, 0x10, 0x5d, 0x7b, 0xd0, 0x46, 0x45, 0x2e, 0x77, 0x5f, 0xaf, 0x36, 0x6a, 0x35, 0xb4, 0x88, - 0x31, 0x14, 0x6b, 0x0d, 0x55, 0x6e, 0x92, 0x48, 0x1a, 0x31, 0x87, 0x02, 0x9c, 0x38, 0xf3, 0x02, - 0x2e, 0x40, 0x56, 0xd6, 0x75, 0xf9, 0x21, 0xd7, 0x88, 0xd9, 0x61, 0x5b, 0x6d, 0x4d, 0xe5, 0xd0, - 0x45, 0x46, 0x64, 0xd0, 0x26, 0x07, 0x2f, 0xb1, 0xe3, 0xda, 0x1d, 0xbd, 0xa1, 0xd6, 0x39, 0xfc, - 0x1a, 0xf7, 0xba, 0xd1, 0xe1, 0x21, 0x78, 0x9d, 0x39, 0xc2, 0x00, 0x4d, 0x47, 0x97, 0x71, 0x06, - 0x52, 0x15, 0x4d, 0xd7, 0x51, 0x09, 0x97, 0xe0, 0x52, 0x4b, 0xd1, 0x2b, 0x8a, 0xda, 0x69, 0x34, - 0x15, 0x52, 0x6d, 0xb4, 0x2b, 0xa4, 0xb1, 0xdd, 0x6a, 0xa2, 0x37, 0xce, 0x50, 0x2a, 0x9a, 0xda, - 0x09, 0x28, 0x4b, 0xf8, 0x22, 0x2c, 0x72, 0x1b, 0xb4, 0xcd, 0x2d, 0xa5, 0x12, 0x04, 0xf1, 0x4d, - 0x7c, 0x09, 0x50, 0x60, 0x4a, 0x0c, 0xfb, 0x16, 0xb3, 0x60, 0x47, 0xd6, 0x49, 0x4b, 0x6b, 0xa1, - 0x1f, 0x04, 0xe6, 0x31, 0xb7, 0x38, 0xbc, 0x8c, 0x17, 0x21, 0xd7, 0xee, 0x90, 0x6d, 0xf9, 0x9e, - 0xd2, 0x6c, 0xa8, 0x0a, 0xba, 0xc2, 0xdc, 0x69, 0x77, 0x88, 0xb2, 0xdb, 0x51, 0xd4, 0x0e, 0x5a, - 0x61, 0xbe, 0xb6, 0x3b, 0xa4, 0xab, 0x36, 0x34, 0x15, 0x5d, 0x0d, 0xa4, 0x49, 0x45, 0x6b, 0x36, - 0x95, 0x4a, 0x07, 0x95, 0x19, 0x73, 0x45, 0x0b, 0x95, 0x5f, 0x0b, 0x42, 0xcd, 0xc0, 0xb6, 0xbc, - 0xdd, 0x42, 0xd7, 0x59, 0x74, 0x75, 0xa5, 0xae, 0xb3, 0x1c, 0x31, 0x2f, 0x5a, 0x1d, 0xf4, 0x36, - 0xb3, 0x86, 0xe3, 0xf4, 0x0d, 0x74, 0x83, 0x09, 0x70, 0xa0, 0xdd, 0xd4, 0x5a, 0x0a, 0xba, 0xc9, - 0x4e, 0x0b, 0xe0, 0xdd, 0x5d, 0xb4, 0x7a, 0x0a, 0x3d, 0x7c, 0x88, 0x6e, 0xc5, 0x68, 0x0f, 0xd1, - 0xed, 0x48, 0x32, 0x28, 0x9a, 0x77, 0x98, 0x25, 0x1c, 0x96, 0x77, 0xea, 0xbb, 0xe8, 0x4e, 0x1c, - 0x7c, 0x88, 0xde, 0x2d, 0xdf, 0x81, 0x14, 0x6b, 0x65, 0x16, 0x73, 0xb9, 0xdb, 0xd1, 0xd0, 0x1c, - 0x2f, 0xa9, 0x8a, 0xdc, 0x94, 0x75, 0x24, 0x31, 0x5d, 0xaa, 0xa6, 0x12, 0x01, 0x27, 0xca, 0x7f, - 0x93, 0xa0, 0xd8, 0x72, 0x9d, 0x4f, 0x69, 0xcf, 0x6f, 0xd3, 0xe0, 0xc5, 0xf8, 0x0b, 0x48, 0xb3, - 0x31, 0x1a, 0xbe, 0xac, 0x66, 0xe9, 0x9c, 0x40, 0x10, 0xd7, 0xe1, 0xc2, 0x80, 0xda, 0xd4, 0x35, - 0xfc, 0xd8, 0xab, 0x33, 0x78, 0x5d, 0x3d, 0x6b, 0x0e, 0xa3, 0x48, 0x28, 0xdc, 0x0c, 0x6f, 0x02, - 0xb2, 0xc7, 0xfc, 0xdb, 0x81, 0x47, 0x46, 0xd4, 0x25, 0x03, 0x6a, 0x07, 0x2f, 0x2b, 0xbd, 0x60, - 0x8f, 0x87, 0x6c, 0xdc, 0xb5, 0xa8, 0x5b, 0xa7, 0x76, 0xf9, 0xeb, 0x02, 0xe4, 0x1f, 0x98, 0x76, - 0xdf, 0x39, 0x12, 0x37, 0xc9, 0x0a, 0xff, 0xfe, 0xeb, 0x9b, 0x7c, 0xb0, 0x9e, 0x88, 0x27, 0x6f, - 0x1c, 0x85, 0xdb, 0x90, 0x3d, 0xe2, 0x12, 0xb5, 0xc8, 0xb8, 0xf5, 0xe9, 0xae, 0xc6, 0x95, 0x0b, - 0xa0, 0x16, 0xcd, 0x8b, 0x48, 0xcf, 0xd2, 0x5f, 0x24, 0x31, 0x29, 0xda, 0x50, 0x08, 0xe7, 0x38, - 0xad, 0xbd, 0xe8, 0xd4, 0xd4, 0x27, 0x75, 0xe0, 0xfb, 0x00, 0xe2, 0x28, 0xa6, 0x31, 0xc1, 0x35, - 0xbe, 0x37, 0x9b, 0xcd, 0x4c, 0x6b, 0x4c, 0xc9, 0x87, 0xa9, 0x27, 0x9f, 0x5f, 0x91, 0x96, 0x3e, - 0x5f, 0x80, 0x74, 0xcd, 0x35, 0x86, 0x14, 0xdf, 0x83, 0xd4, 0xd0, 0xe9, 0x53, 0x61, 0xee, 0xf3, - 0x2a, 0xe7, 0xb2, 0x6b, 0xdb, 0x4e, 0x3f, 0xba, 0x41, 0x98, 0x12, 0x7c, 0x1f, 0xe6, 0xf7, 0x9c, - 0xb1, 0xdd, 0xf7, 0xa6, 0xec, 0x29, 0xcf, 0x56, 0xb7, 0xc9, 0x45, 0xc3, 0xbd, 0x30, 0x50, 0x84, - 0x3f, 0x82, 0x2c, 0x3d, 0xee, 0x59, 0x63, 0xbe, 0x20, 0x24, 0xb9, 0x91, 0xef, 0xcf, 0xa4, 0x55, - 0x09, 0xa5, 0xa3, 0x57, 0x7c, 0x88, 0x58, 0xfa, 0x8f, 0x04, 0x69, 0x7e, 0x28, 0x3b, 0x85, 0x9f, - 0xc7, 0x1a, 0x49, 0x84, 0xe2, 0xfd, 0xd9, 0x6d, 0x8f, 0xdd, 0xa8, 0xa7, 0xea, 0xf0, 0x35, 0x00, - 0xd3, 0xf6, 0x89, 0xb3, 0xbf, 0xef, 0xd1, 0xe0, 0xfe, 0x0b, 0xff, 0x14, 0x93, 0x35, 0x6d, 0x5f, - 0xe3, 0x68, 0x7c, 0x15, 0xf2, 0xac, 0x2b, 0xfa, 0x21, 0x1b, 0xf3, 0x34, 0xaf, 0xe7, 0x38, 0x4e, - 0xb0, 0x6c, 0x41, 0x2e, 0x20, 0xf2, 0x3f, 0x60, 0x8a, 0xad, 0x7d, 0x86, 0x3f, 0xef, 0x41, 0x20, - 0xcd, 0x6c, 0x5a, 0xfa, 0x83, 0x04, 0xf3, 0x41, 0xb8, 0xb1, 0x0a, 0x69, 0xcf, 0x37, 0x5c, 0x5f, - 0x6c, 0x5f, 0x1b, 0xb3, 0xbb, 0x1d, 0x7d, 0xda, 0x67, 0x6a, 0x70, 0x15, 0x92, 0xd4, 0xee, 0x8b, - 0x02, 0x78, 0x01, 0x6d, 0x3a, 0x13, 0x2f, 0xdf, 0x84, 0x14, 0xab, 0x2e, 0x76, 0x75, 0xea, 0xb2, - 0x5a, 0x57, 0xd0, 0x1c, 0x9b, 0x6f, 0xfc, 0x96, 0x93, 0xd8, 0x7c, 0xab, 0xeb, 0x5a, 0xb7, 0xd5, - 0x46, 0x89, 0xf2, 0x67, 0x90, 0x8d, 0x62, 0x8f, 0x2f, 0xc3, 0xc5, 0xae, 0xba, 0xa9, 0x75, 0xd5, - 0xaa, 0x52, 0x25, 0x2d, 0x5d, 0xa9, 0x28, 0xd5, 0x86, 0x5a, 0x47, 0x73, 0x93, 0x84, 0x9a, 0xd6, - 0x6c, 0x6a, 0x0f, 0x18, 0x41, 0x62, 0xb7, 0x8a, 0x56, 0xab, 0xb5, 0x95, 0x4e, 0x8c, 0x3d, 0x11, - 0xc3, 0x9e, 0xf2, 0x26, 0xd9, 0x75, 0x52, 0xe9, 0xea, 0xba, 0x12, 0x5c, 0xb7, 0x28, 0x55, 0xfe, - 0x18, 0xb2, 0x51, 0x75, 0xb1, 0x9b, 0x55, 0xd5, 0x88, 0xb2, 0x5b, 0x69, 0x76, 0xdb, 0xec, 0x42, - 0xe1, 0x87, 0x72, 0xb0, 0xaa, 0x90, 0xb8, 0x9c, 0x84, 0x2f, 0x40, 0x21, 0x24, 0x70, 0x3f, 0x50, - 0x82, 0x49, 0x87, 0xa8, 0x4e, 0x43, 0x69, 0xa3, 0xe4, 0xd2, 0x3f, 0x12, 0x90, 0x09, 0xe7, 0x0e, - 0x56, 0x62, 0xab, 0x58, 0x6e, 0xe3, 0x9d, 0xe7, 0x8d, 0xea, 0xd9, 0x45, 0xac, 0x0a, 0x99, 0xe8, - 0x81, 0x94, 0x9a, 0xf1, 0xab, 0x4a, 0x24, 0xc9, 0xde, 0xa9, 0xfb, 0x2c, 0x5f, 0xe2, 0xb5, 0x77, - 0x67, 0x96, 0x1c, 0xeb, 0x81, 0x28, 0x5e, 0x85, 0x89, 0xd5, 0x8e, 0xbf, 0x39, 0xd2, 0xe1, 0x42, - 0x3c, 0xb1, 0xf4, 0x2d, 0x41, 0xc6, 0x70, 0x07, 0x5e, 0xa3, 0x7f, 0xec, 0x95, 0x16, 0xf8, 0x54, - 0x8f, 0x60, 0xa6, 0x25, 0x78, 0x92, 0x09, 0x2d, 0x99, 0xd8, 0x53, 0x6a, 0x82, 0xb2, 0x95, 0xca, - 0x24, 0x50, 0x52, 0x6c, 0x77, 0x7f, 0x92, 0x00, 0x4e, 0xa7, 0x23, 0xbf, 0x6d, 0xb5, 0x07, 0x44, - 0xed, 0x6e, 0x6f, 0x2a, 0xba, 0xa8, 0x33, 0x59, 0xbd, 0x17, 0xdc, 0x9d, 0x55, 0x45, 0x6d, 0x2b, - 0x84, 0xc3, 0x3c, 0x49, 0x62, 0x63, 0x09, 0x30, 0x49, 0xbe, 0x23, 0x74, 0xb7, 0xf9, 0x5e, 0xd3, - 0x09, 0x16, 0x3d, 0xbe, 0xcd, 0x04, 0x8b, 0x5e, 0x53, 0xae, 0xa3, 0x79, 0xa6, 0xae, 0xa9, 0xc8, - 0x55, 0xb4, 0xc0, 0xea, 0xa7, 0xd6, 0xd0, 0xdb, 0x1d, 0xb2, 0x23, 0x37, 0xbb, 0x0a, 0xca, 0x30, - 0xfd, 0x4d, 0x39, 0x82, 0xb3, 0x4c, 0x9b, 0xda, 0xb9, 0x2b, 0x40, 0xb8, 0xfd, 0x13, 0x28, 0x4e, - 0xfe, 0x89, 0x82, 0x15, 0x7e, 0xab, 0xbb, 0xd9, 0x6c, 0x54, 0xd0, 0x1c, 0x7e, 0x03, 0x5e, 0x0b, - 0x7e, 0xb3, 0xf5, 0x8b, 0x6f, 0xa8, 0x82, 0x24, 0x6d, 0xbe, 0xfb, 0xe4, 0xeb, 0xe5, 0xb9, 0x27, - 0x4f, 0x97, 0xa5, 0x2f, 0x9e, 0x2e, 0x4b, 0x5f, 0x3e, 0x5d, 0x96, 0xfe, 0xf9, 0x74, 0x59, 0xfa, - 0xed, 0x37, 0xcb, 0x73, 0x5f, 0x7c, 0xb3, 0x3c, 0xf7, 0xe5, 0x37, 0xcb, 0x73, 0x1f, 0xe5, 0x62, - 0xff, 0x3d, 0xe1, 0x7f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x1f, 0xb5, 0xe2, 0x09, 0x8e, 0x21, 0x00, - 0x00, + proto.RegisterFile("sql/execinfrapb/processors_sql.proto", fileDescriptor_processors_sql_6df5e05de401c165) +} + +var fileDescriptor_processors_sql_6df5e05de401c165 = []byte{ + // 2895 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0xcf, 0x6f, 0x1b, 0xc7, + 0xf5, 0xd7, 0xf2, 0x87, 0x44, 0x3e, 0xfe, 0xd0, 0x78, 0xec, 0xc4, 0x8c, 0x92, 0xaf, 0x2c, 0xd3, + 0x8e, 0x2d, 0x3b, 0x8e, 0xf4, 0x8d, 0xbe, 0x5f, 0xa4, 0x48, 0x5a, 0x14, 0x5d, 0x91, 0x4b, 0x9a, + 0x32, 0xb5, 0x4b, 0x2f, 0x49, 0x59, 0x4e, 0x80, 0x0c, 0x56, 0xe4, 0x88, 0xda, 0x78, 0xb9, 0x4b, + 0xed, 0x2e, 0x2d, 0x29, 0xa7, 0xa2, 0x97, 0x9e, 0x0a, 0xb4, 0xe8, 0xa5, 0xa7, 0x22, 0x28, 0xd0, + 0x4b, 0xcf, 0xfd, 0x13, 0x7a, 0xf0, 0x31, 0xa7, 0x36, 0xa7, 0xa0, 0x71, 0x6e, 0xbd, 0xf6, 0x50, + 0xf4, 0x56, 0xcc, 0xec, 0xec, 0x6a, 0x29, 0x88, 0xae, 0x69, 0x3b, 0xf1, 0xc5, 0xd0, 0xbc, 0xf7, + 0x3e, 0x6f, 0xde, 0xbc, 0x5f, 0xf3, 0x66, 0x69, 0xb8, 0xee, 0x1d, 0x5a, 0xeb, 0xf4, 0x98, 0xf6, + 0x4c, 0x7b, 0xdf, 0x35, 0x46, 0x7b, 0xeb, 0x23, 0xd7, 0xe9, 0x51, 0xcf, 0x73, 0x5c, 0x8f, 0x78, + 0x87, 0xd6, 0xda, 0xc8, 0x75, 0x7c, 0x07, 0x97, 0x7a, 0x4e, 0xef, 0x91, 0xeb, 0x18, 0xbd, 0x83, + 0x35, 0x46, 0xec, 0x9b, 0x9e, 0xef, 0x1d, 0x5a, 0xee, 0xd8, 0x5e, 0xba, 0xc6, 0xf0, 0x3d, 0xc3, + 0x37, 0x2c, 0x67, 0xb0, 0xde, 0xa7, 0x5e, 0x6f, 0xb4, 0xb7, 0xee, 0xf9, 0xee, 0xb8, 0xe7, 0x8f, + 0x5d, 0xda, 0x0f, 0xe0, 0x4b, 0xe5, 0x73, 0x84, 0x3e, 0x77, 0x4c, 0x9b, 0xf8, 0x27, 0x23, 0x2a, + 0x64, 0x56, 0xce, 0x91, 0xb1, 0x9c, 0xde, 0x23, 0xd3, 0x1e, 0x08, 0x89, 0x37, 0x98, 0x04, 0x43, + 0x78, 0xc1, 0xbf, 0x82, 0xbc, 0x74, 0xf6, 0x04, 0x7d, 0xc3, 0x37, 0x04, 0xef, 0xdd, 0x67, 0x9c, + 0x6e, 0xcf, 0xf0, 0xc2, 0xbd, 0x6f, 0x33, 0x31, 0x67, 0xe4, 0xaf, 0x9b, 0xf6, 0x63, 0xea, 0xfa, + 0xb4, 0x4f, 0x8f, 0x47, 0xee, 0xba, 0x37, 0x32, 0x6c, 0xc2, 0xfe, 0xa2, 0x9e, 0x67, 0x3a, 0xb6, + 0x90, 0xbd, 0x34, 0x70, 0x06, 0x0e, 0xff, 0x73, 0x9d, 0xfd, 0x15, 0x50, 0xcb, 0xbf, 0x95, 0xa0, + 0xb8, 0x63, 0x58, 0x63, 0xea, 0x55, 0x1c, 0x97, 0xb6, 0x47, 0xb4, 0x87, 0x2b, 0xb0, 0xd0, 0x73, + 0xac, 0xf1, 0xd0, 0xf6, 0x4a, 0xd2, 0x4a, 0x72, 0x35, 0xb7, 0x71, 0x6d, 0x6d, 0x9a, 0x17, 0xd7, + 0xaa, 0x86, 0x3f, 0x1e, 0x36, 0xec, 0x7d, 0x67, 0x33, 0xf5, 0xe4, 0x9b, 0x2b, 0x73, 0x7a, 0x88, + 0xc4, 0x6f, 0x43, 0xd6, 0x35, 0x8e, 0xc8, 0xde, 0x89, 0x4f, 0xbd, 0x52, 0x62, 0x25, 0xb9, 0x9a, + 0xd7, 0x33, 0xae, 0x71, 0xb4, 0xc9, 0xd6, 0xf8, 0x0a, 0x64, 0xec, 0xf1, 0x90, 0xb8, 0xce, 0x91, + 0x57, 0x4a, 0xae, 0x48, 0xab, 0xa9, 0x10, 0x6d, 0x8f, 0x87, 0xba, 0x73, 0xe4, 0x95, 0x7f, 0x31, + 0x0f, 0x8b, 0x1d, 0x63, 0xcf, 0xa2, 0x3a, 0x35, 0xfa, 0xd4, 0xe5, 0x66, 0x6d, 0x42, 0xda, 0x67, + 0xa4, 0x92, 0xb4, 0x22, 0xad, 0xe6, 0x36, 0x6e, 0x9c, 0x31, 0xca, 0x3b, 0xb4, 0xb8, 0x63, 0x38, + 0xac, 0x4a, 0xbd, 0x9e, 0x6b, 0x8e, 0x7c, 0xc7, 0x15, 0x9a, 0x03, 0x28, 0xbe, 0x0a, 0x59, 0xd3, + 0xee, 0xd3, 0x63, 0x62, 0xf6, 0x8f, 0x4b, 0x89, 0x15, 0x69, 0xb5, 0x20, 0xf8, 0x19, 0x4e, 0x6e, + 0xf4, 0x8f, 0xf1, 0x32, 0x2c, 0xb8, 0xf4, 0x31, 0x75, 0x3d, 0xca, 0x4d, 0xcb, 0x84, 0xa6, 0x09, + 0x22, 0x56, 0x20, 0xcd, 0xfc, 0xeb, 0x95, 0x52, 0xdc, 0x37, 0xb7, 0xa6, 0xfb, 0x66, 0xe2, 0x00, + 0x86, 0x1d, 0x5a, 0xc2, 0xd1, 0xf8, 0x1a, 0x80, 0x65, 0x0e, 0x4d, 0x9f, 0x1c, 0x98, 0xb6, 0x5f, + 0x4a, 0xaf, 0x48, 0xab, 0x49, 0x21, 0x90, 0xe5, 0xf4, 0xbb, 0xa6, 0xed, 0x33, 0x3f, 0x99, 0x1e, + 0xe9, 0x1d, 0xd0, 0xde, 0xa3, 0xd2, 0x7c, 0xdc, 0x18, 0xd3, 0xab, 0x30, 0x22, 0x56, 0x01, 0x1e, + 0x9b, 0x9e, 0xb9, 0x67, 0x5a, 0xa6, 0x7f, 0x52, 0x5a, 0x58, 0x91, 0x56, 0x8b, 0x1b, 0xab, 0xd3, + 0x2d, 0x6a, 0xf7, 0x0c, 0x7b, 0x27, 0x92, 0x17, 0xca, 0x62, 0x1a, 0xf0, 0x8f, 0xe1, 0xf2, 0xd0, + 0x38, 0x26, 0xbe, 0x39, 0xa4, 0x9e, 0x6f, 0x0c, 0x47, 0xc4, 0x18, 0x50, 0x62, 0x1b, 0xb6, 0xe3, + 0x95, 0xb2, 0xb1, 0x38, 0x5d, 0x1a, 0x1a, 0xc7, 0x9d, 0x50, 0x46, 0x1e, 0x50, 0x95, 0x49, 0xe0, + 0x4f, 0x01, 0x89, 0xbc, 0x27, 0x9e, 0xef, 0x52, 0x7b, 0xe0, 0x1f, 0x94, 0x80, 0x9b, 0x74, 0x7b, + 0x4a, 0xac, 0x98, 0x3d, 0xcd, 0x00, 0xd2, 0x16, 0x08, 0xb1, 0xc3, 0xa2, 0x35, 0x49, 0xc6, 0x7b, + 0x70, 0x31, 0x54, 0x7e, 0x64, 0x98, 0x3e, 0x19, 0x39, 0x96, 0xd9, 0x3b, 0x29, 0xe5, 0xb8, 0xfe, + 0x3b, 0xff, 0x5d, 0xff, 0x03, 0xc3, 0xf4, 0x5b, 0x1c, 0x23, 0x76, 0xb8, 0x60, 0x9d, 0x65, 0xe0, + 0x1b, 0x90, 0x1b, 0x19, 0xae, 0x61, 0x59, 0xd4, 0x32, 0xbf, 0xa0, 0xa5, 0x7c, 0xcc, 0xe3, 0x71, + 0x06, 0xde, 0x00, 0x7c, 0x60, 0x78, 0xc4, 0x3b, 0xf1, 0x7c, 0x3a, 0x24, 0x61, 0xad, 0x14, 0x63, + 0xe2, 0xe8, 0xc0, 0xf0, 0xda, 0x9c, 0x5d, 0x11, 0xf5, 0xf0, 0x2e, 0x14, 0x6d, 0x4a, 0xfb, 0xb4, + 0x1f, 0xc9, 0x2f, 0xae, 0x24, 0x57, 0x0b, 0x7a, 0x21, 0xa0, 0x0a, 0xb1, 0xad, 0x54, 0x26, 0x83, + 0xb2, 0x5b, 0xa9, 0x4c, 0x01, 0x15, 0xcb, 0x3a, 0xe4, 0x6b, 0xa6, 0xe5, 0x53, 0x37, 0x2a, 0x80, + 0xf9, 0x7d, 0xbe, 0x16, 0x15, 0x70, 0x7d, 0x7a, 0xa0, 0x95, 0xa8, 0xf8, 0x85, 0x41, 0x02, 0x59, + 0xfe, 0x77, 0x12, 0x4a, 0x0d, 0x96, 0xea, 0xed, 0x47, 0xe6, 0xe8, 0x35, 0x55, 0x58, 0x54, 0x41, + 0xc9, 0x97, 0xaa, 0xa0, 0xc9, 0xdc, 0x4f, 0xbd, 0x74, 0xee, 0xc7, 0x0a, 0x3f, 0x7d, 0x5e, 0xe1, + 0x9f, 0x97, 0xde, 0xf3, 0xdf, 0x73, 0x7a, 0x2f, 0xbc, 0xc2, 0xf4, 0x2e, 0xff, 0x7c, 0x01, 0x8a, + 0x5b, 0x8e, 0x69, 0xff, 0xf0, 0x11, 0xbf, 0x05, 0x45, 0xcb, 0x71, 0x1e, 0x8d, 0x47, 0x51, 0xf2, + 0xb3, 0xd0, 0x17, 0x36, 0x13, 0x48, 0xd2, 0x0b, 0x01, 0x27, 0xac, 0x93, 0x0a, 0x2c, 0x38, 0xc1, + 0xe5, 0xc5, 0x43, 0x3a, 0x63, 0x96, 0x3b, 0x36, 0xa3, 0xe1, 0x8f, 0x20, 0xc5, 0x2e, 0x5a, 0x11, + 0x9e, 0x2b, 0x53, 0x4e, 0xc5, 0x7c, 0xd1, 0x39, 0x19, 0x51, 0x01, 0xe6, 0x90, 0x57, 0xde, 0x51, + 0x3f, 0x82, 0x37, 0x27, 0x8f, 0x4e, 0x0c, 0x97, 0x92, 0x47, 0xf4, 0xa4, 0x94, 0x89, 0x25, 0xd9, + 0xc5, 0x09, 0x27, 0xc8, 0x2e, 0xbd, 0x47, 0x4f, 0xce, 0x4d, 0xb8, 0xec, 0xf7, 0x9c, 0x70, 0xf0, + 0x2a, 0xfb, 0xe9, 0x07, 0x70, 0x61, 0x68, 0x98, 0xb6, 0x6f, 0x98, 0x36, 0x71, 0xdc, 0x3e, 0x75, + 0x4d, 0x7b, 0xc0, 0x3b, 0x76, 0xd4, 0x26, 0x43, 0xb6, 0x26, 0xb8, 0x53, 0x5a, 0x6b, 0xe1, 0x99, + 0xad, 0xb5, 0x0a, 0xef, 0x58, 0x74, 0xdf, 0x27, 0x7c, 0x30, 0x3b, 0x32, 0xfd, 0x03, 0x32, 0x32, + 0x4c, 0x97, 0xf6, 0x39, 0x81, 0xba, 0x13, 0x8d, 0xb9, 0xc4, 0x24, 0x59, 0xe0, 0x1f, 0x98, 0xfe, + 0x41, 0x8b, 0x8b, 0x6d, 0x71, 0x29, 0xfc, 0x00, 0x6e, 0x38, 0x63, 0x7f, 0x34, 0xf6, 0xc9, 0xc0, + 0x75, 0x78, 0xb8, 0x6c, 0xdf, 0xb4, 0xc7, 0x86, 0x6f, 0x3a, 0x36, 0xd9, 0x77, 0x5c, 0xc2, 0xf7, + 0x70, 0x9d, 0xa3, 0xd2, 0x62, 0x4c, 0xdf, 0xd5, 0x00, 0x53, 0x67, 0x90, 0x4a, 0x0c, 0x51, 0x73, + 0xdc, 0x26, 0xdd, 0xf7, 0x75, 0xe7, 0x68, 0x2b, 0x95, 0x49, 0xa3, 0xf9, 0xad, 0x54, 0x26, 0x8f, + 0x0a, 0x6c, 0xda, 0x82, 0xb6, 0xe3, 0xfa, 0xa2, 0xfc, 0xee, 0xc3, 0xa2, 0xd8, 0x33, 0x72, 0x4f, + 0x50, 0x88, 0xe5, 0xe9, 0x19, 0x17, 0xba, 0x4a, 0x18, 0x50, 0x0c, 0x14, 0xc4, 0x1d, 0x18, 0xea, + 0x22, 0x43, 0xc3, 0xef, 0x1d, 0x10, 0x8b, 0xda, 0x13, 0x65, 0x89, 0x42, 0xfe, 0x36, 0x63, 0x37, + 0xa9, 0x5d, 0xfe, 0x4d, 0x02, 0xf2, 0x55, 0xd3, 0xf3, 0x4d, 0xbb, 0xe7, 0x73, 0xbb, 0x6e, 0xc2, + 0x22, 0x17, 0x8a, 0xdd, 0x56, 0x12, 0xbf, 0xad, 0x8a, 0x82, 0x1c, 0xba, 0xfe, 0x16, 0xa0, 0xbe, + 0x00, 0x46, 0x92, 0x09, 0x2e, 0xb9, 0x18, 0xd2, 0x43, 0xd1, 0x0d, 0xc0, 0xf6, 0xd8, 0xb2, 0x82, + 0xfc, 0x0f, 0x99, 0x13, 0x23, 0x16, 0xe2, 0x7c, 0xd9, 0xa5, 0xa1, 0x2d, 0xf8, 0x06, 0xe4, 0xa9, + 0xeb, 0x3a, 0x2e, 0x71, 0x6c, 0xd2, 0x1f, 0x8f, 0x78, 0x47, 0xc8, 0x86, 0x45, 0xc6, 0x39, 0x9a, + 0x5d, 0x1d, 0x8f, 0xce, 0xf3, 0x63, 0xfa, 0xe5, 0xfc, 0x58, 0x46, 0x50, 0xd4, 0xdc, 0xbe, 0x69, + 0x1b, 0xac, 0x8a, 0x99, 0x53, 0xca, 0xbf, 0x4b, 0x02, 0xfa, 0xc4, 0x1c, 0x7c, 0x61, 0x0c, 0x82, + 0x8c, 0xe1, 0x9e, 0xaa, 0xc2, 0x3c, 0xef, 0x82, 0xe1, 0xa8, 0x3c, 0x5b, 0x07, 0x15, 0x58, 0x5c, + 0x03, 0xa0, 0x87, 0x13, 0x0e, 0xcc, 0x6d, 0x5c, 0x9d, 0x6e, 0xba, 0x70, 0x69, 0x38, 0x2f, 0xd2, + 0xc3, 0xd3, 0x70, 0x14, 0x83, 0x56, 0xec, 0x04, 0xa6, 0x4f, 0xf4, 0x59, 0xce, 0x11, 0x67, 0x7a, + 0x45, 0x7d, 0xf6, 0x1e, 0xe4, 0xf7, 0xcd, 0x63, 0xda, 0x27, 0x8f, 0xf9, 0x0b, 0xa2, 0x94, 0xe6, + 0x96, 0x3f, 0xa3, 0x5d, 0x4e, 0xbe, 0x34, 0xf4, 0x1c, 0x47, 0x07, 0xc4, 0x97, 0x68, 0xda, 0xe5, + 0xbf, 0x26, 0x61, 0x71, 0x9b, 0xba, 0x03, 0x1a, 0x8b, 0xcc, 0x36, 0x14, 0x78, 0xc5, 0xbe, 0x70, + 0x65, 0xe5, 0x19, 0x3c, 0xaa, 0x2b, 0x0d, 0x8a, 0xae, 0x39, 0x38, 0x88, 0xe9, 0x4b, 0xcc, 0xa8, + 0xaf, 0xc0, 0xf1, 0x91, 0xc2, 0x58, 0x00, 0xd2, 0xaf, 0xe3, 0xa2, 0xbb, 0x05, 0x05, 0x56, 0x6f, + 0x84, 0x1e, 0x8e, 0x8d, 0xe8, 0xae, 0x0b, 0x4b, 0x31, 0xcf, 0x58, 0x8a, 0xe0, 0xe0, 0x8f, 0xe1, + 0x32, 0x77, 0xe5, 0x69, 0x8e, 0x4e, 0xb9, 0xc4, 0xe8, 0xbe, 0xaf, 0x1c, 0x4e, 0x5e, 0x62, 0x3f, + 0x81, 0x52, 0xe0, 0xb7, 0x73, 0xc0, 0xd9, 0x18, 0xf8, 0x12, 0x97, 0x3a, 0x83, 0x2e, 0xff, 0x23, + 0x01, 0xc5, 0xbb, 0x86, 0x77, 0x10, 0x8b, 0xeb, 0x6d, 0x58, 0x3c, 0x63, 0x4c, 0xd0, 0x9b, 0xc4, + 0x30, 0x11, 0x37, 0x01, 0xdf, 0x01, 0x74, 0x76, 0xf3, 0xa0, 0x3d, 0x71, 0xe1, 0xe2, 0xe4, 0x96, + 0xaf, 0x3d, 0x22, 0xaf, 0xcd, 0xcd, 0x5b, 0xa9, 0xcc, 0x02, 0xca, 0x94, 0xbf, 0x49, 0x02, 0x6e, + 0x88, 0xef, 0x08, 0x31, 0x87, 0xff, 0x40, 0x33, 0xa2, 0x06, 0x85, 0xf0, 0x23, 0xc6, 0x8b, 0xb6, + 0xa5, 0x7c, 0xa8, 0x80, 0x47, 0xe2, 0x75, 0x87, 0xf3, 0xdc, 0xe9, 0x67, 0xe1, 0x99, 0xd3, 0xcf, + 0xf3, 0xcf, 0x20, 0x99, 0x59, 0x67, 0x90, 0x24, 0x4a, 0x95, 0xff, 0x90, 0x84, 0x4b, 0x61, 0x80, + 0x27, 0x5e, 0x96, 0x6b, 0x80, 0x22, 0xdf, 0xf7, 0x1c, 0x8b, 0x47, 0x49, 0x8a, 0x45, 0xa9, 0x18, + 0x72, 0x2b, 0x8e, 0xc5, 0x62, 0xf5, 0xd9, 0xd9, 0x58, 0x05, 0xbd, 0xf0, 0xff, 0xce, 0xb8, 0xc7, + 0x19, 0xf9, 0x6b, 0xf1, 0x0f, 0x53, 0x6b, 0xec, 0x11, 0x77, 0xea, 0xea, 0x96, 0xeb, 0xf8, 0xce, + 0xb9, 0xa1, 0xfb, 0x1c, 0x2e, 0x8c, 0x5c, 0x4a, 0xf6, 0x85, 0x8d, 0xc4, 0x1b, 0xd1, 0x1e, 0x0f, + 0x41, 0x6e, 0xe3, 0xa7, 0xd3, 0x83, 0x78, 0xde, 0xd1, 0xd6, 0x5a, 0x2e, 0x8d, 0xaf, 0xf5, 0xc5, + 0xd1, 0x24, 0x61, 0xe9, 0x57, 0x12, 0x2c, 0x9e, 0x11, 0xc2, 0x5b, 0x00, 0xa7, 0x9f, 0xcf, 0x5e, + 0xe0, 0xb5, 0x1d, 0x43, 0xe3, 0x35, 0x91, 0x41, 0x81, 0x8b, 0x96, 0xce, 0x66, 0x10, 0x1d, 0xae, + 0x05, 0xdf, 0x04, 0x3b, 0xe2, 0x2e, 0xfb, 0x67, 0x16, 0x8a, 0xf2, 0x60, 0xe0, 0xd2, 0x81, 0xe1, + 0x3b, 0x81, 0x39, 0x57, 0x01, 0xc2, 0x7c, 0xb0, 0xe2, 0x0d, 0x2c, 0x3b, 0x08, 0x42, 0x6e, 0x79, + 0xf8, 0x33, 0xc8, 0x1b, 0x02, 0x64, 0x3a, 0xd1, 0xd3, 0xfa, 0xff, 0xa7, 0xdb, 0x3c, 0xb9, 0x45, + 0xb4, 0x8c, 0x15, 0x53, 0x5c, 0x1f, 0xfe, 0x5f, 0x31, 0x56, 0xd2, 0x3e, 0x89, 0x99, 0x92, 0x8a, + 0x4c, 0x41, 0x82, 0x5b, 0x8f, 0x2c, 0xaa, 0x8b, 0x73, 0xa7, 0x79, 0xe5, 0xbc, 0xff, 0xdc, 0x96, + 0x9c, 0xad, 0xa3, 0xa5, 0x5f, 0x26, 0x20, 0x17, 0x33, 0x8f, 0x29, 0xde, 0x1f, 0xdb, 0x3d, 0x1e, + 0x96, 0x59, 0x14, 0xd7, 0xc6, 0x76, 0x2f, 0x54, 0xcc, 0x14, 0xe0, 0x15, 0xc8, 0x44, 0x73, 0x68, + 0x22, 0x56, 0x4f, 0x11, 0x15, 0x5f, 0x87, 0x62, 0x90, 0x83, 0x51, 0x55, 0xb0, 0xa6, 0x54, 0xd0, + 0xf3, 0x01, 0x55, 0x54, 0xc3, 0x65, 0xfe, 0xbd, 0x94, 0xb3, 0xd3, 0x7c, 0xf6, 0x9d, 0xef, 0x05, + 0x8c, 0xbb, 0x90, 0x35, 0xdc, 0xc1, 0x78, 0x48, 0x6d, 0xdf, 0x2b, 0xcd, 0xf3, 0x88, 0xcc, 0x92, + 0x45, 0xa7, 0x60, 0x51, 0xbf, 0x7f, 0x4a, 0x43, 0x8a, 0x9d, 0x02, 0x23, 0xc8, 0xcb, 0xea, 0x43, + 0xa2, 0x6a, 0x1d, 0xa2, 0x76, 0x9b, 0x4d, 0x34, 0x87, 0x17, 0x20, 0x29, 0xef, 0xd4, 0x91, 0x84, + 0xf3, 0x90, 0xd9, 0xd4, 0xb4, 0x26, 0x91, 0xd5, 0x2a, 0x4a, 0xe0, 0x1c, 0x2c, 0xf0, 0x95, 0xa6, + 0xa3, 0x24, 0x2e, 0x02, 0x54, 0x34, 0xb5, 0x22, 0x77, 0x88, 0x5c, 0xaf, 0xa3, 0x14, 0xce, 0x42, + 0xba, 0xa2, 0x75, 0xd5, 0x0e, 0x4a, 0x33, 0xf8, 0xb6, 0xbc, 0x8b, 0x16, 0xf8, 0x1f, 0x0d, 0x15, + 0x65, 0x30, 0xc0, 0x7c, 0xbb, 0x53, 0xad, 0x2a, 0x3b, 0x28, 0xcb, 0x88, 0xed, 0xee, 0x36, 0x02, + 0xa6, 0xae, 0xdd, 0xdd, 0x26, 0x0d, 0xb5, 0x83, 0x72, 0x6c, 0xa7, 0x1d, 0x59, 0x6f, 0xc8, 0x6a, + 0x45, 0x41, 0x79, 0xc6, 0xda, 0xd5, 0x74, 0xae, 0xb9, 0x10, 0xec, 0xd4, 0x55, 0x3b, 0x44, 0xd7, + 0x1e, 0xb4, 0x51, 0x91, 0xe3, 0xee, 0xeb, 0xd5, 0x46, 0xad, 0x86, 0x16, 0x31, 0x86, 0x62, 0xad, + 0xa1, 0xca, 0x4d, 0x12, 0xa1, 0x11, 0x3b, 0x50, 0x40, 0x13, 0x7b, 0x5e, 0xc0, 0x05, 0xc8, 0xca, + 0xba, 0x2e, 0x3f, 0xe4, 0x1a, 0x31, 0xdb, 0x6c, 0xab, 0xad, 0xa9, 0x7c, 0x75, 0x91, 0x31, 0xd9, + 0x6a, 0x93, 0x2f, 0x2f, 0xb1, 0xed, 0xda, 0x1d, 0xbd, 0xa1, 0xd6, 0xf9, 0xfa, 0x0d, 0x7e, 0xea, + 0x46, 0x87, 0xbb, 0xe0, 0x4d, 0x76, 0x10, 0xb6, 0xd0, 0x74, 0x74, 0x19, 0x67, 0x20, 0x55, 0xd1, + 0x74, 0x1d, 0x95, 0x70, 0x09, 0x2e, 0xb5, 0x14, 0xbd, 0xa2, 0xa8, 0x9d, 0x46, 0x53, 0x21, 0xd5, + 0x46, 0xbb, 0x42, 0x1a, 0xdb, 0xad, 0x26, 0x7a, 0xeb, 0x0c, 0xa7, 0xa2, 0xa9, 0x9d, 0x80, 0xb3, + 0x84, 0x2f, 0xc2, 0x22, 0xb7, 0x41, 0xdb, 0xdc, 0x52, 0x2a, 0x81, 0x13, 0xdf, 0xc6, 0x97, 0x00, + 0x05, 0xa6, 0xc4, 0xa8, 0xef, 0x30, 0x0b, 0x76, 0x64, 0x9d, 0xb4, 0xb4, 0x16, 0xfa, 0x9f, 0xc0, + 0x3c, 0x76, 0x2c, 0xbe, 0x5e, 0xc6, 0x8b, 0x90, 0x6b, 0x77, 0xc8, 0xb6, 0x7c, 0x4f, 0x69, 0x36, + 0x54, 0x05, 0x5d, 0x61, 0xc7, 0x69, 0x77, 0x88, 0xb2, 0xdb, 0x51, 0xd4, 0x0e, 0x5a, 0x61, 0x67, + 0x6d, 0x77, 0x48, 0x57, 0x6d, 0x68, 0x2a, 0xba, 0x1a, 0xa0, 0x49, 0x45, 0x6b, 0x36, 0x95, 0x4a, + 0x07, 0x95, 0x99, 0x70, 0x45, 0x0b, 0x95, 0x5f, 0x0b, 0x5c, 0xcd, 0x96, 0x6d, 0x79, 0xbb, 0x85, + 0xae, 0x33, 0xef, 0xea, 0x4a, 0x5d, 0x67, 0x31, 0x62, 0xa7, 0x68, 0x75, 0xd0, 0xbb, 0xcc, 0x1a, + 0x4e, 0xd3, 0x37, 0xd0, 0x0d, 0x06, 0xe0, 0x8b, 0x76, 0x53, 0x6b, 0x29, 0xe8, 0x26, 0xdb, 0x2d, + 0x58, 0xef, 0xee, 0xa2, 0xd5, 0xd3, 0xd5, 0xc3, 0x87, 0xe8, 0x56, 0x8c, 0xf7, 0x10, 0xdd, 0x8e, + 0x90, 0x41, 0xd2, 0xbc, 0xc7, 0x2c, 0xe1, 0x6b, 0x79, 0xa7, 0xbe, 0x8b, 0xee, 0xc4, 0x97, 0x0f, + 0xd1, 0xfb, 0xe5, 0x3b, 0x90, 0x62, 0xa5, 0xcc, 0x7c, 0x2e, 0x77, 0x3b, 0x1a, 0x9a, 0xe3, 0x29, + 0x55, 0x91, 0x9b, 0xb2, 0x8e, 0x24, 0xa6, 0x4b, 0xd5, 0x54, 0x22, 0xd6, 0x89, 0xf2, 0x5f, 0x24, + 0x28, 0xb6, 0x5c, 0xe7, 0x73, 0xda, 0xf3, 0xdb, 0x34, 0x78, 0x84, 0xfe, 0x0c, 0xd2, 0xac, 0x8d, + 0x86, 0x2f, 0xab, 0x59, 0x2a, 0x27, 0x00, 0xe2, 0x3a, 0x5c, 0x18, 0x50, 0x9b, 0xba, 0x86, 0x1f, + 0x7b, 0xc8, 0x06, 0xaf, 0xab, 0x67, 0xf5, 0x61, 0x14, 0x81, 0xc2, 0xc9, 0xf0, 0x26, 0x20, 0x7b, + 0xcc, 0x3f, 0x47, 0x78, 0x64, 0x44, 0x5d, 0x32, 0xa0, 0x76, 0xf0, 0xb2, 0xd2, 0x0b, 0xf6, 0x78, + 0xc8, 0xda, 0x5d, 0x8b, 0xba, 0x75, 0x6a, 0x97, 0xbf, 0x2d, 0x40, 0xfe, 0x81, 0x69, 0xf7, 0x9d, + 0x23, 0x71, 0x93, 0xac, 0xf0, 0x4f, 0xca, 0xbe, 0xc9, 0x1b, 0xeb, 0x89, 0x78, 0x45, 0xc7, 0x49, + 0xb8, 0x0d, 0xd9, 0x23, 0x8e, 0xa8, 0x45, 0xc6, 0xad, 0x4f, 0x3f, 0x6a, 0x5c, 0xb9, 0x58, 0xd4, + 0xa2, 0x7e, 0x11, 0xe9, 0x59, 0xfa, 0xb3, 0x24, 0x3a, 0x45, 0x1b, 0x0a, 0x61, 0x1f, 0xa7, 0xb5, + 0x17, 0xed, 0x9a, 0xfa, 0xa4, 0x0e, 0x7c, 0x1f, 0x40, 0x6c, 0xc5, 0x34, 0x26, 0xb8, 0xc6, 0x0f, + 0x66, 0xb3, 0x99, 0x69, 0x8d, 0x29, 0xf9, 0x38, 0xf5, 0xe4, 0xcb, 0x2b, 0xd2, 0xd2, 0x97, 0x0b, + 0x90, 0xae, 0xb9, 0xc6, 0x90, 0xe2, 0x7b, 0x90, 0x1a, 0x3a, 0x7d, 0x2a, 0xcc, 0x7d, 0x5e, 0xe5, + 0x1c, 0xbb, 0xb6, 0xed, 0xf4, 0xa3, 0x1b, 0x84, 0x29, 0xc1, 0xf7, 0x61, 0x7e, 0xcf, 0x19, 0xdb, + 0x7d, 0x6f, 0xca, 0x9c, 0xf2, 0x6c, 0x75, 0x9b, 0x1c, 0x1a, 0xce, 0x85, 0x81, 0x22, 0xfc, 0x09, + 0x64, 0xe9, 0x71, 0xcf, 0x1a, 0xf3, 0x01, 0x21, 0xc9, 0x8d, 0xfc, 0x70, 0x26, 0xad, 0x4a, 0x88, + 0x8e, 0x5e, 0xf1, 0x21, 0x61, 0xe9, 0x5f, 0x12, 0xa4, 0xf9, 0xa6, 0x6c, 0x17, 0xbe, 0x1f, 0x2b, + 0x24, 0xe1, 0x8a, 0x0f, 0x67, 0xb7, 0x3d, 0x76, 0xa3, 0x9e, 0xaa, 0xc3, 0xd7, 0x00, 0x4c, 0xdb, + 0x27, 0xce, 0xfe, 0xbe, 0x47, 0x83, 0xfb, 0x2f, 0xfc, 0x75, 0x27, 0x6b, 0xda, 0xbe, 0xc6, 0xc9, + 0xf8, 0x2a, 0xe4, 0x59, 0x55, 0xf4, 0x43, 0x31, 0x76, 0xd2, 0xbc, 0x9e, 0xe3, 0x34, 0x21, 0xb2, + 0x05, 0xb9, 0x80, 0xc9, 0x7f, 0x13, 0x15, 0x53, 0xfb, 0x0c, 0xbf, 0x18, 0x42, 0x80, 0x66, 0x36, + 0x2d, 0xfd, 0x5e, 0x82, 0xf9, 0xc0, 0xdd, 0x58, 0x85, 0xb4, 0xe7, 0x1b, 0xae, 0x2f, 0xa6, 0xaf, + 0x8d, 0xd9, 0x8f, 0x1d, 0xfd, 0x5a, 0xc0, 0xd4, 0xe0, 0x2a, 0x24, 0xa9, 0xdd, 0x17, 0x09, 0xf0, + 0x02, 0xda, 0x74, 0x06, 0x2f, 0xdf, 0x84, 0x14, 0xcb, 0x2e, 0x76, 0x75, 0xea, 0xb2, 0x5a, 0x57, + 0xd0, 0x1c, 0xeb, 0x6f, 0xfc, 0x96, 0x93, 0x58, 0x7f, 0xab, 0xeb, 0x5a, 0xb7, 0xd5, 0x46, 0x89, + 0xf2, 0x17, 0x90, 0x8d, 0x7c, 0x8f, 0x2f, 0xc3, 0xc5, 0xae, 0xba, 0xa9, 0x75, 0xd5, 0xaa, 0x52, + 0x25, 0x2d, 0x5d, 0xa9, 0x28, 0xd5, 0x86, 0x5a, 0x47, 0x73, 0x93, 0x8c, 0x9a, 0xd6, 0x6c, 0x6a, + 0x0f, 0x18, 0x43, 0x62, 0xb7, 0x8a, 0x56, 0xab, 0xb5, 0x95, 0x4e, 0x4c, 0x3c, 0x11, 0xa3, 0x9e, + 0xca, 0x26, 0xd9, 0x75, 0x52, 0xe9, 0xea, 0xba, 0x12, 0x5c, 0xb7, 0x28, 0x55, 0xfe, 0x14, 0xb2, + 0x51, 0x76, 0xb1, 0x9b, 0x55, 0xd5, 0x88, 0xb2, 0x5b, 0x69, 0x76, 0xdb, 0xec, 0x42, 0xe1, 0x9b, + 0xf2, 0x65, 0x55, 0x21, 0x71, 0x9c, 0x84, 0x2f, 0x40, 0x21, 0x64, 0xf0, 0x73, 0xa0, 0x04, 0x43, + 0x87, 0xa4, 0x4e, 0x43, 0x69, 0xa3, 0xe4, 0xd2, 0xdf, 0x12, 0x90, 0x09, 0xfb, 0x0e, 0x56, 0x62, + 0xa3, 0x58, 0x6e, 0xe3, 0xbd, 0xe7, 0xf5, 0xea, 0xd9, 0x41, 0xac, 0x0a, 0x99, 0xe8, 0x81, 0x94, + 0x9a, 0xf1, 0xab, 0x4a, 0x84, 0x64, 0xef, 0xd4, 0x7d, 0x16, 0x2f, 0xf1, 0xda, 0xbb, 0x33, 0x4b, + 0x8c, 0xf5, 0x00, 0x8a, 0x57, 0x61, 0x62, 0xb4, 0xe3, 0x6f, 0x8e, 0x74, 0x38, 0x10, 0x4f, 0x0c, + 0x7d, 0x4b, 0x90, 0x31, 0xdc, 0x81, 0xd7, 0xe8, 0x1f, 0x7b, 0xa5, 0x05, 0xde, 0xd5, 0xa3, 0x35, + 0xd3, 0x12, 0x3c, 0xc9, 0x84, 0x96, 0x4c, 0xec, 0x29, 0x35, 0xc1, 0xd9, 0x4a, 0x65, 0x12, 0x28, + 0x29, 0xa6, 0xbb, 0x3f, 0x4a, 0x00, 0xa7, 0xdd, 0x91, 0xdf, 0xb6, 0xda, 0x03, 0xa2, 0x76, 0xb7, + 0x37, 0x15, 0x5d, 0xe4, 0x99, 0xac, 0xde, 0x0b, 0xee, 0xce, 0xaa, 0xa2, 0xb6, 0x15, 0xc2, 0xd7, + 0x3c, 0x48, 0x62, 0x62, 0x09, 0x28, 0x49, 0x3e, 0x23, 0x74, 0xb7, 0xf9, 0x5c, 0xd3, 0x09, 0x06, + 0x3d, 0x3e, 0xcd, 0x04, 0x83, 0x5e, 0x53, 0xae, 0xa3, 0x79, 0xa6, 0xae, 0xa9, 0xc8, 0x55, 0xb4, + 0xc0, 0xf2, 0xa7, 0xd6, 0xd0, 0xdb, 0x1d, 0xb2, 0x23, 0x37, 0xbb, 0x0a, 0xca, 0x30, 0xfd, 0x4d, + 0x39, 0x5a, 0x67, 0x99, 0x36, 0xb5, 0x73, 0x57, 0x2c, 0xe1, 0xf6, 0x8f, 0xa0, 0x38, 0xf9, 0xab, + 0x07, 0x4b, 0xfc, 0x56, 0x77, 0xb3, 0xd9, 0xa8, 0xa0, 0x39, 0xfc, 0x16, 0xbc, 0x11, 0xfc, 0xcd, + 0xc6, 0x2f, 0x3e, 0xa1, 0x0a, 0x96, 0xb4, 0xf9, 0xfe, 0x93, 0x6f, 0x97, 0xe7, 0x9e, 0x3c, 0x5d, + 0x96, 0xbe, 0x7a, 0xba, 0x2c, 0x7d, 0xfd, 0x74, 0x59, 0xfa, 0xfb, 0xd3, 0x65, 0xe9, 0xd7, 0xdf, + 0x2d, 0xcf, 0x7d, 0xf5, 0xdd, 0xf2, 0xdc, 0xd7, 0xdf, 0x2d, 0xcf, 0x7d, 0x92, 0x8b, 0xfd, 0x8f, + 0x87, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0xf2, 0x30, 0x61, 0x69, 0xe1, 0x21, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/processors_sql.proto b/pkg/sql/execinfrapb/processors_sql.proto index 2ae09ea3269f..99e82fc62f15 100644 --- a/pkg/sql/execinfrapb/processors_sql.proto +++ b/pkg/sql/execinfrapb/processors_sql.proto @@ -390,6 +390,10 @@ message DistinctSpec { // UPSERT and INSERT..ON CONFLICT statements, both of which prohibit the same // row from being changed twice. optional string error_on_dup = 4 [(gogoproto.nullable) = false]; + // OutputOrdering specifies the required ordering of the output produced by + // the distinct. The input to the processor *must* already be ordered + // according to it. + optional Ordering output_ordering = 5 [(gogoproto.nullable) = false]; } // The specification for a WITH ORDINALITY processor. It adds a new column to diff --git a/pkg/sql/logictest/testdata/logic_test/explain_analyze_plans b/pkg/sql/logictest/testdata/logic_test/explain_analyze_plans index 825a78c135f0..413f136e77e3 100644 --- a/pkg/sql/logictest/testdata/logic_test/explain_analyze_plans +++ b/pkg/sql/logictest/testdata/logic_test/explain_analyze_plans @@ -66,7 +66,7 @@ https://cockroachdb.github.io/distsqlplan/decode.html#eJzkWt1yozYYve9TaHSVzOLFEn query T SELECT url FROM [EXPLAIN ANALYZE (DISTSQL) SELECT DISTINCT(kw.w) FROM kv JOIN kw ON kv.k = kw.w ORDER BY kw.w] ---- -https://cockroachdb.github.io/distsqlplan/decode.html#eJzkm9Fu4kYXx--_pxjN1a4-s2bGBhKklZLdpCrbFLZJVHW74sKxp8ECbGoPIWmUx-oL9MkqQwgxxnNsKmfGw12MGeB3zvH5-z8nfsTxnxPcxVfnF-efr9E8mqAfLgc_o-_nv329OO310Wn_9OLb7-fo3Vnv6vrql4v36PmtyXGv__kavRsvPizer5aN79CXQa-Pxgs06KPx3Ycx-oiS82hweXZ-iT59Wx4NsYGD0GN9Z8pi3P2OCTYwxQa2sIFtbOAWHhp4FoUui-MwSt7yuFzQ8-5xt2lgP5jNefLy0MBuGDHcfcTc5xOGu_jauZmwS-Z4LDKb2MAe444_WX7N-O5kFvlTJ3rABr6aOUHcRQ0z-eLBnHfRSfIzfvoVcX_Kuqj5z9_x6tgNA84C7odB5hSfzyYsRhFzvC56Xn7zwF9eOkKfsIFvHO6OWIzCOZ8lX5T8rOeV65cIHj4ZeHW04l3z3DygkROP0iQnyfuHBo65c8twlzwZ-wWovRWgRTZAJnkdIlrXENHcEG0-Zx6Ekcci5qU-aZishN6yI84_OvHoS-gHLDLJViFO2B_83Ql5_zHyb0fLv1IBZvfMnWeDOXXu0ZRNw-gBOZNJ6DqceV3UXMYvORe7URJF5PnxOPuOPSK8iZ5VpsDO_Jj7gctN0sqkZC-4yovD_g_FIQoAbUoLwAauVSZ3V2HEWWTSTOb-r2RdtgskblfalkA7c9cPG-HMtLYSl0Ne4Le3cn97J_XbSfGmTWBVM2nDtHTTtRIh6hTQtVSINNE18sa6RrTSNaDANrrWrqWulSsOoa4R5XQNyN1a1zKZq4Ou0eKNjxbQBqth2rppQ4kQHRXQhlSINNEG-sbaQLXSBqDANtrQqaU2lCsOoTZQ5bQByN1aGzKZq4M2WMUbn1VAG-yG2dJNG0qE6LiANqRCpIk2WG-sDZZW2gAU2EYbjmqpDeWKQ6gNlnLaAORurQ2ZzNVBG-zijc8uoA2thm7KUCJALVgZGiZBTuAhgkI-YpFuGmG_sUbYWmkEUGobjTiupUaUKw6hRtjKaQSQu7VGZDJXB40A5kGXLJ6FQcy2r-6dn9xMEsu8W7aqgjicRy77GoXu8mtWh4PluuV-ncdivjpLVwe9YH0q5g4XDV4qD18TL1tYcYTjXISA8UUYjdHE4SxwH14Y1q8vHJ-n6TwWs8h3Jv5fThZ9vexFI1zm3y3ZXp1ci8XLucq4SUs6OBGAl1PKUjVLpIPLyTjtaAtOSnQpqmaXEiMc5yLonbqtLiUDvKouBdQskQ4u6WLtaAtOt8Gbr8GtFHdze7EljJqt3I3YjooXIxB9dRkA11eXxeDUOlRwfU2HLexxLXGPawkXt9NB217cFnfXdHdpKtAhd0Svs6fbrnvViLnzr5bapE4Bty0ndxq7bQBcvqpX5VyO9nTbdU-4mDvfoivUpcQICrhtObnT2G0D4PLddlVd6lh4P0ia4ltRIt5Lzffbda8XAFzg0hXaaYAg5GuzpPRpbLkhcn09N8nsqKZbHQVanXhfsi09bpXtwAO7NNLBKysYMbjGVwqwE5vv0hW6h4fKVr7jlnXBaizrmX3VA_HcALjGc3ExuMCqq9SqgLKVb7sra1UAuXzfXVnhiqc4pA3cjWbmOIdivMXgGo_HIfJ8WVdpy0EMobPxBsg1thOZEVC61R0BrS6zN3soxjuzN3soxlsMLv9KkZRxjcfjUK2rt5--B4S-xpuWeQBGp9t4AFy-8a6qVQHgGs_IoVrPd-sKtSoIQl_jTcVjIAqMgWhmDHQgxhsA19h4Q-Qay7qYXODWFdpygCDk24nK0pcZBKVbnQ20OvF_qetrvGlmb1Yx4y0JXOMrRQyusfGGal1jWQfI1dtP3wUB7CpLv42vrFWJweUb78rKFhgDaWy8gVqXPyaXRZ7v1lVqVeIxEAXGQFT8L_oaG2_gaRSNFRog19h4A8-j6Dsmh8jVe8p1182J-KkaC3iqxspszupqvIdP__s3AAD__-SSa8c= +https://cockroachdb.github.io/distsqlplan/decode.html#eJzkW9Fu4kYUfe9XjOZpVzVrZmwgQVop2U2qsk1hm0RVtyseHHsaLMCm9hCSRvms_kC_rLIJIcZ4rp3ImWF4i7EHfO69PmfOvfE9jv-e4C6-OD07_XyJ5tEE_XQ--BV9P_3j69lxr4-O-8dn3_48Re9OeheXF7-dvUePlybHvf7nS_RuvPiweL9cNr5BXwa9Phov0KCPxjcfxugjSs6jwfnJ6Tn69C09GmIDB6HH-s6Uxbj7HRNsYIoNbGED29jALTw08CwKXRbHYZRccp8u6Hm3uNs0sB_M5jz5eGhgN4wY7t5j7vMJw1186VxN2DlzPBaZTWxgj3HHn6Q_M745mkX-1InusIEvZk4Qd1HDTH54MOdddJTcxi-_I-5PWRc1__s3Xh67YcBZwP0wyJ3i89mExShijtdFj8uv7vjTRwfoEzbwlcPdEYtROOez5IeS23pcufqI4OGDgZdHS7wrPFd3aOTEoyySo-T6oYFj7lwz3CUPxssC1N4I0CIfIJM8DxHd1RDRwhCtv2cehJHHIuZlvmmYrIQu2RLnn5149CX0AxaZZKMQJ-wv_u6IvP8Y-dej9K9MgNktc-f5YE6dWzRl0zC6Q85kEroOZ14XNdP4JediN0qiiDw_HueveEGE19GzqhTYiR9zP3C5SVq5lKgDrlr52K8oH1GIaFPhEK3ht6rk_yKMOItMmsv-j0pia5dI7bbEpoC2ZrcfNsKZaW2ktgB5iXtvFd57J3PvpDzxE1gZTdowLd20sUKIOiW0MRMiTbSRvLE2Eq20ESiwtTa2FSb-12hjtfIRaiNROERF2gjkf6WNuezvgjbS8uRJS-iL1TBt3fSlQogOSuhLJkSa6At9Y32hWukLUGBrfekoTJ6v0Zdq5SPUF6pwiIr0Bcj_Sl9y2d8FfbHKk6dVQl_shtnSTV8qhOiwhL5kQqSJvlhvrC-WVvoCFNhaXw4UJs_X6Eu18hHqi6VwiIr0Bcj_Sl9y2d8FfbHLk6ddQl9aDd3UpUKAWrC6NEyCnMBDBIV8xCLddMZ-Y52xtdIZoNTWOnOoMIm-RmeqlY9QZ2yFQ1SkM0D-VzqTy_4u6AwwHztn8SwMYrbJEFu_uZmknnnXbFkncTiPXPY1Ct30Z5aHg3Rd2nv0WMyXZ-nyoBesTsXc4aJBVO3ha-KUBstDOCyEEDC-CKMxmjicBe7dE4bV5wvH51l0HotZ5DsT_x8nD3217ElnXObfpNienVwJztO52nCTlnTgRAC8mtpWqlkiHbicjNOOtsBJBZaiarKUGMJhIQS9U7fBUjKA18VSQM0S6cAlPawdbYHTTeDN58CtDO7m5mJLGDVbuY3YlooXQyD66jIAXF9dFgOn1r4C19d02EKOa4k5riVc3M4GbXNxW8yuWXZpKsCQW6LXeaHb3vWqEeMuflp2JnUKuG05udPYbQPA5at6Xc7l4IVue9cTLsZdbNEVYikxBAXctpzcaey2AeDy3XZdLHUo3A-SpngrSsS91GK_vev1AgAXuHSFOg0QCPnaLCl9GltuCLm-npvkOqpZqqMA1Yn7km3pcautAw90aaQDr61gxMA1flKATmyxS1doDw-VrXzHLeuB1VjWc33VPfHcAHCN5-Ji4AKrrhJVAWUr33bXRlUAcvm-u7bCFU9xSBvYjebmOPtivMXANR6PQ8iLZV2lloMYhM7GG0CusZ3IjYCyVHcAUF2uN7svxjvXm90X4y0GLv9JkZRxjcfjUK2r109_AQh9jTet8gKMTtt4ALh8410XVQHANZ6RQ7Ve7NYVoioIhL7Gm4rHQBQYA9HcGGhPjDcAXGPjDSHXWNbFyAVuXaGWAwRCvp2oLX25QVCW6myA6sT_pa6v8aa53qxixlsScI2fFDFwjY03VOsayzqAXL1--jYQQFdZ-ja-NqoSA5dvvGsrW2AMpLHxBmpd_phcFvJit64SVYnHQBQYA1Hxv-hrbLyBt1E0VmgAucbGG3gfRd8xOYRcvbdct21OxG_VWMBbNVauOaur8R4-_PB_AAAA___z_Nwz # This query verifies stats collection for WITH ORDINALITY and the hashJoiner. query T diff --git a/pkg/sql/opt/exec/factory.opt b/pkg/sql/opt/exec/factory.opt index 7e8859f7c8c9..301467e80eda 100644 --- a/pkg/sql/opt/exec/factory.opt +++ b/pkg/sql/opt/exec/factory.opt @@ -147,10 +147,11 @@ define ScalarGroupBy { } # Distinct filters out rows such that only the first row is kept for each set of -# values along the distinct columns. The orderedCols are a subset of +# values along the distinct columns. The orderedCols are a subset of # distinctCols; the input is required to be ordered along these columns (i.e. # all rows with the same values on these columns are a contiguous part of the -# input). +# input). reqOrdering specifies the required output ordering, and if not empty, +# the input is already ordered according to it. define Distinct { Input exec.Node DistinctCols exec.NodeColumnOrdinalSet