From fa8f5db050be436a14b2be2bc67128fb6c483957 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Thu, 8 Apr 2021 19:49:47 -0700 Subject: [PATCH 1/6] colexec: clean up external distinct test a bit Release note: None --- pkg/sql/colexec/external_distinct_test.go | 32 +++++++++-------------- pkg/sql/colexec/external_sort_test.go | 3 --- 2 files changed, 13 insertions(+), 22 deletions(-) diff --git a/pkg/sql/colexec/external_distinct_test.go b/pkg/sql/colexec/external_distinct_test.go index a2242483556e..1e265fc310c9 100644 --- a/pkg/sql/colexec/external_distinct_test.go +++ b/pkg/sql/colexec/external_distinct_test.go @@ -67,38 +67,35 @@ func TestExternalDistinct(t *testing.T) { for tcIdx, tc := range distinctTestCases { log.Infof(context.Background(), "spillForced=%t/%d", spillForced, tcIdx) var semsToCheck []semaphore.Semaphore + var outputOrdering execinfrapb.Ordering + verifier := colexectestutils.UnorderedVerifier + // Check that the external distinct and the disk-backed sort + // were added as Closers. + numExpectedClosers := 2 + if tc.isOrderedOnDistinctCols { + outputOrdering = convertDistinctColsToOrdering(tc.distinctCols) + verifier = colexectestutils.OrderedVerifier + // The final disk-backed sort must also be added as a + // Closer. + numExpectedClosers++ + } colexectestutils.RunTestsWithTyps( t, testAllocator, []colexectestutils.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. - colexectestutils.UnorderedVerifier, + verifier, func(input []colexecop.Operator) (colexecop.Operator, error) { // A sorter should never exceed ExternalSorterMinPartitions, even // during repartitioning. A panic will happen if a sorter requests // more than this number of file descriptors. sem := colexecop.NewTestingSemaphore(colexecop.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, nil /* spillingCallbackFn */, numForcedRepartitions, ) - // 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...) @@ -385,9 +382,6 @@ func createExternalDistinct( } args.TestingKnobs.SpillingCallbackFn = spillingCallbackFn args.TestingKnobs.NumForcedRepartitions = numForcedRepartitions - // 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 := colexecargs.TestNewColOperator(ctx, flowCtx, args) return result.Op, result.OpAccounts, result.OpMonitors, result.ToClose, err } diff --git a/pkg/sql/colexec/external_sort_test.go b/pkg/sql/colexec/external_sort_test.go index 2d549729172c..3ddad4369053 100644 --- a/pkg/sql/colexec/external_sort_test.go +++ b/pkg/sql/colexec/external_sort_test.go @@ -487,9 +487,6 @@ func createDiskBackedSorter( 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. args.TestingKnobs.SpillingCallbackFn = spillingCallbackFn args.TestingKnobs.NumForcedRepartitions = numForcedRepartitions args.TestingKnobs.DelegateFDAcquisitions = delegateFDAcquisitions From a72d1bb037e72234efbda73bff41a56127fe2656 Mon Sep 17 00:00:00 2001 From: Angela Xu Date: Thu, 8 Apr 2021 16:09:22 -0700 Subject: [PATCH 2/6] opt: fix duplicate empty JSON value spans for <@ expressions Previously, there were some duplicate spans produced for empty JSON values, such as {"a": []} and {"a": {}}. This was discovered in the spans produced from contained by and fetch/contained by expressions. This commit fixes this issue, only allowing one empty object or array span to be created per level of nesting, in encodedContainedInvertedIndexSpans. Informs: #61430 Release note: None --- .../exec/execbuilder/testdata/inverted_index | 4 +- pkg/sql/opt/xform/testdata/rules/select | 6 -- pkg/util/json/encoded.go | 4 +- pkg/util/json/json.go | 85 +++++++++++-------- 4 files changed, 54 insertions(+), 45 deletions(-) diff --git a/pkg/sql/opt/exec/execbuilder/testdata/inverted_index b/pkg/sql/opt/exec/execbuilder/testdata/inverted_index index 6d560e310ed2..c3be2181789e 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/inverted_index +++ b/pkg/sql/opt/exec/execbuilder/testdata/inverted_index @@ -1112,13 +1112,13 @@ vectorized: true └── • inverted filter │ columns: (a, b_inverted_key) │ inverted column: b_inverted_key - │ num spans: 10 + │ num spans: 8 │ └── • scan columns: (a, b_inverted_key) estimated row count: 111 (missing stats) table: d@foo_inv - spans: /"f"-/"f"/PrefixEnd /[]-/{} /Arr/"f"-/Arr/"f"/PrefixEnd /Arr/{}-/Arr/{}/PrefixEnd /Arr/"a"/"b"-/Arr/"a"/"b"/PrefixEnd /Arr/"c"/{}-/Arr/"c"/{}/PrefixEnd /Arr/"c"/{}-/Arr/"c"/{}/PrefixEnd /Arr/"c"/"d"/[]-/Arr/"c"/"d"/{} /Arr/"c"/"d"/[]-/Arr/"c"/"d"/{} /Arr/"c"/"d"/Arr/"e"-/Arr/"c"/"d"/Arr/"e"/PrefixEnd + spans: /"f"-/"f"/PrefixEnd /[]-/{} /Arr/"f"-/Arr/"f"/PrefixEnd /Arr/{}-/Arr/{}/PrefixEnd /Arr/"a"/"b"-/Arr/"a"/"b"/PrefixEnd /Arr/"c"/{}-/Arr/"c"/{}/PrefixEnd /Arr/"c"/"d"/[]-/Arr/"c"/"d"/{} /Arr/"c"/"d"/Arr/"e"-/Arr/"c"/"d"/Arr/"e"/PrefixEnd # Ensure that an inverted index with a composite primary key still encodes # the primary key data in the composite value. diff --git a/pkg/sql/opt/xform/testdata/rules/select b/pkg/sql/opt/xform/testdata/rules/select index 5de7717a8ee8..ff1858c6dd5a 100644 --- a/pkg/sql/opt/xform/testdata/rules/select +++ b/pkg/sql/opt/xform/testdata/rules/select @@ -2506,7 +2506,6 @@ select │ │ ├── ["7\x00\x03\x00\x01*\x02\x00", "7\x00\x03\x00\x01*\x02\x00"] │ │ ├── ["7\x00\x03\x00\x019", "7\x00\x03\x00\x019"] │ │ ├── ["7\x00\x03a\x00\x019", "7\x00\x03a\x00\x019"] - │ │ ├── ["7\x00\x03a\x00\x02\x00\x019", "7\x00\x03a\x00\x02\x00\x019"] │ │ └── ["7\x00\x03a\x00\x02d\x00\x01\n", "7\x00\x03a\x00\x02d\x00\x01\n"] │ ├── key: (1) │ └── scan b@j_inv_idx @@ -2520,7 +2519,6 @@ select │ │ ├── ["7\x00\x03\x00\x01*\x02\x00", "7\x00\x03\x00\x01*\x02\x00"] │ │ ├── ["7\x00\x03\x00\x019", "7\x00\x03\x00\x019"] │ │ ├── ["7\x00\x03a\x00\x019", "7\x00\x03a\x00\x019"] - │ │ ├── ["7\x00\x03a\x00\x02\x00\x019", "7\x00\x03a\x00\x02\x00\x019"] │ │ └── ["7\x00\x03a\x00\x02d\x00\x01\n", "7\x00\x03a\x00\x02d\x00\x01\n"] │ ├── key: (1) │ └── fd: (1)-->(6) @@ -2553,7 +2551,6 @@ select │ │ ├── ["7\x00\x03\x00\x01*\x04\x00", "7\x00\x03\x00\x01*\x04\x00"] │ │ ├── ["7\x00\x03\x00\x01*\x06\x00", "7\x00\x03\x00\x01*\x06\x00"] │ │ ├── ["7a\x00\x018", "7a\x00\x018"] - │ │ ├── ["7a\x00\x02\x00\x018", "7a\x00\x02\x00\x018"] │ │ └── ["7a\x00\x02\x00\x03\x00\x01*\x06\x00", "7a\x00\x02\x00\x03\x00\x01*\x06\x00"] │ ├── key: (1) │ └── scan b@j_inv_idx @@ -2568,7 +2565,6 @@ select │ │ ├── ["7\x00\x03\x00\x01*\x04\x00", "7\x00\x03\x00\x01*\x04\x00"] │ │ ├── ["7\x00\x03\x00\x01*\x06\x00", "7\x00\x03\x00\x01*\x06\x00"] │ │ ├── ["7a\x00\x018", "7a\x00\x018"] - │ │ ├── ["7a\x00\x02\x00\x018", "7a\x00\x02\x00\x018"] │ │ └── ["7a\x00\x02\x00\x03\x00\x01*\x06\x00", "7a\x00\x02\x00\x03\x00\x01*\x06\x00"] │ ├── key: (1) │ └── fd: (1)-->(6) @@ -2599,7 +2595,6 @@ select │ │ │ └── union spans │ │ │ ├── ["7\x00\x019", "7\x00\x019"] │ │ │ ├── ["7a\x00\x018", "7a\x00\x018"] - │ │ │ ├── ["7a\x00\x02\x00\x018", "7a\x00\x02\x00\x018"] │ │ │ └── ["7a\x00\x02\x00\x03\x00\x01*\x06\x00", "7a\x00\x02\x00\x03\x00\x01*\x06\x00"] │ │ └── span expression │ │ ├── tight: false, unique: false @@ -2624,7 +2619,6 @@ select │ │ ├── ["7\x00\x03\x00\x01*\x04\x00", "7\x00\x03\x00\x01*\x04\x00"] │ │ ├── ["7\x00\x03\x00\x01*\x06\x00", "7\x00\x03\x00\x01*\x06\x00"] │ │ ├── ["7a\x00\x018", "7a\x00\x018"] - │ │ ├── ["7a\x00\x02\x00\x018", "7a\x00\x02\x00\x018"] │ │ └── ["7a\x00\x02\x00\x03\x00\x01*\x06\x00", "7a\x00\x02\x00\x03\x00\x01*\x06\x00"] │ ├── key: (1) │ └── fd: (1)-->(6) diff --git a/pkg/util/json/encoded.go b/pkg/util/json/encoded.go index e9f316bb784a..f1a53e939d35 100644 --- a/pkg/util/json/encoded.go +++ b/pkg/util/json/encoded.go @@ -745,13 +745,13 @@ func (j *jsonEncoded) encodeContainingInvertedIndexSpans( } func (j *jsonEncoded) encodeContainedInvertedIndexSpans( - b []byte, isRoot bool, + b []byte, isRoot, isObjectValue bool, ) (inverted.Expression, error) { decoded, err := j.decode() if err != nil { return nil, err } - return decoded.encodeContainedInvertedIndexSpans(b, isRoot) + return decoded.encodeContainedInvertedIndexSpans(b, isRoot, isObjectValue) } // numInvertedIndexEntries implements the JSON interface. diff --git a/pkg/util/json/json.go b/pkg/util/json/json.go index e4cdef78451e..60b282ea1af1 100644 --- a/pkg/util/json/json.go +++ b/pkg/util/json/json.go @@ -105,9 +105,11 @@ type JSON interface { // comments in the SpanExpression definition for details. // // If isRoot is true, this function is being called at the root level of the - // JSON hierarchy. + // JSON hierarchy. If isObjectValue is true, the given JSON is the value of a + // JSON object key. Note that isRoot and isObjectValue cannot both be true at + // the same time. encodeContainedInvertedIndexSpans( - b []byte, isRoot bool, + b []byte, isRoot, isObjectValue bool, ) (invertedExpr inverted.Expression, err error) // numInvertedIndexEntries returns the number of entries that will be @@ -814,7 +816,7 @@ func EncodeContainedInvertedIndexSpans( b []byte, json JSON, ) (invertedExpr inverted.Expression, err error) { invertedExpr, err = json.encodeContainedInvertedIndexSpans( - encoding.EncodeJSONAscending(b), true, /* isRoot */ + encoding.EncodeJSONAscending(b), true /* isRoot */, false, /* isObjectValue */ ) if err != nil { return nil, err @@ -843,7 +845,7 @@ func (j jsonNull) encodeContainingInvertedIndexSpans( } func (j jsonNull) encodeContainedInvertedIndexSpans( - b []byte, isRoot bool, + b []byte, isRoot, isObjectValue bool, ) (inverted.Expression, error) { invertedExpr, err := encodeContainedInvertedIndexSpansFromLeaf(j, b, isRoot) return invertedExpr, err @@ -861,7 +863,7 @@ func (j jsonTrue) encodeContainingInvertedIndexSpans( } func (j jsonTrue) encodeContainedInvertedIndexSpans( - b []byte, isRoot bool, + b []byte, isRoot, isObjectValue bool, ) (inverted.Expression, error) { invertedExpr, err := encodeContainedInvertedIndexSpansFromLeaf(j, b, isRoot) return invertedExpr, err @@ -879,7 +881,7 @@ func (j jsonFalse) encodeContainingInvertedIndexSpans( } func (j jsonFalse) encodeContainedInvertedIndexSpans( - b []byte, isRoot bool, + b []byte, isRoot, isObjectValue bool, ) (inverted.Expression, error) { invertedExpr, err := encodeContainedInvertedIndexSpansFromLeaf(j, b, isRoot) return invertedExpr, err @@ -897,7 +899,7 @@ func (j jsonString) encodeContainingInvertedIndexSpans( } func (j jsonString) encodeContainedInvertedIndexSpans( - b []byte, isRoot bool, + b []byte, isRoot, isObjectValue bool, ) (inverted.Expression, error) { invertedExpr, err := encodeContainedInvertedIndexSpansFromLeaf(j, b, isRoot) return invertedExpr, err @@ -916,7 +918,7 @@ func (j jsonNumber) encodeContainingInvertedIndexSpans( } func (j jsonNumber) encodeContainedInvertedIndexSpans( - b []byte, isRoot bool, + b []byte, isRoot, isObjectValue bool, ) (inverted.Expression, error) { invertedExpr, err := encodeContainedInvertedIndexSpansFromLeaf(j, b, isRoot) return invertedExpr, err @@ -992,29 +994,37 @@ func (j jsonArray) encodeContainingInvertedIndexSpans( } func (j jsonArray) encodeContainedInvertedIndexSpans( - b []byte, isRoot bool, + b []byte, isRoot, isObjectValue bool, ) (invertedExpr inverted.Expression, err error) { - // The empty array should always be added to the spans, since it is contained - // by everything. - emptyArrSpanExpr := inverted.ExprForSpan( - inverted.MakeSingleValSpan(encoding.EncodeJSONEmptyArray(b[:len(b):len(b)])), false, /* tight */ - ) - emptyArrSpanExpr.Unique = true + if !isObjectValue || len(j) == 0 { + // The empty array should always be added to the spans, since it is contained + // by everything. Empty array values are already accounted for when getting + // the spans for a non-empty object value, so they should be excluded. + emptyArrSpanExpr := inverted.ExprForSpan( + inverted.MakeSingleValSpan(encoding.EncodeJSONEmptyArray(b[:len(b):len(b)])), false, /* tight */ + ) + emptyArrSpanExpr.Unique = true + invertedExpr = emptyArrSpanExpr + } + // If the given jsonArray is empty, we return the SpanExpression. if len(j) == 0 { - return emptyArrSpanExpr, nil + return invertedExpr, nil } - invertedExpr = emptyArrSpanExpr prefix := encoding.EncodeArrayAscending(b[:len(b):len(b)]) for i := range j { childWithPrefix, err := j[i].encodeContainedInvertedIndexSpans( - prefix[:len(prefix):len(prefix)], false, /* isRoot */ + prefix[:len(prefix):len(prefix)], false /* isRoot */, false, /* isObjectValue */ ) if err != nil { return nil, err } - invertedExpr = inverted.Or(invertedExpr, childWithPrefix) + if invertedExpr == nil { + invertedExpr = childWithPrefix + } else { + invertedExpr = inverted.Or(invertedExpr, childWithPrefix) + } // Scalars inside the array should also be included in the spans // without the array prefix, since they are contained by the array. This @@ -1028,7 +1038,7 @@ func (j jsonArray) encodeContainedInvertedIndexSpans( // and empty arrays/objects. if isRoot && isEnd(j[i]) { childWithoutPrefix, err := j[i].encodeContainedInvertedIndexSpans( - b[:len(b):len(b)], false, /* isRoot */ + b[:len(b):len(b)], false /* isRoot */, false, /* isObjectValue */ ) if err != nil { return nil, err @@ -1112,34 +1122,41 @@ func (j jsonObject) encodeContainingInvertedIndexSpans( } func (j jsonObject) encodeContainedInvertedIndexSpans( - b []byte, isRoot bool, + b []byte, isRoot, isObjectValue bool, ) (invertedExpr inverted.Expression, err error) { // The empty object should always be added to the spans, since it is contained - // by everything. - emptyObjSpanExpr := inverted.ExprForSpan( - inverted.MakeSingleValSpan(encoding.EncodeJSONEmptyObject(b[:len(b):len(b)])), false, /* tight */ - ) - emptyObjSpanExpr.Unique = true + // by everything. Empty object values are already accounted for when getting + // the spans for a non-empty object value, so they should be excluded. + if !isObjectValue || len(j) == 0 { + emptyObjSpanExpr := inverted.ExprForSpan( + inverted.MakeSingleValSpan(encoding.EncodeJSONEmptyObject(b[:len(b):len(b)])), false, /* tight */ + ) + emptyObjSpanExpr.Unique = true + invertedExpr = emptyObjSpanExpr + } // If the given jsonObject is empty, we return the SpanExpression. if len(j) == 0 { - return emptyObjSpanExpr, nil + return invertedExpr, nil } - invertedExpr = emptyObjSpanExpr for i := range j { // We're trying to see if this is the end of the JSON path. If it is, then // we don't want to add an extra separator. end := isEnd(j[i].v) prefix := encoding.EncodeJSONKeyStringAscending(b[:len(b):len(b)], string(j[i].k), end) - child, err := j[i].v.encodeContainedInvertedIndexSpans( - prefix, false, /* isRoot */ + prefix, false /* isRoot */, true, /* isObjectValue */ ) if err != nil { return nil, err } - invertedExpr = inverted.Or(invertedExpr, child) + + if invertedExpr == nil { + invertedExpr = child + } else { + invertedExpr = inverted.Or(invertedExpr, child) + } // When we have a nested object or array, we want to include the empty // object or array span with the prefix. For example, '{"a": {"b": "c"}}' @@ -1150,7 +1167,7 @@ func (j jsonObject) encodeContainedInvertedIndexSpans( if v != nil { prefixWithEnd := encoding.EncodeJSONKeyStringAscending(b[:len(b):len(b)], string(j[i].k), true) childWithEnd, err := v.encodeContainedInvertedIndexSpans( - prefixWithEnd, false, /* isRoot */ + prefixWithEnd, false /* isRoot */, true, /* isObjectValue */ ) if err != nil { return nil, err @@ -1349,9 +1366,7 @@ func encodeContainingInvertedIndexSpansFromLeaf( // array spans are not encoded by this function. // // If isRoot is true, this function is being called at the root level of the -// JSON hierarchy. If isObjectValue is true, the given JSON is the value of a -// JSON object key. Note that isRoot and isObjectValue cannot both be true at -// the same time. +// JSON hierarchy. func encodeContainedInvertedIndexSpansFromLeaf( j JSON, b []byte, isRoot bool, ) (invertedExpr inverted.Expression, err error) { From 8c7cbf7dcfa2cb4de54f33dd967fca52993e4cc7 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Thu, 8 Apr 2021 21:39:12 -0700 Subject: [PATCH 3/6] execinfrapb: introduce aliases for agg funcs and use everywhere This commit introduces nicer aliases for the specification of the aggregate functions and uses the aliases throughout the code base. Release note: None --- pkg/sql/colexec/aggregators_test.go | 145 +++++++++--------- pkg/sql/colexec/colexecagg/aggregate_funcs.go | 44 +++--- pkg/sql/colexec/default_agg_test.go | 22 +-- .../colexec/external_hash_aggregator_test.go | 2 +- pkg/sql/colexec/hash_aggregator_test.go | 4 +- pkg/sql/colflow/vectorized_flow_space_test.go | 2 +- pkg/sql/distsql/columnar_operators_test.go | 108 ++++++------- pkg/sql/distsql_physical_planner.go | 2 +- pkg/sql/execinfrapb/BUILD.bazel | 1 + pkg/sql/execinfrapb/aggregate_funcs.go | 61 ++++++++ pkg/sql/execinfrapb/processors.go | 6 +- pkg/sql/physicalplan/aggregator_funcs.go | 102 ++++++------ pkg/sql/physicalplan/aggregator_funcs_test.go | 6 +- pkg/sql/rowexec/aggregator_test.go | 42 ++--- pkg/sql/rowexec/processors_test.go | 48 +++--- pkg/sql/rowexec/windower_test.go | 2 +- 16 files changed, 328 insertions(+), 269 deletions(-) create mode 100644 pkg/sql/execinfrapb/aggregate_funcs.go diff --git a/pkg/sql/colexec/aggregators_test.go b/pkg/sql/colexec/aggregators_test.go index e24259ab03b3..7232a405394b 100644 --- a/pkg/sql/colexec/aggregators_test.go +++ b/pkg/sql/colexec/aggregators_test.go @@ -36,13 +36,10 @@ import ( "github.com/stretchr/testify/require" ) -// TODO(yuzefovich): introduce nicer aliases for the protobuf generated ones for -// the aggregate functions and use those throughout the codebase. - var ( defaultGroupCols = []uint32{0} defaultAggCols = [][]uint32{{1}} - defaultAggFns = []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_SUM_INT} + defaultAggFns = []execinfrapb.AggregatorSpec_Func{execinfrapb.SumInt} defaultTyps = []*types.T{types.Int, types.Int} ) @@ -286,8 +283,8 @@ var aggregatorsTestCases = []aggregatorTestCase{ }, { aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_COUNT_ROWS, + execinfrapb.AnyNotNull, + execinfrapb.CountRows, }, aggCols: [][]uint32{{0}, {}}, typs: []*types.T{types.Int}, @@ -316,7 +313,7 @@ var aggregatorsTestCases = []aggregatorTestCase{ }, }, { - aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_SUM_INT, execinfrapb.AggregatorSpec_SUM_INT}, + aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.SumInt, execinfrapb.SumInt}, aggCols: [][]uint32{ {2}, {1}, }, @@ -331,7 +328,7 @@ var aggregatorsTestCases = []aggregatorTestCase{ name: "OutputOrder", }, { - aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_SUM, execinfrapb.AggregatorSpec_SUM_INT}, + aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.Sum, execinfrapb.SumInt}, aggCols: [][]uint32{ {2}, {1}, }, @@ -350,7 +347,7 @@ var aggregatorsTestCases = []aggregatorTestCase{ convToDecimal: true, }, { - aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_AVG, execinfrapb.AggregatorSpec_SUM}, + aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.Avg, execinfrapb.Sum}, aggCols: [][]uint32{ {1}, {1}, }, @@ -371,8 +368,8 @@ var aggregatorsTestCases = []aggregatorTestCase{ }, { aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_BOOL_AND, - execinfrapb.AggregatorSpec_BOOL_OR, + execinfrapb.BoolAnd, + execinfrapb.BoolOr, }, aggCols: [][]uint32{ {1}, {1}, @@ -412,11 +409,11 @@ var aggregatorsTestCases = []aggregatorTestCase{ }, { aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_MIN, - execinfrapb.AggregatorSpec_SUM, + execinfrapb.AnyNotNull, + execinfrapb.AnyNotNull, + execinfrapb.AnyNotNull, + execinfrapb.Min, + execinfrapb.Sum, }, input: colexectestutils.Tuples{ {2, 1.0, "1.0", 2.0}, @@ -436,8 +433,8 @@ var aggregatorsTestCases = []aggregatorTestCase{ }, { aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_SUM_INT, + execinfrapb.AnyNotNull, + execinfrapb.SumInt, }, input: colexectestutils.Tuples{ {tree.NewDTimeTZFromOffset(timeofday.FromInt(0), 0), -1}, @@ -476,7 +473,7 @@ var aggregatorsTestCases = []aggregatorTestCase{ {3.0, 3.0, 3.0, 3.0, 3.0, duration.MakeDuration(3, 3, 3)}, }, typs: []*types.T{types.Int, types.Int2, types.Int4, types.Int, types.Decimal, types.Float, types.Interval}, - aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_AVG, execinfrapb.AggregatorSpec_AVG, execinfrapb.AggregatorSpec_AVG, execinfrapb.AggregatorSpec_AVG, execinfrapb.AggregatorSpec_AVG, execinfrapb.AggregatorSpec_AVG}, + aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.Avg, execinfrapb.Avg, execinfrapb.Avg, execinfrapb.Avg, execinfrapb.Avg, execinfrapb.Avg}, aggCols: [][]uint32{{1}, {2}, {3}, {4}, {5}, {6}}, name: "AVG on all types", }, @@ -501,24 +498,24 @@ var aggregatorsTestCases = []aggregatorTestCase{ {nil}, }, typs: []*types.T{types.Int, types.Bytes}, - aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_CONCAT_AGG}, + aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.ConcatAgg}, groupCols: []uint32{0}, aggCols: [][]uint32{{1}}, }, { aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_AVG, - execinfrapb.AggregatorSpec_COUNT_ROWS, - execinfrapb.AggregatorSpec_COUNT, - execinfrapb.AggregatorSpec_SUM, - execinfrapb.AggregatorSpec_SUM_INT, - execinfrapb.AggregatorSpec_MIN, - execinfrapb.AggregatorSpec_MAX, - execinfrapb.AggregatorSpec_BOOL_AND, - execinfrapb.AggregatorSpec_BOOL_OR, - execinfrapb.AggregatorSpec_CONCAT_AGG, + execinfrapb.AnyNotNull, + execinfrapb.AnyNotNull, + execinfrapb.Avg, + execinfrapb.CountRows, + execinfrapb.Count, + execinfrapb.Sum, + execinfrapb.SumInt, + execinfrapb.Min, + execinfrapb.Max, + execinfrapb.BoolAnd, + execinfrapb.BoolOr, + execinfrapb.ConcatAgg, }, aggCols: [][]uint32{{0}, {4}, {1}, {}, {1}, {1}, {2}, {2}, {2}, {3}, {3}, {4}}, typs: []*types.T{types.Int, types.Decimal, types.Int, types.Bool, types.Bytes}, @@ -543,18 +540,18 @@ var aggregatorsTestCases = []aggregatorTestCase{ // Test case for null handling. { aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_COUNT_ROWS, - execinfrapb.AggregatorSpec_COUNT, - execinfrapb.AggregatorSpec_SUM, - execinfrapb.AggregatorSpec_SUM_INT, - execinfrapb.AggregatorSpec_MIN, - execinfrapb.AggregatorSpec_MAX, - execinfrapb.AggregatorSpec_AVG, - execinfrapb.AggregatorSpec_BOOL_AND, - execinfrapb.AggregatorSpec_BOOL_OR, - execinfrapb.AggregatorSpec_CONCAT_AGG, + execinfrapb.AnyNotNull, + execinfrapb.AnyNotNull, + execinfrapb.CountRows, + execinfrapb.Count, + execinfrapb.Sum, + execinfrapb.SumInt, + execinfrapb.Min, + execinfrapb.Max, + execinfrapb.Avg, + execinfrapb.BoolAnd, + execinfrapb.BoolOr, + execinfrapb.ConcatAgg, }, aggCols: [][]uint32{{0}, {1}, {}, {1}, {1}, {2}, {2}, {2}, {1}, {3}, {3}, {4}}, typs: []*types.T{types.Int, types.Decimal, types.Int, types.Bool, types.Bytes}, @@ -576,11 +573,11 @@ var aggregatorsTestCases = []aggregatorTestCase{ // Test DISTINCT aggregation. { aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_COUNT, - execinfrapb.AggregatorSpec_COUNT, - execinfrapb.AggregatorSpec_SUM_INT, - execinfrapb.AggregatorSpec_SUM_INT, + execinfrapb.AnyNotNull, + execinfrapb.Count, + execinfrapb.Count, + execinfrapb.SumInt, + execinfrapb.SumInt, }, aggCols: [][]uint32{{0}, {1}, {1}, {1}, {1}}, aggDistinct: []bool{false, false, true, false, true}, @@ -605,9 +602,9 @@ var aggregatorsTestCases = []aggregatorTestCase{ // Test aggregation with FILTERs. { aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_COUNT_ROWS, - execinfrapb.AggregatorSpec_SUM_INT, + execinfrapb.AnyNotNull, + execinfrapb.CountRows, + execinfrapb.SumInt, }, aggCols: [][]uint32{{0}, {}, {1}}, aggFilter: []int{tree.NoColumnIdx, 2, 2}, @@ -632,9 +629,9 @@ var aggregatorsTestCases = []aggregatorTestCase{ // Test aggregation with FILTERs when the whole groups are filtered out. { aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_COUNT_ROWS, - execinfrapb.AggregatorSpec_SUM_INT, + execinfrapb.AnyNotNull, + execinfrapb.CountRows, + execinfrapb.SumInt, }, aggCols: [][]uint32{{0}, {}, {1}}, aggFilter: []int{tree.NoColumnIdx, 2, 2}, @@ -660,13 +657,13 @@ var aggregatorsTestCases = []aggregatorTestCase{ // Test aggregation with FILTERs and DISTINCTs intertwined. { aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_COUNT, - execinfrapb.AggregatorSpec_COUNT, - execinfrapb.AggregatorSpec_COUNT, - execinfrapb.AggregatorSpec_SUM_INT, - execinfrapb.AggregatorSpec_SUM_INT, - execinfrapb.AggregatorSpec_SUM_INT, + execinfrapb.AnyNotNull, + execinfrapb.Count, + execinfrapb.Count, + execinfrapb.Count, + execinfrapb.SumInt, + execinfrapb.SumInt, + execinfrapb.SumInt, }, aggCols: [][]uint32{{0}, {1}, {1}, {1}, {1}, {1}, {1}}, aggDistinct: []bool{false, false, true, true, false, true, true}, @@ -831,12 +828,12 @@ func TestAggregatorRandom(t *testing.T) { tc := aggregatorTestCase{ typs: typs, aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_COUNT_ROWS, - execinfrapb.AggregatorSpec_COUNT, - execinfrapb.AggregatorSpec_SUM, - execinfrapb.AggregatorSpec_MIN, - execinfrapb.AggregatorSpec_MAX, - execinfrapb.AggregatorSpec_AVG, + execinfrapb.CountRows, + execinfrapb.Count, + execinfrapb.Sum, + execinfrapb.Min, + execinfrapb.Max, + execinfrapb.Avg, }, groupCols: []uint32{0}, aggCols: [][]uint32{{}, {1}, {1}, {1}, {1}, {1}}, @@ -931,7 +928,7 @@ func benchmarkAggregateFunction( BytesFixedLength: bytesFixedLength, }) } - if aggFn == execinfrapb.AggregatorSpec_SUM_INT { + if aggFn == execinfrapb.SumInt { // Integer summation of random Int64 values can lead // to overflow, and we will panic. To go around it, we // restrict the range of values. @@ -1037,7 +1034,7 @@ func benchmarkAggregateFunction( // benchmark is measuring the performance of the aggregators themselves // depending on the parameters of the input. func BenchmarkAggregator(b *testing.B) { - aggFn := execinfrapb.AggregatorSpec_MIN + aggFn := execinfrapb.Min numRows := []int{1, 32, coldata.BatchSize(), 32 * coldata.BatchSize(), 1024 * coldata.BatchSize()} groupSizes := []int{1, 2, 32, 128, coldata.BatchSize()} if testing.Short() { @@ -1075,11 +1072,11 @@ func BenchmarkAllOptimizedAggregateFunctions(b *testing.B) { for _, agg := range aggTypes { var aggInputTypes []*types.T switch aggFn { - case execinfrapb.AggregatorSpec_BOOL_AND, execinfrapb.AggregatorSpec_BOOL_OR: + case execinfrapb.BoolAnd, execinfrapb.BoolOr: aggInputTypes = []*types.T{types.Bool} - case execinfrapb.AggregatorSpec_CONCAT_AGG: + case execinfrapb.ConcatAgg: aggInputTypes = []*types.T{types.Bytes} - case execinfrapb.AggregatorSpec_COUNT_ROWS: + case execinfrapb.CountRows: default: aggInputTypes = []*types.T{types.Int} } @@ -1094,7 +1091,7 @@ func BenchmarkAllOptimizedAggregateFunctions(b *testing.B) { } func BenchmarkDistinctAggregation(b *testing.B) { - aggFn := execinfrapb.AggregatorSpec_COUNT + aggFn := execinfrapb.Count for _, agg := range aggTypes { for _, numInputRows := range []int{32, 32 * coldata.BatchSize()} { for _, groupSize := range []int{1, 2, 32, 128, coldata.BatchSize()} { diff --git a/pkg/sql/colexec/colexecagg/aggregate_funcs.go b/pkg/sql/colexec/colexecagg/aggregate_funcs.go index 713839ff65df..227e436a284b 100644 --- a/pkg/sql/colexec/colexecagg/aggregate_funcs.go +++ b/pkg/sql/colexec/colexecagg/aggregate_funcs.go @@ -29,17 +29,17 @@ import ( // IsAggOptimized returns whether aggFn has an optimized implementation. func IsAggOptimized(aggFn execinfrapb.AggregatorSpec_Func) bool { switch aggFn { - case execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_AVG, - execinfrapb.AggregatorSpec_SUM, - execinfrapb.AggregatorSpec_SUM_INT, - execinfrapb.AggregatorSpec_CONCAT_AGG, - execinfrapb.AggregatorSpec_COUNT_ROWS, - execinfrapb.AggregatorSpec_COUNT, - execinfrapb.AggregatorSpec_MIN, - execinfrapb.AggregatorSpec_MAX, - execinfrapb.AggregatorSpec_BOOL_AND, - execinfrapb.AggregatorSpec_BOOL_OR: + case execinfrapb.AnyNotNull, + execinfrapb.Avg, + execinfrapb.Sum, + execinfrapb.SumInt, + execinfrapb.ConcatAgg, + execinfrapb.CountRows, + execinfrapb.Count, + execinfrapb.Min, + execinfrapb.Max, + execinfrapb.BoolAnd, + execinfrapb.BoolOr: return true default: return false @@ -225,67 +225,67 @@ func NewAggregateFuncsAlloc( for i, aggFn := range args.Spec.Aggregations { var err error switch aggFn.Func { - case execinfrapb.AggregatorSpec_ANY_NOT_NULL: + case execinfrapb.AnyNotNull: if isHashAgg { funcAllocs[i], err = newAnyNotNullHashAggAlloc(args.Allocator, args.InputTypes[aggFn.ColIdx[0]], allocSize) } else { funcAllocs[i], err = newAnyNotNullOrderedAggAlloc(args.Allocator, args.InputTypes[aggFn.ColIdx[0]], allocSize) } - case execinfrapb.AggregatorSpec_AVG: + case execinfrapb.Avg: if isHashAgg { funcAllocs[i], err = newAvgHashAggAlloc(args.Allocator, args.InputTypes[aggFn.ColIdx[0]], allocSize) } else { funcAllocs[i], err = newAvgOrderedAggAlloc(args.Allocator, args.InputTypes[aggFn.ColIdx[0]], allocSize) } - case execinfrapb.AggregatorSpec_SUM: + case execinfrapb.Sum: if isHashAgg { funcAllocs[i], err = newSumHashAggAlloc(args.Allocator, args.InputTypes[aggFn.ColIdx[0]], allocSize) } else { funcAllocs[i], err = newSumOrderedAggAlloc(args.Allocator, args.InputTypes[aggFn.ColIdx[0]], allocSize) } - case execinfrapb.AggregatorSpec_SUM_INT: + case execinfrapb.SumInt: if isHashAgg { funcAllocs[i], err = newSumIntHashAggAlloc(args.Allocator, args.InputTypes[aggFn.ColIdx[0]], allocSize) } else { funcAllocs[i], err = newSumIntOrderedAggAlloc(args.Allocator, args.InputTypes[aggFn.ColIdx[0]], allocSize) } - case execinfrapb.AggregatorSpec_CONCAT_AGG: + case execinfrapb.ConcatAgg: if isHashAgg { funcAllocs[i] = newConcatHashAggAlloc(args.Allocator, allocSize) } else { funcAllocs[i] = newConcatOrderedAggAlloc(args.Allocator, allocSize) } - case execinfrapb.AggregatorSpec_COUNT_ROWS: + case execinfrapb.CountRows: if isHashAgg { funcAllocs[i] = newCountRowsHashAggAlloc(args.Allocator, allocSize) } else { funcAllocs[i] = newCountRowsOrderedAggAlloc(args.Allocator, allocSize) } - case execinfrapb.AggregatorSpec_COUNT: + case execinfrapb.Count: if isHashAgg { funcAllocs[i] = newCountHashAggAlloc(args.Allocator, allocSize) } else { funcAllocs[i] = newCountOrderedAggAlloc(args.Allocator, allocSize) } - case execinfrapb.AggregatorSpec_MIN: + case execinfrapb.Min: if isHashAgg { funcAllocs[i] = newMinHashAggAlloc(args.Allocator, args.InputTypes[aggFn.ColIdx[0]], allocSize) } else { funcAllocs[i] = newMinOrderedAggAlloc(args.Allocator, args.InputTypes[aggFn.ColIdx[0]], allocSize) } - case execinfrapb.AggregatorSpec_MAX: + case execinfrapb.Max: if isHashAgg { funcAllocs[i] = newMaxHashAggAlloc(args.Allocator, args.InputTypes[aggFn.ColIdx[0]], allocSize) } else { funcAllocs[i] = newMaxOrderedAggAlloc(args.Allocator, args.InputTypes[aggFn.ColIdx[0]], allocSize) } - case execinfrapb.AggregatorSpec_BOOL_AND: + case execinfrapb.BoolAnd: if isHashAgg { funcAllocs[i] = newBoolAndHashAggAlloc(args.Allocator, allocSize) } else { funcAllocs[i] = newBoolAndOrderedAggAlloc(args.Allocator, allocSize) } - case execinfrapb.AggregatorSpec_BOOL_OR: + case execinfrapb.BoolOr: if isHashAgg { funcAllocs[i] = newBoolOrHashAggAlloc(args.Allocator, allocSize) } else { diff --git a/pkg/sql/colexec/default_agg_test.go b/pkg/sql/colexec/default_agg_test.go index 6719263cefb2..9f15e1464bc1 100644 --- a/pkg/sql/colexec/default_agg_test.go +++ b/pkg/sql/colexec/default_agg_test.go @@ -34,8 +34,8 @@ func TestDefaultAggregateFunc(t *testing.T) { testCases := []aggregatorTestCase{ { aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_STRING_AGG, + execinfrapb.AnyNotNull, + execinfrapb.StringAgg, }, input: colexectestutils.Tuples{ {nil, "a", "1"}, @@ -58,8 +58,8 @@ func TestDefaultAggregateFunc(t *testing.T) { }, { aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_STRING_AGG, + execinfrapb.AnyNotNull, + execinfrapb.StringAgg, }, input: colexectestutils.Tuples{ {nil, "a"}, @@ -83,10 +83,10 @@ func TestDefaultAggregateFunc(t *testing.T) { }, { aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_JSON_AGG, - execinfrapb.AggregatorSpec_JSON_AGG, - execinfrapb.AggregatorSpec_STRING_AGG, + execinfrapb.AnyNotNull, + execinfrapb.JSONAgg, + execinfrapb.JSONAgg, + execinfrapb.StringAgg, }, typs: []*types.T{types.Int, types.Jsonb, types.String}, input: colexectestutils.Tuples{ @@ -109,8 +109,8 @@ func TestDefaultAggregateFunc(t *testing.T) { }, { aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_XOR_AGG, + execinfrapb.AnyNotNull, + execinfrapb.XorAgg, }, input: colexectestutils.Tuples{ {nil, 3}, @@ -166,7 +166,7 @@ func TestDefaultAggregateFunc(t *testing.T) { } func BenchmarkDefaultAggregateFunction(b *testing.B) { - aggFn := execinfrapb.AggregatorSpec_STRING_AGG + aggFn := execinfrapb.StringAgg for _, agg := range aggTypes { for _, numInputRows := range []int{32, 32 * coldata.BatchSize()} { for _, groupSize := range []int{1, 2, 32, 128, coldata.BatchSize()} { diff --git a/pkg/sql/colexec/external_hash_aggregator_test.go b/pkg/sql/colexec/external_hash_aggregator_test.go index 1d3db5c3bb28..c70a200345c3 100644 --- a/pkg/sql/colexec/external_hash_aggregator_test.go +++ b/pkg/sql/colexec/external_hash_aggregator_test.go @@ -166,7 +166,7 @@ func BenchmarkExternalHashAggregator(b *testing.B) { queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(b, false /* inMem */) defer cleanup() - aggFn := execinfrapb.AggregatorSpec_MIN + aggFn := execinfrapb.Min numRows := []int{coldata.BatchSize(), 64 * coldata.BatchSize(), 4096 * coldata.BatchSize()} groupSizes := []int{1, 2, 32, 128, coldata.BatchSize()} if testing.Short() { diff --git a/pkg/sql/colexec/hash_aggregator_test.go b/pkg/sql/colexec/hash_aggregator_test.go index 16919b177e58..e4a37ec13e4f 100644 --- a/pkg/sql/colexec/hash_aggregator_test.go +++ b/pkg/sql/colexec/hash_aggregator_test.go @@ -102,7 +102,7 @@ var hashAggregatorTestCases = []aggregatorTestCase{ typs: []*types.T{types.Int, types.Int, types.Decimal}, convToDecimal: true, - aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_SUM, execinfrapb.AggregatorSpec_SUM_INT}, + aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.Sum, execinfrapb.SumInt}, groupCols: []uint32{0, 1}, aggCols: [][]uint32{ {2}, {1}, @@ -189,7 +189,7 @@ func BenchmarkHashAggregatorInputTuplesTracking(b *testing.B) { queueCfg.CacheMode = colcontainer.DiskQueueCacheModeReuseCache queueCfg.SetDefaultBufferSizeBytesForCacheMode() - aggFn := execinfrapb.AggregatorSpec_MIN + aggFn := execinfrapb.Min numRows := []int{1, 32, coldata.BatchSize(), 32 * coldata.BatchSize(), 1024 * coldata.BatchSize()} groupSizes := []int{1, 2, 32, 128, coldata.BatchSize()} if testing.Short() { diff --git a/pkg/sql/colflow/vectorized_flow_space_test.go b/pkg/sql/colflow/vectorized_flow_space_test.go index f44d97599312..427403957ae7 100644 --- a/pkg/sql/colflow/vectorized_flow_space_test.go +++ b/pkg/sql/colflow/vectorized_flow_space_test.go @@ -164,7 +164,7 @@ func TestVectorizeAllocatorSpaceError(t *testing.T) { Type: execinfrapb.AggregatorSpec_SCALAR, Aggregations: []execinfrapb.AggregatorSpec_Aggregation{ { - Func: execinfrapb.AggregatorSpec_MAX, + Func: execinfrapb.Max, ColIdx: []uint32{0}, }, }, diff --git a/pkg/sql/distsql/columnar_operators_test.go b/pkg/sql/distsql/columnar_operators_test.go index b8ececccc303..b5844153fa6b 100644 --- a/pkg/sql/distsql/columnar_operators_test.go +++ b/pkg/sql/distsql/columnar_operators_test.go @@ -36,51 +36,51 @@ const nullProbability = 0.2 const randTypesProbability = 0.5 var aggregateFuncToNumArguments = map[execinfrapb.AggregatorSpec_Func]int{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL: 1, - execinfrapb.AggregatorSpec_AVG: 1, - execinfrapb.AggregatorSpec_BOOL_AND: 1, - execinfrapb.AggregatorSpec_BOOL_OR: 1, - execinfrapb.AggregatorSpec_CONCAT_AGG: 1, - execinfrapb.AggregatorSpec_COUNT: 1, - execinfrapb.AggregatorSpec_MAX: 1, - execinfrapb.AggregatorSpec_MIN: 1, - execinfrapb.AggregatorSpec_STDDEV: 1, - execinfrapb.AggregatorSpec_SUM: 1, - execinfrapb.AggregatorSpec_SUM_INT: 1, - execinfrapb.AggregatorSpec_VARIANCE: 1, - execinfrapb.AggregatorSpec_XOR_AGG: 1, - execinfrapb.AggregatorSpec_COUNT_ROWS: 0, - execinfrapb.AggregatorSpec_SQRDIFF: 1, - execinfrapb.AggregatorSpec_FINAL_VARIANCE: 3, - execinfrapb.AggregatorSpec_FINAL_STDDEV: 3, - execinfrapb.AggregatorSpec_ARRAY_AGG: 1, - execinfrapb.AggregatorSpec_JSON_AGG: 1, - execinfrapb.AggregatorSpec_JSONB_AGG: 1, - execinfrapb.AggregatorSpec_STRING_AGG: 2, - execinfrapb.AggregatorSpec_BIT_AND: 1, - execinfrapb.AggregatorSpec_BIT_OR: 1, - execinfrapb.AggregatorSpec_CORR: 2, - execinfrapb.AggregatorSpec_PERCENTILE_DISC_IMPL: 2, - execinfrapb.AggregatorSpec_PERCENTILE_CONT_IMPL: 2, - execinfrapb.AggregatorSpec_JSON_OBJECT_AGG: 2, - execinfrapb.AggregatorSpec_JSONB_OBJECT_AGG: 2, - execinfrapb.AggregatorSpec_VAR_POP: 1, - execinfrapb.AggregatorSpec_STDDEV_POP: 1, - execinfrapb.AggregatorSpec_ST_MAKELINE: 1, - execinfrapb.AggregatorSpec_ST_EXTENT: 1, - execinfrapb.AggregatorSpec_ST_UNION: 1, - execinfrapb.AggregatorSpec_ST_COLLECT: 1, - execinfrapb.AggregatorSpec_COVAR_POP: 2, - execinfrapb.AggregatorSpec_COVAR_SAMP: 2, - execinfrapb.AggregatorSpec_REGR_INTERCEPT: 2, - execinfrapb.AggregatorSpec_REGR_R2: 2, - execinfrapb.AggregatorSpec_REGR_SLOPE: 2, - execinfrapb.AggregatorSpec_REGR_SXX: 2, - execinfrapb.AggregatorSpec_REGR_SXY: 2, - execinfrapb.AggregatorSpec_REGR_SYY: 2, - execinfrapb.AggregatorSpec_REGR_COUNT: 2, - execinfrapb.AggregatorSpec_REGR_AVGX: 2, - execinfrapb.AggregatorSpec_REGR_AVGY: 2, + execinfrapb.AnyNotNull: 1, + execinfrapb.Avg: 1, + execinfrapb.BoolAnd: 1, + execinfrapb.BoolOr: 1, + execinfrapb.ConcatAgg: 1, + execinfrapb.Count: 1, + execinfrapb.Max: 1, + execinfrapb.Min: 1, + execinfrapb.Stddev: 1, + execinfrapb.Sum: 1, + execinfrapb.SumInt: 1, + execinfrapb.Variance: 1, + execinfrapb.XorAgg: 1, + execinfrapb.CountRows: 0, + execinfrapb.Sqrdiff: 1, + execinfrapb.FinalVariance: 3, + execinfrapb.FinalStddev: 3, + execinfrapb.ArrayAgg: 1, + execinfrapb.JSONAgg: 1, + execinfrapb.JSONBAgg: 1, + execinfrapb.StringAgg: 2, + execinfrapb.BitAnd: 1, + execinfrapb.BitOr: 1, + execinfrapb.Corr: 2, + execinfrapb.PercentileDiscImpl: 2, + execinfrapb.PercentileContImpl: 2, + execinfrapb.JSONObjectAgg: 2, + execinfrapb.JSONBObjectAgg: 2, + execinfrapb.VarPop: 1, + execinfrapb.StddevPop: 1, + execinfrapb.StMakeline: 1, + execinfrapb.StExtent: 1, + execinfrapb.StUnion: 1, + execinfrapb.StCollect: 1, + execinfrapb.CovarPop: 2, + execinfrapb.CovarSamp: 2, + execinfrapb.RegrIntercept: 2, + execinfrapb.RegrR2: 2, + execinfrapb.RegrSlope: 2, + execinfrapb.RegrSxx: 2, + execinfrapb.RegrSxy: 2, + execinfrapb.RegrSyy: 2, + execinfrapb.RegrCount: 2, + execinfrapb.RegrAvgx: 2, + execinfrapb.RegrAvgy: 2, } // TestAggregateFuncToNumArguments ensures that all aggregate functions are @@ -129,11 +129,11 @@ func TestAggregatorAgainstProcessor(t *testing.T) { continue } switch aggFn { - case execinfrapb.AggregatorSpec_ANY_NOT_NULL: + case execinfrapb.AnyNotNull: // We skip ANY_NOT_NULL aggregate function because it returns // non-deterministic results. - case execinfrapb.AggregatorSpec_PERCENTILE_DISC_IMPL, - execinfrapb.AggregatorSpec_PERCENTILE_CONT_IMPL: + case execinfrapb.PercentileDiscImpl, + execinfrapb.PercentileContImpl: // We skip percentile functions because those can only be // planned as window functions. default: @@ -209,12 +209,12 @@ func TestAggregatorAgainstProcessor(t *testing.T) { // checking, so we hard-code an exception here. invalid := false switch aggFn { - case execinfrapb.AggregatorSpec_CONCAT_AGG, - execinfrapb.AggregatorSpec_STRING_AGG, - execinfrapb.AggregatorSpec_ST_MAKELINE, - execinfrapb.AggregatorSpec_ST_EXTENT, - execinfrapb.AggregatorSpec_ST_UNION, - execinfrapb.AggregatorSpec_ST_COLLECT: + case execinfrapb.ConcatAgg, + execinfrapb.StringAgg, + execinfrapb.StMakeline, + execinfrapb.StExtent, + execinfrapb.StUnion, + execinfrapb.StCollect: for _, typ := range aggFnInputTypes { if typ.Family() == types.TupleFamily { invalid = true diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 40ab3d9137bd..935f52356b3e 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -1768,7 +1768,7 @@ func (dsp *DistSQLPlanner) planAggregators( finalOrderedGroupCols := make([]uint32, 0, len(orderedGroupCols)) for i, groupColIdx := range groupCols { agg := execinfrapb.AggregatorSpec_Aggregation{ - Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, + Func: execinfrapb.AnyNotNull, ColIdx: []uint32{groupColIdx}, } // See if there already is an aggregation like the one diff --git a/pkg/sql/execinfrapb/BUILD.bazel b/pkg/sql/execinfrapb/BUILD.bazel index 6f77e231df38..2d5deaa75516 100644 --- a/pkg/sql/execinfrapb/BUILD.bazel +++ b/pkg/sql/execinfrapb/BUILD.bazel @@ -5,6 +5,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "execinfrapb", srcs = [ + "aggregate_funcs.go", "api.go", "component_stats.go", "data.go", diff --git a/pkg/sql/execinfrapb/aggregate_funcs.go b/pkg/sql/execinfrapb/aggregate_funcs.go new file mode 100644 index 000000000000..0fed665498d6 --- /dev/null +++ b/pkg/sql/execinfrapb/aggregate_funcs.go @@ -0,0 +1,61 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package execinfrapb + +// Prettier aliases for AggregatorSpec_Func values. +const ( + AnyNotNull = AggregatorSpec_ANY_NOT_NULL + Avg = AggregatorSpec_AVG + BoolAnd = AggregatorSpec_BOOL_AND + BoolOr = AggregatorSpec_BOOL_OR + ConcatAgg = AggregatorSpec_CONCAT_AGG + Count = AggregatorSpec_COUNT + Max = AggregatorSpec_MAX + Min = AggregatorSpec_MIN + Stddev = AggregatorSpec_STDDEV + Sum = AggregatorSpec_SUM + SumInt = AggregatorSpec_SUM_INT + Variance = AggregatorSpec_VARIANCE + XorAgg = AggregatorSpec_XOR_AGG + CountRows = AggregatorSpec_COUNT_ROWS + Sqrdiff = AggregatorSpec_SQRDIFF + FinalVariance = AggregatorSpec_FINAL_VARIANCE + FinalStddev = AggregatorSpec_FINAL_STDDEV + ArrayAgg = AggregatorSpec_ARRAY_AGG + JSONAgg = AggregatorSpec_JSON_AGG + // JSONBAgg is an alias for JSONAgg, they do the same thing. + JSONBAgg = AggregatorSpec_JSONB_AGG + StringAgg = AggregatorSpec_STRING_AGG + BitAnd = AggregatorSpec_BIT_AND + BitOr = AggregatorSpec_BIT_OR + Corr = AggregatorSpec_CORR + PercentileDiscImpl = AggregatorSpec_PERCENTILE_DISC_IMPL + PercentileContImpl = AggregatorSpec_PERCENTILE_CONT_IMPL + JSONObjectAgg = AggregatorSpec_JSON_OBJECT_AGG + JSONBObjectAgg = AggregatorSpec_JSONB_OBJECT_AGG + VarPop = AggregatorSpec_VAR_POP + StddevPop = AggregatorSpec_STDDEV_POP + StMakeline = AggregatorSpec_ST_MAKELINE + StExtent = AggregatorSpec_ST_EXTENT + StUnion = AggregatorSpec_ST_UNION + StCollect = AggregatorSpec_ST_COLLECT + CovarPop = AggregatorSpec_COVAR_POP + CovarSamp = AggregatorSpec_COVAR_SAMP + RegrIntercept = AggregatorSpec_REGR_INTERCEPT + RegrR2 = AggregatorSpec_REGR_R2 + RegrSlope = AggregatorSpec_REGR_SLOPE + RegrSxx = AggregatorSpec_REGR_SXX + RegrSyy = AggregatorSpec_REGR_SYY + RegrSxy = AggregatorSpec_REGR_SXY + RegrCount = AggregatorSpec_REGR_COUNT + RegrAvgx = AggregatorSpec_REGR_AVGX + RegrAvgy = AggregatorSpec_REGR_AVGY +) diff --git a/pkg/sql/execinfrapb/processors.go b/pkg/sql/execinfrapb/processors.go index 3097270d741c..75c30852e943 100644 --- a/pkg/sql/execinfrapb/processors.go +++ b/pkg/sql/execinfrapb/processors.go @@ -45,7 +45,7 @@ type AggregateConstructor func(*tree.EvalContext, tree.Datums) tree.AggregateFun func GetAggregateInfo( fn AggregatorSpec_Func, inputTypes ...*types.T, ) (aggregateConstructor AggregateConstructor, returnType *types.T, err error) { - if fn == AggregatorSpec_ANY_NOT_NULL { + if fn == AnyNotNull { // The ANY_NOT_NULL builtin does not have a fixed return type; // handle it separately. if len(inputTypes) != 1 { @@ -165,7 +165,7 @@ func (spec *AggregatorSpec) IsScalar() bool { func (spec *AggregatorSpec) IsRowCount() bool { return len(spec.Aggregations) == 1 && spec.Aggregations[0].FilterColIdx == nil && - spec.Aggregations[0].Func == AggregatorSpec_COUNT_ROWS && + spec.Aggregations[0].Func == CountRows && !spec.Aggregations[0].Distinct && spec.IsScalar() } @@ -186,7 +186,7 @@ func GetWindowFuncIdx(funcName string) (int32, error) { func GetWindowFunctionInfo( fn WindowerSpec_Func, inputTypes ...*types.T, ) (windowConstructor func(*tree.EvalContext) tree.WindowFunc, returnType *types.T, err error) { - if fn.AggregateFunc != nil && *fn.AggregateFunc == AggregatorSpec_ANY_NOT_NULL { + if fn.AggregateFunc != nil && *fn.AggregateFunc == AnyNotNull { // The ANY_NOT_NULL builtin does not have a fixed return type; // handle it separately. if len(inputTypes) != 1 { diff --git a/pkg/sql/physicalplan/aggregator_funcs.go b/pkg/sql/physicalplan/aggregator_funcs.go index f4e31c579b18..9ba677704bf6 100644 --- a/pkg/sql/physicalplan/aggregator_funcs.go +++ b/pkg/sql/physicalplan/aggregator_funcs.go @@ -85,11 +85,11 @@ var passThroughLocalIdxs = []uint32{0} // DistAggregationTable is DistAggregationInfo look-up table. Functions that // don't have an entry in the table are not optimized with a local stage. var DistAggregationTable = map[execinfrapb.AggregatorSpec_Func]DistAggregationInfo{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL: { - LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_ANY_NOT_NULL}, + execinfrapb.AnyNotNull: { + LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AnyNotNull}, FinalStage: []FinalStageInfo{ { - Fn: execinfrapb.AggregatorSpec_ANY_NOT_NULL, + Fn: execinfrapb.AnyNotNull, LocalIdxs: passThroughLocalIdxs, }, }, @@ -102,18 +102,18 @@ var DistAggregationTable = map[execinfrapb.AggregatorSpec_Func]DistAggregationIn // - a final rendering then divides the two results. // // At a high level, this is analogous to rewriting AVG(x) as SUM(x)/COUNT(x). - execinfrapb.AggregatorSpec_AVG: { + execinfrapb.Avg: { LocalStage: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_SUM, - execinfrapb.AggregatorSpec_COUNT, + execinfrapb.Sum, + execinfrapb.Count, }, FinalStage: []FinalStageInfo{ { - Fn: execinfrapb.AggregatorSpec_SUM, + Fn: execinfrapb.Sum, LocalIdxs: []uint32{0}, }, { - Fn: execinfrapb.AggregatorSpec_SUM_INT, + Fn: execinfrapb.SumInt, LocalIdxs: []uint32{1}, }, }, @@ -145,85 +145,85 @@ var DistAggregationTable = map[execinfrapb.AggregatorSpec_Func]DistAggregationIn }, }, - execinfrapb.AggregatorSpec_BOOL_AND: { - LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_BOOL_AND}, + execinfrapb.BoolAnd: { + LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.BoolAnd}, FinalStage: []FinalStageInfo{ { - Fn: execinfrapb.AggregatorSpec_BOOL_AND, + Fn: execinfrapb.BoolAnd, LocalIdxs: passThroughLocalIdxs, }, }, }, - execinfrapb.AggregatorSpec_BOOL_OR: { - LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_BOOL_OR}, + execinfrapb.BoolOr: { + LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.BoolOr}, FinalStage: []FinalStageInfo{ { - Fn: execinfrapb.AggregatorSpec_BOOL_OR, + Fn: execinfrapb.BoolOr, LocalIdxs: passThroughLocalIdxs, }, }, }, - execinfrapb.AggregatorSpec_COUNT: { - LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_COUNT}, + execinfrapb.Count: { + LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.Count}, FinalStage: []FinalStageInfo{ { - Fn: execinfrapb.AggregatorSpec_SUM_INT, + Fn: execinfrapb.SumInt, LocalIdxs: passThroughLocalIdxs, }, }, }, - execinfrapb.AggregatorSpec_MAX: { - LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_MAX}, + execinfrapb.Max: { + LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.Max}, FinalStage: []FinalStageInfo{ { - Fn: execinfrapb.AggregatorSpec_MAX, + Fn: execinfrapb.Max, LocalIdxs: passThroughLocalIdxs, }, }, }, - execinfrapb.AggregatorSpec_MIN: { - LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_MIN}, + execinfrapb.Min: { + LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.Min}, FinalStage: []FinalStageInfo{ { - Fn: execinfrapb.AggregatorSpec_MIN, + Fn: execinfrapb.Min, LocalIdxs: passThroughLocalIdxs, }, }, }, - execinfrapb.AggregatorSpec_STDDEV: { + execinfrapb.Stddev: { LocalStage: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_SQRDIFF, - execinfrapb.AggregatorSpec_SUM, - execinfrapb.AggregatorSpec_COUNT, + execinfrapb.Sqrdiff, + execinfrapb.Sum, + execinfrapb.Count, }, FinalStage: []FinalStageInfo{ { - Fn: execinfrapb.AggregatorSpec_FINAL_STDDEV, + Fn: execinfrapb.FinalStddev, LocalIdxs: []uint32{0, 1, 2}, }, }, }, - execinfrapb.AggregatorSpec_SUM: { - LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_SUM}, + execinfrapb.Sum: { + LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.Sum}, FinalStage: []FinalStageInfo{ { - Fn: execinfrapb.AggregatorSpec_SUM, + Fn: execinfrapb.Sum, LocalIdxs: passThroughLocalIdxs, }, }, }, - execinfrapb.AggregatorSpec_SUM_INT: { - LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_SUM_INT}, + execinfrapb.SumInt: { + LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.SumInt}, FinalStage: []FinalStageInfo{ { - Fn: execinfrapb.AggregatorSpec_SUM_INT, + Fn: execinfrapb.SumInt, LocalIdxs: passThroughLocalIdxs, }, }, @@ -237,11 +237,11 @@ var DistAggregationTable = map[execinfrapb.AggregatorSpec_Func]DistAggregationIn // // At a high level, this is analogous to rewriting VARIANCE(x) as // SQRDIFF(x)/(COUNT(x) - 1) (and STDDEV(x) as sqrt(VARIANCE(x))). - execinfrapb.AggregatorSpec_VARIANCE: { + execinfrapb.Variance: { LocalStage: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_SQRDIFF, - execinfrapb.AggregatorSpec_SUM, - execinfrapb.AggregatorSpec_COUNT, + execinfrapb.Sqrdiff, + execinfrapb.Sum, + execinfrapb.Count, }, // Instead of have a SUM_SQRDIFFS and SUM_INT (for COUNT) stage // for VARIANCE (and STDDEV) then tailoring a FinalRendering @@ -255,47 +255,47 @@ var DistAggregationTable = map[execinfrapb.AggregatorSpec_Func]DistAggregationIn // have one or the other FinalStage: []FinalStageInfo{ { - Fn: execinfrapb.AggregatorSpec_FINAL_VARIANCE, + Fn: execinfrapb.FinalVariance, LocalIdxs: []uint32{0, 1, 2}, }, }, }, - execinfrapb.AggregatorSpec_XOR_AGG: { - LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_XOR_AGG}, + execinfrapb.XorAgg: { + LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.XorAgg}, FinalStage: []FinalStageInfo{ { - Fn: execinfrapb.AggregatorSpec_XOR_AGG, + Fn: execinfrapb.XorAgg, LocalIdxs: passThroughLocalIdxs, }, }, }, - execinfrapb.AggregatorSpec_COUNT_ROWS: { - LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_COUNT_ROWS}, + execinfrapb.CountRows: { + LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.CountRows}, FinalStage: []FinalStageInfo{ { - Fn: execinfrapb.AggregatorSpec_SUM_INT, + Fn: execinfrapb.SumInt, LocalIdxs: passThroughLocalIdxs, }, }, }, - execinfrapb.AggregatorSpec_BIT_AND: { - LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_BIT_AND}, + execinfrapb.BitAnd: { + LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.BitAnd}, FinalStage: []FinalStageInfo{ { - Fn: execinfrapb.AggregatorSpec_BIT_AND, + Fn: execinfrapb.BitAnd, LocalIdxs: passThroughLocalIdxs, }, }, }, - execinfrapb.AggregatorSpec_BIT_OR: { - LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_BIT_OR}, + execinfrapb.BitOr: { + LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.BitOr}, FinalStage: []FinalStageInfo{ { - Fn: execinfrapb.AggregatorSpec_BIT_OR, + Fn: execinfrapb.BitOr, LocalIdxs: passThroughLocalIdxs, }, }, diff --git a/pkg/sql/physicalplan/aggregator_funcs_test.go b/pkg/sql/physicalplan/aggregator_funcs_test.go index 1f64719d4732..3cec613eb343 100644 --- a/pkg/sql/physicalplan/aggregator_funcs_test.go +++ b/pkg/sql/physicalplan/aggregator_funcs_test.go @@ -451,12 +451,12 @@ func TestDistAggregationTable(t *testing.T) { desc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t") for fn, info := range DistAggregationTable { - if fn == execinfrapb.AggregatorSpec_ANY_NOT_NULL { + if fn == execinfrapb.AnyNotNull { // ANY_NOT_NULL only has a definite result if all rows have the same value // on the relevant column; skip testing this trivial case. continue } - if fn == execinfrapb.AggregatorSpec_COUNT_ROWS { + if fn == execinfrapb.CountRows { // COUNT_ROWS takes no arguments; skip it in this test. continue } @@ -472,7 +472,7 @@ func TestDistAggregationTable(t *testing.T) { if err != nil { continue } - if fn == execinfrapb.AggregatorSpec_SUM_INT && col.Ordinal() == 2 { + if fn == execinfrapb.SumInt && col.Ordinal() == 2 { // When using sum_int over int2 column we're likely to hit an // integer out of range error since we insert random DInts into // that column, so we'll skip such config. diff --git a/pkg/sql/rowexec/aggregator_test.go b/pkg/sql/rowexec/aggregator_test.go index 84c9513448fc..908bbbe4ecb1 100644 --- a/pkg/sql/rowexec/aggregator_test.go +++ b/pkg/sql/rowexec/aggregator_test.go @@ -401,16 +401,16 @@ func BenchmarkAggregation(b *testing.B) { const numRows = 1000 aggFuncs := []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_AVG, - execinfrapb.AggregatorSpec_COUNT, - execinfrapb.AggregatorSpec_MAX, - execinfrapb.AggregatorSpec_MIN, - execinfrapb.AggregatorSpec_STDDEV, - execinfrapb.AggregatorSpec_SUM, - execinfrapb.AggregatorSpec_SUM_INT, - execinfrapb.AggregatorSpec_VARIANCE, - execinfrapb.AggregatorSpec_XOR_AGG, + execinfrapb.AnyNotNull, + execinfrapb.Avg, + execinfrapb.Count, + execinfrapb.Max, + execinfrapb.Min, + execinfrapb.Stddev, + execinfrapb.Sum, + execinfrapb.SumInt, + execinfrapb.Variance, + execinfrapb.XorAgg, } ctx := context.Background() @@ -457,7 +457,7 @@ func BenchmarkCountRows(b *testing.B) { spec := &execinfrapb.AggregatorSpec{ Aggregations: []execinfrapb.AggregatorSpec_Aggregation{ { - Func: execinfrapb.AggregatorSpec_COUNT_ROWS, + Func: execinfrapb.CountRows, }, }, } @@ -531,16 +531,16 @@ func benchmarkAggregationWithGrouping(b *testing.B, numOrderedCols int) { var allOrderedGroupCols = [2]uint32{0, 1} aggFuncs := []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AggregatorSpec_ANY_NOT_NULL, - execinfrapb.AggregatorSpec_AVG, - execinfrapb.AggregatorSpec_COUNT, - execinfrapb.AggregatorSpec_MAX, - execinfrapb.AggregatorSpec_MIN, - execinfrapb.AggregatorSpec_STDDEV, - execinfrapb.AggregatorSpec_SUM, - execinfrapb.AggregatorSpec_SUM_INT, - execinfrapb.AggregatorSpec_VARIANCE, - execinfrapb.AggregatorSpec_XOR_AGG, + execinfrapb.AnyNotNull, + execinfrapb.Avg, + execinfrapb.Count, + execinfrapb.Max, + execinfrapb.Min, + execinfrapb.Stddev, + execinfrapb.Sum, + execinfrapb.SumInt, + execinfrapb.Variance, + execinfrapb.XorAgg, } ctx := context.Background() diff --git a/pkg/sql/rowexec/processors_test.go b/pkg/sql/rowexec/processors_test.go index 4e0e4ca5aec2..dddc56683863 100644 --- a/pkg/sql/rowexec/processors_test.go +++ b/pkg/sql/rowexec/processors_test.go @@ -260,69 +260,69 @@ func TestAggregatorSpecAggregationEquals(t *testing.T) { }{ // Func tests. { - a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL}, - b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL}, + a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull}, + b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull}, expected: true, }, { - a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL}, - b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_AVG}, + a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull}, + b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.Avg}, expected: false, }, // ColIdx tests. { - a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, ColIdx: []uint32{1, 2}}, - b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, ColIdx: []uint32{1, 2}}, + a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, ColIdx: []uint32{1, 2}}, + b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, ColIdx: []uint32{1, 2}}, expected: true, }, { - a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, ColIdx: []uint32{1}}, - b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, ColIdx: []uint32{1, 3}}, + a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, ColIdx: []uint32{1}}, + b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, ColIdx: []uint32{1, 3}}, expected: false, }, { - a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, ColIdx: []uint32{1, 2}}, - b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, ColIdx: []uint32{1, 3}}, + a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, ColIdx: []uint32{1, 2}}, + b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, ColIdx: []uint32{1, 3}}, expected: false, }, // FilterColIdx tests. { - a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, FilterColIdx: &colIdx1}, - b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, FilterColIdx: &colIdx1}, + a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, FilterColIdx: &colIdx1}, + b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, FilterColIdx: &colIdx1}, expected: true, }, { - a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, FilterColIdx: &colIdx1}, - b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL}, + a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, FilterColIdx: &colIdx1}, + b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull}, expected: false, }, { - a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, FilterColIdx: &colIdx1}, - b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, FilterColIdx: &colIdx2}, + a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, FilterColIdx: &colIdx1}, + b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, FilterColIdx: &colIdx2}, expected: false, }, // Distinct tests. { - a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, Distinct: true}, - b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, Distinct: true}, + a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, Distinct: true}, + b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, Distinct: true}, expected: true, }, { - a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, Distinct: false}, - b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, Distinct: false}, + a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, Distinct: false}, + b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, Distinct: false}, expected: true, }, { - a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, Distinct: false}, - b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL}, + a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, Distinct: false}, + b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull}, expected: true, }, { - a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL, Distinct: true}, - b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AggregatorSpec_ANY_NOT_NULL}, + a: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull, Distinct: true}, + b: execinfrapb.AggregatorSpec_Aggregation{Func: execinfrapb.AnyNotNull}, expected: false, }, } { diff --git a/pkg/sql/rowexec/windower_test.go b/pkg/sql/rowexec/windower_test.go index 74ef2f3b9535..6dc4fa81c674 100644 --- a/pkg/sql/rowexec/windower_test.go +++ b/pkg/sql/rowexec/windower_test.go @@ -67,7 +67,7 @@ func TestWindowerAccountingForResults(t *testing.T) { post := &execinfrapb.PostProcessSpec{} input := execinfra.NewRepeatableRowSource(rowenc.OneIntCol, rowenc.MakeIntRows(1000, 1)) - aggSpec := execinfrapb.AggregatorSpec_ARRAY_AGG + aggSpec := execinfrapb.ArrayAgg spec := execinfrapb.WindowerSpec{ PartitionBy: []uint32{}, WindowFns: []execinfrapb.WindowerSpec_WindowFn{{ From d511fb48d01156b268bcb106715d0ab50bba7f2b Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Thu, 8 Apr 2021 22:12:26 -0700 Subject: [PATCH 4/6] colexec: clean up aggregator test cases This commit is only a test change. It cleans up the aggregator test cases in the following ways: - removing some of the defaults in favor of explicit setting (easier to read each test case in isolation) - reordering the fields to have uniform assignment order - inserting any_not_null aggregates for the cases when the input is ordered (this will be needed by the follow up commit that will enforce a particular order on the output). This change simulates how specs are created in the production. - removing a couple of impossible in production test cases (when some columns are unused). Release note: None --- pkg/sql/colexec/aggregators_test.go | 517 +++++++++++++----------- pkg/sql/colexec/default_agg_test.go | 71 ++-- pkg/sql/colexec/hash_aggregator_test.go | 95 ++--- 3 files changed, 347 insertions(+), 336 deletions(-) diff --git a/pkg/sql/colexec/aggregators_test.go b/pkg/sql/colexec/aggregators_test.go index 7232a405394b..037af0f19eb2 100644 --- a/pkg/sql/colexec/aggregators_test.go +++ b/pkg/sql/colexec/aggregators_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils/skip" @@ -36,29 +37,21 @@ import ( "github.com/stretchr/testify/require" ) -var ( - defaultGroupCols = []uint32{0} - defaultAggCols = [][]uint32{{1}} - defaultAggFns = []execinfrapb.AggregatorSpec_Func{execinfrapb.SumInt} - defaultTyps = []*types.T{types.Int, types.Int} -) - type aggregatorTestCase struct { - // typs, aggFns, groupCols, and aggCols will be set to their default - // values before running a test if nil. - typs []*types.T - aggFns []execinfrapb.AggregatorSpec_Func - groupCols []uint32 - aggCols [][]uint32 + name string + typs []*types.T + input colexectestutils.Tuples + groupCols []uint32 + aggCols [][]uint32 + aggFns []execinfrapb.AggregatorSpec_Func + expected colexectestutils.Tuples + constArguments [][]execinfrapb.Expression // spec will be populated during init(). spec *execinfrapb.AggregatorSpec aggDistinct []bool aggFilter []int - input colexectestutils.Tuples unorderedInput bool - expected colexectestutils.Tuples - name string // convToDecimal will convert any float64s to apd.Decimals. If a string is // encountered, a best effort is made to convert that string to an @@ -115,18 +108,6 @@ func (tc *aggregatorTestCase) init() error { } } } - if tc.groupCols == nil { - tc.groupCols = defaultGroupCols - } - if tc.aggFns == nil { - tc.aggFns = defaultAggFns - } - if tc.aggCols == nil { - tc.aggCols = defaultAggCols - } - if tc.typs == nil { - tc.typs = defaultTyps - } aggregations := make([]execinfrapb.AggregatorSpec_Aggregation, len(tc.aggFns)) for i, aggFn := range tc.aggFns { aggregations[i].Func = aggFn @@ -151,25 +132,41 @@ func (tc *aggregatorTestCase) init() error { var aggregatorsTestCases = []aggregatorTestCase{ { + name: "OneTuple", + typs: rowenc.TwoIntCols, input: colexectestutils.Tuples{ {0, 1}, }, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.SumInt, + }, expected: colexectestutils.Tuples{ - {1}, + {0, 1}, }, - name: "OneTuple", }, { + name: "OneGroup", + typs: rowenc.TwoIntCols, input: colexectestutils.Tuples{ {0, 1}, {0, 1}, }, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.SumInt, + }, expected: colexectestutils.Tuples{ - {2}, + {0, 2}, }, - name: "OneGroup", }, { + name: "MultiGroup", + typs: rowenc.TwoIntCols, input: colexectestutils.Tuples{ {0, 1}, {0, 0}, @@ -177,14 +174,21 @@ var aggregatorsTestCases = []aggregatorTestCase{ {1, 4}, {2, 5}, }, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.SumInt, + }, expected: colexectestutils.Tuples{ - {2}, - {4}, - {5}, + {0, 2}, + {1, 4}, + {2, 5}, }, - name: "MultiGroup", }, { + name: "CarryBetweenInputBatches", + typs: rowenc.TwoIntCols, input: colexectestutils.Tuples{ {0, 1}, {0, 2}, @@ -192,13 +196,20 @@ var aggregatorsTestCases = []aggregatorTestCase{ {1, 4}, {1, 5}, }, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.SumInt, + }, expected: colexectestutils.Tuples{ - {6}, - {9}, + {0, 6}, + {1, 9}, }, - name: "CarryBetweenInputBatches", }, { + name: "CarryBetweenOutputBatches", + typs: rowenc.TwoIntCols, input: colexectestutils.Tuples{ {0, 1}, {0, 2}, @@ -207,14 +218,21 @@ var aggregatorsTestCases = []aggregatorTestCase{ {1, 5}, {2, 6}, }, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.SumInt, + }, expected: colexectestutils.Tuples{ - {10}, - {5}, - {6}, + {0, 10}, + {1, 5}, + {2, 6}, }, - name: "CarryBetweenOutputBatches", }, { + name: "CarryBetweenInputAndOutputBatches", + typs: rowenc.TwoIntCols, input: colexectestutils.Tuples{ {0, 1}, {0, 1}, @@ -228,67 +246,65 @@ var aggregatorsTestCases = []aggregatorTestCase{ {6, 7}, {7, 8}, }, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.SumInt, + }, expected: colexectestutils.Tuples{ - {2}, - {2}, - {6}, - {8}, - {5}, - {6}, - {7}, - {8}, + {0, 2}, + {1, 2}, + {2, 6}, + {3, 8}, + {4, 5}, + {5, 6}, + {6, 7}, + {7, 8}, }, - name: "CarryBetweenInputAndOutputBatches", }, { + name: "NoGroupingCols", + typs: rowenc.TwoIntCols, input: colexectestutils.Tuples{ {0, 1}, {0, 2}, {0, 3}, {0, 4}, }, - expected: colexectestutils.Tuples{ - {10}, - }, - name: "NoGroupingCols", groupCols: []uint32{}, - }, - { - input: colexectestutils.Tuples{ - {1, 0, 0}, - {2, 0, 0}, - {3, 0, 0}, - {4, 0, 0}, + aggCols: [][]uint32{{0}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.SumInt, }, expected: colexectestutils.Tuples{ - {10}, + {0, 10}, }, - name: "UnusedInputColumns", - typs: []*types.T{types.Int, types.Int, types.Int}, - groupCols: []uint32{1, 2}, - aggCols: [][]uint32{{0}}, }, { + name: "UnorderedWithNullsInGroupingCol", + typs: rowenc.TwoIntCols, input: colexectestutils.Tuples{ {nil, 1}, {4, 42}, {nil, 2}, }, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.SumInt, + }, expected: colexectestutils.Tuples{ - {3}, - {42}, + {nil, 3}, + {4, 42}, }, - name: "UnorderedWithNullsInGroupingCol", unorderedInput: true, }, { - aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AnyNotNull, - execinfrapb.CountRows, - }, - aggCols: [][]uint32{{0}, {}}, - typs: []*types.T{types.Int}, - unorderedInput: true, + name: "CountRows", + typs: rowenc.OneIntCol, input: colexectestutils.Tuples{ {1}, {2}, @@ -304,6 +320,12 @@ var aggregatorsTestCases = []aggregatorTestCase{ {4}, {2}, }, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.CountRows, + }, expected: colexectestutils.Tuples{ {nil, 2}, {1, 4}, @@ -311,46 +333,51 @@ var aggregatorsTestCases = []aggregatorTestCase{ {3, 2}, {4, 2}, }, + unorderedInput: true, }, { - aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.SumInt, execinfrapb.SumInt}, - aggCols: [][]uint32{ - {2}, {1}, - }, + name: "OutputOrder", + typs: rowenc.ThreeIntCols, input: colexectestutils.Tuples{ {0, 1, 2}, {0, 1, 2}, }, - typs: []*types.T{types.Int, types.Int, types.Int}, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {2}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.SumInt, + execinfrapb.SumInt, + }, expected: colexectestutils.Tuples{ - {4, 2}, + {0, 4, 2}, }, - name: "OutputOrder", }, { - aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.Sum, execinfrapb.SumInt}, - aggCols: [][]uint32{ - {2}, {1}, - }, + name: "SumMultiType", + typs: []*types.T{types.Int, types.Int, types.Decimal}, input: colexectestutils.Tuples{ {0, 1, 1.3}, {0, 1, 1.6}, {0, 1, 0.5}, {1, 1, 1.2}, }, - typs: []*types.T{types.Int, types.Int, types.Decimal}, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {2}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.Sum, + execinfrapb.SumInt, + }, expected: colexectestutils.Tuples{ - {3.4, 3}, - {1.2, 1}, + {0, 3.4, 3}, + {1, 1.2, 1}, }, - name: "SumMultiType", convToDecimal: true, }, { - aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.Avg, execinfrapb.Sum}, - aggCols: [][]uint32{ - {1}, {1}, - }, + name: "AvgSumSingleInputBatch", + typs: []*types.T{types.Int, types.Decimal}, input: colexectestutils.Tuples{ {0, 1.1}, {0, 1.2}, @@ -358,22 +385,22 @@ var aggregatorsTestCases = []aggregatorTestCase{ {1, 6.21}, {1, 2.43}, }, - typs: []*types.T{types.Int, types.Decimal}, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.Avg, + execinfrapb.Sum, + }, expected: colexectestutils.Tuples{ - {"1.5333333333333333333", 4.6}, - {4.32, 8.64}, + {0, "1.5333333333333333333", 4.6}, + {1, 4.32, 8.64}, }, - name: "AvgSumSingleInputBatch", convToDecimal: true, }, { - aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.BoolAnd, - execinfrapb.BoolOr, - }, - aggCols: [][]uint32{ - {1}, {1}, - }, + name: "BoolAndOrBatch", + typs: []*types.T{types.Int, types.Bool}, input: colexectestutils.Tuples{ {0, true}, {1, false}, @@ -393,21 +420,35 @@ var aggregatorsTestCases = []aggregatorTestCase{ {8, nil}, {8, nil}, }, - typs: []*types.T{types.Int, types.Bool}, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.BoolAnd, + execinfrapb.BoolOr, + }, expected: colexectestutils.Tuples{ - {true, true}, - {false, false}, - {false, true}, - {true, true}, - {false, false}, - {false, false}, - {true, true}, - {false, true}, - {nil, nil}, + {0, true, true}, + {1, false, false}, + {2, false, true}, + {3, true, true}, + {4, false, false}, + {5, false, false}, + {6, true, true}, + {7, false, true}, + {8, nil, nil}, }, - name: "BoolAndOrBatch", }, { + name: "MultiGroupColsWithPointerTypes", + typs: []*types.T{types.Int, types.Decimal, types.Bytes, types.Decimal}, + input: colexectestutils.Tuples{ + {2, 1.0, "1.0", 2.0}, + {2, 1.0, "1.0", 4.0}, + {2, 2.0, "2.0", 6.0}, + }, + groupCols: []uint32{0, 1, 2}, + aggCols: [][]uint32{{0}, {1}, {2}, {3}, {3}}, aggFns: []execinfrapb.AggregatorSpec_Func{ execinfrapb.AnyNotNull, execinfrapb.AnyNotNull, @@ -415,27 +456,14 @@ var aggregatorsTestCases = []aggregatorTestCase{ execinfrapb.Min, execinfrapb.Sum, }, - input: colexectestutils.Tuples{ - {2, 1.0, "1.0", 2.0}, - {2, 1.0, "1.0", 4.0}, - {2, 2.0, "2.0", 6.0}, - }, expected: colexectestutils.Tuples{ {2, 1.0, "1.0", 2.0, 6.0}, {2, 2.0, "2.0", 6.0, 6.0}, }, - typs: []*types.T{types.Int, types.Decimal, types.Bytes, types.Decimal}, - name: "MultiGroupColsWithPointerTypes", - groupCols: []uint32{0, 1, 2}, - aggCols: [][]uint32{ - {0}, {1}, {2}, {3}, {3}, - }, }, { - aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AnyNotNull, - execinfrapb.SumInt, - }, + name: "GroupOnTimeTZColumns", + typs: []*types.T{types.TimeTZ, types.Int}, input: colexectestutils.Tuples{ {tree.NewDTimeTZFromOffset(timeofday.FromInt(0), 0), -1}, {tree.NewDTimeTZFromOffset(timeofday.FromInt(0), 1), 1}, @@ -446,6 +474,12 @@ var aggregatorsTestCases = []aggregatorTestCase{ {tree.NewDTimeTZFromOffset(timeofday.FromInt(0), 3), 101}, {tree.NewDTimeTZFromOffset(timeofday.FromInt(0), 4), 102}, }, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.SumInt, + }, expected: colexectestutils.Tuples{ {tree.NewDTimeTZFromOffset(timeofday.FromInt(0), 0), -1}, {tree.NewDTimeTZFromOffset(timeofday.FromInt(0), 1), 3}, @@ -453,14 +487,10 @@ var aggregatorsTestCases = []aggregatorTestCase{ {tree.NewDTimeTZFromOffset(timeofday.FromInt(0), 3), 201}, {tree.NewDTimeTZFromOffset(timeofday.FromInt(0), 4), 102}, }, - typs: []*types.T{types.TimeTZ, types.Int}, - name: "GroupOnTimeTZColumns", - groupCols: []uint32{0}, - aggCols: [][]uint32{ - {0}, {1}, - }, }, { + name: "AVG on all types", + typs: []*types.T{types.Int, types.Int2, types.Int4, types.Int, types.Decimal, types.Float, types.Interval}, input: colexectestutils.Tuples{ {0, nil, 1, 1, 1.0, 1.0, duration.MakeDuration(1, 1, 1)}, {0, 1, nil, 2, 2.0, 2.0, duration.MakeDuration(2, 2, 2)}, @@ -469,15 +499,24 @@ var aggregatorsTestCases = []aggregatorTestCase{ {0, 4, 4, 4, 4.0, nil, duration.MakeDuration(5, 5, 5)}, {0, 5, 5, 5, 5.0, 5.0, nil}, }, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1}, {2}, {3}, {4}, {5}, {6}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.Avg, + execinfrapb.Avg, + execinfrapb.Avg, + execinfrapb.Avg, + execinfrapb.Avg, + execinfrapb.Avg, + }, expected: colexectestutils.Tuples{ - {3.0, 3.0, 3.0, 3.0, 3.0, duration.MakeDuration(3, 3, 3)}, + {0, 3.0, 3.0, 3.0, 3.0, 3.0, duration.MakeDuration(3, 3, 3)}, }, - typs: []*types.T{types.Int, types.Int2, types.Int4, types.Int, types.Decimal, types.Float, types.Interval}, - aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.Avg, execinfrapb.Avg, execinfrapb.Avg, execinfrapb.Avg, execinfrapb.Avg, execinfrapb.Avg}, - aggCols: [][]uint32{{1}, {2}, {3}, {4}, {5}, {6}}, - name: "AVG on all types", }, { + name: "ConcatAgg", + typs: []*types.T{types.Int, types.Bytes}, input: colexectestutils.Tuples{ {1, "1"}, {1, "2"}, @@ -491,34 +530,22 @@ var aggregatorsTestCases = []aggregatorTestCase{ {4, nil}, {4, nil}, }, - expected: colexectestutils.Tuples{ - {"123"}, - {"12"}, - {"12"}, - {nil}, - }, - typs: []*types.T{types.Int, types.Bytes}, - aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.ConcatAgg}, groupCols: []uint32{0}, - aggCols: [][]uint32{{1}}, - }, - { + aggCols: [][]uint32{{0}, {1}}, aggFns: []execinfrapb.AggregatorSpec_Func{ execinfrapb.AnyNotNull, - execinfrapb.AnyNotNull, - execinfrapb.Avg, - execinfrapb.CountRows, - execinfrapb.Count, - execinfrapb.Sum, - execinfrapb.SumInt, - execinfrapb.Min, - execinfrapb.Max, - execinfrapb.BoolAnd, - execinfrapb.BoolOr, execinfrapb.ConcatAgg, }, - aggCols: [][]uint32{{0}, {4}, {1}, {}, {1}, {1}, {2}, {2}, {2}, {3}, {3}, {4}}, - typs: []*types.T{types.Int, types.Decimal, types.Int, types.Bool, types.Bytes}, + expected: colexectestutils.Tuples{ + {1, "123"}, + {2, "12"}, + {3, "12"}, + {4, nil}, + }, + }, + { + name: "All", + typs: []*types.T{types.Int, types.Decimal, types.Int, types.Bool, types.Bytes}, input: colexectestutils.Tuples{ {0, 3.1, 2, true, "zero"}, {0, 1.1, 3, false, "zero"}, @@ -528,33 +555,33 @@ var aggregatorsTestCases = []aggregatorTestCase{ {3, 4.1, 0, false, "three"}, {3, 5.1, 0, true, "three"}, }, - expected: colexectestutils.Tuples{ - {0, "zero", 2.1, 2, 2, 4.2, 5, 2, 3, false, true, "zerozero"}, - {1, "one", 2.6, 2, 2, 5.2, 1, 0, 1, false, false, "oneone"}, - {2, "two", 1.1, 1, 1, 1.1, 1, 1, 1, true, true, "two"}, - {3, "three", 4.6, 2, 2, 9.2, 0, 0, 0, false, true, "threethree"}, - }, - convToDecimal: true, - }, - - // Test case for null handling. - { + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {}, {1}, {1}, {1}, {2}, {2}, {2}, {3}, {3}, {4}, {4}}, aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AnyNotNull, execinfrapb.AnyNotNull, execinfrapb.CountRows, + execinfrapb.Avg, execinfrapb.Count, execinfrapb.Sum, execinfrapb.SumInt, execinfrapb.Min, execinfrapb.Max, - execinfrapb.Avg, execinfrapb.BoolAnd, execinfrapb.BoolOr, + execinfrapb.AnyNotNull, execinfrapb.ConcatAgg, }, - aggCols: [][]uint32{{0}, {1}, {}, {1}, {1}, {2}, {2}, {2}, {1}, {3}, {3}, {4}}, - typs: []*types.T{types.Int, types.Decimal, types.Int, types.Bool, types.Bytes}, + expected: colexectestutils.Tuples{ + {0, 2, 2.1, 2, 4.2, 5, 2, 3, false, true, "zero", "zerozero"}, + {1, 2, 2.6, 2, 5.2, 1, 0, 1, false, false, "one", "oneone"}, + {2, 1, 1.1, 1, 1.1, 1, 1, 1, true, true, "two", "two"}, + {3, 2, 4.6, 2, 9.2, 0, 0, 0, false, true, "three", "threethree"}, + }, + convToDecimal: true, + }, + { + name: "NullHandling", + typs: []*types.T{types.Int, types.Decimal, types.Int, types.Bool, types.Bytes}, input: colexectestutils.Tuples{ {nil, 1.1, 4, true, "a"}, {0, nil, nil, nil, nil}, @@ -562,26 +589,32 @@ var aggregatorsTestCases = []aggregatorTestCase{ {1, nil, nil, nil, nil}, {1, nil, nil, false, nil}, }, - expected: colexectestutils.Tuples{ - {nil, 1.1, 1, 1, 1.1, 4, 4, 4, 1.1, true, true, "a"}, - {0, 3.1, 2, 1, 3.1, 5, 5, 5, 3.1, nil, nil, "b"}, - {1, nil, 2, 0, nil, nil, nil, nil, nil, false, false, nil}, - }, - convToDecimal: true, - }, - - // Test DISTINCT aggregation. - { + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {}, {1}, {1}, {1}, {1}, {2}, {2}, {2}, {3}, {3}, {4}}, aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.CountRows, execinfrapb.AnyNotNull, execinfrapb.Count, - execinfrapb.Count, - execinfrapb.SumInt, + execinfrapb.Sum, + execinfrapb.Avg, execinfrapb.SumInt, + execinfrapb.Min, + execinfrapb.Max, + execinfrapb.BoolAnd, + execinfrapb.BoolOr, + execinfrapb.ConcatAgg, }, - aggCols: [][]uint32{{0}, {1}, {1}, {1}, {1}}, - aggDistinct: []bool{false, false, true, false, true}, - typs: []*types.T{types.Int, types.Int}, + expected: colexectestutils.Tuples{ + {nil, 1, 1.1, 1, 1.1, 1.1, 4, 4, 4, true, true, "a"}, + {0, 2, 3.1, 1, 3.1, 3.1, 5, 5, 5, nil, nil, "b"}, + {1, 2, nil, 0, nil, nil, nil, nil, nil, false, false, nil}, + }, + convToDecimal: true, + }, + { + name: "DistinctAggregation", + typs: rowenc.TwoIntCols, input: colexectestutils.Tuples{ {0, 1}, {0, 2}, @@ -593,22 +626,24 @@ var aggregatorsTestCases = []aggregatorTestCase{ {1, 2}, {1, 2}, }, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1}, {1}, {1}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.Count, + execinfrapb.Count, + execinfrapb.SumInt, + execinfrapb.SumInt, + }, expected: colexectestutils.Tuples{ {0, 4, 2, 6, 3}, {1, 3, 2, 5, 3}, }, + aggDistinct: []bool{false, false, true, false, true}, }, - - // Test aggregation with FILTERs. { - aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AnyNotNull, - execinfrapb.CountRows, - execinfrapb.SumInt, - }, - aggCols: [][]uint32{{0}, {}, {1}}, - aggFilter: []int{tree.NoColumnIdx, 2, 2}, - typs: []*types.T{types.Int, types.Int, types.Bool}, + name: "FilteringAggregation", + typs: []*types.T{types.Int, types.Int, types.Bool}, input: colexectestutils.Tuples{ {0, 1, false}, {0, 2, true}, @@ -620,22 +655,22 @@ var aggregatorsTestCases = []aggregatorTestCase{ {1, 2, nil}, {1, 2, true}, }, - expected: colexectestutils.Tuples{ - {0, 3, 4}, - {1, 2, 3}, - }, - }, - - // Test aggregation with FILTERs when the whole groups are filtered out. - { + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {}, {1}}, aggFns: []execinfrapb.AggregatorSpec_Func{ execinfrapb.AnyNotNull, execinfrapb.CountRows, execinfrapb.SumInt, }, - aggCols: [][]uint32{{0}, {}, {1}}, + expected: colexectestutils.Tuples{ + {0, 3, 4}, + {1, 2, 3}, + }, aggFilter: []int{tree.NoColumnIdx, 2, 2}, - typs: []*types.T{types.Int, types.Int, types.Bool}, + }, + { + name: "AllGroupsFilteredOut", + typs: []*types.T{types.Int, types.Int, types.Bool}, input: colexectestutils.Tuples{ {0, 1, false}, {0, nil, nil}, @@ -647,28 +682,23 @@ var aggregatorsTestCases = []aggregatorTestCase{ {2, nil, nil}, {2, 2, nil}, }, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.CountRows, + execinfrapb.SumInt, + }, expected: colexectestutils.Tuples{ {0, 0, nil}, {1, 2, 3}, {2, 0, nil}, }, + aggFilter: []int{tree.NoColumnIdx, 2, 2}, }, - - // Test aggregation with FILTERs and DISTINCTs intertwined. { - aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AnyNotNull, - execinfrapb.Count, - execinfrapb.Count, - execinfrapb.Count, - execinfrapb.SumInt, - execinfrapb.SumInt, - execinfrapb.SumInt, - }, - aggCols: [][]uint32{{0}, {1}, {1}, {1}, {1}, {1}, {1}}, - aggDistinct: []bool{false, false, true, true, false, true, true}, - aggFilter: []int{tree.NoColumnIdx, 2, tree.NoColumnIdx, 2, 2, tree.NoColumnIdx, 2}, - typs: []*types.T{types.Int, types.Int, types.Bool}, + name: "DistinctFilteringAggregation", + typs: []*types.T{types.Int, types.Int, types.Bool}, input: colexectestutils.Tuples{ {0, 1, false}, {0, 2, true}, @@ -680,10 +710,23 @@ var aggregatorsTestCases = []aggregatorTestCase{ {1, 2, nil}, {1, 2, true}, }, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1}, {1}, {1}, {1}, {1}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.Count, + execinfrapb.Count, + execinfrapb.Count, + execinfrapb.SumInt, + execinfrapb.SumInt, + execinfrapb.SumInt, + }, expected: colexectestutils.Tuples{ {0, 2, 2, 1, 4, 3, 2}, {1, 2, 2, 2, 3, 3, 3}, }, + aggDistinct: []bool{false, false, true, true, false, true, true}, + aggFilter: []int{tree.NoColumnIdx, 2, tree.NoColumnIdx, 2, 2, tree.NoColumnIdx, 2}, }, } @@ -826,7 +869,9 @@ func TestAggregatorRandom(t *testing.T) { source := colexectestutils.NewChunkingBatchSource(testAllocator, typs, cols, nTuples) tc := aggregatorTestCase{ - typs: typs, + typs: typs, + groupCols: []uint32{0}, + aggCols: [][]uint32{{}, {1}, {1}, {1}, {1}, {1}}, aggFns: []execinfrapb.AggregatorSpec_Func{ execinfrapb.CountRows, execinfrapb.Count, @@ -835,8 +880,6 @@ func TestAggregatorRandom(t *testing.T) { execinfrapb.Max, execinfrapb.Avg, }, - groupCols: []uint32{0}, - aggCols: [][]uint32{{}, {1}, {1}, {1}, {1}, {1}}, } require.NoError(t, tc.init()) constructors, constArguments, outputTypes, err := colexecagg.ProcessAggregations( @@ -945,9 +988,9 @@ func benchmarkAggregateFunction( } tc := aggregatorTestCase{ typs: typs, - aggFns: []execinfrapb.AggregatorSpec_Func{aggFn}, groupCols: []uint32{0}, aggCols: [][]uint32{aggCols}, + aggFns: []execinfrapb.AggregatorSpec_Func{aggFn}, } if distinctProb > 0 { if !typs[0].Identical(types.Int) { diff --git a/pkg/sql/colexec/default_agg_test.go b/pkg/sql/colexec/default_agg_test.go index 9f15e1464bc1..bd640ef66210 100644 --- a/pkg/sql/colexec/default_agg_test.go +++ b/pkg/sql/colexec/default_agg_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -33,10 +34,8 @@ func TestDefaultAggregateFunc(t *testing.T) { defer log.Scope(t).Close(t) testCases := []aggregatorTestCase{ { - aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AnyNotNull, - execinfrapb.StringAgg, - }, + name: "StringAgg", + typs: []*types.T{types.Int, types.String, types.String}, input: colexectestutils.Tuples{ {nil, "a", "1"}, {nil, "b", "2"}, @@ -46,21 +45,21 @@ func TestDefaultAggregateFunc(t *testing.T) { {1, "f", "6"}, {1, "g", "7"}, }, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1, 2}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.StringAgg, + }, expected: colexectestutils.Tuples{ {nil, "a2b"}, {0, "c4d5e"}, {1, "f7g"}, }, - typs: []*types.T{types.Int, types.String, types.String}, - name: "StringAgg", - groupCols: []uint32{0}, - aggCols: [][]uint32{{0}, {1, 2}}, }, { - aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AnyNotNull, - execinfrapb.StringAgg, - }, + name: "StringAggWithConstDelimiter", + typs: []*types.T{types.Int, types.String}, input: colexectestutils.Tuples{ {nil, "a"}, {nil, "b"}, @@ -70,24 +69,21 @@ func TestDefaultAggregateFunc(t *testing.T) { {1, "f"}, {1, "g"}, }, - constArguments: [][]execinfrapb.Expression{nil, {{Expr: "'_'"}}}, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.StringAgg, + }, expected: colexectestutils.Tuples{ {nil, "a_b"}, {0, "c_d_e"}, {1, "f_g"}, }, - typs: []*types.T{types.Int, types.String}, - name: "StringAggWithConstDelimiter", - groupCols: []uint32{0}, - aggCols: [][]uint32{{0}, {1}}, + constArguments: [][]execinfrapb.Expression{nil, {{Expr: "'_'"}}}, }, { - aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AnyNotNull, - execinfrapb.JSONAgg, - execinfrapb.JSONAgg, - execinfrapb.StringAgg, - }, + name: "JsonAggWithStringAgg", typs: []*types.T{types.Int, types.Jsonb, types.String}, input: colexectestutils.Tuples{ {nil, `'{"id": 1}'`, "a"}, @@ -97,21 +93,24 @@ func TestDefaultAggregateFunc(t *testing.T) { {0, `'{"id": 2}'`, "e"}, {1, `'{"id": 3}'`, "f"}, }, - constArguments: [][]execinfrapb.Expression{nil, nil, nil, {{Expr: "'_'"}}}, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1}, {2}, {2}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.JSONAgg, + execinfrapb.JSONAgg, + execinfrapb.StringAgg, + }, expected: colexectestutils.Tuples{ {nil, `'[{"id": 1}, {"id": 2}]'`, `'["a", "b"]'`, "a_b"}, {0, `'[{"id": 1}, {"id": 2}, {"id": 2}]'`, `'["c", "d", "e"]'`, "c_d_e"}, {1, `'[{"id": 3}]'`, `'["f"]'`, "f"}, }, - name: "JsonAggWithStringAgg", - groupCols: []uint32{0}, - aggCols: [][]uint32{{0}, {1}, {2}, {2}}, + constArguments: [][]execinfrapb.Expression{nil, nil, nil, {{Expr: "'_'"}}}, }, { - aggFns: []execinfrapb.AggregatorSpec_Func{ - execinfrapb.AnyNotNull, - execinfrapb.XorAgg, - }, + name: "XorAgg", + typs: rowenc.TwoIntCols, input: colexectestutils.Tuples{ {nil, 3}, {nil, 1}, @@ -119,14 +118,16 @@ func TestDefaultAggregateFunc(t *testing.T) { {0, -1}, {0, 0}, }, + groupCols: []uint32{0}, + aggCols: [][]uint32{{0}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.XorAgg, + }, expected: colexectestutils.Tuples{ {nil, 2}, {0, 4}, }, - typs: []*types.T{types.Int, types.Int}, - name: "XorAgg", - groupCols: []uint32{0}, - aggCols: [][]uint32{{0}, {1}}, }, } diff --git a/pkg/sql/colexec/hash_aggregator_test.go b/pkg/sql/colexec/hash_aggregator_test.go index e4a37ec13e4f..902c9d35613f 100644 --- a/pkg/sql/colexec/hash_aggregator_test.go +++ b/pkg/sql/colexec/hash_aggregator_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils/colcontainerutils" @@ -35,7 +36,8 @@ import ( var hashAggregatorTestCases = []aggregatorTestCase{ { - // Test carry between output batches. + name: "carryBetweenBatches", + typs: rowenc.TwoIntCols, input: colexectestutils.Tuples{ {0, 1}, {1, 5}, @@ -45,35 +47,22 @@ var hashAggregatorTestCases = []aggregatorTestCase{ {0, 3}, {0, 7}, }, - typs: []*types.T{types.Int, types.Int}, groupCols: []uint32{0}, - aggCols: [][]uint32{{1}}, - - expected: colexectestutils.Tuples{ - {5}, - {6}, - {17}, - }, - - name: "carryBetweenBatches", - }, - { - // Test a single row input source. - input: colexectestutils.Tuples{ - {5}, + aggCols: [][]uint32{{0}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.SumInt, }, - typs: []*types.T{types.Int}, - groupCols: []uint32{0}, - aggCols: [][]uint32{{0}}, - expected: colexectestutils.Tuples{ - {5}, + {1, 5}, + {2, 6}, + {0, 17}, }, - - name: "singleRowInput", + unorderedInput: true, }, { - // Test bucket collisions. + name: "bucketCollision", + typs: rowenc.TwoIntCols, input: colexectestutils.Tuples{ {0, 3}, {0, 4}, @@ -81,61 +70,39 @@ var hashAggregatorTestCases = []aggregatorTestCase{ {0, 5}, {coldata.BatchSize(), 7}, }, - typs: []*types.T{types.Int, types.Int}, groupCols: []uint32{0}, - aggCols: [][]uint32{{1}}, - + aggCols: [][]uint32{{0}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.SumInt, + }, expected: colexectestutils.Tuples{ - {12}, - {13}, + {0, 12}, + {coldata.BatchSize(), 13}, }, - - name: "bucketCollision", }, { + name: "decimalSums", + typs: []*types.T{types.Int, types.Int, types.Decimal}, input: colexectestutils.Tuples{ {0, 1, 1.3}, {0, 1, 1.6}, {0, 1, 0.5}, {1, 1, 1.2}, }, - typs: []*types.T{types.Int, types.Int, types.Decimal}, - convToDecimal: true, - - aggFns: []execinfrapb.AggregatorSpec_Func{execinfrapb.Sum, execinfrapb.SumInt}, groupCols: []uint32{0, 1}, - aggCols: [][]uint32{ - {2}, {1}, + aggCols: [][]uint32{{0}, {1}, {2}, {1}}, + aggFns: []execinfrapb.AggregatorSpec_Func{ + execinfrapb.AnyNotNull, + execinfrapb.AnyNotNull, + execinfrapb.Sum, + execinfrapb.SumInt, }, - expected: colexectestutils.Tuples{ - {3.4, 3}, - {1.2, 1}, + {0, 1, 3.4, 3}, + {1, 1, 1.2, 1}, }, - - name: "decimalSums", - }, - { - // Test unused input columns. - input: colexectestutils.Tuples{ - {0, 1, 2, 3}, - {0, 1, 4, 5}, - {1, 1, 3, 7}, - {1, 2, 4, 9}, - {0, 1, 6, 11}, - {1, 2, 6, 13}, - }, - typs: []*types.T{types.Int, types.Int, types.Int, types.Int}, - groupCols: []uint32{0, 1}, - aggCols: [][]uint32{{3}}, - - expected: colexectestutils.Tuples{ - {7}, - {19}, - {22}, - }, - - name: "unusedInputCol", + convToDecimal: true, }, } From ef1323208db40946714b95ee8e4009b1bc9586a0 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Thu, 8 Apr 2021 19:42:55 -0700 Subject: [PATCH 5/6] colexec: fix hash aggregator when spilling to disk In some cases the aggregation is expected to maintain the required ordering in order to eliminate an explicit sort afterwards. It is always the case that the required ordering is a prefix of ordered grouping columns. With the introduction of disk spilling for the vectorized hash aggregator in 21.1 release the ordering was no longer maintained if the spilling occurs. In all previous cases (row-by-row processors and in-memory columnar operator) the ordering was maintained by construction, but with `hashBasedPartitioner` the ordering can be arbitrary. In order to fix this issue we now do what we did for the external distinct - we plan an external sort on top of the external hash aggregator to restore the required ordering. Note that this will only kick in if the spilling to disk occurred. This required changes to the AggregatorSpec to propagate the required output ordering. Release note (bug fix): In 21.1 alpha and beta releases CockroachDB could return the output in an incorrect order if the query containing hash aggregation was executed via the vectorized engine and spilling to temporary storage was required, in some cases. --- pkg/sql/colexec/aggregators_test.go | 11 +- pkg/sql/colexec/colexectestutils/utils.go | 3 +- pkg/sql/colexec/external_distinct.go | 3 + pkg/sql/colexec/external_hash_aggregator.go | 19 +- .../colexec/external_hash_aggregator_test.go | 33 +- pkg/sql/colexec/hash_aggregator_test.go | 6 +- pkg/sql/distsql_physical_planner.go | 7 + pkg/sql/execinfrapb/processors_sql.pb.go | 516 ++++++++++-------- pkg/sql/execinfrapb/processors_sql.proto | 9 +- .../logictest/testdata/logic_test/aggregate | 15 + pkg/sql/opt/exec/factory.opt | 3 + 11 files changed, 372 insertions(+), 253 deletions(-) diff --git a/pkg/sql/colexec/aggregators_test.go b/pkg/sql/colexec/aggregators_test.go index 037af0f19eb2..5fa1b295557a 100644 --- a/pkg/sql/colexec/aggregators_test.go +++ b/pkg/sql/colexec/aggregators_test.go @@ -127,6 +127,15 @@ func (tc *aggregatorTestCase) init() error { GroupCols: tc.groupCols, Aggregations: aggregations, } + if !tc.unorderedInput { + // If we have ordered on grouping columns input, then we'll require the + // output to also have the same ordering. + outputOrdering := execinfrapb.Ordering{Columns: make([]execinfrapb.Ordering_Column, len(tc.groupCols))} + for i, col := range tc.groupCols { + outputOrdering.Columns[i].ColIdx = col + } + tc.spec.OutputOrdering = outputOrdering + } return nil } @@ -762,7 +771,7 @@ func TestAggregators(t *testing.T) { } log.Infof(ctx, "%s/%s", tc.name, agg.name) verifier := colexectestutils.OrderedVerifier - if agg.name == "hash" { + if tc.unorderedInput { verifier = colexectestutils.UnorderedVerifier } colexectestutils.RunTestsWithTyps(t, testAllocator, []colexectestutils.Tuples{tc.input}, [][]*types.T{tc.typs}, tc.expected, verifier, diff --git a/pkg/sql/colexec/colexectestutils/utils.go b/pkg/sql/colexec/colexectestutils/utils.go index 52d704fa1386..51d65e8e0bb9 100644 --- a/pkg/sql/colexec/colexectestutils/utils.go +++ b/pkg/sql/colexec/colexectestutils/utils.go @@ -455,6 +455,8 @@ func RunTestsWithoutAllNullsInjection( if err != nil { t.Fatal(err) } + // We might short-circuit, so defer the closing of the operator. + defer closeIfCloser(ctx, t, op) op.Init() b := op.Next(ctx) if b.Length() == 0 { @@ -495,7 +497,6 @@ func RunTestsWithoutAllNullsInjection( assert.False(t, maybeHasNulls(b)) } } - closeIfCloser(ctx, t, op) } } diff --git a/pkg/sql/colexec/external_distinct.go b/pkg/sql/colexec/external_distinct.go index 815508f135d0..f27dbe96de4a 100644 --- a/pkg/sql/colexec/external_distinct.go +++ b/pkg/sql/colexec/external_distinct.go @@ -112,6 +112,9 @@ func NewExternalDistinct( // No particular output ordering is required. return ed } + // TODO(yuzefovich): the fact that we're planning an additional external + // sort isn't accounted for when considering the number file descriptors to + // acquire. Not urgent, but it should be fixed. maxNumberActivePartitions := calculateMaxNumberActivePartitions(flowCtx, args, numRequiredActivePartitions) return createDiskBackedSorter(ed, inputTypes, outputOrdering.Columns, maxNumberActivePartitions) } diff --git a/pkg/sql/colexec/external_hash_aggregator.go b/pkg/sql/colexec/external_hash_aggregator.go index 2f07895cecf7..60d5a0e360ba 100644 --- a/pkg/sql/colexec/external_hash_aggregator.go +++ b/pkg/sql/colexec/external_hash_aggregator.go @@ -71,7 +71,7 @@ func NewExternalHashAggregator( } return diskBackedFallbackOp } - return newHashBasedPartitioner( + eha := newHashBasedPartitioner( newAggArgs.Allocator, flowCtx, args, @@ -84,6 +84,23 @@ func NewExternalHashAggregator( diskAcc, ehaNumRequiredActivePartitions, ) + // 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 hash aggregation 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.Aggregator.OutputOrdering + if len(outputOrdering.Columns) == 0 { + // No particular output ordering is required. + return eha + } + // TODO(yuzefovich): the fact that we're planning an additional external + // sort isn't accounted for when considering the number file descriptors to + // acquire. Not urgent, but it should be fixed. + maxNumberActivePartitions := calculateMaxNumberActivePartitions(flowCtx, args, ehaNumRequiredActivePartitions) + return createDiskBackedSorter(eha, args.Spec.ResultTypes, outputOrdering.Columns, maxNumberActivePartitions) } // HashAggregationDiskSpillingEnabled is a cluster setting that allows to diff --git a/pkg/sql/colexec/external_hash_aggregator_test.go b/pkg/sql/colexec/external_hash_aggregator_test.go index c70a200345c3..25c975c8e25b 100644 --- a/pkg/sql/colexec/external_hash_aggregator_test.go +++ b/pkg/sql/colexec/external_hash_aggregator_test.go @@ -85,6 +85,25 @@ func TestExternalHashAggregator(t *testing.T) { &evalCtx, nil /* semaCtx */, tc.spec.Aggregations, tc.typs, ) require.NoError(t, err) + verifier := colexectestutils.OrderedVerifier + if tc.unorderedInput { + verifier = colexectestutils.UnorderedVerifier + } + var numExpectedClosers int + if diskSpillingEnabled { + // The external sorter and the disk spiller should be added + // as Closers (the latter is responsible for closing the + // in-memory hash aggregator as well as the external one). + numExpectedClosers = 2 + if len(tc.spec.OutputOrdering.Columns) > 0 { + // When the output ordering is required, we also plan + // another external sort. + numExpectedClosers++ + } + } else { + // Only the in-memory hash aggregator should be added. + numExpectedClosers = 1 + } var semsToCheck []semaphore.Semaphore colexectestutils.RunTestsWithTyps( t, @@ -92,7 +111,7 @@ func TestExternalHashAggregator(t *testing.T) { []colexectestutils.Tuples{tc.input}, [][]*types.T{tc.typs}, tc.expected, - colexectestutils.UnorderedVerifier, + verifier, func(input []colexecop.Operator) (colexecop.Operator, error) { sem := colexecop.NewTestingSemaphore(ehaNumRequiredFDs) semsToCheck = append(semsToCheck, sem) @@ -112,16 +131,8 @@ func TestExternalHashAggregator(t *testing.T) { ) accounts = append(accounts, accs...) monitors = append(monitors, mons...) - if diskSpillingEnabled { - // Check that the external sorter and the disk - // spiller were added as Closers (the latter is - // responsible for closing the in-memory hash - // aggregator as well as the external one). - require.Equal(t, 2, len(closers)) - } else { - // Only the in-memory hash aggregator has been - // created. - require.Equal(t, 1, len(closers)) + require.Equal(t, numExpectedClosers, len(closers)) + if !diskSpillingEnabled { // Sanity check that indeed only the in-memory hash // aggregator was created. _, isHashAgg := op.(*hashAggregator) diff --git a/pkg/sql/colexec/hash_aggregator_test.go b/pkg/sql/colexec/hash_aggregator_test.go index 902c9d35613f..67c3da2fe2de 100644 --- a/pkg/sql/colexec/hash_aggregator_test.go +++ b/pkg/sql/colexec/hash_aggregator_test.go @@ -125,7 +125,11 @@ func TestHashAggregator(t *testing.T) { &evalCtx, nil /* semaCtx */, tc.spec.Aggregations, tc.typs, ) require.NoError(t, err) - colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.input}, tc.expected, colexectestutils.UnorderedVerifier, func(sources []colexecop.Operator) (colexecop.Operator, error) { + verifier := colexectestutils.OrderedVerifier + if tc.unorderedInput { + verifier = colexectestutils.UnorderedVerifier + } + colexectestutils.RunTests(t, testAllocator, []colexectestutils.Tuples{tc.input}, tc.expected, verifier, func(sources []colexecop.Operator) (colexecop.Operator, error) { return NewHashAggregator(&colexecagg.NewAggregatorArgs{ Allocator: testAllocator, MemAccount: testMemAcc, diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 935f52356b3e..ad3485aefbea 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -1593,12 +1593,17 @@ func (dsp *DistSQLPlanner) planAggregators( var finalAggsSpec execinfrapb.AggregatorSpec var finalAggsPost execinfrapb.PostProcessSpec + // Note that we pass in nil as the second argument because we will have a + // simple 1-to-1 PlanToStreamColMap in the end. + finalOutputOrdering := dsp.convertOrdering(info.reqOrdering, nil /* planToStreamColMap */) + if !multiStage { finalAggsSpec = execinfrapb.AggregatorSpec{ Type: aggType, Aggregations: info.aggregations, GroupCols: groupCols, OrderedGroupCols: orderedGroupCols, + OutputOrdering: finalOutputOrdering, } } else { // Some aggregations might need multiple aggregation as part of @@ -1820,6 +1825,7 @@ func (dsp *DistSQLPlanner) planAggregators( Aggregations: localAggs, GroupCols: groupCols, OrderedGroupCols: orderedGroupCols, + OutputOrdering: execinfrapb.Ordering{Columns: ordCols}, } p.AddNoGroupingStage( @@ -1834,6 +1840,7 @@ func (dsp *DistSQLPlanner) planAggregators( Aggregations: finalAggs, GroupCols: finalGroupCols, OrderedGroupCols: finalOrderedGroupCols, + OutputOrdering: finalOutputOrdering, } if needRender { diff --git a/pkg/sql/execinfrapb/processors_sql.pb.go b/pkg/sql/execinfrapb/processors_sql.pb.go index 42d7ce4f0dd8..ce3d62e6e211 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_34caf8b4267e5165, []int{0} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []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_34caf8b4267e5165, []int{13, 0} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []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_34caf8b4267e5165, []int{13, 1} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []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_34caf8b4267e5165, []int{15, 0} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []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_34caf8b4267e5165, []int{15, 1, 0} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []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_34caf8b4267e5165, []int{15, 1, 1} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []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_34caf8b4267e5165, []int{15, 1, 2} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []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_34caf8b4267e5165, []int{0} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{0} } func (m *ValuesCoreSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -611,7 +611,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_34caf8b4267e5165, []int{1} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{1} } func (m *TableReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -648,7 +648,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_34caf8b4267e5165, []int{2} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{2} } func (m *FiltererSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -703,7 +703,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_34caf8b4267e5165, []int{3} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{3} } func (m *IndexSkipTableReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -909,7 +909,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_34caf8b4267e5165, []int{4} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{4} } func (m *JoinReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -953,7 +953,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_34caf8b4267e5165, []int{5} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{5} } func (m *SorterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1019,7 +1019,7 @@ 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_34caf8b4267e5165, []int{6} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{6} } func (m *DistinctSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1054,7 +1054,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_34caf8b4267e5165, []int{7} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{7} } func (m *OrdinalitySpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1112,7 +1112,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_34caf8b4267e5165, []int{8} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{8} } func (m *ZigzagJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1192,7 +1192,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_34caf8b4267e5165, []int{9} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{9} } func (m *MergeJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1265,7 +1265,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_34caf8b4267e5165, []int{10} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{10} } func (m *HashJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1400,7 +1400,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_34caf8b4267e5165, []int{11} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{11} } func (m *InvertedJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1461,7 +1461,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_34caf8b4267e5165, []int{12} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{12} } func (m *InvertedFiltererSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1499,7 +1499,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_34caf8b4267e5165, []int{12, 0} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{12, 0} } func (m *InvertedFiltererSpec_PreFiltererSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1532,20 +1532,24 @@ var xxx_messageInfo_InvertedFiltererSpec_PreFiltererSpec proto.InternalMessageIn // // The "internal columns" of an Aggregator map 1-1 to the aggregations. type AggregatorSpec struct { - Type AggregatorSpec_Type `protobuf:"varint,5,opt,name=type,enum=cockroach.sql.distsqlrun.AggregatorSpec_Type" json:"type"` // The group key is a subset of the columns in the input stream schema on the // basis of which we define our groups. GroupCols []uint32 `protobuf:"varint,2,rep,packed,name=group_cols,json=groupCols" json:"group_cols,omitempty"` Aggregations []AggregatorSpec_Aggregation `protobuf:"bytes,3,rep,name=aggregations" json:"aggregations"` // A subset of the GROUP BY columns which are ordered in the input. - OrderedGroupCols []uint32 `protobuf:"varint,4,rep,packed,name=ordered_group_cols,json=orderedGroupCols" json:"ordered_group_cols,omitempty"` + OrderedGroupCols []uint32 `protobuf:"varint,4,rep,packed,name=ordered_group_cols,json=orderedGroupCols" json:"ordered_group_cols,omitempty"` + Type AggregatorSpec_Type `protobuf:"varint,5,opt,name=type,enum=cockroach.sql.distsqlrun.AggregatorSpec_Type" json:"type"` + // OutputOrdering specifies the required ordering of the output produced by + // the aggregator. The input to the processor *must* already be ordered + // according to it. + OutputOrdering Ordering `protobuf:"bytes,6,opt,name=output_ordering,json=outputOrdering" json:"output_ordering"` } 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_34caf8b4267e5165, []int{13} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{13} } func (m *AggregatorSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1596,7 +1600,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_34caf8b4267e5165, []int{13, 0} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{13, 0} } func (m *AggregatorSpec_Aggregation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1636,7 +1640,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_34caf8b4267e5165, []int{14} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{14} } func (m *ProjectSetSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1678,7 +1682,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_34caf8b4267e5165, []int{15} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{15} } func (m *WindowerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1714,7 +1718,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_34caf8b4267e5165, []int{15, 0} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{15, 0} } func (m *WindowerSpec_Func) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1750,7 +1754,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_34caf8b4267e5165, []int{15, 1} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{15, 1} } func (m *WindowerSpec_Frame) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1791,7 +1795,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_34caf8b4267e5165, []int{15, 1, 0} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{15, 1, 0} } func (m *WindowerSpec_Frame_Bound) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1827,7 +1831,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_34caf8b4267e5165, []int{15, 1, 1} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{15, 1, 1} } func (m *WindowerSpec_Frame_Bounds) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1877,7 +1881,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_34caf8b4267e5165, []int{15, 2} + return fileDescriptor_processors_sql_6b6703e0ac4ff0b7, []int{15, 2} } func (m *WindowerSpec_WindowFn) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2831,6 +2835,14 @@ func (m *AggregatorSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x28 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Type)) + dAtA[i] = 0x32 + i++ + i = encodeVarintProcessorsSql(dAtA, i, uint64(m.OutputOrdering.Size())) + n36, err := m.OutputOrdering.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n36 return i, nil } @@ -3022,11 +3034,11 @@ func (m *WindowerSpec_Frame) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Bounds.Size())) - n36, err := m.Bounds.MarshalTo(dAtA[i:]) + n37, err := m.Bounds.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n36 + i += n37 dAtA[i] = 0x18 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Exclusion)) @@ -3063,11 +3075,11 @@ func (m *WindowerSpec_Frame_Bound) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.OffsetType.Size())) - n37, err := m.OffsetType.MarshalTo(dAtA[i:]) + n38, err := m.OffsetType.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n37 + i += n38 return i, nil } @@ -3089,20 +3101,20 @@ func (m *WindowerSpec_Frame_Bounds) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Start.Size())) - n38, err := m.Start.MarshalTo(dAtA[i:]) + n39, err := m.Start.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n38 + i += n39 if m.End != nil { dAtA[i] = 0x12 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.End.Size())) - n39, err := m.End.MarshalTo(dAtA[i:]) + n40, err := m.End.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n39 + i += n40 } return i, nil } @@ -3125,28 +3137,28 @@ func (m *WindowerSpec_WindowFn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Func.Size())) - n40, err := m.Func.MarshalTo(dAtA[i:]) + n41, err := m.Func.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n40 + i += n41 dAtA[i] = 0x22 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Ordering.Size())) - n41, err := m.Ordering.MarshalTo(dAtA[i:]) + n42, err := m.Ordering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n41 + i += n42 if m.Frame != nil { dAtA[i] = 0x2a i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Frame.Size())) - n42, err := m.Frame.MarshalTo(dAtA[i:]) + n43, err := m.Frame.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n42 + i += n43 } dAtA[i] = 0x30 i++ @@ -3509,6 +3521,8 @@ func (m *AggregatorSpec) Size() (n int) { n += 1 + sovProcessorsSql(uint64(l)) + l } n += 1 + sovProcessorsSql(uint64(m.Type)) + l = m.OutputOrdering.Size() + n += 1 + l + sovProcessorsSql(uint64(l)) return n } @@ -6890,6 +6904,36 @@ func (m *AggregatorSpec) Unmarshal(dAtA []byte) error { break } } + case 6: + 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:]) @@ -8293,194 +8337,194 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors_sql.proto", fileDescriptor_processors_sql_34caf8b4267e5165) -} - -var fileDescriptor_processors_sql_34caf8b4267e5165 = []byte{ - // 2949 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0xcb, 0x6f, 0x1b, 0xd7, - 0xd5, 0x17, 0x5f, 0x12, 0x79, 0xf8, 0xd0, 0xf5, 0xb5, 0x1d, 0x33, 0x4a, 0x3e, 0x59, 0xa6, 0x1d, - 0x5b, 0x76, 0x1c, 0xf9, 0x8b, 0xbe, 0x0f, 0x29, 0x92, 0x16, 0x45, 0x29, 0x72, 0x48, 0x53, 0xa6, - 0x66, 0xe8, 0x21, 0x29, 0xdb, 0x09, 0xd0, 0x8b, 0x11, 0x79, 0x49, 0x4d, 0x3c, 0x9c, 0xa1, 0x66, - 0x86, 0x96, 0x94, 0x7f, 0xa0, 0xab, 0x02, 0x2d, 0xba, 0x29, 0xb2, 0x28, 0xb2, 0xe9, 0xa6, 0xeb, - 0xfe, 0x09, 0x5d, 0x78, 0x53, 0x20, 0xe8, 0xa2, 0xcd, 0xaa, 0x68, 0x9c, 0x5d, 0xb7, 0x5d, 0x14, - 0xdd, 0x15, 0xf7, 0x31, 0xa3, 0xa1, 0x2a, 0xba, 0xa1, 0xed, 0xc4, 0x1b, 0x43, 0xf7, 0x9c, 0xf3, - 0x3b, 0xf7, 0xdc, 0xf3, 0xba, 0xe7, 0x0e, 0x0d, 0xd7, 0xbc, 0x03, 0xeb, 0x0e, 0x3d, 0xa2, 0x3d, - 0xd3, 0x1e, 0xb8, 0xc6, 0x78, 0xef, 0xce, 0xd8, 0x75, 0x7a, 0xd4, 0xf3, 0x1c, 0xd7, 0x23, 0xde, - 0x81, 0xb5, 0x31, 0x76, 0x1d, 0xdf, 0xc1, 0xc5, 0x9e, 0xd3, 0x7b, 0xec, 0x3a, 0x46, 0x6f, 0x7f, - 0x83, 0x11, 0xfb, 0xa6, 0xe7, 0x7b, 0x07, 0x96, 0x3b, 0xb1, 0x57, 0xae, 0x32, 0x7c, 0xcf, 0xf0, - 0x0d, 0xcb, 0x19, 0xde, 0xe9, 0x53, 0xaf, 0x37, 0xde, 0xbb, 0xe3, 0xf9, 0xee, 0xa4, 0xe7, 0x4f, - 0x5c, 0xda, 0x17, 0xf0, 0x95, 0xd2, 0x19, 0x42, 0x9f, 0x3a, 0xa6, 0x4d, 0xfc, 0xe3, 0x31, 0x95, - 0x32, 0x6b, 0x67, 0xc8, 0x58, 0x4e, 0xef, 0xb1, 0x69, 0x0f, 0xa5, 0xc4, 0x45, 0x26, 0xc1, 0x10, - 0x9e, 0xf8, 0x57, 0x92, 0x57, 0x4e, 0x9f, 0xa0, 0x6f, 0xf8, 0x86, 0xe4, 0xbd, 0xf3, 0x9c, 0xd3, - 0xed, 0x19, 0x1e, 0x8d, 0xda, 0x67, 0xda, 0x4f, 0xa8, 0xeb, 0xd3, 0xfe, 0x1d, 0x6f, 0x6c, 0xd8, - 0x84, 0x1e, 0x8d, 0x5d, 0xea, 0x79, 0xa6, 0x63, 0x4b, 0x99, 0x0b, 0x43, 0x67, 0xe8, 0xf0, 0x3f, - 0xef, 0xb0, 0xbf, 0x04, 0xb5, 0xf4, 0xab, 0x18, 0x14, 0x76, 0x0d, 0x6b, 0x42, 0xbd, 0x8a, 0xe3, - 0xd2, 0xf6, 0x98, 0xf6, 0x70, 0x05, 0x96, 0x7a, 0x8e, 0x35, 0x19, 0xd9, 0x5e, 0x31, 0xb6, 0x96, - 0x58, 0xcf, 0x6e, 0x5e, 0xdd, 0x98, 0xe5, 0xbd, 0x8d, 0xaa, 0xe1, 0x4f, 0x46, 0x0d, 0x7b, 0xe0, - 0x6c, 0x25, 0x9f, 0xfe, 0xf5, 0xf2, 0x82, 0x1e, 0x20, 0xf1, 0x5b, 0x90, 0x71, 0x8d, 0x43, 0xb2, - 0x77, 0xec, 0x53, 0xaf, 0x18, 0x5f, 0x4b, 0xac, 0xe7, 0xf4, 0xb4, 0x6b, 0x1c, 0x6e, 0xb1, 0x35, - 0xbe, 0x0c, 0x69, 0x7b, 0x32, 0x22, 0xae, 0x73, 0xe8, 0x15, 0x13, 0x6b, 0xb1, 0xf5, 0x64, 0x80, - 0xb6, 0x27, 0x23, 0xdd, 0x39, 0xf4, 0x4a, 0x7f, 0x5a, 0x84, 0xe5, 0x8e, 0xb1, 0x67, 0x51, 0x9d, - 0x1a, 0x7d, 0xea, 0x72, 0xb3, 0xb6, 0x20, 0xe5, 0x33, 0x52, 0x31, 0xb6, 0x16, 0x5b, 0xcf, 0x6e, - 0x5e, 0x3f, 0x65, 0x94, 0x77, 0x60, 0x71, 0x87, 0x70, 0x58, 0x95, 0x7a, 0x3d, 0xd7, 0x1c, 0xfb, - 0x8e, 0x2b, 0x35, 0x0b, 0x28, 0xbe, 0x02, 0x19, 0xd3, 0xee, 0xd3, 0x23, 0x62, 0xf6, 0x8f, 0x8a, - 0xf1, 0xb5, 0xd8, 0x7a, 0x5e, 0xf2, 0xd3, 0x9c, 0xdc, 0xe8, 0x1f, 0xe1, 0x55, 0x58, 0x72, 0xe9, - 0x13, 0xea, 0x7a, 0x94, 0x9b, 0x96, 0x0e, 0x4c, 0x93, 0x44, 0xac, 0x40, 0x8a, 0xf9, 0xd7, 0x2b, - 0x26, 0xb9, 0x6f, 0x6e, 0xce, 0xf6, 0xcd, 0xd4, 0x01, 0x0c, 0x3b, 0xb0, 0x84, 0xa3, 0xf1, 0x55, - 0x00, 0xcb, 0x1c, 0x99, 0x3e, 0xd9, 0x37, 0x6d, 0xbf, 0x98, 0x5a, 0x8b, 0xad, 0x27, 0xa4, 0x40, - 0x86, 0xd3, 0xef, 0x9a, 0xb6, 0xcf, 0xfc, 0x64, 0x7a, 0xa4, 0xb7, 0x4f, 0x7b, 0x8f, 0x8b, 0x8b, - 0x51, 0x63, 0x4c, 0xaf, 0xc2, 0x88, 0x58, 0x05, 0x78, 0x62, 0x7a, 0xe6, 0x9e, 0x69, 0x99, 0xfe, - 0x71, 0x71, 0x69, 0x2d, 0xb6, 0x5e, 0xd8, 0x5c, 0x9f, 0x6d, 0x51, 0xbb, 0x67, 0xd8, 0xbb, 0xa1, - 0xbc, 0x54, 0x16, 0xd1, 0x80, 0x7f, 0x08, 0x97, 0x46, 0xc6, 0x11, 0xf1, 0xcd, 0x11, 0xf5, 0x7c, - 0x63, 0x34, 0x26, 0xc6, 0x90, 0x12, 0xdb, 0xb0, 0x1d, 0xaf, 0x98, 0x89, 0xc4, 0xe9, 0xc2, 0xc8, - 0x38, 0xea, 0x04, 0x32, 0xe5, 0x21, 0x55, 0x99, 0x04, 0xfe, 0x04, 0x90, 0xcc, 0x77, 0xe2, 0xf9, - 0x2e, 0xb5, 0x87, 0xfe, 0x7e, 0x11, 0xb8, 0x49, 0xb7, 0x66, 0xc4, 0x8a, 0xd9, 0xd3, 0x14, 0x90, - 0xb6, 0x44, 0xc8, 0x1d, 0x96, 0xad, 0x69, 0x32, 0xde, 0x83, 0xf3, 0x81, 0xf2, 0x43, 0xc3, 0xf4, - 0xc9, 0xd8, 0xb1, 0xcc, 0xde, 0x71, 0x31, 0xcb, 0xf5, 0xdf, 0xfe, 0xef, 0xfa, 0x1f, 0x18, 0xa6, - 0xdf, 0xe2, 0x18, 0xb9, 0xc3, 0x39, 0xeb, 0x34, 0x03, 0x5f, 0x87, 0xec, 0xd8, 0x70, 0x0d, 0xcb, - 0xa2, 0x96, 0xf9, 0x19, 0x2d, 0xe6, 0x22, 0x1e, 0x8f, 0x32, 0xf0, 0x26, 0xe0, 0x7d, 0xc3, 0x23, - 0xde, 0xb1, 0xe7, 0xd3, 0x11, 0x09, 0x6a, 0xa5, 0x10, 0x11, 0x47, 0xfb, 0x86, 0xd7, 0xe6, 0xec, - 0x8a, 0xac, 0x87, 0x77, 0xa0, 0x60, 0x53, 0xda, 0xa7, 0xfd, 0x50, 0x7e, 0x79, 0x2d, 0xb1, 0x9e, - 0xd7, 0xf3, 0x82, 0x1a, 0x88, 0xa9, 0x50, 0x78, 0x62, 0xba, 0xfe, 0xc4, 0xb0, 0xa4, 0x5c, 0x11, - 0xf1, 0x6c, 0xbf, 0x31, 0xe3, 0x84, 0x02, 0x77, 0x92, 0xee, 0x7a, 0x5e, 0xc2, 0x05, 0x63, 0x3b, - 0x99, 0x4e, 0xa3, 0xcc, 0x76, 0x32, 0x9d, 0x47, 0x85, 0x92, 0x0e, 0xb9, 0x9a, 0x69, 0xf9, 0xd4, - 0x0d, 0x0b, 0x6a, 0x71, 0xc0, 0xd7, 0xb2, 0xa2, 0xae, 0xcd, 0x4e, 0x1c, 0x25, 0x6c, 0x26, 0xf2, - 0x80, 0x12, 0x59, 0xfa, 0x57, 0x02, 0x8a, 0x0d, 0x56, 0x3a, 0xed, 0xc7, 0xe6, 0xf8, 0x35, 0x55, - 0x6c, 0x58, 0x91, 0x89, 0x97, 0xaa, 0xc8, 0xe9, 0x5a, 0x4a, 0xbe, 0x74, 0x2d, 0x45, 0x1a, 0x49, - 0xea, 0xac, 0x46, 0x72, 0x56, 0xb9, 0x2c, 0x7e, 0xc7, 0xe5, 0xb2, 0xf4, 0x0a, 0xcb, 0xa5, 0xf4, - 0xc7, 0x25, 0x28, 0x6c, 0x3b, 0xa6, 0xfd, 0xfd, 0x47, 0xfc, 0x26, 0x14, 0x2c, 0xc7, 0x79, 0x3c, - 0x19, 0x87, 0xc5, 0xc4, 0x42, 0x9f, 0xdf, 0x8a, 0xa3, 0x98, 0x9e, 0x17, 0x9c, 0xa0, 0xa0, 0x2a, - 0xb0, 0xe4, 0x88, 0xcb, 0x90, 0x87, 0x74, 0xce, 0x2c, 0x77, 0x6c, 0x46, 0xc3, 0x1f, 0x42, 0x92, - 0x5d, 0xd8, 0x32, 0x3c, 0x97, 0x67, 0x9c, 0x8a, 0xf9, 0xa2, 0x73, 0x3c, 0xa6, 0x12, 0xcc, 0x21, - 0xaf, 0xbc, 0x43, 0x7f, 0x08, 0x6f, 0x4c, 0x1f, 0x9d, 0x18, 0x2e, 0x25, 0x8f, 0xe9, 0x71, 0x31, - 0x1d, 0x49, 0xb2, 0xf3, 0x53, 0x4e, 0x28, 0xbb, 0xf4, 0x1e, 0x3d, 0x3e, 0x33, 0xe1, 0x32, 0xdf, - 0x71, 0xc2, 0xc1, 0xab, 0xec, 0xcf, 0xef, 0xc3, 0xb9, 0x91, 0x61, 0xda, 0xbe, 0x61, 0xda, 0xc4, - 0x71, 0xfb, 0xd4, 0x35, 0xed, 0x21, 0xbf, 0x01, 0xc2, 0xb6, 0x1b, 0xb0, 0x35, 0xc9, 0x9d, 0xd1, - 0xaa, 0xf3, 0xcf, 0x6d, 0xd5, 0x55, 0x78, 0xdb, 0xa2, 0x03, 0x9f, 0xf0, 0x01, 0xef, 0xd0, 0xf4, - 0xf7, 0xc9, 0xd8, 0x30, 0x5d, 0xda, 0xe7, 0x04, 0xea, 0x4e, 0x35, 0xfa, 0x22, 0x93, 0x64, 0x81, - 0x7f, 0x60, 0xfa, 0xfb, 0x2d, 0x2e, 0xb6, 0xcd, 0xa5, 0xf0, 0x03, 0xb8, 0xee, 0x4c, 0xfc, 0xf1, - 0xc4, 0x27, 0x43, 0xd7, 0xe1, 0xe1, 0xb2, 0x7d, 0xd3, 0x9e, 0x18, 0xbe, 0xe9, 0xd8, 0x64, 0xe0, - 0xb8, 0x84, 0xef, 0xe1, 0x3a, 0x87, 0xc5, 0xe5, 0x88, 0xbe, 0x2b, 0x02, 0x53, 0x67, 0x90, 0x4a, - 0x04, 0x51, 0x73, 0xdc, 0x26, 0x1d, 0xf8, 0xba, 0x73, 0x88, 0xef, 0x41, 0x56, 0x66, 0x00, 0xcf, - 0x6a, 0x34, 0x77, 0x56, 0x83, 0x80, 0x33, 0xfa, 0x76, 0x32, 0x9d, 0x42, 0x8b, 0xdb, 0xc9, 0x74, - 0x0e, 0xe5, 0xd9, 0x28, 0x08, 0x6d, 0xc7, 0xf5, 0x65, 0x2d, 0xdf, 0x87, 0x65, 0x79, 0x80, 0xd0, - 0xd7, 0xa2, 0xaa, 0x4b, 0xb3, 0xf7, 0x0a, 0xfc, 0x2e, 0x77, 0x2a, 0x08, 0x05, 0xd1, 0x68, 0x04, - 0xba, 0xc8, 0xc8, 0xf0, 0x7b, 0xfb, 0xc4, 0xa2, 0xf6, 0x54, 0x8d, 0xa3, 0x80, 0xbf, 0xc3, 0xd8, - 0x4d, 0x6a, 0x97, 0x7e, 0x19, 0x87, 0x5c, 0xd5, 0xf4, 0x7c, 0xd3, 0xee, 0xf9, 0xdc, 0xae, 0x1b, - 0xb0, 0xcc, 0x85, 0x22, 0x57, 0x69, 0x8c, 0x5f, 0xa5, 0x05, 0x49, 0x0e, 0xe2, 0x78, 0x13, 0x50, - 0x5f, 0x02, 0x43, 0xc9, 0x38, 0x97, 0x5c, 0x0e, 0xe8, 0x81, 0xe8, 0x26, 0x60, 0x7b, 0x62, 0x59, - 0xa2, 0x98, 0x02, 0xe6, 0xd4, 0xfc, 0x87, 0x38, 0xbf, 0xec, 0xd2, 0xc0, 0x16, 0x7c, 0x1d, 0x72, - 0xd4, 0x75, 0x1d, 0x97, 0x38, 0x36, 0xe9, 0x4f, 0xc6, 0xbc, 0xbd, 0x64, 0x02, 0x17, 0x73, 0x8e, - 0x66, 0x57, 0x27, 0xe3, 0xb3, 0xfc, 0x98, 0x7a, 0x39, 0x3f, 0x96, 0x10, 0x14, 0x34, 0xb7, 0x6f, - 0xda, 0x06, 0x6b, 0x09, 0xcc, 0x29, 0xa5, 0x5f, 0x27, 0x00, 0x7d, 0x6c, 0x0e, 0x3f, 0x33, 0x86, - 0x22, 0xfd, 0xb8, 0xa7, 0xaa, 0xb0, 0xc8, 0x5b, 0x6a, 0x30, 0xc7, 0xcf, 0xd7, 0x8e, 0x25, 0x16, - 0xd7, 0x00, 0xe8, 0xc1, 0x94, 0x03, 0xb3, 0x9b, 0x57, 0x66, 0x9b, 0x2e, 0x5d, 0x1a, 0x0c, 0xb3, - 0xf4, 0xe0, 0x24, 0x1c, 0x05, 0xd1, 0xd7, 0x1d, 0x61, 0xfa, 0x54, 0xd3, 0xe6, 0x1c, 0x79, 0xa6, - 0x57, 0xd4, 0xb4, 0xef, 0x41, 0x6e, 0x60, 0x1e, 0xd1, 0x3e, 0x79, 0xc2, 0x9f, 0x37, 0xc5, 0x14, - 0xb7, 0xfc, 0x39, 0xbd, 0x77, 0xfa, 0x19, 0xa4, 0x67, 0x39, 0x5a, 0x10, 0x5f, 0xe2, 0x06, 0x28, - 0xfd, 0x39, 0x01, 0xcb, 0x3b, 0xd4, 0x1d, 0xd2, 0x48, 0x64, 0x76, 0x20, 0xcf, 0xcb, 0xff, 0x85, - 0x2b, 0x2b, 0xc7, 0xe0, 0x61, 0x5d, 0x69, 0x50, 0x70, 0xcd, 0xe1, 0x7e, 0x44, 0x5f, 0x7c, 0x4e, - 0x7d, 0x79, 0x8e, 0x0f, 0x15, 0x46, 0x02, 0x90, 0x7a, 0x1d, 0xb7, 0xe6, 0x4d, 0xc8, 0xb3, 0x7a, - 0x23, 0xf4, 0x60, 0x62, 0x84, 0x17, 0x67, 0x50, 0x8a, 0x39, 0xc6, 0x52, 0x24, 0x07, 0x7f, 0x04, - 0x97, 0xb8, 0x2b, 0x4f, 0x72, 0x74, 0xc6, 0x8d, 0x48, 0x07, 0xbe, 0x72, 0x30, 0x7d, 0x23, 0xfe, - 0x08, 0x8a, 0xc2, 0x6f, 0x67, 0x80, 0x33, 0x11, 0xf0, 0x05, 0x2e, 0x75, 0x0a, 0x5d, 0xfa, 0x7b, - 0x1c, 0x0a, 0x77, 0x0d, 0x6f, 0x3f, 0x12, 0xd7, 0x5b, 0xb0, 0x7c, 0xca, 0x18, 0xd1, 0x9b, 0xe4, - 0x64, 0x12, 0x35, 0x01, 0xdf, 0x06, 0x74, 0x7a, 0x73, 0xd1, 0x9e, 0xb8, 0x70, 0x61, 0x7a, 0xcb, - 0xd7, 0x1e, 0x91, 0xd7, 0xe6, 0xe6, 0xed, 0x64, 0x7a, 0x09, 0xa5, 0x4b, 0x9f, 0x27, 0x01, 0x37, - 0xe4, 0x07, 0x8e, 0x88, 0xc3, 0xbf, 0xa7, 0x81, 0x53, 0x83, 0x7c, 0xf0, 0x75, 0xe5, 0x45, 0xdb, - 0x52, 0x2e, 0x50, 0xc0, 0x23, 0xf1, 0xba, 0xc3, 0x79, 0xe6, 0x28, 0xb5, 0xf4, 0xdc, 0x51, 0xea, - 0xdb, 0x0f, 0x34, 0xe9, 0xf9, 0x06, 0x9a, 0x8f, 0xe0, 0xd2, 0xd8, 0xa5, 0x03, 0xf3, 0x28, 0x2c, - 0xf7, 0xb0, 0x1e, 0x32, 0x61, 0x3d, 0x5c, 0x14, 0x22, 0x41, 0xd9, 0xcb, 0x14, 0xd9, 0x4e, 0xa6, - 0x13, 0x28, 0x59, 0xfa, 0x3c, 0x01, 0x17, 0x82, 0xe4, 0x98, 0x7a, 0xe2, 0x6e, 0x00, 0x0a, 0xe3, - 0xd6, 0x73, 0x2c, 0x1e, 0xe1, 0x58, 0x24, 0xc2, 0x85, 0x80, 0x5b, 0x71, 0x2c, 0x16, 0xe7, 0xdd, - 0xd3, 0x71, 0x16, 0x7d, 0xf4, 0xdd, 0x53, 0xae, 0x0d, 0x64, 0x36, 0xd8, 0x2b, 0xf2, 0x24, 0x3c, - 0x2d, 0xd7, 0xf1, 0x9d, 0x33, 0xc3, 0xfd, 0x29, 0x9c, 0x1b, 0xbb, 0x94, 0x0c, 0xa4, 0x6d, 0xc4, - 0x1b, 0xd3, 0x1e, 0x0f, 0x5b, 0x76, 0xf3, 0xc7, 0xb3, 0x03, 0x7f, 0xd6, 0x91, 0x36, 0x5a, 0x2e, - 0x8d, 0xae, 0xf5, 0xe5, 0xf1, 0x34, 0x61, 0xe5, 0xe7, 0x31, 0x58, 0x3e, 0x25, 0x84, 0xb7, 0x01, - 0x4e, 0xbe, 0x07, 0xbe, 0xc0, 0x73, 0x3f, 0x82, 0xc6, 0x1b, 0x32, 0xeb, 0x84, 0x6b, 0x56, 0x4e, - 0x67, 0x1d, 0x1d, 0x6d, 0x88, 0x8f, 0x9b, 0x1d, 0x79, 0xff, 0xfd, 0x23, 0x03, 0x85, 0xf2, 0x70, - 0xe8, 0xd2, 0xa1, 0xe1, 0x3b, 0xc2, 0x9c, 0x2b, 0x00, 0x41, 0x0e, 0x59, 0xd1, 0xa6, 0x97, 0x19, - 0x8a, 0x34, 0xb1, 0x3c, 0xfc, 0x53, 0xc8, 0x19, 0x12, 0x64, 0x3a, 0xe1, 0xdb, 0xfe, 0xff, 0x67, - 0xdb, 0x3c, 0xbd, 0x45, 0xb8, 0x8c, 0x14, 0x60, 0x54, 0x1f, 0xfe, 0x5f, 0x39, 0x8a, 0xd2, 0x3e, - 0x89, 0x98, 0x92, 0x0c, 0x4d, 0x41, 0x92, 0x5b, 0x0f, 0x2d, 0xaa, 0xcb, 0x73, 0xa7, 0x78, 0xb5, - 0xbd, 0xf7, 0xad, 0x2d, 0x39, 0x5d, 0x7b, 0x2b, 0x3f, 0x8b, 0x43, 0x36, 0x62, 0x1e, 0x53, 0x3c, - 0x98, 0xd8, 0x3d, 0x1e, 0x96, 0x79, 0x14, 0xd7, 0x26, 0x76, 0x2f, 0x50, 0xcc, 0x14, 0xe0, 0x35, - 0x48, 0x87, 0xb3, 0x6b, 0x3c, 0x52, 0x83, 0x21, 0x15, 0x5f, 0x83, 0x82, 0xc8, 0xc1, 0xb0, 0x1a, - 0x58, 0x23, 0xcb, 0xeb, 0x39, 0x41, 0x95, 0x55, 0x70, 0x89, 0x7f, 0x00, 0xe6, 0xec, 0x14, 0x9f, - 0x97, 0x17, 0x7b, 0x82, 0x71, 0x17, 0x32, 0x86, 0x3b, 0x9c, 0x8c, 0xa8, 0xed, 0x7b, 0xc5, 0x45, - 0x1e, 0x91, 0x79, 0xb2, 0xe8, 0x04, 0x2c, 0xeb, 0xf6, 0x77, 0x29, 0x48, 0xb2, 0x53, 0x60, 0x04, - 0xb9, 0xb2, 0xfa, 0x88, 0xa8, 0x5a, 0x87, 0xa8, 0xdd, 0x66, 0x13, 0x2d, 0xe0, 0x25, 0x48, 0x94, - 0x77, 0xeb, 0x28, 0x86, 0x73, 0x90, 0xde, 0xd2, 0xb4, 0x26, 0x29, 0xab, 0x55, 0x14, 0xc7, 0x59, - 0x58, 0xe2, 0x2b, 0x4d, 0x47, 0x09, 0x5c, 0x00, 0xa8, 0x68, 0x6a, 0xa5, 0xdc, 0x21, 0xe5, 0x7a, - 0x1d, 0x25, 0x71, 0x06, 0x52, 0x15, 0xad, 0xab, 0x76, 0x50, 0x8a, 0xc1, 0x77, 0xca, 0x0f, 0xd1, - 0x12, 0xff, 0xa3, 0xa1, 0xa2, 0x34, 0x06, 0x58, 0x6c, 0x77, 0xaa, 0x55, 0x65, 0x17, 0x65, 0x18, - 0xb1, 0xdd, 0xdd, 0x41, 0xc0, 0xd4, 0xb5, 0xbb, 0x3b, 0xa4, 0xa1, 0x76, 0x50, 0x96, 0xed, 0xb4, - 0x5b, 0xd6, 0x1b, 0x65, 0xb5, 0xa2, 0xa0, 0x1c, 0x63, 0x3d, 0xd4, 0x74, 0xae, 0x39, 0x2f, 0x76, - 0xea, 0xaa, 0x1d, 0xa2, 0x6b, 0x0f, 0xda, 0xa8, 0xc0, 0x71, 0xf7, 0xf5, 0x6a, 0xa3, 0x56, 0x43, - 0xcb, 0x18, 0x43, 0xa1, 0xd6, 0x50, 0xcb, 0x4d, 0x12, 0xa2, 0x11, 0x3b, 0x90, 0xa0, 0xc9, 0x3d, - 0xcf, 0xe1, 0x3c, 0x64, 0xca, 0xba, 0x5e, 0x7e, 0xc4, 0x35, 0x62, 0xb6, 0xd9, 0x76, 0x5b, 0x53, - 0xf9, 0xea, 0x3c, 0x63, 0xb2, 0xd5, 0x16, 0x5f, 0x5e, 0x60, 0xdb, 0xb5, 0x3b, 0x7a, 0x43, 0xad, - 0xf3, 0xf5, 0x45, 0x7e, 0xea, 0x46, 0x87, 0xbb, 0xe0, 0x0d, 0x76, 0x10, 0xb6, 0xd0, 0x74, 0x74, - 0x09, 0xa7, 0x21, 0x59, 0xd1, 0x74, 0x1d, 0x15, 0x71, 0x11, 0x2e, 0xb4, 0x14, 0xbd, 0xa2, 0xa8, - 0x9d, 0x46, 0x53, 0x21, 0xd5, 0x46, 0xbb, 0x42, 0x1a, 0x3b, 0xad, 0x26, 0x7a, 0xf3, 0x14, 0xa7, - 0xa2, 0xa9, 0x1d, 0xc1, 0x59, 0xc1, 0xe7, 0x61, 0x99, 0xdb, 0xa0, 0x6d, 0x6d, 0x2b, 0x15, 0xe1, - 0xc4, 0xb7, 0xf0, 0x05, 0x40, 0xc2, 0x94, 0x08, 0xf5, 0x6d, 0x66, 0xc1, 0x6e, 0x59, 0x27, 0x2d, - 0xad, 0x85, 0xfe, 0x47, 0x98, 0xc7, 0x8e, 0xc5, 0xd7, 0xab, 0x78, 0x19, 0xb2, 0xed, 0x0e, 0xd9, - 0x29, 0xdf, 0x53, 0x9a, 0x0d, 0x55, 0x41, 0x97, 0xd9, 0x71, 0xda, 0x1d, 0xa2, 0x3c, 0xec, 0x28, - 0x6a, 0x07, 0xad, 0xb1, 0xb3, 0xb6, 0x3b, 0xa4, 0xab, 0x36, 0x34, 0x15, 0x5d, 0x11, 0x68, 0x52, - 0xd1, 0x9a, 0x4d, 0xa5, 0xd2, 0x41, 0x25, 0x26, 0x5c, 0xd1, 0x02, 0xe5, 0x57, 0x85, 0xab, 0xd9, - 0xb2, 0x5d, 0xde, 0x69, 0xa1, 0x6b, 0xcc, 0xbb, 0xba, 0x52, 0xd7, 0x59, 0x8c, 0xd8, 0x29, 0x5a, - 0x1d, 0xf4, 0x0e, 0xb3, 0x86, 0xd3, 0xf4, 0x4d, 0x74, 0x9d, 0x01, 0xf8, 0xa2, 0xdd, 0xd4, 0x5a, - 0x0a, 0xba, 0xc1, 0x76, 0x13, 0xeb, 0x87, 0x0f, 0xd1, 0xfa, 0xc9, 0xea, 0xd1, 0x23, 0x74, 0x33, - 0xc2, 0x7b, 0x84, 0x6e, 0x85, 0x48, 0x91, 0x34, 0xef, 0x32, 0x4b, 0xf8, 0xba, 0xbc, 0x5b, 0x7f, - 0x88, 0x6e, 0x47, 0x97, 0x8f, 0xd0, 0x7b, 0xa5, 0xdb, 0x90, 0x64, 0xa5, 0xcc, 0x7c, 0x5e, 0xee, - 0x76, 0x34, 0xb4, 0xc0, 0x53, 0xaa, 0x52, 0x6e, 0x96, 0x75, 0x14, 0x63, 0xba, 0x54, 0x4d, 0x25, - 0x72, 0x1d, 0x2f, 0xfd, 0x21, 0x06, 0x85, 0x96, 0xeb, 0x7c, 0x4a, 0x7b, 0x7e, 0x9b, 0x8a, 0x87, - 0xeb, 0x4f, 0x20, 0xc5, 0xda, 0x68, 0xf0, 0x1a, 0x9b, 0xa7, 0x72, 0x04, 0x10, 0xd7, 0xe1, 0xdc, - 0x90, 0xda, 0xd4, 0x35, 0xfc, 0xc8, 0xe3, 0x57, 0xbc, 0xc8, 0x9e, 0xd7, 0x87, 0x51, 0x08, 0x0a, - 0xa6, 0xc9, 0x1b, 0x80, 0xec, 0x09, 0xff, 0x1e, 0xe2, 0x91, 0x31, 0x75, 0xc9, 0x90, 0xda, 0xe2, - 0x35, 0xa6, 0xe7, 0xed, 0xc9, 0x88, 0xb5, 0xbb, 0x16, 0x75, 0xeb, 0xd4, 0x2e, 0x7d, 0x9d, 0x87, - 0xdc, 0x03, 0xd3, 0xee, 0x3b, 0x87, 0xf2, 0x26, 0x59, 0xe3, 0xdf, 0xc8, 0x7d, 0x93, 0x37, 0xd6, - 0x63, 0xf9, 0xf2, 0x8e, 0x92, 0x70, 0x1b, 0x32, 0x87, 0x1c, 0x51, 0x0b, 0x8d, 0xbb, 0x33, 0xfb, - 0xa8, 0x51, 0xe5, 0x72, 0x51, 0x0b, 0xfb, 0x45, 0xa8, 0x67, 0xe5, 0xf7, 0x31, 0xd9, 0x29, 0xda, - 0x90, 0x0f, 0xfa, 0x38, 0xad, 0xbd, 0x68, 0xd7, 0xd4, 0xa7, 0x75, 0xe0, 0xfb, 0x00, 0x72, 0x2b, - 0xa6, 0x31, 0xce, 0x35, 0xbe, 0x3f, 0x9f, 0xcd, 0x4c, 0x6b, 0x44, 0xc9, 0x47, 0xc9, 0xa7, 0x5f, - 0x5c, 0x8e, 0xad, 0x7c, 0xb1, 0x04, 0xa9, 0x9a, 0x6b, 0x8c, 0x28, 0xbe, 0x07, 0xc9, 0x91, 0xd3, - 0xa7, 0xd2, 0xdc, 0x6f, 0xab, 0x9c, 0x63, 0x37, 0x76, 0x9c, 0x7e, 0x78, 0x83, 0x30, 0x25, 0xf8, - 0x3e, 0x2c, 0xee, 0x39, 0x13, 0xbb, 0xef, 0xc9, 0x4b, 0xf8, 0xff, 0xe6, 0x52, 0xb7, 0xc5, 0xa1, - 0xc1, 0x2c, 0x29, 0x14, 0xe1, 0x8f, 0x21, 0x43, 0x8f, 0x7a, 0xd6, 0x84, 0x0f, 0x08, 0x09, 0x6e, - 0xe4, 0x07, 0x73, 0x69, 0x55, 0x02, 0x74, 0xf8, 0xf2, 0x0f, 0x08, 0x2b, 0xff, 0x8c, 0x41, 0x8a, - 0x6f, 0xca, 0x76, 0xe1, 0xfb, 0xb1, 0x42, 0x92, 0xae, 0xf8, 0x60, 0x7e, 0xdb, 0x23, 0x37, 0xea, - 0x89, 0x3a, 0x7c, 0x15, 0xc0, 0xb4, 0x7d, 0xe2, 0x0c, 0x06, 0x1e, 0x15, 0xf7, 0x5f, 0xf0, 0x73, - 0x55, 0xc6, 0xb4, 0x7d, 0x8d, 0x93, 0xf1, 0x15, 0xc8, 0xb1, 0xaa, 0xe8, 0x07, 0x62, 0xec, 0xa4, - 0x39, 0x3d, 0xcb, 0x69, 0x52, 0x64, 0x1b, 0xb2, 0x82, 0xc9, 0x7f, 0xdc, 0x95, 0x93, 0xfe, 0x1c, - 0x3f, 0x81, 0x82, 0x40, 0x33, 0x9b, 0x56, 0x7e, 0x13, 0x83, 0x45, 0xe1, 0x6e, 0xac, 0x42, 0xca, - 0xf3, 0x0d, 0xd7, 0x97, 0xd3, 0xd7, 0xe6, 0xfc, 0xc7, 0x0e, 0x7f, 0xae, 0x60, 0x6a, 0x70, 0x15, - 0x12, 0xd4, 0xee, 0xcb, 0x04, 0x78, 0x01, 0x6d, 0x3a, 0x83, 0x97, 0x6e, 0x40, 0x92, 0x65, 0x17, - 0xbb, 0x3a, 0xf5, 0xb2, 0x5a, 0x57, 0xd0, 0x02, 0xeb, 0x6f, 0xfc, 0x96, 0x8b, 0xb1, 0xfe, 0x56, - 0xd7, 0xb5, 0x6e, 0xab, 0x8d, 0xe2, 0xa5, 0xcf, 0x20, 0x13, 0xfa, 0x1e, 0x5f, 0x82, 0xf3, 0x5d, - 0x75, 0x4b, 0xeb, 0xaa, 0x55, 0xa5, 0x4a, 0x5a, 0xba, 0x52, 0x51, 0xaa, 0x0d, 0xb5, 0x8e, 0x16, - 0xa6, 0x19, 0x35, 0xad, 0xd9, 0xd4, 0x1e, 0x30, 0x46, 0x8c, 0xdd, 0x2a, 0x5a, 0xad, 0xd6, 0x56, - 0x3a, 0x11, 0xf1, 0x78, 0x84, 0x7a, 0x22, 0x9b, 0x60, 0xd7, 0x49, 0xa5, 0xab, 0xeb, 0x8a, 0xb8, - 0x6e, 0x51, 0xb2, 0xf4, 0x09, 0x64, 0xc2, 0xec, 0x62, 0x37, 0xab, 0xaa, 0x11, 0xe5, 0x61, 0xa5, - 0xd9, 0x6d, 0xb3, 0x0b, 0x85, 0x6f, 0xca, 0x97, 0x55, 0x85, 0x44, 0x71, 0x31, 0x7c, 0x0e, 0xf2, - 0x01, 0x83, 0x9f, 0x03, 0xc5, 0x19, 0x3a, 0x20, 0x75, 0x1a, 0x4a, 0x1b, 0x25, 0x56, 0xfe, 0x12, - 0x87, 0x74, 0xd0, 0x77, 0xb0, 0x12, 0x19, 0xc5, 0xfe, 0x73, 0xec, 0x9f, 0xe9, 0xd5, 0xd3, 0x83, - 0x58, 0x15, 0xd2, 0xe1, 0xa3, 0x2a, 0x39, 0xe7, 0x97, 0x98, 0x10, 0xc9, 0xde, 0xb6, 0x03, 0x16, - 0x2f, 0xf9, 0x42, 0xbc, 0x3d, 0x4f, 0x8c, 0x75, 0x01, 0xc5, 0xeb, 0x30, 0x35, 0xda, 0xf1, 0x37, - 0x47, 0x2a, 0x18, 0x88, 0xa7, 0x86, 0xbe, 0x15, 0x48, 0x1b, 0xee, 0xd0, 0x6b, 0xf4, 0x8f, 0xbc, - 0xe2, 0x12, 0xef, 0xea, 0xe1, 0x9a, 0x69, 0x11, 0xcf, 0x38, 0xa9, 0x25, 0x1d, 0x79, 0x42, 0x4d, - 0x71, 0xb6, 0x93, 0xe9, 0x38, 0x4a, 0xc8, 0xe9, 0xee, 0xb7, 0x31, 0x80, 0x93, 0xee, 0xc8, 0x6f, - 0x5b, 0xed, 0x01, 0x51, 0xbb, 0x3b, 0x5b, 0x8a, 0x2e, 0xf3, 0xac, 0xac, 0xde, 0x13, 0x77, 0x67, - 0x55, 0x51, 0xdb, 0x0a, 0xe1, 0x6b, 0x1e, 0x24, 0x39, 0xb1, 0x08, 0x4a, 0x82, 0xcf, 0x08, 0xdd, - 0x1d, 0x3e, 0xd7, 0x74, 0xc4, 0xa0, 0xc7, 0xa7, 0x19, 0x31, 0xe8, 0x35, 0xcb, 0x75, 0xb4, 0xc8, - 0xd4, 0x35, 0x95, 0x72, 0x15, 0x2d, 0xb1, 0xfc, 0xa9, 0x35, 0xf4, 0x76, 0x87, 0xec, 0x96, 0x9b, - 0x5d, 0x05, 0xa5, 0x99, 0xfe, 0x66, 0x39, 0x5c, 0x67, 0x98, 0x36, 0xb5, 0x73, 0x57, 0x2e, 0xe1, - 0xd6, 0x0f, 0xa0, 0x30, 0xfd, 0xb3, 0x0b, 0x4b, 0xfc, 0x56, 0x77, 0xab, 0xd9, 0xa8, 0xa0, 0x05, - 0xfc, 0x26, 0x5c, 0x14, 0x7f, 0xb3, 0xf1, 0x8b, 0x4f, 0xa8, 0x92, 0x15, 0xdb, 0x7a, 0xef, 0xe9, - 0xd7, 0xab, 0x0b, 0x4f, 0x9f, 0xad, 0xc6, 0xbe, 0x7c, 0xb6, 0x1a, 0xfb, 0xea, 0xd9, 0x6a, 0xec, - 0x6f, 0xcf, 0x56, 0x63, 0xbf, 0xf8, 0x66, 0x75, 0xe1, 0xcb, 0x6f, 0x56, 0x17, 0xbe, 0xfa, 0x66, - 0x75, 0xe1, 0xe3, 0x6c, 0xe4, 0xbf, 0x6e, 0xfc, 0x3b, 0x00, 0x00, 0xff, 0xff, 0xac, 0x86, 0xc6, - 0xbb, 0xaa, 0x22, 0x00, 0x00, + proto.RegisterFile("sql/execinfrapb/processors_sql.proto", fileDescriptor_processors_sql_6b6703e0ac4ff0b7) +} + +var fileDescriptor_processors_sql_6b6703e0ac4ff0b7 = []byte{ + // 2954 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x3a, 0x4b, 0x6f, 0x1b, 0xd7, + 0xb9, 0xe2, 0x4b, 0x22, 0x3f, 0x3e, 0x74, 0x7c, 0x6c, 0xc7, 0x8c, 0x92, 0x2b, 0xcb, 0xb4, 0x63, + 0xcb, 0x8e, 0x23, 0xdf, 0xe8, 0x5e, 0xe4, 0x22, 0xb9, 0x45, 0x51, 0x8a, 0x1c, 0xd2, 0x94, 0xa9, + 0x19, 0x7a, 0x48, 0xca, 0x76, 0x02, 0xf4, 0x60, 0x44, 0x1e, 0x52, 0x13, 0x0f, 0x67, 0xa8, 0x99, + 0xa1, 0x25, 0xe5, 0x0f, 0x74, 0x55, 0xa0, 0x45, 0x37, 0x45, 0x16, 0x45, 0x36, 0xdd, 0x74, 0xdd, + 0x9f, 0x50, 0x14, 0xde, 0x14, 0x08, 0xba, 0x68, 0xb3, 0x2a, 0x1a, 0x67, 0xd7, 0x3f, 0x50, 0x74, + 0x57, 0x9c, 0xc7, 0x8c, 0x86, 0x8a, 0xe8, 0x86, 0xb6, 0x13, 0x6f, 0x0c, 0x9d, 0xef, 0x75, 0xbe, + 0xf7, 0xf7, 0x9d, 0xa1, 0xe1, 0x9a, 0x77, 0x60, 0xdd, 0xa1, 0x47, 0xb4, 0x67, 0xda, 0x03, 0xd7, + 0x18, 0xef, 0xdd, 0x19, 0xbb, 0x4e, 0x8f, 0x7a, 0x9e, 0xe3, 0x7a, 0xc4, 0x3b, 0xb0, 0x36, 0xc6, + 0xae, 0xe3, 0x3b, 0xb8, 0xd8, 0x73, 0x7a, 0x8f, 0x5d, 0xc7, 0xe8, 0xed, 0x6f, 0x30, 0x60, 0xdf, + 0xf4, 0x7c, 0xef, 0xc0, 0x72, 0x27, 0xf6, 0xca, 0x55, 0xc6, 0xdf, 0x33, 0x7c, 0xc3, 0x72, 0x86, + 0x77, 0xfa, 0xd4, 0xeb, 0x8d, 0xf7, 0xee, 0x78, 0xbe, 0x3b, 0xe9, 0xf9, 0x13, 0x97, 0xf6, 0x05, + 0xfb, 0x4a, 0xe9, 0x0c, 0xa2, 0x4f, 0x1d, 0xd3, 0x26, 0xfe, 0xf1, 0x98, 0x4a, 0x9a, 0xb5, 0x33, + 0x68, 0x2c, 0xa7, 0xf7, 0xd8, 0xb4, 0x87, 0x92, 0xe2, 0x22, 0xa3, 0x60, 0x1c, 0x9e, 0xf8, 0x57, + 0x82, 0x57, 0x4e, 0x5b, 0xd0, 0x37, 0x7c, 0x43, 0xe2, 0xde, 0x79, 0x8e, 0x75, 0x7b, 0x86, 0x47, + 0xa3, 0xfa, 0x99, 0xf6, 0x13, 0xea, 0xfa, 0xb4, 0x7f, 0xc7, 0x1b, 0x1b, 0x36, 0xa1, 0x47, 0x63, + 0x97, 0x7a, 0x9e, 0xe9, 0xd8, 0x92, 0xe6, 0xc2, 0xd0, 0x19, 0x3a, 0xfc, 0xcf, 0x3b, 0xec, 0x2f, + 0x01, 0x2d, 0xfd, 0x2a, 0x06, 0x85, 0x5d, 0xc3, 0x9a, 0x50, 0xaf, 0xe2, 0xb8, 0xb4, 0x3d, 0xa6, + 0x3d, 0x5c, 0x81, 0xa5, 0x9e, 0x63, 0x4d, 0x46, 0xb6, 0x57, 0x8c, 0xad, 0x25, 0xd6, 0xb3, 0x9b, + 0x57, 0x37, 0x66, 0x79, 0x6f, 0xa3, 0x6a, 0xf8, 0x93, 0x51, 0xc3, 0x1e, 0x38, 0x5b, 0xc9, 0xa7, + 0x7f, 0xbb, 0xbc, 0xa0, 0x07, 0x9c, 0xf8, 0x2d, 0xc8, 0xb8, 0xc6, 0x21, 0xd9, 0x3b, 0xf6, 0xa9, + 0x57, 0x8c, 0xaf, 0x25, 0xd6, 0x73, 0x7a, 0xda, 0x35, 0x0e, 0xb7, 0xd8, 0x19, 0x5f, 0x86, 0xb4, + 0x3d, 0x19, 0x11, 0xd7, 0x39, 0xf4, 0x8a, 0x89, 0xb5, 0xd8, 0x7a, 0x32, 0xe0, 0xb6, 0x27, 0x23, + 0xdd, 0x39, 0xf4, 0x4a, 0x7f, 0x5e, 0x84, 0xe5, 0x8e, 0xb1, 0x67, 0x51, 0x9d, 0x1a, 0x7d, 0xea, + 0x72, 0xb5, 0xb6, 0x20, 0xe5, 0x33, 0x50, 0x31, 0xb6, 0x16, 0x5b, 0xcf, 0x6e, 0x5e, 0x3f, 0xa5, + 0x94, 0x77, 0x60, 0x71, 0x87, 0x70, 0xb6, 0x2a, 0xf5, 0x7a, 0xae, 0x39, 0xf6, 0x1d, 0x57, 0x4a, + 0x16, 0xac, 0xf8, 0x0a, 0x64, 0x4c, 0xbb, 0x4f, 0x8f, 0x88, 0xd9, 0x3f, 0x2a, 0xc6, 0xd7, 0x62, + 0xeb, 0x79, 0x89, 0x4f, 0x73, 0x70, 0xa3, 0x7f, 0x84, 0x57, 0x61, 0xc9, 0xa5, 0x4f, 0xa8, 0xeb, + 0x51, 0xae, 0x5a, 0x3a, 0x50, 0x4d, 0x02, 0xb1, 0x02, 0x29, 0xe6, 0x5f, 0xaf, 0x98, 0xe4, 0xbe, + 0xb9, 0x39, 0xdb, 0x37, 0x53, 0x06, 0x18, 0x76, 0xa0, 0x09, 0xe7, 0xc6, 0x57, 0x01, 0x2c, 0x73, + 0x64, 0xfa, 0x64, 0xdf, 0xb4, 0xfd, 0x62, 0x6a, 0x2d, 0xb6, 0x9e, 0x90, 0x04, 0x19, 0x0e, 0xbf, + 0x6b, 0xda, 0x3e, 0xf3, 0x93, 0xe9, 0x91, 0xde, 0x3e, 0xed, 0x3d, 0x2e, 0x2e, 0x46, 0x95, 0x31, + 0xbd, 0x0a, 0x03, 0x62, 0x15, 0xe0, 0x89, 0xe9, 0x99, 0x7b, 0xa6, 0x65, 0xfa, 0xc7, 0xc5, 0xa5, + 0xb5, 0xd8, 0x7a, 0x61, 0x73, 0x7d, 0xb6, 0x46, 0xed, 0x9e, 0x61, 0xef, 0x86, 0xf4, 0x52, 0x58, + 0x44, 0x02, 0xfe, 0x7f, 0xb8, 0x34, 0x32, 0x8e, 0x88, 0x6f, 0x8e, 0xa8, 0xe7, 0x1b, 0xa3, 0x31, + 0x31, 0x86, 0x94, 0xd8, 0x86, 0xed, 0x78, 0xc5, 0x4c, 0x24, 0x4e, 0x17, 0x46, 0xc6, 0x51, 0x27, + 0xa0, 0x29, 0x0f, 0xa9, 0xca, 0x28, 0xf0, 0x27, 0x80, 0x64, 0xbe, 0x13, 0xcf, 0x77, 0xa9, 0x3d, + 0xf4, 0xf7, 0x8b, 0xc0, 0x55, 0xba, 0x35, 0x23, 0x56, 0x4c, 0x9f, 0xa6, 0x60, 0x69, 0x4b, 0x0e, + 0x79, 0xc3, 0xb2, 0x35, 0x0d, 0xc6, 0x7b, 0x70, 0x3e, 0x10, 0x7e, 0x68, 0x98, 0x3e, 0x19, 0x3b, + 0x96, 0xd9, 0x3b, 0x2e, 0x66, 0xb9, 0xfc, 0xdb, 0xff, 0x59, 0xfe, 0x03, 0xc3, 0xf4, 0x5b, 0x9c, + 0x47, 0xde, 0x70, 0xce, 0x3a, 0x8d, 0xc0, 0xd7, 0x21, 0x3b, 0x36, 0x5c, 0xc3, 0xb2, 0xa8, 0x65, + 0x7e, 0x46, 0x8b, 0xb9, 0x88, 0xc7, 0xa3, 0x08, 0xbc, 0x09, 0x78, 0xdf, 0xf0, 0x88, 0x77, 0xec, + 0xf9, 0x74, 0x44, 0x82, 0x5a, 0x29, 0x44, 0xc8, 0xd1, 0xbe, 0xe1, 0xb5, 0x39, 0xba, 0x22, 0xeb, + 0xe1, 0x1d, 0x28, 0xd8, 0x94, 0xf6, 0x69, 0x3f, 0xa4, 0x5f, 0x5e, 0x4b, 0xac, 0xe7, 0xf5, 0xbc, + 0x80, 0x06, 0x64, 0x2a, 0x14, 0x9e, 0x98, 0xae, 0x3f, 0x31, 0x2c, 0x49, 0x57, 0x44, 0x3c, 0xdb, + 0x6f, 0xcc, 0xb0, 0x50, 0xf0, 0x9d, 0xa4, 0xbb, 0x9e, 0x97, 0xec, 0x02, 0xb1, 0x9d, 0x4c, 0xa7, + 0x51, 0x66, 0x3b, 0x99, 0xce, 0xa3, 0x42, 0x49, 0x87, 0x5c, 0xcd, 0xb4, 0x7c, 0xea, 0x86, 0x05, + 0xb5, 0x38, 0xe0, 0x67, 0x59, 0x51, 0xd7, 0x66, 0x27, 0x8e, 0x12, 0x36, 0x13, 0x69, 0xa0, 0xe4, + 0x2c, 0xfd, 0x2b, 0x01, 0xc5, 0x06, 0x2b, 0x9d, 0xf6, 0x63, 0x73, 0xfc, 0x9a, 0x2a, 0x36, 0xac, + 0xc8, 0xc4, 0x4b, 0x55, 0xe4, 0x74, 0x2d, 0x25, 0x5f, 0xba, 0x96, 0x22, 0x8d, 0x24, 0x75, 0x56, + 0x23, 0x39, 0xab, 0x5c, 0x16, 0xbf, 0xe7, 0x72, 0x59, 0x7a, 0x85, 0xe5, 0x52, 0xfa, 0xd3, 0x12, + 0x14, 0xb6, 0x1d, 0xd3, 0xfe, 0xe1, 0x23, 0x7e, 0x13, 0x0a, 0x96, 0xe3, 0x3c, 0x9e, 0x8c, 0xc3, + 0x62, 0x62, 0xa1, 0xcf, 0x6f, 0xc5, 0x51, 0x4c, 0xcf, 0x0b, 0x4c, 0x50, 0x50, 0x15, 0x58, 0x72, + 0xc4, 0x30, 0xe4, 0x21, 0x9d, 0x33, 0xcb, 0x1d, 0x9b, 0xc1, 0xf0, 0x87, 0x90, 0x64, 0x03, 0x5b, + 0x86, 0xe7, 0xf2, 0x0c, 0xab, 0x98, 0x2f, 0x3a, 0xc7, 0x63, 0x2a, 0x99, 0x39, 0xcb, 0x2b, 0xef, + 0xd0, 0x1f, 0xc2, 0x1b, 0xd3, 0xa6, 0x13, 0xc3, 0xa5, 0xe4, 0x31, 0x3d, 0x2e, 0xa6, 0x23, 0x49, + 0x76, 0x7e, 0xca, 0x09, 0x65, 0x97, 0xde, 0xa3, 0xc7, 0x67, 0x26, 0x5c, 0xe6, 0x7b, 0x4e, 0x38, + 0x78, 0x95, 0xfd, 0xf9, 0x7d, 0x38, 0x37, 0x32, 0x4c, 0xdb, 0x37, 0x4c, 0x9b, 0x38, 0x6e, 0x9f, + 0xba, 0xa6, 0x3d, 0xe4, 0x13, 0x20, 0x6c, 0xbb, 0x01, 0x5a, 0x93, 0xd8, 0x19, 0xad, 0x3a, 0xff, + 0xdc, 0x56, 0x5d, 0x85, 0xb7, 0x2d, 0x3a, 0xf0, 0x09, 0x5f, 0xf0, 0x0e, 0x4d, 0x7f, 0x9f, 0x8c, + 0x0d, 0xd3, 0xa5, 0x7d, 0x0e, 0xa0, 0xee, 0x54, 0xa3, 0x2f, 0x32, 0x4a, 0x16, 0xf8, 0x07, 0xa6, + 0xbf, 0xdf, 0xe2, 0x64, 0xdb, 0x9c, 0x0a, 0x3f, 0x80, 0xeb, 0xce, 0xc4, 0x1f, 0x4f, 0x7c, 0x32, + 0x74, 0x1d, 0x1e, 0x2e, 0xdb, 0x37, 0xed, 0x89, 0xe1, 0x9b, 0x8e, 0x4d, 0x06, 0x8e, 0x4b, 0xf8, + 0x1d, 0xae, 0x73, 0x58, 0x5c, 0x8e, 0xc8, 0xbb, 0x22, 0x78, 0xea, 0x8c, 0xa5, 0x12, 0xe1, 0xa8, + 0x39, 0x6e, 0x93, 0x0e, 0x7c, 0xdd, 0x39, 0xc4, 0xf7, 0x20, 0x2b, 0x33, 0x80, 0x67, 0x35, 0x9a, + 0x3b, 0xab, 0x41, 0xb0, 0x33, 0xf8, 0x76, 0x32, 0x9d, 0x42, 0x8b, 0xdb, 0xc9, 0x74, 0x0e, 0xe5, + 0xd9, 0x2a, 0x08, 0x6d, 0xc7, 0xf5, 0x65, 0x2d, 0xdf, 0x87, 0x65, 0x69, 0x40, 0xe8, 0x6b, 0x51, + 0xd5, 0xa5, 0xd9, 0x77, 0x05, 0x7e, 0x97, 0x37, 0x15, 0x84, 0x80, 0x68, 0x34, 0x02, 0x59, 0x64, + 0x64, 0xf8, 0xbd, 0x7d, 0x62, 0x51, 0x7b, 0xaa, 0xc6, 0x51, 0x80, 0xdf, 0x61, 0xe8, 0x26, 0xb5, + 0x4b, 0xbf, 0x8c, 0x43, 0xae, 0x6a, 0x7a, 0xbe, 0x69, 0xf7, 0x7c, 0xae, 0xd7, 0x0d, 0x58, 0xe6, + 0x44, 0x91, 0x51, 0x1a, 0xe3, 0xa3, 0xb4, 0x20, 0xc1, 0x41, 0x1c, 0x6f, 0x02, 0xea, 0x4b, 0xc6, + 0x90, 0x32, 0xce, 0x29, 0x97, 0x03, 0x78, 0x40, 0xba, 0x09, 0xd8, 0x9e, 0x58, 0x96, 0x28, 0xa6, + 0x00, 0x39, 0xb5, 0xff, 0x21, 0x8e, 0x2f, 0xbb, 0x34, 0xd0, 0x05, 0x5f, 0x87, 0x1c, 0x75, 0x5d, + 0xc7, 0x25, 0x8e, 0x4d, 0xfa, 0x93, 0x31, 0x6f, 0x2f, 0x99, 0xc0, 0xc5, 0x1c, 0xa3, 0xd9, 0xd5, + 0xc9, 0xf8, 0x2c, 0x3f, 0xa6, 0x5e, 0xce, 0x8f, 0x25, 0x04, 0x05, 0xcd, 0xed, 0x9b, 0xb6, 0xc1, + 0x5a, 0x02, 0x73, 0x4a, 0xe9, 0xd7, 0x09, 0x40, 0x1f, 0x9b, 0xc3, 0xcf, 0x8c, 0xa1, 0x48, 0x3f, + 0xee, 0xa9, 0x2a, 0x2c, 0xf2, 0x96, 0x1a, 0xec, 0xf1, 0xf3, 0xb5, 0x63, 0xc9, 0x8b, 0x6b, 0x00, + 0xf4, 0x60, 0xca, 0x81, 0xd9, 0xcd, 0x2b, 0xb3, 0x55, 0x97, 0x2e, 0x0d, 0x96, 0x59, 0x7a, 0x70, + 0x12, 0x8e, 0x82, 0xe8, 0xeb, 0x8e, 0x50, 0x7d, 0xaa, 0x69, 0x73, 0x8c, 0xb4, 0xe9, 0x15, 0x35, + 0xed, 0x7b, 0x90, 0x1b, 0x98, 0x47, 0xb4, 0x4f, 0x9e, 0xf0, 0xe7, 0x4d, 0x31, 0xc5, 0x35, 0x7f, + 0x4e, 0xef, 0x9d, 0x7e, 0x06, 0xe9, 0x59, 0xce, 0x2d, 0x80, 0x2f, 0x31, 0x01, 0x4a, 0x7f, 0x49, + 0xc0, 0xf2, 0x0e, 0x75, 0x87, 0x34, 0x12, 0x99, 0x1d, 0xc8, 0xf3, 0xf2, 0x7f, 0xe1, 0xca, 0xca, + 0x31, 0xf6, 0xb0, 0xae, 0x34, 0x28, 0xb8, 0xe6, 0x70, 0x3f, 0x22, 0x2f, 0x3e, 0xa7, 0xbc, 0x3c, + 0xe7, 0x0f, 0x05, 0x46, 0x02, 0x90, 0x7a, 0x1d, 0x53, 0xf3, 0x26, 0xe4, 0x59, 0xbd, 0x11, 0x7a, + 0x30, 0x31, 0xc2, 0xc1, 0x19, 0x94, 0x62, 0x8e, 0xa1, 0x14, 0x89, 0xc1, 0x1f, 0xc1, 0x25, 0xee, + 0xca, 0x93, 0x1c, 0x9d, 0x31, 0x11, 0xe9, 0xc0, 0x57, 0x0e, 0xa6, 0x27, 0xe2, 0x8f, 0xa0, 0x28, + 0xfc, 0x76, 0x06, 0x73, 0x26, 0xc2, 0x7c, 0x81, 0x53, 0x9d, 0xe2, 0x2e, 0xfd, 0x23, 0x0e, 0x85, + 0xbb, 0x86, 0xb7, 0x1f, 0x89, 0xeb, 0x2d, 0x58, 0x3e, 0xa5, 0x8c, 0xe8, 0x4d, 0x72, 0x33, 0x89, + 0xaa, 0x80, 0x6f, 0x03, 0x3a, 0x7d, 0xb9, 0x68, 0x4f, 0x9c, 0xb8, 0x30, 0x7d, 0xe5, 0x6b, 0x8f, + 0xc8, 0x6b, 0x73, 0xf3, 0x76, 0x32, 0xbd, 0x84, 0xd2, 0xa5, 0xcf, 0x93, 0x80, 0x1b, 0xf2, 0x03, + 0x47, 0xc4, 0xe1, 0x3f, 0xd0, 0xc2, 0xa9, 0x41, 0x3e, 0xf8, 0xba, 0xf2, 0xa2, 0x6d, 0x29, 0x17, + 0x08, 0xe0, 0x91, 0x78, 0xdd, 0xe1, 0x3c, 0x73, 0x95, 0x5a, 0x7a, 0xee, 0x2a, 0xf5, 0xdd, 0x17, + 0x9a, 0xf4, 0x7c, 0x0b, 0xcd, 0x47, 0x70, 0x69, 0xec, 0xd2, 0x81, 0x79, 0x14, 0x96, 0x7b, 0x58, + 0x0f, 0x99, 0xb0, 0x1e, 0x2e, 0x0a, 0x92, 0xa0, 0xec, 0x65, 0x8a, 0x6c, 0x27, 0xd3, 0x09, 0x94, + 0x2c, 0x7d, 0x9e, 0x80, 0x0b, 0x41, 0x72, 0x4c, 0x3d, 0x71, 0x37, 0x00, 0x85, 0x71, 0xeb, 0x39, + 0x16, 0x8f, 0x70, 0x2c, 0x12, 0xe1, 0x42, 0x80, 0xad, 0x38, 0x16, 0x8b, 0xf3, 0xee, 0xe9, 0x38, + 0x8b, 0x3e, 0xfa, 0xee, 0x29, 0xd7, 0x06, 0x34, 0x1b, 0xec, 0x15, 0x79, 0x12, 0x9e, 0x96, 0xeb, + 0xf8, 0xce, 0x99, 0xe1, 0xfe, 0x14, 0xce, 0x8d, 0x5d, 0x4a, 0x06, 0x52, 0x37, 0xe2, 0x8d, 0x69, + 0x8f, 0x87, 0x2d, 0xbb, 0xf9, 0xe3, 0xd9, 0x81, 0x3f, 0xcb, 0xa4, 0x8d, 0x96, 0x4b, 0xa3, 0x67, + 0x7d, 0x79, 0x3c, 0x0d, 0x58, 0xf9, 0x79, 0x0c, 0x96, 0x4f, 0x11, 0xe1, 0x6d, 0x80, 0x93, 0xef, + 0x81, 0x2f, 0xf0, 0xdc, 0x8f, 0x70, 0xe3, 0x0d, 0x99, 0x75, 0xc2, 0x35, 0x2b, 0xa7, 0xb3, 0x8e, + 0x8e, 0x36, 0xc4, 0xc7, 0xcd, 0x8e, 0x9c, 0x7f, 0x7f, 0x04, 0x28, 0x94, 0x87, 0x43, 0x97, 0x0e, + 0x0d, 0xdf, 0x11, 0xea, 0x5c, 0x01, 0x08, 0x72, 0xc8, 0x8a, 0x36, 0xbd, 0xcc, 0x50, 0xa4, 0x89, + 0xe5, 0xe1, 0x9f, 0x42, 0xce, 0x90, 0x4c, 0xa6, 0x13, 0xbe, 0xed, 0xff, 0x77, 0xb6, 0xce, 0xd3, + 0x57, 0x84, 0xc7, 0x48, 0x01, 0x46, 0xe5, 0xe1, 0xff, 0x96, 0xab, 0x28, 0xed, 0x93, 0x88, 0x2a, + 0xc9, 0x50, 0x15, 0x24, 0xb1, 0xf5, 0x50, 0xa3, 0xba, 0xb4, 0x3b, 0xc5, 0xab, 0xed, 0xbd, 0xef, + 0xac, 0xc9, 0xb7, 0x6a, 0xef, 0x8c, 0x85, 0x70, 0xf1, 0xe5, 0x16, 0xc2, 0x95, 0x9f, 0xc5, 0x21, + 0x1b, 0xb1, 0x98, 0xe9, 0x3a, 0x98, 0xd8, 0x3d, 0x1e, 0xe9, 0x79, 0x74, 0xad, 0x4d, 0xec, 0x5e, + 0xa0, 0x2b, 0x13, 0x80, 0xd7, 0x20, 0x1d, 0xae, 0xc3, 0xf1, 0x48, 0x59, 0x87, 0x50, 0x7c, 0x0d, + 0x0a, 0x22, 0xad, 0xc3, 0x02, 0x63, 0xbd, 0x31, 0xaf, 0xe7, 0x04, 0x54, 0x16, 0xd6, 0x25, 0xfe, + 0x4d, 0x99, 0xa3, 0x53, 0x7c, 0x05, 0x5f, 0xec, 0x09, 0xc4, 0x5d, 0xc8, 0x18, 0xee, 0x70, 0x32, + 0xa2, 0xb6, 0xef, 0x15, 0x17, 0x79, 0x90, 0xe7, 0x49, 0xcc, 0x13, 0x66, 0xd9, 0x0a, 0x7e, 0x97, + 0x82, 0x24, 0xb3, 0x02, 0x23, 0xc8, 0x95, 0xd5, 0x47, 0x44, 0xd5, 0x3a, 0x44, 0xed, 0x36, 0x9b, + 0x68, 0x01, 0x2f, 0x41, 0xa2, 0xbc, 0x5b, 0x47, 0x31, 0x9c, 0x83, 0xf4, 0x96, 0xa6, 0x35, 0x49, + 0x59, 0xad, 0xa2, 0x38, 0xce, 0xc2, 0x12, 0x3f, 0x69, 0x3a, 0x4a, 0xe0, 0x02, 0x40, 0x45, 0x53, + 0x2b, 0xe5, 0x0e, 0x29, 0xd7, 0xeb, 0x28, 0x89, 0x33, 0x90, 0xaa, 0x68, 0x5d, 0xb5, 0x83, 0x52, + 0x8c, 0x7d, 0xa7, 0xfc, 0x10, 0x2d, 0xf1, 0x3f, 0x1a, 0x2a, 0x4a, 0x63, 0x80, 0xc5, 0x76, 0xa7, + 0x5a, 0x55, 0x76, 0x51, 0x86, 0x01, 0xdb, 0xdd, 0x1d, 0x04, 0x4c, 0x5c, 0xbb, 0xbb, 0x43, 0x1a, + 0x6a, 0x07, 0x65, 0xd9, 0x4d, 0xbb, 0x65, 0xbd, 0x51, 0x56, 0x2b, 0x0a, 0xca, 0x31, 0xd4, 0x43, + 0x4d, 0xe7, 0x92, 0xf3, 0xe2, 0xa6, 0xae, 0xda, 0x21, 0xba, 0xf6, 0xa0, 0x8d, 0x0a, 0x9c, 0xef, + 0xbe, 0x5e, 0x6d, 0xd4, 0x6a, 0x68, 0x19, 0x63, 0x28, 0xd4, 0x1a, 0x6a, 0xb9, 0x49, 0x42, 0x6e, + 0xc4, 0x0c, 0x12, 0x30, 0x79, 0xe7, 0x39, 0x9c, 0x87, 0x4c, 0x59, 0xd7, 0xcb, 0x8f, 0xb8, 0x44, + 0xcc, 0x2e, 0xdb, 0x6e, 0x6b, 0x2a, 0x3f, 0x9d, 0x67, 0x48, 0x76, 0xda, 0xe2, 0xc7, 0x0b, 0xec, + 0xba, 0x76, 0x47, 0x6f, 0xa8, 0x75, 0x7e, 0xbe, 0xc8, 0xad, 0x6e, 0x74, 0xb8, 0x0b, 0xde, 0x60, + 0x86, 0xb0, 0x83, 0xa6, 0xa3, 0x4b, 0x38, 0x0d, 0xc9, 0x8a, 0xa6, 0xeb, 0xa8, 0x88, 0x8b, 0x70, + 0xa1, 0xa5, 0xe8, 0x15, 0x45, 0xed, 0x34, 0x9a, 0x0a, 0xa9, 0x36, 0xda, 0x15, 0xd2, 0xd8, 0x69, + 0x35, 0xd1, 0x9b, 0xa7, 0x30, 0x15, 0x4d, 0xed, 0x08, 0xcc, 0x0a, 0x3e, 0x0f, 0xcb, 0x5c, 0x07, + 0x6d, 0x6b, 0x5b, 0xa9, 0x08, 0x27, 0xbe, 0x85, 0x2f, 0x00, 0x12, 0xaa, 0x44, 0xa0, 0x6f, 0x33, + 0x0d, 0x76, 0xcb, 0x3a, 0x69, 0x69, 0x2d, 0xf4, 0x5f, 0x42, 0x3d, 0x66, 0x16, 0x3f, 0xaf, 0xe2, + 0x65, 0xc8, 0xb6, 0x3b, 0x64, 0xa7, 0x7c, 0x4f, 0x69, 0x36, 0x54, 0x05, 0x5d, 0x66, 0xe6, 0xb4, + 0x3b, 0x44, 0x79, 0xd8, 0x51, 0xd4, 0x0e, 0x5a, 0x63, 0xb6, 0xb6, 0x3b, 0xa4, 0xab, 0x36, 0x34, + 0x15, 0x5d, 0x11, 0xdc, 0xa4, 0xa2, 0x35, 0x9b, 0x4a, 0xa5, 0x83, 0x4a, 0x8c, 0xb8, 0xa2, 0x05, + 0xc2, 0xaf, 0x0a, 0x57, 0xb3, 0x63, 0xbb, 0xbc, 0xd3, 0x42, 0xd7, 0x98, 0x77, 0x75, 0xa5, 0xae, + 0xb3, 0x18, 0x31, 0x2b, 0x5a, 0x1d, 0xf4, 0x0e, 0xd3, 0x86, 0xc3, 0xf4, 0x4d, 0x74, 0x9d, 0x31, + 0xf0, 0x43, 0xbb, 0xa9, 0xb5, 0x14, 0x74, 0x83, 0xdd, 0x26, 0xce, 0x0f, 0x1f, 0xa2, 0xf5, 0x93, + 0xd3, 0xa3, 0x47, 0xe8, 0x66, 0x04, 0xf7, 0x08, 0xdd, 0x0a, 0x39, 0x45, 0xd2, 0xbc, 0xcb, 0x34, + 0xe1, 0xe7, 0xf2, 0x6e, 0xfd, 0x21, 0xba, 0x1d, 0x3d, 0x3e, 0x42, 0xef, 0x95, 0x6e, 0x43, 0x92, + 0x75, 0x07, 0xe6, 0xf3, 0x72, 0xb7, 0xa3, 0xa1, 0x05, 0x9e, 0x52, 0x95, 0x72, 0xb3, 0xac, 0xa3, + 0x18, 0x93, 0xa5, 0x6a, 0x2a, 0x91, 0xe7, 0x78, 0xe9, 0x0f, 0x31, 0x28, 0xb4, 0x5c, 0xe7, 0x53, + 0xda, 0xf3, 0xdb, 0x54, 0xbc, 0x85, 0x7f, 0x02, 0x29, 0xd6, 0x99, 0x83, 0x07, 0xde, 0x3c, 0x95, + 0x23, 0x18, 0x71, 0x1d, 0xce, 0x0d, 0xa9, 0x4d, 0x5d, 0xc3, 0x8f, 0xbc, 0xa7, 0xc5, 0x23, 0xef, + 0x79, 0xad, 0x1d, 0x85, 0x4c, 0xc1, 0x82, 0x7a, 0x03, 0x90, 0x3d, 0xe1, 0x9f, 0x58, 0x3c, 0x32, + 0xa6, 0x2e, 0x19, 0x52, 0x5b, 0x3c, 0xf0, 0xf4, 0xbc, 0x3d, 0x19, 0xb1, 0x0e, 0xda, 0xa2, 0x6e, + 0x9d, 0xda, 0xa5, 0xaf, 0xf3, 0x90, 0x7b, 0x60, 0xda, 0x7d, 0xe7, 0x50, 0x0e, 0xa7, 0x35, 0xfe, + 0xd9, 0xdd, 0x37, 0x79, 0xaf, 0x3e, 0x96, 0x8f, 0xf9, 0x28, 0x08, 0xb7, 0x21, 0x73, 0xc8, 0x39, + 0x6a, 0xa1, 0x72, 0x77, 0x66, 0x9b, 0x1a, 0x15, 0x2e, 0x0f, 0xb5, 0xb0, 0x5f, 0x84, 0x72, 0x56, + 0x7e, 0x1f, 0x93, 0x9d, 0xa2, 0x0d, 0xf9, 0x60, 0x34, 0xd0, 0xda, 0x8b, 0x76, 0x4d, 0x7d, 0x5a, + 0x06, 0xbe, 0x0f, 0x20, 0xaf, 0x62, 0x12, 0xe3, 0x5c, 0xe2, 0xfb, 0xf3, 0xe9, 0xcc, 0xa4, 0x46, + 0x84, 0x7c, 0x94, 0x7c, 0xfa, 0xc5, 0xe5, 0xd8, 0xca, 0x17, 0x4b, 0x90, 0xaa, 0xb9, 0xc6, 0x88, + 0xe2, 0x7b, 0x90, 0x1c, 0x39, 0x7d, 0x2a, 0xd5, 0xfd, 0xae, 0xc2, 0x39, 0xef, 0xc6, 0x8e, 0xd3, + 0x0f, 0x87, 0x12, 0x13, 0x82, 0xef, 0xc3, 0xe2, 0x9e, 0x33, 0xb1, 0xfb, 0x9e, 0x9c, 0xeb, 0xff, + 0x33, 0x97, 0xb8, 0x2d, 0xce, 0x1a, 0xac, 0xa7, 0x42, 0x10, 0xfe, 0x18, 0x32, 0xf4, 0xa8, 0x67, + 0x4d, 0xf8, 0xce, 0x91, 0xe0, 0x4a, 0x7e, 0x30, 0x97, 0x54, 0x25, 0xe0, 0x0e, 0x3f, 0x26, 0x04, + 0x80, 0x95, 0x7f, 0xc6, 0x20, 0xc5, 0x2f, 0x65, 0xb7, 0xf0, 0xfb, 0x58, 0x21, 0x49, 0x57, 0x7c, + 0x30, 0xbf, 0xee, 0x91, 0x21, 0x7d, 0x22, 0x0e, 0x5f, 0x05, 0x30, 0x6d, 0x9f, 0x38, 0x83, 0x81, + 0x47, 0xc5, 0xfc, 0x0b, 0x7e, 0x01, 0xcb, 0x98, 0xb6, 0xaf, 0x71, 0x30, 0xbe, 0x02, 0x39, 0x56, + 0x15, 0xfd, 0x80, 0x8c, 0x59, 0x9a, 0xd3, 0xb3, 0x1c, 0x26, 0x49, 0xb6, 0x21, 0x2b, 0x90, 0xfc, + 0xf7, 0x62, 0xf9, 0x78, 0x98, 0xe3, 0x57, 0x55, 0x10, 0xdc, 0x4c, 0xa7, 0x95, 0xdf, 0xc4, 0x60, + 0x51, 0xb8, 0x1b, 0xab, 0x90, 0xf2, 0x7c, 0xc3, 0xf5, 0xe5, 0x42, 0xb7, 0x39, 0xbf, 0xd9, 0xe1, + 0x2f, 0x20, 0x4c, 0x0c, 0xae, 0x42, 0x82, 0xda, 0x7d, 0x99, 0x00, 0x2f, 0x20, 0x4d, 0x67, 0xec, + 0xa5, 0x1b, 0x90, 0x64, 0xd9, 0xc5, 0x46, 0xa7, 0x5e, 0x56, 0xeb, 0x0a, 0x5a, 0x60, 0xfd, 0x8d, + 0x4f, 0xb9, 0x18, 0xeb, 0x6f, 0x75, 0x5d, 0xeb, 0xb6, 0xda, 0x28, 0x5e, 0xfa, 0x0c, 0x32, 0xa1, + 0xef, 0xf1, 0x25, 0x38, 0xdf, 0x55, 0xb7, 0xb4, 0xae, 0x5a, 0x55, 0xaa, 0xa4, 0xa5, 0x2b, 0x15, + 0xa5, 0xda, 0x50, 0xeb, 0x68, 0x61, 0x1a, 0x51, 0xd3, 0x9a, 0x4d, 0xed, 0x01, 0x43, 0xc4, 0xd8, + 0x54, 0xd1, 0x6a, 0xb5, 0xb6, 0xd2, 0x89, 0x90, 0xc7, 0x23, 0xd0, 0x13, 0xda, 0x04, 0x1b, 0x27, + 0x95, 0xae, 0xae, 0x2b, 0x62, 0xdc, 0xa2, 0x64, 0xe9, 0x13, 0xc8, 0x84, 0xd9, 0xc5, 0x26, 0xab, + 0xaa, 0x11, 0xe5, 0x61, 0xa5, 0xd9, 0x6d, 0xb3, 0x81, 0xc2, 0x2f, 0xe5, 0xc7, 0xaa, 0x42, 0xa2, + 0x7c, 0x31, 0x7c, 0x0e, 0xf2, 0x01, 0x82, 0xdb, 0x81, 0xe2, 0x8c, 0x3b, 0x00, 0x75, 0x1a, 0x4a, + 0x1b, 0x25, 0x56, 0xfe, 0x1a, 0x87, 0x74, 0xd0, 0x77, 0xb0, 0x12, 0x59, 0xc5, 0xbe, 0xfd, 0x92, + 0x98, 0xe9, 0xd5, 0xd3, 0x8b, 0x58, 0x15, 0xd2, 0xe1, 0xb6, 0x98, 0x9c, 0x73, 0x5b, 0x0c, 0x39, + 0xd9, 0x73, 0x79, 0xc0, 0xe2, 0x25, 0x1f, 0x9d, 0xb7, 0xe7, 0x89, 0xb1, 0x2e, 0x58, 0xf1, 0x3a, + 0x4c, 0xad, 0x76, 0x7c, 0x77, 0x4d, 0x05, 0x3b, 0xf6, 0xd4, 0xd2, 0xb7, 0x02, 0x69, 0xc3, 0x1d, + 0x7a, 0x8d, 0xfe, 0x91, 0x57, 0x5c, 0xe2, 0x5d, 0x3d, 0x3c, 0x33, 0x29, 0x62, 0x87, 0x95, 0x52, + 0xd2, 0x91, 0x57, 0xd9, 0x14, 0x66, 0x3b, 0x99, 0x8e, 0xa3, 0x84, 0xdc, 0xee, 0x7e, 0x1b, 0x03, + 0x38, 0xe9, 0x8e, 0x7c, 0xda, 0x6a, 0x0f, 0x88, 0xda, 0xdd, 0xd9, 0x52, 0x74, 0x99, 0x67, 0x65, + 0xf5, 0x9e, 0x98, 0x9d, 0x55, 0x45, 0x6d, 0x2b, 0x84, 0x9f, 0x79, 0x90, 0xe4, 0xc6, 0x22, 0x20, + 0x09, 0xbe, 0x23, 0x74, 0x77, 0xf8, 0x5e, 0xd3, 0x11, 0x8b, 0x1e, 0xdf, 0x66, 0xc4, 0xa2, 0xd7, + 0x2c, 0xd7, 0xd1, 0x22, 0x13, 0xd7, 0x54, 0xca, 0x55, 0xb4, 0xc4, 0xf2, 0xa7, 0xd6, 0xd0, 0xdb, + 0x1d, 0xb2, 0x5b, 0x6e, 0x76, 0x15, 0x94, 0x66, 0xf2, 0x9b, 0xe5, 0xf0, 0x9c, 0x61, 0xd2, 0xd4, + 0xce, 0x5d, 0x79, 0x84, 0x5b, 0xff, 0x07, 0x85, 0xe9, 0x5f, 0x72, 0x58, 0xe2, 0xb7, 0xba, 0x5b, + 0xcd, 0x46, 0x05, 0x2d, 0xe0, 0x37, 0xe1, 0xa2, 0xf8, 0x9b, 0xad, 0x5f, 0x7c, 0x43, 0x95, 0xa8, + 0xd8, 0xd6, 0x7b, 0x4f, 0xbf, 0x5e, 0x5d, 0x78, 0xfa, 0x6c, 0x35, 0xf6, 0xe5, 0xb3, 0xd5, 0xd8, + 0x57, 0xcf, 0x56, 0x63, 0x7f, 0x7f, 0xb6, 0x1a, 0xfb, 0xc5, 0x37, 0xab, 0x0b, 0x5f, 0x7e, 0xb3, + 0xba, 0xf0, 0xd5, 0x37, 0xab, 0x0b, 0x1f, 0x67, 0x23, 0xff, 0x1b, 0xe4, 0xdf, 0x01, 0x00, 0x00, + 0xff, 0xff, 0xa5, 0x59, 0x1e, 0x2e, 0xfd, 0x22, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/processors_sql.proto b/pkg/sql/execinfrapb/processors_sql.proto index 62a896b14da4..a3fcfc148a82 100644 --- a/pkg/sql/execinfrapb/processors_sql.proto +++ b/pkg/sql/execinfrapb/processors_sql.proto @@ -844,8 +844,6 @@ message AggregatorSpec { reserved 3; } - optional Type type = 5 [(gogoproto.nullable) = false]; - // The group key is a subset of the columns in the input stream schema on the // basis of which we define our groups. repeated uint32 group_cols = 2 [packed = true]; @@ -854,6 +852,13 @@ message AggregatorSpec { // A subset of the GROUP BY columns which are ordered in the input. repeated uint32 ordered_group_cols = 4 [packed = true]; + + optional Type type = 5 [(gogoproto.nullable) = false]; + + // OutputOrdering specifies the required ordering of the output produced by + // the aggregator. The input to the processor *must* already be ordered + // according to it. + optional Ordering output_ordering = 6 [(gogoproto.nullable) = false]; } // ProjectSetSpec is the specification of a processor which applies a set of diff --git a/pkg/sql/logictest/testdata/logic_test/aggregate b/pkg/sql/logictest/testdata/logic_test/aggregate index 63c1d4cfe00a..be1bc7ad454d 100644 --- a/pkg/sql/logictest/testdata/logic_test/aggregate +++ b/pkg/sql/logictest/testdata/logic_test/aggregate @@ -3825,3 +3825,18 @@ query FI SELECT corr(DISTINCT y, x), count(DISTINCT y) FROM t55776 ---- 0.522232967867094 3 + +# Regression test for the vectorized hash aggregator not maintaining the +# required output ordering when spilling to disk (#63159). +statement ok +CREATE TABLE t63159 (a INT, b INT, INDEX(a) STORING (b)); +INSERT INTO t63159 VALUES (1,1), (3,3), (2,2), (5,5), (0,0), (1,1); + +query III +SELECT a, b, count(*) FROM t63159 GROUP BY a,b ORDER BY a +---- +0 0 1 +1 1 2 +2 2 1 +3 3 1 +5 5 1 diff --git a/pkg/sql/opt/exec/factory.opt b/pkg/sql/opt/exec/factory.opt index 8b841d8e6869..4677b200f792 100644 --- a/pkg/sql/opt/exec/factory.opt +++ b/pkg/sql/opt/exec/factory.opt @@ -134,6 +134,9 @@ define GroupBy { # distinct set of values on the set of columns in the ordering). GroupColOrdering colinfo.ColumnOrdering Aggregations []exec.AggInfo + + # If set, the output must have this ordering, but it is guaranteed that + # ReqOrdering is a prefix of GroupColOrdering. ReqOrdering exec.OutputOrdering } From 7d8097a0eeddc576c49c4de7e0cb1c23ad420b4f Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Thu, 8 Apr 2021 20:52:04 -0700 Subject: [PATCH 6/6] colbuilder: enforce unique memory monitor names Our disk-spilling infrastructure (namely, `diskSpillerBase`) uses the name of the limited memory monitor of its in-memory operator in order to distinguish the "memory budget exceeded" errors between the ones it needs to catch and the ones it needs to propagate further. In particular, previously it was possible that the in-memory chains within two disk-backed sorters created for the external distinct (one in the fallback strategy and another when the output ordering is needed, on top of the external distinct) would have the same memory monitor names. This is now fixed by appending a unique suffix to all names and is enforced via an invariants assertion in the test setup. Note that even with the previous behavior no actual bug would occur because the monitors with the same name were on "different levels of the tree" and there was the correct catcher in-between. So this commit simply restores the assumption that we had implicitly without fixing any production bugs. Release note: None --- pkg/sql/colexec/colbuilder/execplan.go | 177 +++++++++++---------- pkg/sql/colexec/colexecargs/BUILD.bazel | 2 + pkg/sql/colexec/colexecargs/op_creation.go | 17 ++ pkg/util/mon/bytes_usage.go | 5 + 4 files changed, 119 insertions(+), 82 deletions(-) diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index d1c70e8a67d0..de69eabc1280 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -383,7 +383,7 @@ func (r opResult) createDiskBackedSort( maxNumberPartitions int, processorID int32, post *execinfrapb.PostProcessSpec, - memMonitorNamePrefix string, + opNamePrefix string, factory coldata.ColumnFactory, ) (colexecop.Operator, error) { streamingMemAccount := args.StreamingMemAccount @@ -400,13 +400,12 @@ func (r opResult) createDiskBackedSort( if matchLen > 0 { // The input is already partially ordered. Use a chunks sorter to avoid // loading all the rows into memory. - sorterMemMonitorName = fmt.Sprintf("%ssort-chunks-%d", memMonitorNamePrefix, processorID) var sortChunksMemAccount *mon.BoundAccount if useStreamingMemAccountForBuffering { sortChunksMemAccount = streamingMemAccount } else { - sortChunksMemAccount = r.createMemAccountForSpillStrategy( - ctx, flowCtx, sorterMemMonitorName, + sortChunksMemAccount, sorterMemMonitorName = r.createMemAccountForSpillStrategy( + ctx, flowCtx, opNamePrefix+"sort-chunks", processorID, ) } inMemorySorter, err = colexec.NewSortChunks( @@ -423,13 +422,12 @@ func (r opResult) createDiskBackedSort( // // TODO(radu): we should not choose this processor when K is very large // - it is slower unless we get significantly more rows than the limit. - sorterMemMonitorName = fmt.Sprintf("%stopk-sort-%d", memMonitorNamePrefix, processorID) var topKSorterMemAccount *mon.BoundAccount if useStreamingMemAccountForBuffering { topKSorterMemAccount = streamingMemAccount } else { - topKSorterMemAccount = r.createMemAccountForSpillStrategy( - ctx, flowCtx, sorterMemMonitorName, + topKSorterMemAccount, sorterMemMonitorName = r.createMemAccountForSpillStrategy( + ctx, flowCtx, opNamePrefix+"topk-sort", processorID, ) } k := post.Limit + post.Offset @@ -439,13 +437,12 @@ func (r opResult) createDiskBackedSort( ) } else { // No optimizations possible. Default to the standard sort operator. - sorterMemMonitorName = fmt.Sprintf("%ssort-all-%d", memMonitorNamePrefix, processorID) var sorterMemAccount *mon.BoundAccount if useStreamingMemAccountForBuffering { sorterMemAccount = streamingMemAccount } else { - sorterMemAccount = r.createMemAccountForSpillStrategy( - ctx, flowCtx, sorterMemMonitorName, + sorterMemAccount, sorterMemMonitorName = r.createMemAccountForSpillStrategy( + ctx, flowCtx, opNamePrefix+"sort-all", processorID, ) } inMemorySorter, err = colexec.NewSorter( @@ -458,6 +455,11 @@ func (r opResult) createDiskBackedSort( if inMemorySorter == nil { return nil, errors.AssertionFailedf("unexpectedly inMemorySorter is nil") } + if useStreamingMemAccountForBuffering || args.TestingKnobs.DiskSpillingDisabled { + // In some testing scenarios we actually don't want to create a + // disk-backed sort. + return inMemorySorter, nil + } // NOTE: when spilling to disk, we're using the same general external // sorter regardless of which sorter variant we have instantiated (i.e. // we don't take advantage of the limits and of partial ordering). We @@ -466,23 +468,23 @@ func (r opResult) createDiskBackedSort( input, inMemorySorter.(colexecop.BufferingInMemoryOperator), sorterMemMonitorName, func(input colexecop.Operator) colexecop.Operator { - monitorNamePrefix := fmt.Sprintf("%sexternal-sorter", memMonitorNamePrefix) + opName := opNamePrefix + "external-sorter" // We are using unlimited memory monitors here because external // sort itself is responsible for making sure that we stay within // the memory limit. sortUnlimitedAllocator := colmem.NewAllocator( ctx, r.createBufferingUnlimitedMemAccount( - ctx, flowCtx, monitorNamePrefix+"-sort", + ctx, flowCtx, opName+"-sort", processorID, ), factory) mergeUnlimitedAllocator := colmem.NewAllocator( ctx, r.createBufferingUnlimitedMemAccount( - ctx, flowCtx, monitorNamePrefix+"-merge", + ctx, flowCtx, opName+"-merge", processorID, ), factory) outputUnlimitedAllocator := colmem.NewAllocator( ctx, r.createBufferingUnlimitedMemAccount( - ctx, flowCtx, monitorNamePrefix+"-output", + ctx, flowCtx, opName+"-output", processorID, ), factory) - diskAccount := r.createDiskAccount(ctx, flowCtx, monitorNamePrefix) + diskAccount := r.createDiskAccount(ctx, flowCtx, opName, processorID) es := colexec.NewExternalSorter( sortUnlimitedAllocator, mergeUnlimitedAllocator, @@ -511,7 +513,7 @@ func (r opResult) makeDiskBackedSorterConstructor( ctx context.Context, flowCtx *execinfra.FlowCtx, args *colexecargs.NewColOperatorArgs, - monitorNamePrefix string, + opNamePrefix string, factory coldata.ColumnFactory, ) colexec.DiskBackedSorterConstructor { return func(input colexecop.Operator, inputTypes []*types.T, orderingCols []execinfrapb.Ordering_Column, maxNumberPartitions int) colexecop.Operator { @@ -531,7 +533,7 @@ func (r opResult) makeDiskBackedSorterConstructor( ctx, flowCtx, &sortArgs, input, inputTypes, execinfrapb.Ordering{Columns: orderingCols}, 0 /* matchLen */, maxNumberPartitions, args.Spec.ProcessorID, - &execinfrapb.PostProcessSpec{}, monitorNamePrefix+"-", factory, + &execinfrapb.PostProcessSpec{}, opNamePrefix+"-", factory, ) if err != nil { colexecerror.InternalError(err) @@ -843,18 +845,18 @@ func NewColOperator( result.ColumnTypes = newAggArgs.OutputTypes if needHash { + opName := "hash-aggregator" // We have separate unit tests that instantiate the in-memory // hash aggregators, so we don't need to look at // args.TestingKnobs.DiskSpillingDisabled and always instantiate // a disk-backed one here. - hashAggregatorMemMonitorName := fmt.Sprintf("hash-aggregator-%d", spec.ProcessorID) diskSpillingDisabled := !colexec.HashAggregationDiskSpillingEnabled.Get(&flowCtx.Cfg.Settings.SV) if diskSpillingDisabled { // The disk spilling is disabled by the cluster setting, so // we give an unlimited memory account to the in-memory // hash aggregator and don't set up the disk spiller. hashAggregatorUnlimitedMemAccount := result.createBufferingUnlimitedMemAccount( - ctx, flowCtx, hashAggregatorMemMonitorName, + ctx, flowCtx, opName, spec.ProcessorID, ) newAggArgs.Allocator = colmem.NewAllocator( ctx, hashAggregatorUnlimitedMemAccount, factory, @@ -869,14 +871,16 @@ func NewColOperator( // two usages - the hash aggregation itself and the input // tuples tracking. totalMemLimit := execinfra.GetWorkMemLimit(flowCtx.Cfg) - hashAggregatorMemAccount := result.createMemAccountForSpillStrategyWithLimit( - ctx, flowCtx, hashAggregatorMemMonitorName, totalMemLimit/2, + hashAggregatorMemAccount, hashAggregatorMemMonitorName := result.createMemAccountForSpillStrategyWithLimit( + ctx, flowCtx, totalMemLimit/2, opName, spec.ProcessorID, ) spillingQueueMemMonitorName := hashAggregatorMemMonitorName + "-spilling-queue" // We need to create a separate memory account for the // spilling queue because it looks at how much memory it has // already used in order to decide when to spill to disk. - spillingQueueMemAccount := result.createBufferingUnlimitedMemAccount(ctx, flowCtx, spillingQueueMemMonitorName) + spillingQueueMemAccount := result.createBufferingUnlimitedMemAccount( + ctx, flowCtx, spillingQueueMemMonitorName, spec.ProcessorID, + ) spillingQueueCfg := args.DiskQueueCfg spillingQueueCfg.CacheMode = colcontainer.DiskQueueCacheModeReuseCache spillingQueueCfg.SetDefaultBufferSizeBytesForCacheMode() @@ -891,14 +895,14 @@ func NewColOperator( MemoryLimit: totalMemLimit / 2, DiskQueueCfg: spillingQueueCfg, FDSemaphore: args.FDSemaphore, - DiskAcc: result.createDiskAccount(ctx, flowCtx, spillingQueueMemMonitorName), + DiskAcc: result.createDiskAccount(ctx, flowCtx, spillingQueueMemMonitorName, spec.ProcessorID), }, ) if err != nil { return r, err } - ehaMonitorNamePrefix := fmt.Sprintf("external-hash-aggregator-%d", spec.ProcessorID) - ehaMemAccount := result.createBufferingUnlimitedMemAccount(ctx, flowCtx, ehaMonitorNamePrefix) + ehaOpName := "external-hash-aggregator" + ehaMemAccount := result.createBufferingUnlimitedMemAccount(ctx, flowCtx, ehaOpName, spec.ProcessorID) // Note that we will use an unlimited memory account here // even for the in-memory hash aggregator since it is easier // to do so than to try to replace the memory account if the @@ -922,8 +926,8 @@ func NewColOperator( flowCtx, args, &newAggArgs, - result.makeDiskBackedSorterConstructor(ctx, flowCtx, args, ehaMonitorNamePrefix, factory), - result.createDiskAccount(ctx, flowCtx, ehaMonitorNamePrefix), + result.makeDiskBackedSorterConstructor(ctx, flowCtx, args, ehaOpName, factory), + result.createDiskAccount(ctx, flowCtx, ehaOpName, spec.ProcessorID), ) }, args.TestingKnobs.SpillingCallbackFn, @@ -950,9 +954,8 @@ func NewColOperator( // 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, + distinctMemAccount, distinctMemMonitorName := result.createMemAccountForSpillStrategy( + ctx, flowCtx, "distinct" /* opName */, spec.ProcessorID, ) // TODO(yuzefovich): we have an implementation of partially // ordered distinct, and we should plan it when we have @@ -962,14 +965,14 @@ func NewColOperator( inMemoryUnorderedDistinct := colexec.NewUnorderedDistinct( allocator, inputs[0], core.Distinct.DistinctColumns, result.ColumnTypes, ) - diskAccount := result.createDiskAccount(ctx, flowCtx, distinctMemMonitorName) + edOpName := "external-distinct" + diskAccount := result.createDiskAccount(ctx, flowCtx, edOpName, spec.ProcessorID) result.Op = colexec.NewOneInputDiskSpiller( inputs[0], inMemoryUnorderedDistinct.(colexecop.BufferingInMemoryOperator), distinctMemMonitorName, func(input colexecop.Operator) colexecop.Operator { - monitorNamePrefix := fmt.Sprintf("external-distinct-%d", spec.ProcessorID) unlimitedAllocator := colmem.NewAllocator( - ctx, result.createBufferingUnlimitedMemAccount(ctx, flowCtx, monitorNamePrefix), factory, + ctx, result.createBufferingUnlimitedMemAccount(ctx, flowCtx, edOpName, spec.ProcessorID), factory, ) return colexec.NewExternalDistinct( unlimitedAllocator, @@ -977,7 +980,7 @@ func NewColOperator( args, input, result.ColumnTypes, - result.makeDiskBackedSorterConstructor(ctx, flowCtx, args, monitorNamePrefix, factory), + result.makeDiskBackedSorterConstructor(ctx, flowCtx, args, edOpName, factory), inMemoryUnorderedDistinct, diskAccount, ) @@ -1008,9 +1011,9 @@ func NewColOperator( if len(core.HashJoiner.LeftEqColumns) == 0 { // We are performing a cross-join, so we need to plan a // specialized operator. - crossJoinerMemMonitorName := fmt.Sprintf("cross-joiner-%d", spec.ProcessorID) - crossJoinerMemAccount := result.createBufferingUnlimitedMemAccount(ctx, flowCtx, crossJoinerMemMonitorName) - crossJoinerDiskAcc := result.createDiskAccount(ctx, flowCtx, crossJoinerMemMonitorName) + opName := "cross-joiner" + crossJoinerMemAccount := result.createBufferingUnlimitedMemAccount(ctx, flowCtx, opName, spec.ProcessorID) + crossJoinerDiskAcc := result.createDiskAccount(ctx, flowCtx, opName, spec.ProcessorID) unlimitedAllocator := colmem.NewAllocator(ctx, crossJoinerMemAccount, factory) result.Op = colexecjoin.NewCrossJoiner( unlimitedAllocator, @@ -1024,18 +1027,19 @@ func NewColOperator( ) result.ToClose = append(result.ToClose, result.Op.(colexecop.Closer)) } else { - hashJoinerMemMonitorName := fmt.Sprintf("hash-joiner-%d", spec.ProcessorID) + var hashJoinerMemMonitorName string var hashJoinerMemAccount *mon.BoundAccount var hashJoinerUnlimitedAllocator *colmem.Allocator if useStreamingMemAccountForBuffering { hashJoinerMemAccount = streamingMemAccount hashJoinerUnlimitedAllocator = streamingAllocator } else { - hashJoinerMemAccount = result.createMemAccountForSpillStrategy( - ctx, flowCtx, hashJoinerMemMonitorName, + opName := "hash-joiner" + hashJoinerMemAccount, hashJoinerMemMonitorName = result.createMemAccountForSpillStrategy( + ctx, flowCtx, opName, spec.ProcessorID, ) hashJoinerUnlimitedAllocator = colmem.NewAllocator( - ctx, result.createBufferingUnlimitedMemAccount(ctx, flowCtx, hashJoinerMemMonitorName), factory, + ctx, result.createBufferingUnlimitedMemAccount(ctx, flowCtx, opName, spec.ProcessorID), factory, ) } hjSpec := colexecjoin.MakeHashJoinerSpec( @@ -1052,20 +1056,20 @@ func NewColOperator( hashJoinerUnlimitedAllocator, hjSpec, inputs[0], inputs[1], colexecjoin.HashJoinerInitialNumBuckets, memoryLimit, ) - if args.TestingKnobs.DiskSpillingDisabled { + if useStreamingMemAccountForBuffering || args.TestingKnobs.DiskSpillingDisabled { // We will not be creating a disk-backed hash joiner because // we're running a test that explicitly asked for only // in-memory hash joiner. result.Op = inMemoryHashJoiner } else { - diskAccount := result.createDiskAccount(ctx, flowCtx, hashJoinerMemMonitorName) + opName := "external-hash-joiner" + diskAccount := result.createDiskAccount(ctx, flowCtx, opName, spec.ProcessorID) result.Op = colexec.NewTwoInputDiskSpiller( inputs[0], inputs[1], inMemoryHashJoiner.(colexecop.BufferingInMemoryOperator), hashJoinerMemMonitorName, func(inputOne, inputTwo colexecop.Operator) colexecop.Operator { - monitorNamePrefix := fmt.Sprintf("external-hash-joiner-%d", spec.ProcessorID) unlimitedAllocator := colmem.NewAllocator( - ctx, result.createBufferingUnlimitedMemAccount(ctx, flowCtx, monitorNamePrefix), factory, + ctx, result.createBufferingUnlimitedMemAccount(ctx, flowCtx, opName, spec.ProcessorID), factory, ) ehj := colexec.NewExternalHashJoiner( unlimitedAllocator, @@ -1073,7 +1077,7 @@ func NewColOperator( args, hjSpec, inputOne, inputTwo, - result.makeDiskBackedSorterConstructor(ctx, flowCtx, args, monitorNamePrefix, factory), + result.makeDiskBackedSorterConstructor(ctx, flowCtx, args, opName, factory), diskAccount, ) result.ToClose = append(result.ToClose, ehj.(colexecop.Closer)) @@ -1116,15 +1120,15 @@ func NewColOperator( onExpr = &core.MergeJoiner.OnExpr } - monitorName := "merge-joiner" + opName := "merge-joiner" // We are using an unlimited memory monitor here because merge // joiner itself is responsible for making sure that we stay within // the memory limit, and it will fall back to disk if necessary. unlimitedAllocator := colmem.NewAllocator( ctx, result.createBufferingUnlimitedMemAccount( - ctx, flowCtx, monitorName, + ctx, flowCtx, opName, spec.ProcessorID, ), factory) - diskAccount := result.createDiskAccount(ctx, flowCtx, monitorName) + diskAccount := result.createDiskAccount(ctx, flowCtx, opName, spec.ProcessorID) mj, err := colexecjoin.NewMergeJoinOp( unlimitedAllocator, execinfra.GetWorkMemLimit(flowCtx.Cfg), args.DiskQueueCfg, args.FDSemaphore, @@ -1159,14 +1163,14 @@ func NewColOperator( matchLen := core.Sorter.OrderingMatchLen result.Op, err = result.createDiskBackedSort( ctx, flowCtx, args, input, result.ColumnTypes, ordering, matchLen, 0, /* maxNumberPartitions */ - spec.ProcessorID, post, "" /* memMonitorNamePrefix */, factory, + spec.ProcessorID, post, "" /* opNamePrefix */, factory, ) case core.Windower != nil: if err := checkNumIn(inputs, 1); err != nil { return r, err } - memMonitorsPrefix := "window-" + opNamePrefix := "window-" input := inputs[0] result.ColumnTypes = make([]*types.T, len(spec.Input[0].ColumnTypes)) copy(result.ColumnTypes, spec.Input[0].ColumnTypes) @@ -1192,7 +1196,7 @@ func NewColOperator( ctx, flowCtx, args, input, inputTypes, execinfrapb.Ordering{Columns: orderingCols}, 0, /* matchLen */ 0 /* maxNumberPartitions */, spec.ProcessorID, - &execinfrapb.PostProcessSpec{}, memMonitorsPrefix, factory) + &execinfrapb.PostProcessSpec{}, opNamePrefix, factory) }, ) // Window partitioner will append a boolean column. @@ -1204,7 +1208,7 @@ func NewColOperator( input, err = result.createDiskBackedSort( ctx, flowCtx, args, input, typs, wf.Ordering, 0 /* matchLen */, 0, /* maxNumberPartitions */ - spec.ProcessorID, &execinfrapb.PostProcessSpec{}, memMonitorsPrefix, factory, + spec.ProcessorID, &execinfrapb.PostProcessSpec{}, opNamePrefix, factory, ) } } @@ -1237,11 +1241,11 @@ func NewColOperator( // relative rank operators themselves are responsible for // making sure that we stay within the memory limit, and // they will fall back to disk if necessary. - memAccName := memMonitorsPrefix + "relative-rank" + opName := opNamePrefix + "relative-rank" unlimitedAllocator := colmem.NewAllocator( - ctx, result.createBufferingUnlimitedMemAccount(ctx, flowCtx, memAccName), factory, + ctx, result.createBufferingUnlimitedMemAccount(ctx, flowCtx, opName, spec.ProcessorID), factory, ) - diskAcc := result.createDiskAccount(ctx, flowCtx, memAccName) + diskAcc := result.createDiskAccount(ctx, flowCtx, opName, spec.ProcessorID) result.Op, err = colexecwindow.NewRelativeRankOperator( unlimitedAllocator, execinfra.GetWorkMemLimit(flowCtx.Cfg), args.DiskQueueCfg, args.FDSemaphore, input, typs, windowFn, wf.Ordering.Columns, @@ -1394,6 +1398,9 @@ func NewColOperator( for i := range args.MetadataSources { r.MetadataSources = append(r.MetadataSources, args.MetadataSources[i]...) } + if util.CrdbTestBuild { + r.AssertInvariants() + } return r, err } @@ -1518,51 +1525,46 @@ func (r *postProcessResult) planPostProcessSpec( return nil } -// createBufferingUnlimitedMemMonitor instantiates an unlimited memory monitor. -// These should only be used when spilling to disk and an operator is made aware -// of a memory usage limit separately. -// The receiver is updated to have a reference to the unlimited memory monitor. -func (r opResult) createBufferingUnlimitedMemMonitor( - ctx context.Context, flowCtx *execinfra.FlowCtx, name string, -) *mon.BytesMonitor { - bufferingOpUnlimitedMemMonitor := execinfra.NewMonitor( - ctx, flowCtx.EvalCtx.Mon, name+"-unlimited", - ) - r.OpMonitors = append(r.OpMonitors, bufferingOpUnlimitedMemMonitor) - return bufferingOpUnlimitedMemMonitor +// getMemMonitorName returns a unique (for this opResult) memory monitor name. +func (r opResult) getMemMonitorName(opName string, processorID int32, suffix string) string { + return fmt.Sprintf("%s-%d-%s-%d", opName, processorID, suffix, len(r.OpMonitors)) } // createMemAccountForSpillStrategy instantiates a memory monitor and a memory // account to be used with a buffering Operator that can fall back to disk. // The default memory limit is used, if flowCtx.Cfg.ForceDiskSpill is used, this -// will be 1. The receiver is updated to have references to both objects. +// will be 1. The receiver is updated to have references to both objects. Memory +// monitor name is also returned. func (r opResult) createMemAccountForSpillStrategy( - ctx context.Context, flowCtx *execinfra.FlowCtx, name string, -) *mon.BoundAccount { + ctx context.Context, flowCtx *execinfra.FlowCtx, opName string, processorID int32, +) (*mon.BoundAccount, string) { + monitorName := r.getMemMonitorName(opName, processorID, "limited" /* suffix */) bufferingOpMemMonitor := execinfra.NewLimitedMonitor( - ctx, flowCtx.EvalCtx.Mon, flowCtx.Cfg, name+"-limited", + ctx, flowCtx.EvalCtx.Mon, flowCtx.Cfg, monitorName, ) r.OpMonitors = append(r.OpMonitors, bufferingOpMemMonitor) bufferingMemAccount := bufferingOpMemMonitor.MakeBoundAccount() r.OpAccounts = append(r.OpAccounts, &bufferingMemAccount) - return &bufferingMemAccount + return &bufferingMemAccount, monitorName } // createMemAccountForSpillStrategyWithLimit is the same as // createMemAccountForSpillStrategy except that it takes in a custom limit -// instead of using the number obtained via execinfra.GetWorkMemLimit. +// instead of using the number obtained via execinfra.GetWorkMemLimit. Memory +// monitor name is also returned. func (r opResult) createMemAccountForSpillStrategyWithLimit( - ctx context.Context, flowCtx *execinfra.FlowCtx, name string, limit int64, -) *mon.BoundAccount { + ctx context.Context, flowCtx *execinfra.FlowCtx, limit int64, opName string, processorID int32, +) (*mon.BoundAccount, string) { if flowCtx.Cfg.TestingKnobs.ForceDiskSpill { limit = 1 } - bufferingOpMemMonitor := mon.NewMonitorInheritWithLimit(name+"-limited", limit, flowCtx.EvalCtx.Mon) + monitorName := r.getMemMonitorName(opName, processorID, "limited" /* suffix */) + bufferingOpMemMonitor := mon.NewMonitorInheritWithLimit(monitorName, limit, flowCtx.EvalCtx.Mon) bufferingOpMemMonitor.Start(ctx, flowCtx.EvalCtx.Mon, mon.BoundAccount{}) r.OpMonitors = append(r.OpMonitors, bufferingOpMemMonitor) bufferingMemAccount := bufferingOpMemMonitor.MakeBoundAccount() r.OpAccounts = append(r.OpAccounts, &bufferingMemAccount) - return &bufferingMemAccount + return &bufferingMemAccount, monitorName } // createBufferingUnlimitedMemAccount instantiates an unlimited memory monitor @@ -1570,10 +1572,17 @@ func (r opResult) createMemAccountForSpillStrategyWithLimit( // receiver is updated to have references to both objects. Note that the // returned account is only "unlimited" in that it does not have a hard limit // that it enforces, but a limit might be enforced by a root monitor. +// +// Note that the memory monitor name is not returned (unlike above) because no +// caller actually needs it. func (r opResult) createBufferingUnlimitedMemAccount( - ctx context.Context, flowCtx *execinfra.FlowCtx, name string, + ctx context.Context, flowCtx *execinfra.FlowCtx, opName string, processorID int32, ) *mon.BoundAccount { - bufferingOpUnlimitedMemMonitor := r.createBufferingUnlimitedMemMonitor(ctx, flowCtx, name) + monitorName := r.getMemMonitorName(opName, processorID, "unlimited" /* suffix */) + bufferingOpUnlimitedMemMonitor := execinfra.NewMonitor( + ctx, flowCtx.EvalCtx.Mon, monitorName, + ) + r.OpMonitors = append(r.OpMonitors, bufferingOpUnlimitedMemMonitor) bufferingMemAccount := bufferingOpUnlimitedMemMonitor.MakeBoundAccount() r.OpAccounts = append(r.OpAccounts, &bufferingMemAccount) return &bufferingMemAccount @@ -1583,10 +1592,14 @@ func (r opResult) createBufferingUnlimitedMemAccount( // to be used for disk spilling infrastructure in vectorized engine. // TODO(azhng): consolidates all allocation monitors/account manage into one // place after branch cut for 20.1. +// +// Note that the memory monitor name is not returned (unlike above) because no +// caller actually needs it. func (r opResult) createDiskAccount( - ctx context.Context, flowCtx *execinfra.FlowCtx, name string, + ctx context.Context, flowCtx *execinfra.FlowCtx, opName string, processorID int32, ) *mon.BoundAccount { - opDiskMonitor := execinfra.NewMonitor(ctx, flowCtx.DiskMonitor, name) + monitorName := r.getMemMonitorName(opName, processorID, "disk" /* suffix */) + opDiskMonitor := execinfra.NewMonitor(ctx, flowCtx.DiskMonitor, monitorName) r.OpMonitors = append(r.OpMonitors, opDiskMonitor) opDiskAccount := opDiskMonitor.MakeBoundAccount() r.OpAccounts = append(r.OpAccounts, &opDiskAccount) diff --git a/pkg/sql/colexec/colexecargs/BUILD.bazel b/pkg/sql/colexec/colexecargs/BUILD.bazel index 2ee9fc45ee50..79b4b2c66557 100644 --- a/pkg/sql/colexec/colexecargs/BUILD.bazel +++ b/pkg/sql/colexec/colexecargs/BUILD.bazel @@ -11,6 +11,7 @@ go_library( deps = [ "//pkg/col/coldata", "//pkg/sql/colcontainer", + "//pkg/sql/colexecerror", "//pkg/sql/colexecop", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", @@ -18,6 +19,7 @@ go_library( "//pkg/sql/sem/tree", "//pkg/sql/types", "//pkg/util/mon", + "@com_github_cockroachdb_errors//:errors", "@com_github_marusama_semaphore//:semaphore", ], ) diff --git a/pkg/sql/colexec/colexecargs/op_creation.go b/pkg/sql/colexec/colexecargs/op_creation.go index 6bdeb172d79f..e25fd362cbfa 100644 --- a/pkg/sql/colexec/colexecargs/op_creation.go +++ b/pkg/sql/colexec/colexecargs/op_creation.go @@ -16,11 +16,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/mon" + "github.com/cockroachdb/errors" "github.com/marusama/semaphore" ) @@ -107,6 +109,21 @@ type NewColOperatorResult struct { var _ execinfra.Releasable = &NewColOperatorResult{} +// AssertInvariants confirms that all invariants are maintained by +// NewColOperatorResult. +func (r *NewColOperatorResult) AssertInvariants() { + // Check that all memory monitor names are unique (colexec.diskSpillerBase + // relies on this in order to catch "memory budget exceeded" errors only + // from "its own" component). + names := make(map[string]struct{}, len(r.OpMonitors)) + for _, m := range r.OpMonitors { + if _, seen := names[m.Name()]; seen { + colexecerror.InternalError(errors.AssertionFailedf("monitor named %q encountered twice", m.Name())) + } + names[m.Name()] = struct{}{} + } +} + var newColOperatorResultPool = sync.Pool{ New: func() interface{} { return &NewColOperatorResult{} diff --git a/pkg/util/mon/bytes_usage.go b/pkg/util/mon/bytes_usage.go index 74b3c24f9e82..036b25be8162 100644 --- a/pkg/util/mon/bytes_usage.go +++ b/pkg/util/mon/bytes_usage.go @@ -392,6 +392,11 @@ func (mm *BytesMonitor) Stop(ctx context.Context) { mm.doStop(ctx, true) } +// Name returns the name of the monitor. +func (mm *BytesMonitor) Name() string { + return mm.name +} + const bytesMaxUsageLoggingThreshold = 100 * 1024 func (mm *BytesMonitor) doStop(ctx context.Context, check bool) {