From cd63a655813cb45dc60fe2c48a1ce8d8ac9041d1 Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Tue, 3 Aug 2021 18:04:42 -0600 Subject: [PATCH] colexec: optimize min and max window functions with default exclusion This commit modifies the 'min' and 'max' aggregate window functions to implement the `slidingWindowAggregateFunc` interface, which allows them to be used in a sliding window context. However, this is only usable when the window frame never shrinks - e.g. it always contains all rows from the previous frame. This commit also provides implementations of `min` and `max` for use when the window frame can shrink. The indices of the 'next best' minimum or maximum values are stored in a priority queue that is updated for each row. Using the priority queue allows the `min` and `max` operators to avoid fully aggregating over the window frame even when the previous best value goes out of scope. Note that this implementation currently does not handle the case of non-default exclusion clause, in which case we must fall back to the quadratic approach. Release note: None --- Makefile | 1 + pkg/sql/colexec/colbuilder/execplan.go | 23 +- .../colexec/colexecagg/hash_min_max_agg.eg.go | 496 +- .../colexec/colexecagg/min_max_agg_tmpl.go | 47 +- .../colexecagg/ordered_min_max_agg.eg.go | 1024 +++-- .../colexecagg/window_min_max_agg.eg.go | 752 +++- pkg/sql/colexec/colexecwindow/BUILD.bazel | 5 + .../colexec/colexecwindow/min_max_queue.go | 177 + .../colexecwindow/min_max_queue_test.go | 67 + .../colexecwindow/min_max_removable_agg.eg.go | 3978 +++++++++++++++++ .../min_max_removable_agg_tmpl.go | 320 ++ .../colexecwindow/window_aggregator.eg.go | 42 +- .../colexecwindow/window_aggregator_tmpl.go | 42 +- .../colexecwindow/window_functions_test.go | 5 +- .../colexecwindow/window_functions_util.go | 28 + .../colexec/execgen/cmd/execgen/BUILD.bazel | 1 + .../cmd/execgen/min_max_removable_agg_gen.go | 64 + pkg/sql/memsize/constants.go | 3 + 18 files changed, 6084 insertions(+), 991 deletions(-) create mode 100644 pkg/sql/colexec/colexecwindow/min_max_queue.go create mode 100644 pkg/sql/colexec/colexecwindow/min_max_queue_test.go create mode 100644 pkg/sql/colexec/colexecwindow/min_max_removable_agg.eg.go create mode 100644 pkg/sql/colexec/colexecwindow/min_max_removable_agg_tmpl.go create mode 100644 pkg/sql/colexec/execgen/cmd/execgen/min_max_removable_agg_gen.go diff --git a/Makefile b/Makefile index a73828e3f280..560f132e7096 100644 --- a/Makefile +++ b/Makefile @@ -885,6 +885,7 @@ EXECGEN_TARGETS = \ pkg/sql/colexec/colexecwindow/lag.eg.go \ pkg/sql/colexec/colexecwindow/last_value.eg.go \ pkg/sql/colexec/colexecwindow/lead.eg.go \ + pkg/sql/colexec/colexecwindow/min_max_removable_agg.eg.go \ pkg/sql/colexec/colexecwindow/ntile.eg.go \ pkg/sql/colexec/colexecwindow/nth_value.eg.go \ pkg/sql/colexec/colexecwindow/range_offset_handler.eg.go \ diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 93d58cdf74eb..9c3d1de5fbfe 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -1358,6 +1358,7 @@ func NewColOperator( opName := opNamePrefix + strings.ToLower(wf.Func.AggregateFunc.String()) result.finishBufferedWindowerArgs( ctx, flowCtx, windowArgs, opName, spec.ProcessorID, factory, true /* needsBuffer */) + aggType := *wf.Func.AggregateFunc switch *wf.Func.AggregateFunc { case execinfrapb.CountRows: // count_rows has a specialized implementation. @@ -1375,20 +1376,28 @@ func NewColOperator( colIdx[i] = uint32(i) } aggregations := []execinfrapb.AggregatorSpec_Aggregation{{ - Func: *wf.Func.AggregateFunc, + Func: aggType, ColIdx: colIdx, }} semaCtx := flowCtx.TypeResolverFactory.NewSemaContext(evalCtx.Txn) aggArgs.Constructors, aggArgs.ConstArguments, aggArgs.OutputTypes, err = colexecagg.ProcessAggregations(evalCtx, semaCtx, aggregations, argTypes) - aggFnsAlloc, _, toClose, err := colexecagg.NewAggregateFuncsAlloc( - &aggArgs, aggregations, 1 /* allocSize */, colexecagg.WindowAggKind, - ) - if err != nil { - colexecerror.InternalError(err) + var toClose colexecop.Closers + var aggFnsAlloc *colexecagg.AggregateFuncsAlloc + if (aggType != execinfrapb.Min && aggType != execinfrapb.Max) || + wf.Frame.Exclusion != execinfrapb.WindowerSpec_Frame_NO_EXCLUSION || + !colexecwindow.WindowFrameCanShrink(wf.Frame, &wf.Ordering) { + // Min and max window functions have specialized implementations + // when the frame can shrink and has a default exclusion clause. + aggFnsAlloc, _, toClose, err = colexecagg.NewAggregateFuncsAlloc( + &aggArgs, aggregations, 1 /* allocSize */, colexecagg.WindowAggKind, + ) + if err != nil { + colexecerror.InternalError(err) + } } result.Root = colexecwindow.NewWindowAggregatorOperator( - windowArgs, wf.Frame, &wf.Ordering, argIdxs, + windowArgs, aggType, wf.Frame, &wf.Ordering, argIdxs, aggArgs.OutputTypes[0], aggFnsAlloc, toClose) } } else { diff --git a/pkg/sql/colexec/colexecagg/hash_min_max_agg.eg.go b/pkg/sql/colexec/colexecagg/hash_min_max_agg.eg.go index 2965a211e7fc..d02bb69cd302 100644 --- a/pkg/sql/colexec/colexecagg/hash_min_max_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/hash_min_max_agg.eg.go @@ -116,11 +116,11 @@ type minBoolHashAgg struct { col coldata.Bools // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg bool - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minBoolHashAgg{} @@ -145,10 +145,9 @@ func (a *minBoolHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -171,6 +170,7 @@ func (a *minBoolHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -179,10 +179,9 @@ func (a *minBoolHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -205,6 +204,7 @@ func (a *minBoolHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -221,7 +221,7 @@ func (a *minBoolHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -229,7 +229,7 @@ func (a *minBoolHashAgg) Flush(outputIdx int) { } func (a *minBoolHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minBoolHashAggAlloc struct { @@ -259,11 +259,11 @@ type minBytesHashAgg struct { col *coldata.Bytes // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg []byte - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minBytesHashAgg{} @@ -288,10 +288,9 @@ func (a *minBytesHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -306,6 +305,7 @@ func (a *minBytesHashAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } else { @@ -314,10 +314,9 @@ func (a *minBytesHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -332,6 +331,7 @@ func (a *minBytesHashAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } @@ -348,19 +348,24 @@ func (a *minBytesHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) } oldCurAggSize := len(a.curAgg) - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *minBytesHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + oldCurAggSize := len(a.curAgg) + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type minBytesHashAggAlloc struct { @@ -390,11 +395,11 @@ type minDecimalHashAgg struct { col coldata.Decimals // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minDecimalHashAgg{} @@ -419,10 +424,9 @@ func (a *minDecimalHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -437,6 +441,7 @@ func (a *minDecimalHashAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } else { @@ -445,10 +450,9 @@ func (a *minDecimalHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -463,6 +467,7 @@ func (a *minDecimalHashAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } @@ -479,7 +484,7 @@ func (a *minDecimalHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -487,7 +492,7 @@ func (a *minDecimalHashAgg) Flush(outputIdx int) { } func (a *minDecimalHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minDecimalHashAggAlloc struct { @@ -517,11 +522,11 @@ type minInt16HashAgg struct { col coldata.Int16s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int16 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt16HashAgg{} @@ -546,10 +551,9 @@ func (a *minInt16HashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -575,6 +579,7 @@ func (a *minInt16HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -583,10 +588,9 @@ func (a *minInt16HashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -612,6 +616,7 @@ func (a *minInt16HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -628,7 +633,7 @@ func (a *minInt16HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -636,7 +641,7 @@ func (a *minInt16HashAgg) Flush(outputIdx int) { } func (a *minInt16HashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt16HashAggAlloc struct { @@ -666,11 +671,11 @@ type minInt32HashAgg struct { col coldata.Int32s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int32 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt32HashAgg{} @@ -695,10 +700,9 @@ func (a *minInt32HashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -724,6 +728,7 @@ func (a *minInt32HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -732,10 +737,9 @@ func (a *minInt32HashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -761,6 +765,7 @@ func (a *minInt32HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -777,7 +782,7 @@ func (a *minInt32HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -785,7 +790,7 @@ func (a *minInt32HashAgg) Flush(outputIdx int) { } func (a *minInt32HashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt32HashAggAlloc struct { @@ -815,11 +820,11 @@ type minInt64HashAgg struct { col coldata.Int64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt64HashAgg{} @@ -844,10 +849,9 @@ func (a *minInt64HashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -873,6 +877,7 @@ func (a *minInt64HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -881,10 +886,9 @@ func (a *minInt64HashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -910,6 +914,7 @@ func (a *minInt64HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -926,7 +931,7 @@ func (a *minInt64HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -934,7 +939,7 @@ func (a *minInt64HashAgg) Flush(outputIdx int) { } func (a *minInt64HashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt64HashAggAlloc struct { @@ -964,11 +969,11 @@ type minFloat64HashAgg struct { col coldata.Float64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minFloat64HashAgg{} @@ -993,10 +998,9 @@ func (a *minFloat64HashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1030,6 +1034,7 @@ func (a *minFloat64HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1038,10 +1043,9 @@ func (a *minFloat64HashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1075,6 +1079,7 @@ func (a *minFloat64HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1091,7 +1096,7 @@ func (a *minFloat64HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1099,7 +1104,7 @@ func (a *minFloat64HashAgg) Flush(outputIdx int) { } func (a *minFloat64HashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minFloat64HashAggAlloc struct { @@ -1129,11 +1134,11 @@ type minTimestampHashAgg struct { col coldata.Times // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg time.Time - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minTimestampHashAgg{} @@ -1158,10 +1163,9 @@ func (a *minTimestampHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1183,6 +1187,7 @@ func (a *minTimestampHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1191,10 +1196,9 @@ func (a *minTimestampHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1216,6 +1220,7 @@ func (a *minTimestampHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1232,7 +1237,7 @@ func (a *minTimestampHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1240,7 +1245,7 @@ func (a *minTimestampHashAgg) Flush(outputIdx int) { } func (a *minTimestampHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minTimestampHashAggAlloc struct { @@ -1270,11 +1275,11 @@ type minIntervalHashAgg struct { col coldata.Durations // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minIntervalHashAgg{} @@ -1299,10 +1304,9 @@ func (a *minIntervalHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1317,6 +1321,7 @@ func (a *minIntervalHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1325,10 +1330,9 @@ func (a *minIntervalHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1343,6 +1347,7 @@ func (a *minIntervalHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1359,7 +1364,7 @@ func (a *minIntervalHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1367,7 +1372,7 @@ func (a *minIntervalHashAgg) Flush(outputIdx int) { } func (a *minIntervalHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minIntervalHashAggAlloc struct { @@ -1397,11 +1402,11 @@ type minJSONHashAgg struct { col *coldata.JSONs // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg json.JSON - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minJSONHashAgg{} @@ -1429,7 +1434,7 @@ func (a *minJSONHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -1443,7 +1448,6 @@ func (a *minJSONHashAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1475,6 +1479,7 @@ func (a *minJSONHashAgg) Compute( } } + a.numNonNull++ } } } else { @@ -1483,7 +1488,7 @@ func (a *minJSONHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -1497,7 +1502,6 @@ func (a *minJSONHashAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1529,6 +1533,7 @@ func (a *minJSONHashAgg) Compute( } } + a.numNonNull++ } } } @@ -1548,7 +1553,7 @@ func (a *minJSONHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1557,13 +1562,21 @@ func (a *minJSONHashAgg) Flush(outputIdx int) { if a.curAgg != nil { oldCurAggSize = a.curAgg.Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *minJSONHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + var oldCurAggSize uintptr + if a.curAgg != nil { + oldCurAggSize = a.curAgg.Size() + } + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type minJSONHashAggAlloc struct { @@ -1593,11 +1606,11 @@ type minDatumHashAgg struct { col coldata.DatumVec // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg interface{} - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minDatumHashAgg{} @@ -1626,10 +1639,9 @@ func (a *minDatumHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1646,6 +1658,7 @@ func (a *minDatumHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1654,10 +1667,9 @@ func (a *minDatumHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1674,6 +1686,7 @@ func (a *minDatumHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1694,7 +1707,7 @@ func (a *minDatumHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1704,13 +1717,22 @@ func (a *minDatumHashAgg) Flush(outputIdx int) { if a.curAgg != nil { oldCurAggSize = a.curAgg.(tree.Datum).Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *minDatumHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + + var oldCurAggSize uintptr + if a.curAgg != nil { + oldCurAggSize = a.curAgg.(tree.Datum).Size() + } + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type minDatumHashAggAlloc struct { @@ -1809,11 +1831,11 @@ type maxBoolHashAgg struct { col coldata.Bools // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg bool - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxBoolHashAgg{} @@ -1838,10 +1860,9 @@ func (a *maxBoolHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1864,6 +1885,7 @@ func (a *maxBoolHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1872,10 +1894,9 @@ func (a *maxBoolHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1898,6 +1919,7 @@ func (a *maxBoolHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1914,7 +1936,7 @@ func (a *maxBoolHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1922,7 +1944,7 @@ func (a *maxBoolHashAgg) Flush(outputIdx int) { } func (a *maxBoolHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxBoolHashAggAlloc struct { @@ -1952,11 +1974,11 @@ type maxBytesHashAgg struct { col *coldata.Bytes // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg []byte - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxBytesHashAgg{} @@ -1981,10 +2003,9 @@ func (a *maxBytesHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1999,6 +2020,7 @@ func (a *maxBytesHashAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } else { @@ -2007,10 +2029,9 @@ func (a *maxBytesHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2025,6 +2046,7 @@ func (a *maxBytesHashAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } @@ -2041,19 +2063,24 @@ func (a *maxBytesHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) } oldCurAggSize := len(a.curAgg) - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *maxBytesHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + oldCurAggSize := len(a.curAgg) + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type maxBytesHashAggAlloc struct { @@ -2083,11 +2110,11 @@ type maxDecimalHashAgg struct { col coldata.Decimals // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxDecimalHashAgg{} @@ -2112,10 +2139,9 @@ func (a *maxDecimalHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2130,6 +2156,7 @@ func (a *maxDecimalHashAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } else { @@ -2138,10 +2165,9 @@ func (a *maxDecimalHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2156,6 +2182,7 @@ func (a *maxDecimalHashAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } @@ -2172,7 +2199,7 @@ func (a *maxDecimalHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2180,7 +2207,7 @@ func (a *maxDecimalHashAgg) Flush(outputIdx int) { } func (a *maxDecimalHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxDecimalHashAggAlloc struct { @@ -2210,11 +2237,11 @@ type maxInt16HashAgg struct { col coldata.Int16s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int16 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt16HashAgg{} @@ -2239,10 +2266,9 @@ func (a *maxInt16HashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2268,6 +2294,7 @@ func (a *maxInt16HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2276,10 +2303,9 @@ func (a *maxInt16HashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2305,6 +2331,7 @@ func (a *maxInt16HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2321,7 +2348,7 @@ func (a *maxInt16HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2329,7 +2356,7 @@ func (a *maxInt16HashAgg) Flush(outputIdx int) { } func (a *maxInt16HashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt16HashAggAlloc struct { @@ -2359,11 +2386,11 @@ type maxInt32HashAgg struct { col coldata.Int32s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int32 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt32HashAgg{} @@ -2388,10 +2415,9 @@ func (a *maxInt32HashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2417,6 +2443,7 @@ func (a *maxInt32HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2425,10 +2452,9 @@ func (a *maxInt32HashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2454,6 +2480,7 @@ func (a *maxInt32HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2470,7 +2497,7 @@ func (a *maxInt32HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2478,7 +2505,7 @@ func (a *maxInt32HashAgg) Flush(outputIdx int) { } func (a *maxInt32HashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt32HashAggAlloc struct { @@ -2508,11 +2535,11 @@ type maxInt64HashAgg struct { col coldata.Int64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt64HashAgg{} @@ -2537,10 +2564,9 @@ func (a *maxInt64HashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2566,6 +2592,7 @@ func (a *maxInt64HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2574,10 +2601,9 @@ func (a *maxInt64HashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2603,6 +2629,7 @@ func (a *maxInt64HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2619,7 +2646,7 @@ func (a *maxInt64HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2627,7 +2654,7 @@ func (a *maxInt64HashAgg) Flush(outputIdx int) { } func (a *maxInt64HashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt64HashAggAlloc struct { @@ -2657,11 +2684,11 @@ type maxFloat64HashAgg struct { col coldata.Float64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxFloat64HashAgg{} @@ -2686,10 +2713,9 @@ func (a *maxFloat64HashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2723,6 +2749,7 @@ func (a *maxFloat64HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2731,10 +2758,9 @@ func (a *maxFloat64HashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2768,6 +2794,7 @@ func (a *maxFloat64HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2784,7 +2811,7 @@ func (a *maxFloat64HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2792,7 +2819,7 @@ func (a *maxFloat64HashAgg) Flush(outputIdx int) { } func (a *maxFloat64HashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxFloat64HashAggAlloc struct { @@ -2822,11 +2849,11 @@ type maxTimestampHashAgg struct { col coldata.Times // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg time.Time - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxTimestampHashAgg{} @@ -2851,10 +2878,9 @@ func (a *maxTimestampHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2876,6 +2902,7 @@ func (a *maxTimestampHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2884,10 +2911,9 @@ func (a *maxTimestampHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2909,6 +2935,7 @@ func (a *maxTimestampHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2925,7 +2952,7 @@ func (a *maxTimestampHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2933,7 +2960,7 @@ func (a *maxTimestampHashAgg) Flush(outputIdx int) { } func (a *maxTimestampHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxTimestampHashAggAlloc struct { @@ -2963,11 +2990,11 @@ type maxIntervalHashAgg struct { col coldata.Durations // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxIntervalHashAgg{} @@ -2992,10 +3019,9 @@ func (a *maxIntervalHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3010,6 +3036,7 @@ func (a *maxIntervalHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -3018,10 +3045,9 @@ func (a *maxIntervalHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3036,6 +3062,7 @@ func (a *maxIntervalHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -3052,7 +3079,7 @@ func (a *maxIntervalHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -3060,7 +3087,7 @@ func (a *maxIntervalHashAgg) Flush(outputIdx int) { } func (a *maxIntervalHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxIntervalHashAggAlloc struct { @@ -3090,11 +3117,11 @@ type maxJSONHashAgg struct { col *coldata.JSONs // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg json.JSON - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxJSONHashAgg{} @@ -3122,7 +3149,7 @@ func (a *maxJSONHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -3136,7 +3163,6 @@ func (a *maxJSONHashAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3168,6 +3194,7 @@ func (a *maxJSONHashAgg) Compute( } } + a.numNonNull++ } } } else { @@ -3176,7 +3203,7 @@ func (a *maxJSONHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -3190,7 +3217,6 @@ func (a *maxJSONHashAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3222,6 +3248,7 @@ func (a *maxJSONHashAgg) Compute( } } + a.numNonNull++ } } } @@ -3241,7 +3268,7 @@ func (a *maxJSONHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -3250,13 +3277,21 @@ func (a *maxJSONHashAgg) Flush(outputIdx int) { if a.curAgg != nil { oldCurAggSize = a.curAgg.Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *maxJSONHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + var oldCurAggSize uintptr + if a.curAgg != nil { + oldCurAggSize = a.curAgg.Size() + } + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type maxJSONHashAggAlloc struct { @@ -3286,11 +3321,11 @@ type maxDatumHashAgg struct { col coldata.DatumVec // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg interface{} - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxDatumHashAgg{} @@ -3319,10 +3354,9 @@ func (a *maxDatumHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3339,6 +3373,7 @@ func (a *maxDatumHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -3347,10 +3382,9 @@ func (a *maxDatumHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3367,6 +3401,7 @@ func (a *maxDatumHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -3387,7 +3422,7 @@ func (a *maxDatumHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -3397,13 +3432,22 @@ func (a *maxDatumHashAgg) Flush(outputIdx int) { if a.curAgg != nil { oldCurAggSize = a.curAgg.(tree.Datum).Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *maxDatumHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + + var oldCurAggSize uintptr + if a.curAgg != nil { + oldCurAggSize = a.curAgg.(tree.Datum).Size() + } + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type maxDatumHashAggAlloc struct { diff --git a/pkg/sql/colexec/colexecagg/min_max_agg_tmpl.go b/pkg/sql/colexec/colexecagg/min_max_agg_tmpl.go index 23919be3330b..749febcaca1d 100644 --- a/pkg/sql/colexec/colexecagg/min_max_agg_tmpl.go +++ b/pkg/sql/colexec/colexecagg/min_max_agg_tmpl.go @@ -97,11 +97,11 @@ type _AGG_TYPE_AGGKINDAgg struct { col _GOTYPESLICE // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg _GOTYPE - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &_AGG_TYPE_AGGKINDAgg{} @@ -192,14 +192,20 @@ func (a *_AGG_TYPE_AGGKINDAgg) Flush(outputIdx int) { outputIdx = a.curIdx a.curIdx++ // {{end}} - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // {{if eq "_AGGKIND" "Window"}} + // We need to copy the value because window functions reuse the aggregation + // between rows. + execgen.COPYVAL(a.curAgg, a.curAgg) + // {{end}} a.col.Set(outputIdx, a.curAgg) } - // {{if or (.IsBytesLike) (eq .VecMethod "Datum")}} + // {{if and (not (eq "_AGGKIND" "Window")) (or (.IsBytesLike) (eq .VecMethod "Datum"))}} execgen.SETVARIABLESIZE(oldCurAggSize, a.curAgg) - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil // {{end}} @@ -209,7 +215,13 @@ func (a *_AGG_TYPE_AGGKINDAgg) Reset() { // {{if eq "_AGGKIND" "Ordered"}} a.orderedAggregateFuncBase.Reset() // {{end}} - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + // {{if or (.IsBytesLike) (eq .VecMethod "Datum")}} + execgen.SETVARIABLESIZE(oldCurAggSize, a.curAgg) + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil + // {{end}} } type _AGG_TYPE_AGGKINDAggAlloc struct { @@ -233,6 +245,17 @@ func (a *_AGG_TYPE_AGGKINDAggAlloc) newAggFunc() AggregateFunc { return f } +// {{if eq "_AGGKIND" "Window"}} + +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*_AGG_TYPE_AGGKINDAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on _AGG_TYPE_AGGKINDAgg")) +} + +// {{end}} // {{end}} // {{end}} // {{end}} @@ -255,13 +278,13 @@ func _ACCUMULATE_MINMAX( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -274,7 +297,7 @@ func _ACCUMULATE_MINMAX( isNull = false // {{end}} if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { // {{if and (.Sliceable) (not .HasSel)}} //gcassert:bce // {{end}} @@ -282,7 +305,6 @@ func _ACCUMULATE_MINMAX( // {{with .Global}} execgen.COPYVAL(a.curAgg, val) // {{end}} - a.foundNonNullForCurrentGroup = true } else { var cmp bool // {{if and (.Sliceable) (not .HasSel)}} @@ -296,6 +318,7 @@ func _ACCUMULATE_MINMAX( } // {{end}} } + a.numNonNull++ } // {{end}} diff --git a/pkg/sql/colexec/colexecagg/ordered_min_max_agg.eg.go b/pkg/sql/colexec/colexecagg/ordered_min_max_agg.eg.go index 5de5ff46bd41..2e8cec106e34 100644 --- a/pkg/sql/colexec/colexecagg/ordered_min_max_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/ordered_min_max_agg.eg.go @@ -116,11 +116,11 @@ type minBoolOrderedAgg struct { col coldata.Bools // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg bool - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minBoolOrderedAgg{} @@ -152,13 +152,13 @@ func (a *minBoolOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -166,10 +166,9 @@ func (a *minBoolOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -192,6 +191,7 @@ func (a *minBoolOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -202,13 +202,13 @@ func (a *minBoolOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -216,10 +216,9 @@ func (a *minBoolOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -242,6 +241,7 @@ func (a *minBoolOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -254,13 +254,13 @@ func (a *minBoolOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -268,10 +268,9 @@ func (a *minBoolOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -294,6 +293,7 @@ func (a *minBoolOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -303,13 +303,13 @@ func (a *minBoolOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -317,10 +317,9 @@ func (a *minBoolOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -343,6 +342,7 @@ func (a *minBoolOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -363,7 +363,7 @@ func (a *minBoolOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -372,7 +372,7 @@ func (a *minBoolOrderedAgg) Flush(outputIdx int) { func (a *minBoolOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minBoolOrderedAggAlloc struct { @@ -402,11 +402,11 @@ type minBytesOrderedAgg struct { col *coldata.Bytes // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg []byte - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minBytesOrderedAgg{} @@ -438,13 +438,13 @@ func (a *minBytesOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -452,10 +452,9 @@ func (a *minBytesOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -470,6 +469,7 @@ func (a *minBytesOrderedAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } else { @@ -480,13 +480,13 @@ func (a *minBytesOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -494,10 +494,9 @@ func (a *minBytesOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -512,6 +511,7 @@ func (a *minBytesOrderedAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } @@ -524,13 +524,13 @@ func (a *minBytesOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -538,10 +538,9 @@ func (a *minBytesOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -556,6 +555,7 @@ func (a *minBytesOrderedAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } else { @@ -565,13 +565,13 @@ func (a *minBytesOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -579,10 +579,9 @@ func (a *minBytesOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -597,6 +596,7 @@ func (a *minBytesOrderedAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } @@ -617,20 +617,25 @@ func (a *minBytesOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) } oldCurAggSize := len(a.curAgg) - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *minBytesOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + oldCurAggSize := len(a.curAgg) + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type minBytesOrderedAggAlloc struct { @@ -660,11 +665,11 @@ type minDecimalOrderedAgg struct { col coldata.Decimals // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minDecimalOrderedAgg{} @@ -696,13 +701,13 @@ func (a *minDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -710,10 +715,9 @@ func (a *minDecimalOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -728,6 +732,7 @@ func (a *minDecimalOrderedAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } else { @@ -738,13 +743,13 @@ func (a *minDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -752,10 +757,9 @@ func (a *minDecimalOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -770,6 +774,7 @@ func (a *minDecimalOrderedAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } @@ -782,13 +787,13 @@ func (a *minDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -796,10 +801,9 @@ func (a *minDecimalOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -814,6 +818,7 @@ func (a *minDecimalOrderedAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } else { @@ -823,13 +828,13 @@ func (a *minDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -837,10 +842,9 @@ func (a *minDecimalOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -855,6 +859,7 @@ func (a *minDecimalOrderedAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } @@ -875,7 +880,7 @@ func (a *minDecimalOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -884,7 +889,7 @@ func (a *minDecimalOrderedAgg) Flush(outputIdx int) { func (a *minDecimalOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minDecimalOrderedAggAlloc struct { @@ -914,11 +919,11 @@ type minInt16OrderedAgg struct { col coldata.Int16s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int16 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt16OrderedAgg{} @@ -950,13 +955,13 @@ func (a *minInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -964,10 +969,9 @@ func (a *minInt16OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -993,6 +997,7 @@ func (a *minInt16OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1003,13 +1008,13 @@ func (a *minInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1017,10 +1022,9 @@ func (a *minInt16OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1046,6 +1050,7 @@ func (a *minInt16OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1058,13 +1063,13 @@ func (a *minInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1072,10 +1077,9 @@ func (a *minInt16OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1101,6 +1105,7 @@ func (a *minInt16OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1110,13 +1115,13 @@ func (a *minInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1124,10 +1129,9 @@ func (a *minInt16OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1153,6 +1157,7 @@ func (a *minInt16OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1173,7 +1178,7 @@ func (a *minInt16OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1182,7 +1187,7 @@ func (a *minInt16OrderedAgg) Flush(outputIdx int) { func (a *minInt16OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt16OrderedAggAlloc struct { @@ -1212,11 +1217,11 @@ type minInt32OrderedAgg struct { col coldata.Int32s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int32 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt32OrderedAgg{} @@ -1248,13 +1253,13 @@ func (a *minInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1262,10 +1267,9 @@ func (a *minInt32OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1291,6 +1295,7 @@ func (a *minInt32OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1301,13 +1306,13 @@ func (a *minInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1315,10 +1320,9 @@ func (a *minInt32OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1344,6 +1348,7 @@ func (a *minInt32OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1356,13 +1361,13 @@ func (a *minInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1370,10 +1375,9 @@ func (a *minInt32OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1399,6 +1403,7 @@ func (a *minInt32OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1408,13 +1413,13 @@ func (a *minInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1422,10 +1427,9 @@ func (a *minInt32OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1451,6 +1455,7 @@ func (a *minInt32OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1471,7 +1476,7 @@ func (a *minInt32OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1480,7 +1485,7 @@ func (a *minInt32OrderedAgg) Flush(outputIdx int) { func (a *minInt32OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt32OrderedAggAlloc struct { @@ -1510,11 +1515,11 @@ type minInt64OrderedAgg struct { col coldata.Int64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt64OrderedAgg{} @@ -1546,13 +1551,13 @@ func (a *minInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1560,10 +1565,9 @@ func (a *minInt64OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1589,6 +1593,7 @@ func (a *minInt64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1599,13 +1604,13 @@ func (a *minInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1613,10 +1618,9 @@ func (a *minInt64OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1642,6 +1646,7 @@ func (a *minInt64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1654,13 +1659,13 @@ func (a *minInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1668,10 +1673,9 @@ func (a *minInt64OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1697,6 +1701,7 @@ func (a *minInt64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1706,13 +1711,13 @@ func (a *minInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1720,10 +1725,9 @@ func (a *minInt64OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1749,6 +1753,7 @@ func (a *minInt64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1769,7 +1774,7 @@ func (a *minInt64OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1778,7 +1783,7 @@ func (a *minInt64OrderedAgg) Flush(outputIdx int) { func (a *minInt64OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt64OrderedAggAlloc struct { @@ -1808,11 +1813,11 @@ type minFloat64OrderedAgg struct { col coldata.Float64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minFloat64OrderedAgg{} @@ -1844,13 +1849,13 @@ func (a *minFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1858,10 +1863,9 @@ func (a *minFloat64OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1895,6 +1899,7 @@ func (a *minFloat64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1905,13 +1910,13 @@ func (a *minFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1919,10 +1924,9 @@ func (a *minFloat64OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1956,6 +1960,7 @@ func (a *minFloat64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1968,13 +1973,13 @@ func (a *minFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1982,10 +1987,9 @@ func (a *minFloat64OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2019,6 +2023,7 @@ func (a *minFloat64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2028,13 +2033,13 @@ func (a *minFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2042,10 +2047,9 @@ func (a *minFloat64OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2079,6 +2083,7 @@ func (a *minFloat64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2099,7 +2104,7 @@ func (a *minFloat64OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2108,7 +2113,7 @@ func (a *minFloat64OrderedAgg) Flush(outputIdx int) { func (a *minFloat64OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minFloat64OrderedAggAlloc struct { @@ -2138,11 +2143,11 @@ type minTimestampOrderedAgg struct { col coldata.Times // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg time.Time - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minTimestampOrderedAgg{} @@ -2174,13 +2179,13 @@ func (a *minTimestampOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2188,10 +2193,9 @@ func (a *minTimestampOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2213,6 +2217,7 @@ func (a *minTimestampOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2223,13 +2228,13 @@ func (a *minTimestampOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2237,10 +2242,9 @@ func (a *minTimestampOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2262,6 +2266,7 @@ func (a *minTimestampOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2274,13 +2279,13 @@ func (a *minTimestampOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2288,10 +2293,9 @@ func (a *minTimestampOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2313,6 +2317,7 @@ func (a *minTimestampOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2322,13 +2327,13 @@ func (a *minTimestampOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2336,10 +2341,9 @@ func (a *minTimestampOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2361,6 +2365,7 @@ func (a *minTimestampOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2381,7 +2386,7 @@ func (a *minTimestampOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2390,7 +2395,7 @@ func (a *minTimestampOrderedAgg) Flush(outputIdx int) { func (a *minTimestampOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minTimestampOrderedAggAlloc struct { @@ -2420,11 +2425,11 @@ type minIntervalOrderedAgg struct { col coldata.Durations // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minIntervalOrderedAgg{} @@ -2456,13 +2461,13 @@ func (a *minIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2470,10 +2475,9 @@ func (a *minIntervalOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2488,6 +2492,7 @@ func (a *minIntervalOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2498,13 +2503,13 @@ func (a *minIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2512,10 +2517,9 @@ func (a *minIntervalOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2530,6 +2534,7 @@ func (a *minIntervalOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2542,13 +2547,13 @@ func (a *minIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2556,10 +2561,9 @@ func (a *minIntervalOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2574,6 +2578,7 @@ func (a *minIntervalOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2583,13 +2588,13 @@ func (a *minIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2597,10 +2602,9 @@ func (a *minIntervalOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2615,6 +2619,7 @@ func (a *minIntervalOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2635,7 +2640,7 @@ func (a *minIntervalOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2644,7 +2649,7 @@ func (a *minIntervalOrderedAgg) Flush(outputIdx int) { func (a *minIntervalOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minIntervalOrderedAggAlloc struct { @@ -2674,11 +2679,11 @@ type minJSONOrderedAgg struct { col *coldata.JSONs // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg json.JSON - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minJSONOrderedAgg{} @@ -2713,13 +2718,13 @@ func (a *minJSONOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2727,7 +2732,7 @@ func (a *minJSONOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -2741,7 +2746,6 @@ func (a *minJSONOrderedAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2773,6 +2777,7 @@ func (a *minJSONOrderedAgg) Compute( } } + a.numNonNull++ } } } else { @@ -2783,13 +2788,13 @@ func (a *minJSONOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2797,7 +2802,7 @@ func (a *minJSONOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -2811,7 +2816,6 @@ func (a *minJSONOrderedAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2843,6 +2847,7 @@ func (a *minJSONOrderedAgg) Compute( } } + a.numNonNull++ } } } @@ -2855,13 +2860,13 @@ func (a *minJSONOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2869,7 +2874,7 @@ func (a *minJSONOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -2883,7 +2888,6 @@ func (a *minJSONOrderedAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2915,6 +2919,7 @@ func (a *minJSONOrderedAgg) Compute( } } + a.numNonNull++ } } } else { @@ -2924,13 +2929,13 @@ func (a *minJSONOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2938,7 +2943,7 @@ func (a *minJSONOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -2952,7 +2957,6 @@ func (a *minJSONOrderedAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2984,6 +2988,7 @@ func (a *minJSONOrderedAgg) Compute( } } + a.numNonNull++ } } } @@ -3007,7 +3012,7 @@ func (a *minJSONOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -3016,14 +3021,22 @@ func (a *minJSONOrderedAgg) Flush(outputIdx int) { if a.curAgg != nil { oldCurAggSize = a.curAgg.Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *minJSONOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + var oldCurAggSize uintptr + if a.curAgg != nil { + oldCurAggSize = a.curAgg.Size() + } + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type minJSONOrderedAggAlloc struct { @@ -3053,11 +3066,11 @@ type minDatumOrderedAgg struct { col coldata.DatumVec // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg interface{} - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minDatumOrderedAgg{} @@ -3093,13 +3106,13 @@ func (a *minDatumOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3107,10 +3120,9 @@ func (a *minDatumOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3127,6 +3139,7 @@ func (a *minDatumOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -3137,13 +3150,13 @@ func (a *minDatumOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3151,10 +3164,9 @@ func (a *minDatumOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3171,6 +3183,7 @@ func (a *minDatumOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -3183,13 +3196,13 @@ func (a *minDatumOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3197,10 +3210,9 @@ func (a *minDatumOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3217,6 +3229,7 @@ func (a *minDatumOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -3226,13 +3239,13 @@ func (a *minDatumOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3240,10 +3253,9 @@ func (a *minDatumOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3260,6 +3272,7 @@ func (a *minDatumOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -3284,7 +3297,7 @@ func (a *minDatumOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -3294,14 +3307,23 @@ func (a *minDatumOrderedAgg) Flush(outputIdx int) { if a.curAgg != nil { oldCurAggSize = a.curAgg.(tree.Datum).Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *minDatumOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + + var oldCurAggSize uintptr + if a.curAgg != nil { + oldCurAggSize = a.curAgg.(tree.Datum).Size() + } + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type minDatumOrderedAggAlloc struct { @@ -3400,11 +3422,11 @@ type maxBoolOrderedAgg struct { col coldata.Bools // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg bool - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxBoolOrderedAgg{} @@ -3436,13 +3458,13 @@ func (a *maxBoolOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3450,10 +3472,9 @@ func (a *maxBoolOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3476,6 +3497,7 @@ func (a *maxBoolOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -3486,13 +3508,13 @@ func (a *maxBoolOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3500,10 +3522,9 @@ func (a *maxBoolOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3526,6 +3547,7 @@ func (a *maxBoolOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -3538,13 +3560,13 @@ func (a *maxBoolOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3552,10 +3574,9 @@ func (a *maxBoolOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3578,6 +3599,7 @@ func (a *maxBoolOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -3587,13 +3609,13 @@ func (a *maxBoolOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3601,10 +3623,9 @@ func (a *maxBoolOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3627,6 +3648,7 @@ func (a *maxBoolOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -3647,7 +3669,7 @@ func (a *maxBoolOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -3656,7 +3678,7 @@ func (a *maxBoolOrderedAgg) Flush(outputIdx int) { func (a *maxBoolOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxBoolOrderedAggAlloc struct { @@ -3686,11 +3708,11 @@ type maxBytesOrderedAgg struct { col *coldata.Bytes // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg []byte - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxBytesOrderedAgg{} @@ -3722,13 +3744,13 @@ func (a *maxBytesOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3736,10 +3758,9 @@ func (a *maxBytesOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3754,6 +3775,7 @@ func (a *maxBytesOrderedAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } else { @@ -3764,13 +3786,13 @@ func (a *maxBytesOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3778,10 +3800,9 @@ func (a *maxBytesOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3796,6 +3817,7 @@ func (a *maxBytesOrderedAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } @@ -3808,13 +3830,13 @@ func (a *maxBytesOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3822,10 +3844,9 @@ func (a *maxBytesOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3840,6 +3861,7 @@ func (a *maxBytesOrderedAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } else { @@ -3849,13 +3871,13 @@ func (a *maxBytesOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3863,10 +3885,9 @@ func (a *maxBytesOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3881,6 +3902,7 @@ func (a *maxBytesOrderedAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } @@ -3901,20 +3923,25 @@ func (a *maxBytesOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) } oldCurAggSize := len(a.curAgg) - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *maxBytesOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + oldCurAggSize := len(a.curAgg) + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type maxBytesOrderedAggAlloc struct { @@ -3944,11 +3971,11 @@ type maxDecimalOrderedAgg struct { col coldata.Decimals // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxDecimalOrderedAgg{} @@ -3980,13 +4007,13 @@ func (a *maxDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3994,10 +4021,9 @@ func (a *maxDecimalOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4012,6 +4038,7 @@ func (a *maxDecimalOrderedAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } else { @@ -4022,13 +4049,13 @@ func (a *maxDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4036,10 +4063,9 @@ func (a *maxDecimalOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4054,6 +4080,7 @@ func (a *maxDecimalOrderedAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } @@ -4066,13 +4093,13 @@ func (a *maxDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4080,10 +4107,9 @@ func (a *maxDecimalOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4098,6 +4124,7 @@ func (a *maxDecimalOrderedAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } else { @@ -4107,13 +4134,13 @@ func (a *maxDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4121,10 +4148,9 @@ func (a *maxDecimalOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4139,6 +4165,7 @@ func (a *maxDecimalOrderedAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } @@ -4159,7 +4186,7 @@ func (a *maxDecimalOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -4168,7 +4195,7 @@ func (a *maxDecimalOrderedAgg) Flush(outputIdx int) { func (a *maxDecimalOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxDecimalOrderedAggAlloc struct { @@ -4198,11 +4225,11 @@ type maxInt16OrderedAgg struct { col coldata.Int16s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int16 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt16OrderedAgg{} @@ -4234,13 +4261,13 @@ func (a *maxInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4248,10 +4275,9 @@ func (a *maxInt16OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4277,6 +4303,7 @@ func (a *maxInt16OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -4287,13 +4314,13 @@ func (a *maxInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4301,10 +4328,9 @@ func (a *maxInt16OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4330,6 +4356,7 @@ func (a *maxInt16OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -4342,13 +4369,13 @@ func (a *maxInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4356,10 +4383,9 @@ func (a *maxInt16OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4385,6 +4411,7 @@ func (a *maxInt16OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -4394,13 +4421,13 @@ func (a *maxInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4408,10 +4435,9 @@ func (a *maxInt16OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4437,6 +4463,7 @@ func (a *maxInt16OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -4457,7 +4484,7 @@ func (a *maxInt16OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -4466,7 +4493,7 @@ func (a *maxInt16OrderedAgg) Flush(outputIdx int) { func (a *maxInt16OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt16OrderedAggAlloc struct { @@ -4496,11 +4523,11 @@ type maxInt32OrderedAgg struct { col coldata.Int32s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int32 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt32OrderedAgg{} @@ -4532,13 +4559,13 @@ func (a *maxInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4546,10 +4573,9 @@ func (a *maxInt32OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4575,6 +4601,7 @@ func (a *maxInt32OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -4585,13 +4612,13 @@ func (a *maxInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4599,10 +4626,9 @@ func (a *maxInt32OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4628,6 +4654,7 @@ func (a *maxInt32OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -4640,13 +4667,13 @@ func (a *maxInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4654,10 +4681,9 @@ func (a *maxInt32OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4683,6 +4709,7 @@ func (a *maxInt32OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -4692,13 +4719,13 @@ func (a *maxInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4706,10 +4733,9 @@ func (a *maxInt32OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4735,6 +4761,7 @@ func (a *maxInt32OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -4755,7 +4782,7 @@ func (a *maxInt32OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -4764,7 +4791,7 @@ func (a *maxInt32OrderedAgg) Flush(outputIdx int) { func (a *maxInt32OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt32OrderedAggAlloc struct { @@ -4794,11 +4821,11 @@ type maxInt64OrderedAgg struct { col coldata.Int64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt64OrderedAgg{} @@ -4830,13 +4857,13 @@ func (a *maxInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4844,10 +4871,9 @@ func (a *maxInt64OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4873,6 +4899,7 @@ func (a *maxInt64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -4883,13 +4910,13 @@ func (a *maxInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4897,10 +4924,9 @@ func (a *maxInt64OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4926,6 +4952,7 @@ func (a *maxInt64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -4938,13 +4965,13 @@ func (a *maxInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4952,10 +4979,9 @@ func (a *maxInt64OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4981,6 +5007,7 @@ func (a *maxInt64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -4990,13 +5017,13 @@ func (a *maxInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5004,10 +5031,9 @@ func (a *maxInt64OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5033,6 +5059,7 @@ func (a *maxInt64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -5053,7 +5080,7 @@ func (a *maxInt64OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -5062,7 +5089,7 @@ func (a *maxInt64OrderedAgg) Flush(outputIdx int) { func (a *maxInt64OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt64OrderedAggAlloc struct { @@ -5092,11 +5119,11 @@ type maxFloat64OrderedAgg struct { col coldata.Float64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxFloat64OrderedAgg{} @@ -5128,13 +5155,13 @@ func (a *maxFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5142,10 +5169,9 @@ func (a *maxFloat64OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5179,6 +5205,7 @@ func (a *maxFloat64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -5189,13 +5216,13 @@ func (a *maxFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5203,10 +5230,9 @@ func (a *maxFloat64OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5240,6 +5266,7 @@ func (a *maxFloat64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -5252,13 +5279,13 @@ func (a *maxFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5266,10 +5293,9 @@ func (a *maxFloat64OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5303,6 +5329,7 @@ func (a *maxFloat64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -5312,13 +5339,13 @@ func (a *maxFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5326,10 +5353,9 @@ func (a *maxFloat64OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5363,6 +5389,7 @@ func (a *maxFloat64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -5383,7 +5410,7 @@ func (a *maxFloat64OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -5392,7 +5419,7 @@ func (a *maxFloat64OrderedAgg) Flush(outputIdx int) { func (a *maxFloat64OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxFloat64OrderedAggAlloc struct { @@ -5422,11 +5449,11 @@ type maxTimestampOrderedAgg struct { col coldata.Times // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg time.Time - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxTimestampOrderedAgg{} @@ -5458,13 +5485,13 @@ func (a *maxTimestampOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5472,10 +5499,9 @@ func (a *maxTimestampOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5497,6 +5523,7 @@ func (a *maxTimestampOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -5507,13 +5534,13 @@ func (a *maxTimestampOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5521,10 +5548,9 @@ func (a *maxTimestampOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5546,6 +5572,7 @@ func (a *maxTimestampOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -5558,13 +5585,13 @@ func (a *maxTimestampOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5572,10 +5599,9 @@ func (a *maxTimestampOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5597,6 +5623,7 @@ func (a *maxTimestampOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -5606,13 +5633,13 @@ func (a *maxTimestampOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5620,10 +5647,9 @@ func (a *maxTimestampOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5645,6 +5671,7 @@ func (a *maxTimestampOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -5665,7 +5692,7 @@ func (a *maxTimestampOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -5674,7 +5701,7 @@ func (a *maxTimestampOrderedAgg) Flush(outputIdx int) { func (a *maxTimestampOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxTimestampOrderedAggAlloc struct { @@ -5704,11 +5731,11 @@ type maxIntervalOrderedAgg struct { col coldata.Durations // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxIntervalOrderedAgg{} @@ -5740,13 +5767,13 @@ func (a *maxIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5754,10 +5781,9 @@ func (a *maxIntervalOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5772,6 +5798,7 @@ func (a *maxIntervalOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -5782,13 +5809,13 @@ func (a *maxIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5796,10 +5823,9 @@ func (a *maxIntervalOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5814,6 +5840,7 @@ func (a *maxIntervalOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -5826,13 +5853,13 @@ func (a *maxIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5840,10 +5867,9 @@ func (a *maxIntervalOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5858,6 +5884,7 @@ func (a *maxIntervalOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -5867,13 +5894,13 @@ func (a *maxIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5881,10 +5908,9 @@ func (a *maxIntervalOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5899,6 +5925,7 @@ func (a *maxIntervalOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -5919,7 +5946,7 @@ func (a *maxIntervalOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -5928,7 +5955,7 @@ func (a *maxIntervalOrderedAgg) Flush(outputIdx int) { func (a *maxIntervalOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxIntervalOrderedAggAlloc struct { @@ -5958,11 +5985,11 @@ type maxJSONOrderedAgg struct { col *coldata.JSONs // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg json.JSON - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxJSONOrderedAgg{} @@ -5997,13 +6024,13 @@ func (a *maxJSONOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -6011,7 +6038,7 @@ func (a *maxJSONOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -6025,7 +6052,6 @@ func (a *maxJSONOrderedAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -6057,6 +6083,7 @@ func (a *maxJSONOrderedAgg) Compute( } } + a.numNonNull++ } } } else { @@ -6067,13 +6094,13 @@ func (a *maxJSONOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -6081,7 +6108,7 @@ func (a *maxJSONOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -6095,7 +6122,6 @@ func (a *maxJSONOrderedAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -6127,6 +6153,7 @@ func (a *maxJSONOrderedAgg) Compute( } } + a.numNonNull++ } } } @@ -6139,13 +6166,13 @@ func (a *maxJSONOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -6153,7 +6180,7 @@ func (a *maxJSONOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -6167,7 +6194,6 @@ func (a *maxJSONOrderedAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -6199,6 +6225,7 @@ func (a *maxJSONOrderedAgg) Compute( } } + a.numNonNull++ } } } else { @@ -6208,13 +6235,13 @@ func (a *maxJSONOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -6222,7 +6249,7 @@ func (a *maxJSONOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -6236,7 +6263,6 @@ func (a *maxJSONOrderedAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -6268,6 +6294,7 @@ func (a *maxJSONOrderedAgg) Compute( } } + a.numNonNull++ } } } @@ -6291,7 +6318,7 @@ func (a *maxJSONOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -6300,14 +6327,22 @@ func (a *maxJSONOrderedAgg) Flush(outputIdx int) { if a.curAgg != nil { oldCurAggSize = a.curAgg.Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *maxJSONOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + var oldCurAggSize uintptr + if a.curAgg != nil { + oldCurAggSize = a.curAgg.Size() + } + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type maxJSONOrderedAggAlloc struct { @@ -6337,11 +6372,11 @@ type maxDatumOrderedAgg struct { col coldata.DatumVec // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg interface{} - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxDatumOrderedAgg{} @@ -6377,13 +6412,13 @@ func (a *maxDatumOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -6391,10 +6426,9 @@ func (a *maxDatumOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -6411,6 +6445,7 @@ func (a *maxDatumOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -6421,13 +6456,13 @@ func (a *maxDatumOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -6435,10 +6470,9 @@ func (a *maxDatumOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -6455,6 +6489,7 @@ func (a *maxDatumOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -6467,13 +6502,13 @@ func (a *maxDatumOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -6481,10 +6516,9 @@ func (a *maxDatumOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -6501,6 +6535,7 @@ func (a *maxDatumOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -6510,13 +6545,13 @@ func (a *maxDatumOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -6524,10 +6559,9 @@ func (a *maxDatumOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -6544,6 +6578,7 @@ func (a *maxDatumOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -6568,7 +6603,7 @@ func (a *maxDatumOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -6578,14 +6613,23 @@ func (a *maxDatumOrderedAgg) Flush(outputIdx int) { if a.curAgg != nil { oldCurAggSize = a.curAgg.(tree.Datum).Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *maxDatumOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + + var oldCurAggSize uintptr + if a.curAgg != nil { + oldCurAggSize = a.curAgg.(tree.Datum).Size() + } + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type maxDatumOrderedAggAlloc struct { diff --git a/pkg/sql/colexec/colexecagg/window_min_max_agg.eg.go b/pkg/sql/colexec/colexecagg/window_min_max_agg.eg.go index 74fc138cad4a..e969b2270340 100644 --- a/pkg/sql/colexec/colexecagg/window_min_max_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/window_min_max_agg.eg.go @@ -116,11 +116,11 @@ type minBoolWindowAgg struct { col coldata.Bools // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg bool - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minBoolWindowAgg{} @@ -146,10 +146,9 @@ func (a *minBoolWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -172,6 +171,7 @@ func (a *minBoolWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -180,10 +180,9 @@ func (a *minBoolWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -206,6 +205,7 @@ func (a *minBoolWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -219,15 +219,18 @@ func (a *minBoolWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *minBoolWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minBoolWindowAggAlloc struct { @@ -251,17 +254,25 @@ func (a *minBoolWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minBoolWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minBoolWindowAgg")) +} + type minBytesWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col *coldata.Bytes // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg []byte - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minBytesWindowAgg{} @@ -287,10 +298,9 @@ func (a *minBytesWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -305,6 +315,7 @@ func (a *minBytesWindowAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } else { @@ -313,10 +324,9 @@ func (a *minBytesWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -331,6 +341,7 @@ func (a *minBytesWindowAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } @@ -344,19 +355,22 @@ func (a *minBytesWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = append(a.curAgg[:0], a.curAgg...) a.col.Set(outputIdx, a.curAgg) } - oldCurAggSize := len(a.curAgg) - // Release the reference to curAgg eagerly. - a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) - a.curAgg = nil } func (a *minBytesWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + oldCurAggSize := len(a.curAgg) + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type minBytesWindowAggAlloc struct { @@ -380,17 +394,25 @@ func (a *minBytesWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minBytesWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minBytesWindowAgg")) +} + type minDecimalWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Decimals // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minDecimalWindowAgg{} @@ -416,10 +438,9 @@ func (a *minDecimalWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -434,6 +455,7 @@ func (a *minDecimalWindowAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } else { @@ -442,10 +464,9 @@ func (a *minDecimalWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -460,6 +481,7 @@ func (a *minDecimalWindowAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } @@ -473,15 +495,18 @@ func (a *minDecimalWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg.Set(&a.curAgg) a.col.Set(outputIdx, a.curAgg) } } func (a *minDecimalWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minDecimalWindowAggAlloc struct { @@ -505,17 +530,25 @@ func (a *minDecimalWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minDecimalWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minDecimalWindowAgg")) +} + type minInt16WindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Int16s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int16 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt16WindowAgg{} @@ -541,10 +574,9 @@ func (a *minInt16WindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -570,6 +602,7 @@ func (a *minInt16WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -578,10 +611,9 @@ func (a *minInt16WindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -607,6 +639,7 @@ func (a *minInt16WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -620,15 +653,18 @@ func (a *minInt16WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *minInt16WindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt16WindowAggAlloc struct { @@ -652,17 +688,25 @@ func (a *minInt16WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minInt16WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minInt16WindowAgg")) +} + type minInt32WindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Int32s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int32 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt32WindowAgg{} @@ -688,10 +732,9 @@ func (a *minInt32WindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -717,6 +760,7 @@ func (a *minInt32WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -725,10 +769,9 @@ func (a *minInt32WindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -754,6 +797,7 @@ func (a *minInt32WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -767,15 +811,18 @@ func (a *minInt32WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *minInt32WindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt32WindowAggAlloc struct { @@ -799,17 +846,25 @@ func (a *minInt32WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minInt32WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minInt32WindowAgg")) +} + type minInt64WindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Int64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt64WindowAgg{} @@ -835,10 +890,9 @@ func (a *minInt64WindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -864,6 +918,7 @@ func (a *minInt64WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -872,10 +927,9 @@ func (a *minInt64WindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -901,6 +955,7 @@ func (a *minInt64WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -914,15 +969,18 @@ func (a *minInt64WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *minInt64WindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt64WindowAggAlloc struct { @@ -946,17 +1004,25 @@ func (a *minInt64WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minInt64WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minInt64WindowAgg")) +} + type minFloat64WindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Float64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minFloat64WindowAgg{} @@ -982,10 +1048,9 @@ func (a *minFloat64WindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1019,6 +1084,7 @@ func (a *minFloat64WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1027,10 +1093,9 @@ func (a *minFloat64WindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1064,6 +1129,7 @@ func (a *minFloat64WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1077,15 +1143,18 @@ func (a *minFloat64WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *minFloat64WindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minFloat64WindowAggAlloc struct { @@ -1109,17 +1178,25 @@ func (a *minFloat64WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minFloat64WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minFloat64WindowAgg")) +} + type minTimestampWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Times // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg time.Time - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minTimestampWindowAgg{} @@ -1145,10 +1222,9 @@ func (a *minTimestampWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1170,6 +1246,7 @@ func (a *minTimestampWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1178,10 +1255,9 @@ func (a *minTimestampWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1203,6 +1279,7 @@ func (a *minTimestampWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1216,15 +1293,18 @@ func (a *minTimestampWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *minTimestampWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minTimestampWindowAggAlloc struct { @@ -1248,17 +1328,25 @@ func (a *minTimestampWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minTimestampWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minTimestampWindowAgg")) +} + type minIntervalWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Durations // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minIntervalWindowAgg{} @@ -1284,10 +1372,9 @@ func (a *minIntervalWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1302,6 +1389,7 @@ func (a *minIntervalWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1310,10 +1398,9 @@ func (a *minIntervalWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1328,6 +1415,7 @@ func (a *minIntervalWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1341,15 +1429,18 @@ func (a *minIntervalWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *minIntervalWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minIntervalWindowAggAlloc struct { @@ -1373,17 +1464,25 @@ func (a *minIntervalWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minIntervalWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minIntervalWindowAgg")) +} + type minJSONWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col *coldata.JSONs // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg json.JSON - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minJSONWindowAgg{} @@ -1412,7 +1511,7 @@ func (a *minJSONWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -1426,7 +1525,6 @@ func (a *minJSONWindowAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1458,6 +1556,7 @@ func (a *minJSONWindowAgg) Compute( } } + a.numNonNull++ } } } else { @@ -1466,7 +1565,7 @@ func (a *minJSONWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -1480,7 +1579,6 @@ func (a *minJSONWindowAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1512,6 +1610,7 @@ func (a *minJSONWindowAgg) Compute( } } + a.numNonNull++ } } } @@ -1528,24 +1627,38 @@ func (a *minJSONWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + + var _err error + var _bytes []byte + _bytes, _err = json.EncodeJSON(nil, a.curAgg) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.curAgg, _err = json.FromEncoding(_bytes) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.col.Set(outputIdx, a.curAgg) } +} + +func (a *minJSONWindowAgg) Reset() { + a.numNonNull = 0 var oldCurAggSize uintptr if a.curAgg != nil { oldCurAggSize = a.curAgg.Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } -func (a *minJSONWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false -} - type minJSONWindowAggAlloc struct { aggAllocBase aggFuncs []minJSONWindowAgg @@ -1567,17 +1680,25 @@ func (a *minJSONWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minJSONWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minJSONWindowAgg")) +} + type minDatumWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.DatumVec // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg interface{} - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minDatumWindowAgg{} @@ -1607,10 +1728,9 @@ func (a *minDatumWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1627,6 +1747,7 @@ func (a *minDatumWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1635,10 +1756,9 @@ func (a *minDatumWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1655,6 +1775,7 @@ func (a *minDatumWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1672,25 +1793,28 @@ func (a *minDatumWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } +} + +func (a *minDatumWindowAgg) Reset() { + a.numNonNull = 0 var oldCurAggSize uintptr if a.curAgg != nil { oldCurAggSize = a.curAgg.(tree.Datum).Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } -func (a *minDatumWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false -} - type minDatumWindowAggAlloc struct { aggAllocBase aggFuncs []minDatumWindowAgg @@ -1712,6 +1836,14 @@ func (a *minDatumWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minDatumWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minDatumWindowAgg")) +} + func newMaxWindowAggAlloc( allocator *colmem.Allocator, t *types.T, allocSize int64, ) aggregateFuncAlloc { @@ -1787,11 +1919,11 @@ type maxBoolWindowAgg struct { col coldata.Bools // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg bool - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxBoolWindowAgg{} @@ -1817,10 +1949,9 @@ func (a *maxBoolWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1843,6 +1974,7 @@ func (a *maxBoolWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1851,10 +1983,9 @@ func (a *maxBoolWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1877,6 +2008,7 @@ func (a *maxBoolWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1890,15 +2022,18 @@ func (a *maxBoolWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *maxBoolWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxBoolWindowAggAlloc struct { @@ -1922,17 +2057,25 @@ func (a *maxBoolWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxBoolWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxBoolWindowAgg")) +} + type maxBytesWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col *coldata.Bytes // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg []byte - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxBytesWindowAgg{} @@ -1958,10 +2101,9 @@ func (a *maxBytesWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1976,6 +2118,7 @@ func (a *maxBytesWindowAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } else { @@ -1984,10 +2127,9 @@ func (a *maxBytesWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2002,6 +2144,7 @@ func (a *maxBytesWindowAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } @@ -2015,19 +2158,22 @@ func (a *maxBytesWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = append(a.curAgg[:0], a.curAgg...) a.col.Set(outputIdx, a.curAgg) } - oldCurAggSize := len(a.curAgg) - // Release the reference to curAgg eagerly. - a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) - a.curAgg = nil } func (a *maxBytesWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + oldCurAggSize := len(a.curAgg) + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type maxBytesWindowAggAlloc struct { @@ -2051,17 +2197,25 @@ func (a *maxBytesWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxBytesWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxBytesWindowAgg")) +} + type maxDecimalWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Decimals // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxDecimalWindowAgg{} @@ -2087,10 +2241,9 @@ func (a *maxDecimalWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2105,6 +2258,7 @@ func (a *maxDecimalWindowAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } else { @@ -2113,10 +2267,9 @@ func (a *maxDecimalWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2131,6 +2284,7 @@ func (a *maxDecimalWindowAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } @@ -2144,15 +2298,18 @@ func (a *maxDecimalWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg.Set(&a.curAgg) a.col.Set(outputIdx, a.curAgg) } } func (a *maxDecimalWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxDecimalWindowAggAlloc struct { @@ -2176,17 +2333,25 @@ func (a *maxDecimalWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxDecimalWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxDecimalWindowAgg")) +} + type maxInt16WindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Int16s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int16 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt16WindowAgg{} @@ -2212,10 +2377,9 @@ func (a *maxInt16WindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2241,6 +2405,7 @@ func (a *maxInt16WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2249,10 +2414,9 @@ func (a *maxInt16WindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2278,6 +2442,7 @@ func (a *maxInt16WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2291,15 +2456,18 @@ func (a *maxInt16WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *maxInt16WindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt16WindowAggAlloc struct { @@ -2323,17 +2491,25 @@ func (a *maxInt16WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxInt16WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxInt16WindowAgg")) +} + type maxInt32WindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Int32s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int32 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt32WindowAgg{} @@ -2359,10 +2535,9 @@ func (a *maxInt32WindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2388,6 +2563,7 @@ func (a *maxInt32WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2396,10 +2572,9 @@ func (a *maxInt32WindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2425,6 +2600,7 @@ func (a *maxInt32WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2438,15 +2614,18 @@ func (a *maxInt32WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *maxInt32WindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt32WindowAggAlloc struct { @@ -2470,17 +2649,25 @@ func (a *maxInt32WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxInt32WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxInt32WindowAgg")) +} + type maxInt64WindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Int64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt64WindowAgg{} @@ -2506,10 +2693,9 @@ func (a *maxInt64WindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2535,6 +2721,7 @@ func (a *maxInt64WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2543,10 +2730,9 @@ func (a *maxInt64WindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2572,6 +2758,7 @@ func (a *maxInt64WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2585,15 +2772,18 @@ func (a *maxInt64WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *maxInt64WindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt64WindowAggAlloc struct { @@ -2617,17 +2807,25 @@ func (a *maxInt64WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxInt64WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxInt64WindowAgg")) +} + type maxFloat64WindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Float64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxFloat64WindowAgg{} @@ -2653,10 +2851,9 @@ func (a *maxFloat64WindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2690,6 +2887,7 @@ func (a *maxFloat64WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2698,10 +2896,9 @@ func (a *maxFloat64WindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2735,6 +2932,7 @@ func (a *maxFloat64WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2748,15 +2946,18 @@ func (a *maxFloat64WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *maxFloat64WindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxFloat64WindowAggAlloc struct { @@ -2780,17 +2981,25 @@ func (a *maxFloat64WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxFloat64WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxFloat64WindowAgg")) +} + type maxTimestampWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Times // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg time.Time - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxTimestampWindowAgg{} @@ -2816,10 +3025,9 @@ func (a *maxTimestampWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2841,6 +3049,7 @@ func (a *maxTimestampWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2849,10 +3058,9 @@ func (a *maxTimestampWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2874,6 +3082,7 @@ func (a *maxTimestampWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2887,15 +3096,18 @@ func (a *maxTimestampWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *maxTimestampWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxTimestampWindowAggAlloc struct { @@ -2919,17 +3131,25 @@ func (a *maxTimestampWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxTimestampWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxTimestampWindowAgg")) +} + type maxIntervalWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Durations // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxIntervalWindowAgg{} @@ -2955,10 +3175,9 @@ func (a *maxIntervalWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2973,6 +3192,7 @@ func (a *maxIntervalWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2981,10 +3201,9 @@ func (a *maxIntervalWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2999,6 +3218,7 @@ func (a *maxIntervalWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -3012,15 +3232,18 @@ func (a *maxIntervalWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *maxIntervalWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxIntervalWindowAggAlloc struct { @@ -3044,17 +3267,25 @@ func (a *maxIntervalWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxIntervalWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxIntervalWindowAgg")) +} + type maxJSONWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col *coldata.JSONs // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg json.JSON - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxJSONWindowAgg{} @@ -3083,7 +3314,7 @@ func (a *maxJSONWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -3097,7 +3328,6 @@ func (a *maxJSONWindowAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3129,6 +3359,7 @@ func (a *maxJSONWindowAgg) Compute( } } + a.numNonNull++ } } } else { @@ -3137,7 +3368,7 @@ func (a *maxJSONWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -3151,7 +3382,6 @@ func (a *maxJSONWindowAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3183,6 +3413,7 @@ func (a *maxJSONWindowAgg) Compute( } } + a.numNonNull++ } } } @@ -3199,24 +3430,38 @@ func (a *maxJSONWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + + var _err error + var _bytes []byte + _bytes, _err = json.EncodeJSON(nil, a.curAgg) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.curAgg, _err = json.FromEncoding(_bytes) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.col.Set(outputIdx, a.curAgg) } +} + +func (a *maxJSONWindowAgg) Reset() { + a.numNonNull = 0 var oldCurAggSize uintptr if a.curAgg != nil { oldCurAggSize = a.curAgg.Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } -func (a *maxJSONWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false -} - type maxJSONWindowAggAlloc struct { aggAllocBase aggFuncs []maxJSONWindowAgg @@ -3238,17 +3483,25 @@ func (a *maxJSONWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxJSONWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxJSONWindowAgg")) +} + type maxDatumWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.DatumVec // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg interface{} - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxDatumWindowAgg{} @@ -3278,10 +3531,9 @@ func (a *maxDatumWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3298,6 +3550,7 @@ func (a *maxDatumWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -3306,10 +3559,9 @@ func (a *maxDatumWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3326,6 +3578,7 @@ func (a *maxDatumWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -3343,25 +3596,28 @@ func (a *maxDatumWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } +} + +func (a *maxDatumWindowAgg) Reset() { + a.numNonNull = 0 var oldCurAggSize uintptr if a.curAgg != nil { oldCurAggSize = a.curAgg.(tree.Datum).Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } -func (a *maxDatumWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false -} - type maxDatumWindowAggAlloc struct { aggAllocBase aggFuncs []maxDatumWindowAgg @@ -3382,3 +3638,11 @@ func (a *maxDatumWindowAggAlloc) newAggFunc() AggregateFunc { a.aggFuncs = a.aggFuncs[1:] return f } + +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxDatumWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxDatumWindowAgg")) +} diff --git a/pkg/sql/colexec/colexecwindow/BUILD.bazel b/pkg/sql/colexec/colexecwindow/BUILD.bazel index 14370e2fd7db..91ca8ea0ca5f 100644 --- a/pkg/sql/colexec/colexecwindow/BUILD.bazel +++ b/pkg/sql/colexec/colexecwindow/BUILD.bazel @@ -6,6 +6,7 @@ go_library( srcs = [ "buffered_window.go", "count_rows_aggregator.go", + "min_max_queue.go", "partitioner.go", "window_functions_util.go", ":gen-exec", # keep @@ -28,6 +29,7 @@ go_library( "//pkg/sql/colmem", # keep "//pkg/sql/execinfra", # keep "//pkg/sql/execinfrapb", # keep + "//pkg/sql/memsize", # keep "//pkg/sql/randgen", "//pkg/sql/rowenc", # keep "//pkg/sql/sem/builtins", # keep @@ -35,6 +37,7 @@ go_library( "//pkg/sql/types", # keep "//pkg/util/duration", # keep "//pkg/util/encoding", + "//pkg/util/json", # keep "//pkg/util/mon", # keep "//pkg/util/timeutil/pgdate", # keep "@com_github_cockroachdb_apd_v2//:apd", # keep @@ -50,6 +53,7 @@ go_test( "dep_test.go", "inject_setup_test.go", "main_test.go", + "min_max_queue_test.go", "window_framer_test.go", "window_functions_test.go", ], @@ -94,6 +98,7 @@ targets = [ ("lag.eg.go", "lead_lag_tmpl.go"), ("last_value.eg.go", "first_last_nth_value_tmpl.go"), ("lead.eg.go", "lead_lag_tmpl.go"), + ("min_max_removable_agg.eg.go", "min_max_removable_agg_tmpl.go"), ("nth_value.eg.go", "first_last_nth_value_tmpl.go"), ("ntile.eg.go", "ntile_tmpl.go"), ("range_offset_handler.eg.go", "range_offset_handler_tmpl.go"), diff --git a/pkg/sql/colexec/colexecwindow/min_max_queue.go b/pkg/sql/colexec/colexecwindow/min_max_queue.go new file mode 100644 index 000000000000..abe8715bb8e2 --- /dev/null +++ b/pkg/sql/colexec/colexecwindow/min_max_queue.go @@ -0,0 +1,177 @@ +// 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 colexecwindow + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/errors" +) + +func newMinMaxQueue(maxLength int) minMaxQueue { + return minMaxQueue{maxLength: maxLength, empty: true} +} + +// minMaxQueue buffers uint32 values that are used by the min and max window +// functions to handle the case where the window frame for the current row need +// not include every row that was in the previous frame. minMaxQueue optimizes +// for operations on the start and end of the buffer, such as removing the last +// n values. +type minMaxQueue struct { + buffer []uint32 // an increasing circular buffer of values + head int // the index of the front of the buffer + tail int // the index of the first position after the end of the buffer + + // Indicates whether the buffer is empty. Necessary to distinguish between an + // empty buffer and a buffer that uses all of its capacity. + empty bool + + // The maximum size to which the minMaxQueue can grow. If reached, no new + // elements can be added to the minMaxQueue until some have been removed. + maxLength int +} + +// len returns the number of elements in the minMaxQueue. +// gcassert:inline +func (q *minMaxQueue) len() int { + if q.empty { + return 0 + } + if q.head < q.tail { + return q.tail - q.head + } + return cap(q.buffer) + q.tail - q.head +} + +// isEmpty returns true if the minMaxQueue has no elements. +// gcassert:inline +func (q *minMaxQueue) isEmpty() bool { + return q.empty +} + +// get returns the element at position pos in the minMaxQueue (zero-based). +// gcassert:inline +func (q *minMaxQueue) get(pos int) uint32 { + if q.empty || pos < 0 || pos >= q.len() { + colexecerror.InternalError(errors.AssertionFailedf("index out of bounds: %d", pos)) + } + return q.buffer[(pos+q.head)%cap(q.buffer)] +} + +// getFirst returns the element at the start of the minMaxQueue. +// gcassert:inline +func (q *minMaxQueue) getFirst() uint32 { + if q.empty { + colexecerror.InternalError(errors.AssertionFailedf("getting first from empty minMaxQueue")) + } + return q.buffer[q.head] +} + +// getLast returns the element at the end of the minMaxQueue. +// gcassert:inline +func (q *minMaxQueue) getLast() uint32 { + if q.empty { + colexecerror.InternalError(errors.AssertionFailedf("getting last from empty minMaxQueue")) + } + return q.buffer[(cap(q.buffer)+q.tail-1)%cap(q.buffer)] +} + +// addLast adds element to the end of the minMaxQueue and doubles it's +// underlying slice if necessary, subject to the max length limit. If the +// minMaxQueue has already reached the maximum length, addLast returns true, +// otherwise false. +// gcassert:inline +func (q *minMaxQueue) addLast(element uint32) (reachedLimit bool) { + if q.maybeGrow() { + return true + } + q.buffer[q.tail] = element + q.tail = (q.tail + 1) % cap(q.buffer) + q.empty = false + return false +} + +// removeLast removes a single element from the end of the minMaxQueue. +// gcassert:inline +func (q *minMaxQueue) removeLast() { + if q.empty { + colexecerror.InternalError(errors.AssertionFailedf("removing last from empty ring buffer")) + } + lastPos := (cap(q.buffer) + q.tail - 1) % cap(q.buffer) + q.tail = lastPos + if q.tail == q.head { + q.empty = true + } +} + +// removeAllBefore removes from the minMaxQueue all values in the range +// [0, val). +// gcassert:inline +func (q *minMaxQueue) removeAllBefore(val uint32) { + if q.empty { + return + } + var idx int + length := q.len() + for ; idx < length && q.get(idx) < val; idx++ { + } + if idx == length { + q.empty = true + } + q.head = (q.head + idx) % cap(q.buffer) +} + +func (q *minMaxQueue) grow(n int) { + newBuffer := make([]uint32, n) + if q.head < q.tail { + copy(newBuffer[:q.len()], q.buffer[q.head:q.tail]) + } else { + copy(newBuffer[:cap(q.buffer)-q.head], q.buffer[q.head:]) + copy(newBuffer[cap(q.buffer)-q.head:q.len()], q.buffer[:q.tail]) + } + q.head = 0 + q.tail = cap(q.buffer) + q.buffer = newBuffer +} + +// maybeGrow attempts to double the size of the minMaxQueue, capped at +// maxLength. If the minMaxQueue has already reached maxLength, returns true. +func (q *minMaxQueue) maybeGrow() (reachedLimit bool) { + if q.len() != cap(q.buffer) { + return false + } + if q.len() == q.maxLength { + return true + } + n := 2 * cap(q.buffer) + if n == 0 { + n = 1 + } + if n > q.maxLength { + n = q.maxLength + } + q.grow(n) + return false +} + +// Reset makes minMaxQueue treat its underlying memory as if it were empty. This +// allows for reusing the same memory again without explicitly removing old +// elements. +// gcassert:inline +func (q *minMaxQueue) reset() { + q.head = 0 + q.tail = 0 + q.empty = true +} + +// close releases the minMaxQueue's underlying memory. +func (q *minMaxQueue) close() { + q.buffer = nil +} diff --git a/pkg/sql/colexec/colexecwindow/min_max_queue_test.go b/pkg/sql/colexec/colexecwindow/min_max_queue_test.go new file mode 100644 index 000000000000..ceb48635942f --- /dev/null +++ b/pkg/sql/colexec/colexecwindow/min_max_queue_test.go @@ -0,0 +1,67 @@ +// 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 colexecwindow + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/util/randutil" + "github.com/stretchr/testify/require" +) + +func TestMinMaxQueue(t *testing.T) { + const ( + chanceToRemove = 0.1 + numIterations = 10 + maxIncrement = 100 + maxValuesToAdd = 1000 + ) + + rng, _ := randutil.NewPseudoRand() + + var queue minMaxQueue + var oracle []uint32 + + for i := 0; i < numIterations; i++ { + oracle = oracle[:0] + queue.reset() + queue.maxLength = rng.Intn(maxValuesToAdd) + if cap(queue.buffer) > queue.maxLength { + // We have to nil out the buffer to ensure that the queue does not grow + // too large, since it will fill out all available capacity. + queue.buffer = nil + } + valuesToAdd := rng.Intn(maxValuesToAdd) + var num uint32 + for j := 0; j < valuesToAdd; j++ { + num += uint32(rng.Intn(maxIncrement)) + 1 // Ensure no duplicates. + if len(oracle) < queue.maxLength { + oracle = append(oracle, num) + } + queue.addLast(num) + if len(oracle) > 0 && rng.Float64() < chanceToRemove { + idx := rng.Intn(len(oracle)) + val := oracle[idx] + oracle = append(oracle[:0], oracle[idx:]...) + queue.removeAllBefore(val) + for k := 0; k < rng.Intn(len(oracle)); k++ { + oracle = oracle[:len(oracle)-1] + queue.removeLast() + } + } + } + require.LessOrEqual(t, queue.len(), queue.maxLength, "queue length exceeds maximum") + require.Equal(t, len(oracle), queue.len(), "expected equal lengths") + for j := 0; j < len(oracle); j++ { + require.Equalf(t, oracle[j], queue.get(j), "wrong value at index: %d", j) + } + } +} diff --git a/pkg/sql/colexec/colexecwindow/min_max_removable_agg.eg.go b/pkg/sql/colexec/colexecwindow/min_max_removable_agg.eg.go new file mode 100644 index 000000000000..b5468c8aec90 --- /dev/null +++ b/pkg/sql/colexec/colexecwindow/min_max_removable_agg.eg.go @@ -0,0 +1,3978 @@ +// Code generated by execgen; DO NOT EDIT. +// Copyright 2021 The Cockroach Authors. +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package colexecwindow + +import ( + "bytes" + "context" + "math" + "time" + + "github.com/cockroachdb/apd/v2" + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldataext" + "github.com/cockroachdb/cockroach/pkg/col/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/memsize" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/json" + "github.com/cockroachdb/errors" +) + +// Workaround for bazel auto-generated code. goimports does not automatically +// pick up the right packages when run within the bazel sandbox. +var ( + _ tree.AggType + _ apd.Context + _ duration.Duration + _ json.JSON + _ = coldataext.CompareDatum + _ = colexecerror.InternalError + _ = memsize.Uint32 +) + +const ( + // The argument column is always the first column in the SpillingBuffer. + argColIdx = 0 + + // The slice of uint32s in the deque can have up to 10,000 values (40KB). + maxQueueLength = 10000 +) + +type minMaxRemovableAggBase struct { + partitionSeekerBase + colexecop.CloserHelper + allocator *colmem.Allocator + outputColIdx int + framer windowFramer + + // A partial deque of indices into the current partition ordered by the value + // of the input column at each index. It contains only indices that are part + // of the current window frame. The first value in the queue is the index of + // the current value for the aggregation (NULL if empty). Under the + // simplifying assumption that the window frame has no exclusion clause, the + // queue does not need to contain any indices smaller than the best index - + // this keeps the queue small in many common cases. + queue minMaxQueue + + // omittedIndex tracks the index where we reached the limit of the length of + // the queue, in which case we may be omitting values that could become + // relevant as the frame shrinks. If the queue becomes empty while this + // index is set, we have to aggregate over the previously omitted values. + // The default (unset) value is -1. + omittedIndex int + + scratchIntervals []windowInterval +} + +// Init implements the bufferedWindower interface. +func (b *minMaxRemovableAggBase) Init(ctx context.Context) { + b.InitHelper.Init(ctx) +} + +// transitionToProcessing implements the bufferedWindower interface. +func (b *minMaxRemovableAggBase) transitionToProcessing() { + b.framer.startPartition(b.Ctx, b.partitionSize, b.buffer) +} + +// startNewPartition implements the bufferedWindower interface. +func (b *minMaxRemovableAggBase) startNewPartition() { + b.partitionSize = 0 + b.buffer.Reset(b.Ctx) + b.queue.reset() +} + +func newMinRemovableAggregator( + args *WindowArgs, framer windowFramer, buffer *colexecutils.SpillingBuffer, argTyp *types.T, +) bufferedWindower { + // Reserve the maximum memory usable by the queue up front to ensure that it + // isn't used by the SpillingBuffer. + args.BufferAllocator.AdjustMemoryUsage(maxQueueLength * memsize.Uint32) + base := minMaxRemovableAggBase{ + partitionSeekerBase: partitionSeekerBase{ + partitionColIdx: args.PartitionColIdx, + buffer: buffer, + }, + allocator: args.MainAllocator, + outputColIdx: args.OutputColIdx, + framer: framer, + queue: newMinMaxQueue(maxQueueLength), + omittedIndex: -1, + } + switch typeconv.TypeFamilyToCanonicalTypeFamily(argTyp.Family()) { + case types.BoolFamily: + switch argTyp.Width() { + case -1: + default: + return &minBoolAggregator{minMaxRemovableAggBase: base} + } + case types.BytesFamily: + switch argTyp.Width() { + case -1: + default: + return &minBytesAggregator{minMaxRemovableAggBase: base} + } + case types.DecimalFamily: + switch argTyp.Width() { + case -1: + default: + return &minDecimalAggregator{minMaxRemovableAggBase: base} + } + case types.IntFamily: + switch argTyp.Width() { + case 16: + return &minInt16Aggregator{minMaxRemovableAggBase: base} + case 32: + return &minInt32Aggregator{minMaxRemovableAggBase: base} + case -1: + default: + return &minInt64Aggregator{minMaxRemovableAggBase: base} + } + case types.FloatFamily: + switch argTyp.Width() { + case -1: + default: + return &minFloat64Aggregator{minMaxRemovableAggBase: base} + } + case types.TimestampTZFamily: + switch argTyp.Width() { + case -1: + default: + return &minTimestampAggregator{minMaxRemovableAggBase: base} + } + case types.IntervalFamily: + switch argTyp.Width() { + case -1: + default: + return &minIntervalAggregator{minMaxRemovableAggBase: base} + } + case types.JsonFamily: + switch argTyp.Width() { + case -1: + default: + return &minJSONAggregator{minMaxRemovableAggBase: base} + } + case typeconv.DatumVecCanonicalTypeFamily: + switch argTyp.Width() { + case -1: + default: + return &minDatumAggregator{minMaxRemovableAggBase: base} + } + } + colexecerror.InternalError( + errors.AssertionFailedf("unexpectedly didn't find min overload for %s type family", argTyp.Name())) + // This code is unreachable, but the compiler cannot infer that. + return nil +} + +type minBoolAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg bool +} + +// processBatch implements the bufferedWindower interface. +func (a *minBoolAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Bool() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Bool() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minBoolAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Bool() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + if !val && a.curAgg { + cmpResult = -1 + } else if val && !a.curAgg { + cmpResult = 1 + } else { + cmpResult = 0 + } + + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Bool().Get(cmpIdx) + + { + var cmpResult int + + if !cmpVal && val { + cmpResult = -1 + } else if cmpVal && !val { + cmpResult = 1 + } else { + cmpResult = 0 + } + + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minBoolAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minBoolAggregator{} +} + +type minBytesAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg []byte +} + +// processBatch implements the bufferedWindower interface. +func (a *minBytesAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Bytes() + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Bytes() + val := col.Get(idx) + a.curAgg = append(a.curAgg[:0], val...) + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = append(a.curAgg[:0], a.curAgg...) + + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minBytesAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Bytes() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + cmpResult = bytes.Compare(val, a.curAgg) + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = append(a.curAgg[:0], val...) + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Bytes().Get(cmpIdx) + + { + var cmpResult int + cmpResult = bytes.Compare(cmpVal, val) + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minBytesAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minBytesAggregator{} +} + +type minDecimalAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg apd.Decimal +} + +// processBatch implements the bufferedWindower interface. +func (a *minDecimalAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Decimal() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Decimal() + val := col.Get(idx) + a.curAgg.Set(&val) + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg.Set(&a.curAgg) + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minDecimalAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Decimal() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + cmpResult = tree.CompareDecimals(&val, &a.curAgg) + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg.Set(&val) + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Decimal().Get(cmpIdx) + + { + var cmpResult int + cmpResult = tree.CompareDecimals(&cmpVal, &val) + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minDecimalAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minDecimalAggregator{} +} + +type minInt16Aggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg int16 +} + +// processBatch implements the bufferedWindower interface. +func (a *minInt16Aggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Int16() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Int16() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minInt16Aggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Int16() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + { + a, b := int64(val), int64(a.curAgg) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Int16().Get(cmpIdx) + + { + var cmpResult int + + { + a, b := int64(cmpVal), int64(val) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minInt16Aggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minInt16Aggregator{} +} + +type minInt32Aggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg int32 +} + +// processBatch implements the bufferedWindower interface. +func (a *minInt32Aggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Int32() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Int32() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minInt32Aggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Int32() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + { + a, b := int64(val), int64(a.curAgg) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Int32().Get(cmpIdx) + + { + var cmpResult int + + { + a, b := int64(cmpVal), int64(val) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minInt32Aggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minInt32Aggregator{} +} + +type minInt64Aggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg int64 +} + +// processBatch implements the bufferedWindower interface. +func (a *minInt64Aggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Int64() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Int64() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minInt64Aggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Int64() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + { + a, b := int64(val), int64(a.curAgg) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Int64().Get(cmpIdx) + + { + var cmpResult int + + { + a, b := int64(cmpVal), int64(val) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minInt64Aggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minInt64Aggregator{} +} + +type minFloat64Aggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg float64 +} + +// processBatch implements the bufferedWindower interface. +func (a *minFloat64Aggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Float64() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Float64() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minFloat64Aggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Float64() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + { + a, b := float64(val), float64(a.curAgg) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else if a == b { + cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 + } + } + + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Float64().Get(cmpIdx) + + { + var cmpResult int + + { + a, b := float64(cmpVal), float64(val) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else if a == b { + cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 + } + } + + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minFloat64Aggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minFloat64Aggregator{} +} + +type minTimestampAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg time.Time +} + +// processBatch implements the bufferedWindower interface. +func (a *minTimestampAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Timestamp() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Timestamp() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minTimestampAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Timestamp() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + if val.Before(a.curAgg) { + cmpResult = -1 + } else if a.curAgg.Before(val) { + cmpResult = 1 + } else { + cmpResult = 0 + } + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Timestamp().Get(cmpIdx) + + { + var cmpResult int + + if cmpVal.Before(val) { + cmpResult = -1 + } else if val.Before(cmpVal) { + cmpResult = 1 + } else { + cmpResult = 0 + } + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minTimestampAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minTimestampAggregator{} +} + +type minIntervalAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg duration.Duration +} + +// processBatch implements the bufferedWindower interface. +func (a *minIntervalAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Interval() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Interval() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minIntervalAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Interval() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + cmpResult = val.Compare(a.curAgg) + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Interval().Get(cmpIdx) + + { + var cmpResult int + cmpResult = cmpVal.Compare(val) + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minIntervalAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minIntervalAggregator{} +} + +type minJSONAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg json.JSON +} + +// processBatch implements the bufferedWindower interface. +func (a *minJSONAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.JSON() + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.JSON() + val := col.Get(idx) + + var _err error + var _bytes []byte + _bytes, _err = json.EncodeJSON(nil, val) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.curAgg, _err = json.FromEncoding(_bytes) + if _err != nil { + colexecerror.ExpectedError(_err) + } + + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + + var _err error + var _bytes []byte + _bytes, _err = json.EncodeJSON(nil, a.curAgg) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.curAgg, _err = json.FromEncoding(_bytes) + if _err != nil { + colexecerror.ExpectedError(_err) + } + + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minJSONAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.JSON() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + var err error + cmpResult, err = val.Compare(a.curAgg) + if err != nil { + colexecerror.ExpectedError(err) + } + + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + + var _err error + var _bytes []byte + _bytes, _err = json.EncodeJSON(nil, val) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.curAgg, _err = json.FromEncoding(_bytes) + if _err != nil { + colexecerror.ExpectedError(_err) + } + + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.JSON().Get(cmpIdx) + + { + var cmpResult int + + var err error + cmpResult, err = cmpVal.Compare(val) + if err != nil { + colexecerror.ExpectedError(err) + } + + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minJSONAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minJSONAggregator{} +} + +type minDatumAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg interface{} +} + +// processBatch implements the bufferedWindower interface. +func (a *minDatumAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Datum() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Datum() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minDatumAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Datum() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + cmpResult = coldataext.CompareDatum(val, col, a.curAgg) + + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Datum().Get(cmpIdx) + + { + var cmpResult int + + cmpResult = coldataext.CompareDatum(cmpVal, col, val) + + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minDatumAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minDatumAggregator{} +} + +func newMaxRemovableAggregator( + args *WindowArgs, framer windowFramer, buffer *colexecutils.SpillingBuffer, argTyp *types.T, +) bufferedWindower { + // Reserve the maximum memory usable by the queue up front to ensure that it + // isn't used by the SpillingBuffer. + args.BufferAllocator.AdjustMemoryUsage(maxQueueLength * memsize.Uint32) + base := minMaxRemovableAggBase{ + partitionSeekerBase: partitionSeekerBase{ + partitionColIdx: args.PartitionColIdx, + buffer: buffer, + }, + allocator: args.MainAllocator, + outputColIdx: args.OutputColIdx, + framer: framer, + queue: newMinMaxQueue(maxQueueLength), + omittedIndex: -1, + } + switch typeconv.TypeFamilyToCanonicalTypeFamily(argTyp.Family()) { + case types.BoolFamily: + switch argTyp.Width() { + case -1: + default: + return &maxBoolAggregator{minMaxRemovableAggBase: base} + } + case types.BytesFamily: + switch argTyp.Width() { + case -1: + default: + return &maxBytesAggregator{minMaxRemovableAggBase: base} + } + case types.DecimalFamily: + switch argTyp.Width() { + case -1: + default: + return &maxDecimalAggregator{minMaxRemovableAggBase: base} + } + case types.IntFamily: + switch argTyp.Width() { + case 16: + return &maxInt16Aggregator{minMaxRemovableAggBase: base} + case 32: + return &maxInt32Aggregator{minMaxRemovableAggBase: base} + case -1: + default: + return &maxInt64Aggregator{minMaxRemovableAggBase: base} + } + case types.FloatFamily: + switch argTyp.Width() { + case -1: + default: + return &maxFloat64Aggregator{minMaxRemovableAggBase: base} + } + case types.TimestampTZFamily: + switch argTyp.Width() { + case -1: + default: + return &maxTimestampAggregator{minMaxRemovableAggBase: base} + } + case types.IntervalFamily: + switch argTyp.Width() { + case -1: + default: + return &maxIntervalAggregator{minMaxRemovableAggBase: base} + } + case types.JsonFamily: + switch argTyp.Width() { + case -1: + default: + return &maxJSONAggregator{minMaxRemovableAggBase: base} + } + case typeconv.DatumVecCanonicalTypeFamily: + switch argTyp.Width() { + case -1: + default: + return &maxDatumAggregator{minMaxRemovableAggBase: base} + } + } + colexecerror.InternalError( + errors.AssertionFailedf("unexpectedly didn't find max overload for %s type family", argTyp.Name())) + // This code is unreachable, but the compiler cannot infer that. + return nil +} + +type maxBoolAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg bool +} + +// processBatch implements the bufferedWindower interface. +func (a *maxBoolAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Bool() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Bool() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxBoolAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Bool() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + if !val && a.curAgg { + cmpResult = -1 + } else if val && !a.curAgg { + cmpResult = 1 + } else { + cmpResult = 0 + } + + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Bool().Get(cmpIdx) + + { + var cmpResult int + + if !cmpVal && val { + cmpResult = -1 + } else if cmpVal && !val { + cmpResult = 1 + } else { + cmpResult = 0 + } + + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxBoolAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxBoolAggregator{} +} + +type maxBytesAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg []byte +} + +// processBatch implements the bufferedWindower interface. +func (a *maxBytesAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Bytes() + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Bytes() + val := col.Get(idx) + a.curAgg = append(a.curAgg[:0], val...) + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = append(a.curAgg[:0], a.curAgg...) + + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxBytesAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Bytes() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + cmpResult = bytes.Compare(val, a.curAgg) + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = append(a.curAgg[:0], val...) + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Bytes().Get(cmpIdx) + + { + var cmpResult int + cmpResult = bytes.Compare(cmpVal, val) + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxBytesAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxBytesAggregator{} +} + +type maxDecimalAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg apd.Decimal +} + +// processBatch implements the bufferedWindower interface. +func (a *maxDecimalAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Decimal() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Decimal() + val := col.Get(idx) + a.curAgg.Set(&val) + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg.Set(&a.curAgg) + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxDecimalAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Decimal() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + cmpResult = tree.CompareDecimals(&val, &a.curAgg) + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg.Set(&val) + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Decimal().Get(cmpIdx) + + { + var cmpResult int + cmpResult = tree.CompareDecimals(&cmpVal, &val) + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxDecimalAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxDecimalAggregator{} +} + +type maxInt16Aggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg int16 +} + +// processBatch implements the bufferedWindower interface. +func (a *maxInt16Aggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Int16() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Int16() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxInt16Aggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Int16() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + { + a, b := int64(val), int64(a.curAgg) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Int16().Get(cmpIdx) + + { + var cmpResult int + + { + a, b := int64(cmpVal), int64(val) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxInt16Aggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxInt16Aggregator{} +} + +type maxInt32Aggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg int32 +} + +// processBatch implements the bufferedWindower interface. +func (a *maxInt32Aggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Int32() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Int32() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxInt32Aggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Int32() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + { + a, b := int64(val), int64(a.curAgg) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Int32().Get(cmpIdx) + + { + var cmpResult int + + { + a, b := int64(cmpVal), int64(val) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxInt32Aggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxInt32Aggregator{} +} + +type maxInt64Aggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg int64 +} + +// processBatch implements the bufferedWindower interface. +func (a *maxInt64Aggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Int64() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Int64() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxInt64Aggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Int64() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + { + a, b := int64(val), int64(a.curAgg) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Int64().Get(cmpIdx) + + { + var cmpResult int + + { + a, b := int64(cmpVal), int64(val) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxInt64Aggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxInt64Aggregator{} +} + +type maxFloat64Aggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg float64 +} + +// processBatch implements the bufferedWindower interface. +func (a *maxFloat64Aggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Float64() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Float64() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxFloat64Aggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Float64() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + { + a, b := float64(val), float64(a.curAgg) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else if a == b { + cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 + } + } + + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Float64().Get(cmpIdx) + + { + var cmpResult int + + { + a, b := float64(cmpVal), float64(val) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else if a == b { + cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 + } + } + + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxFloat64Aggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxFloat64Aggregator{} +} + +type maxTimestampAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg time.Time +} + +// processBatch implements the bufferedWindower interface. +func (a *maxTimestampAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Timestamp() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Timestamp() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxTimestampAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Timestamp() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + if val.Before(a.curAgg) { + cmpResult = -1 + } else if a.curAgg.Before(val) { + cmpResult = 1 + } else { + cmpResult = 0 + } + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Timestamp().Get(cmpIdx) + + { + var cmpResult int + + if cmpVal.Before(val) { + cmpResult = -1 + } else if val.Before(cmpVal) { + cmpResult = 1 + } else { + cmpResult = 0 + } + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxTimestampAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxTimestampAggregator{} +} + +type maxIntervalAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg duration.Duration +} + +// processBatch implements the bufferedWindower interface. +func (a *maxIntervalAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Interval() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Interval() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxIntervalAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Interval() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + cmpResult = val.Compare(a.curAgg) + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Interval().Get(cmpIdx) + + { + var cmpResult int + cmpResult = cmpVal.Compare(val) + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxIntervalAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxIntervalAggregator{} +} + +type maxJSONAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg json.JSON +} + +// processBatch implements the bufferedWindower interface. +func (a *maxJSONAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.JSON() + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.JSON() + val := col.Get(idx) + + var _err error + var _bytes []byte + _bytes, _err = json.EncodeJSON(nil, val) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.curAgg, _err = json.FromEncoding(_bytes) + if _err != nil { + colexecerror.ExpectedError(_err) + } + + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + + var _err error + var _bytes []byte + _bytes, _err = json.EncodeJSON(nil, a.curAgg) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.curAgg, _err = json.FromEncoding(_bytes) + if _err != nil { + colexecerror.ExpectedError(_err) + } + + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxJSONAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.JSON() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + var err error + cmpResult, err = val.Compare(a.curAgg) + if err != nil { + colexecerror.ExpectedError(err) + } + + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + + var _err error + var _bytes []byte + _bytes, _err = json.EncodeJSON(nil, val) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.curAgg, _err = json.FromEncoding(_bytes) + if _err != nil { + colexecerror.ExpectedError(_err) + } + + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.JSON().Get(cmpIdx) + + { + var cmpResult int + + var err error + cmpResult, err = cmpVal.Compare(val) + if err != nil { + colexecerror.ExpectedError(err) + } + + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxJSONAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxJSONAggregator{} +} + +type maxDatumAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg interface{} +} + +// processBatch implements the bufferedWindower interface. +func (a *maxDatumAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Datum() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Datum() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxDatumAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Datum() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + cmpResult = coldataext.CompareDatum(val, col, a.curAgg) + + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Datum().Get(cmpIdx) + + { + var cmpResult int + + cmpResult = coldataext.CompareDatum(cmpVal, col, val) + + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxDatumAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxDatumAggregator{} +} + +// getIntervalsGEIdx returns a set of intervals representing all indexes in the +// 'intervals' slice at or after the given index. +func getIntervalsGEIdx(intervals, scratch []windowInterval, idx int) []windowInterval { + scratch = scratch[:0] + for _, interval := range intervals { + if interval.end <= idx { + continue + } + if interval.start >= idx { + scratch = append(scratch, interval) + continue + } + scratch = append(scratch, windowInterval{start: idx, end: interval.end}) + } + return scratch +} diff --git a/pkg/sql/colexec/colexecwindow/min_max_removable_agg_tmpl.go b/pkg/sql/colexec/colexecwindow/min_max_removable_agg_tmpl.go new file mode 100644 index 000000000000..9a967e31e643 --- /dev/null +++ b/pkg/sql/colexec/colexecwindow/min_max_removable_agg_tmpl.go @@ -0,0 +1,320 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// {{/* +// +build execgen_template +// +// This file is the execgen template for min_max_removable_agg.eg.go. It's +// formatted in a special way, so it's both valid Go and a valid text/template +// input. This permits editing this file with editor support. +// +// */}} + +package colexecwindow + +import ( + "context" + + "github.com/cockroachdb/apd/v2" + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldataext" + "github.com/cockroachdb/cockroach/pkg/col/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/memsize" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/json" + "github.com/cockroachdb/errors" +) + +// Workaround for bazel auto-generated code. goimports does not automatically +// pick up the right packages when run within the bazel sandbox. +var ( + _ tree.AggType + _ apd.Context + _ duration.Duration + _ json.JSON + _ = coldataext.CompareDatum + _ = colexecerror.InternalError + _ = memsize.Uint32 +) + +// {{/* +// Declarations to make the template compile properly. + +// _ASSIGN_CMP is the template function for assigning true to the first input +// if the second input compares successfully to the third input. The comparison +// operator is tree.LT for MIN and is tree.GT for MAX. +func _ASSIGN_CMP(_, _, _, _, _, _ string) bool { + colexecerror.InternalError(errors.AssertionFailedf("")) +} + +// */}} + +const ( + // The argument column is always the first column in the SpillingBuffer. + argColIdx = 0 + + // The slice of uint32s in the deque can have up to 10,000 values (40KB). + maxQueueLength = 10000 +) + +type minMaxRemovableAggBase struct { + partitionSeekerBase + colexecop.CloserHelper + allocator *colmem.Allocator + outputColIdx int + framer windowFramer + + // A partial deque of indices into the current partition ordered by the value + // of the input column at each index. It contains only indices that are part + // of the current window frame. The first value in the queue is the index of + // the current value for the aggregation (NULL if empty). Under the + // simplifying assumption that the window frame has no exclusion clause, the + // queue does not need to contain any indices smaller than the best index - + // this keeps the queue small in many common cases. + queue minMaxQueue + + // omittedIndex tracks the index where we reached the limit of the length of + // the queue, in which case we may be omitting values that could become + // relevant as the frame shrinks. If the queue becomes empty while this + // index is set, we have to aggregate over the previously omitted values. + // The default (unset) value is -1. + omittedIndex int + + scratchIntervals []windowInterval +} + +// Init implements the bufferedWindower interface. +func (b *minMaxRemovableAggBase) Init(ctx context.Context) { + b.InitHelper.Init(ctx) +} + +// transitionToProcessing implements the bufferedWindower interface. +func (b *minMaxRemovableAggBase) transitionToProcessing() { + b.framer.startPartition(b.Ctx, b.partitionSize, b.buffer) +} + +// startNewPartition implements the bufferedWindower interface. +func (b *minMaxRemovableAggBase) startNewPartition() { + b.partitionSize = 0 + b.buffer.Reset(b.Ctx) + b.queue.reset() +} + +// {{range .}} +// {{$agg := .Agg}} + +func new_AGG_TITLERemovableAggregator( + args *WindowArgs, framer windowFramer, buffer *colexecutils.SpillingBuffer, argTyp *types.T, +) bufferedWindower { + // Reserve the maximum memory usable by the queue up front to ensure that it + // isn't used by the SpillingBuffer. + args.BufferAllocator.AdjustMemoryUsage(maxQueueLength * memsize.Uint32) + base := minMaxRemovableAggBase{ + partitionSeekerBase: partitionSeekerBase{ + partitionColIdx: args.PartitionColIdx, + buffer: buffer, + }, + allocator: args.MainAllocator, + outputColIdx: args.OutputColIdx, + framer: framer, + queue: newMinMaxQueue(maxQueueLength), + omittedIndex: -1, + } + switch typeconv.TypeFamilyToCanonicalTypeFamily(argTyp.Family()) { + // {{range .Overloads}} + case _CANONICAL_TYPE_FAMILY: + switch argTyp.Width() { + // {{range .WidthOverloads}} + case _TYPE_WIDTH: + return &_AGG_TYPEAggregator{minMaxRemovableAggBase: base} + // {{end}} + } + // {{end}} + } + colexecerror.InternalError( + errors.AssertionFailedf("unexpectedly didn't find _AGG overload for %s type family", argTyp.Name())) + // This code is unreachable, but the compiler cannot infer that. + return nil +} + +// {{range .Overloads}} +// {{range .WidthOverloads}} + +type _AGG_TYPEAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg _GOTYPE +} + +// processBatch implements the bufferedWindower interface. +func (a *_AGG_TYPEAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.TemplateType() + // {{if not .IsBytesLike}} + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + // {{end}} + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.TemplateType() + val := col.Get(idx) + execgen.COPYVAL(a.curAgg, val) + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + execgen.COPYVAL(a.curAgg, a.curAgg) + + // {{if not .IsBytesLike}} + // gcassert:bce + // {{end}} + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *_AGG_TYPEAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.TemplateType() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + _ASSIGN_CMP(cmp, val, a.curAgg, _, col, _) + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + execgen.COPYVAL(a.curAgg, val) + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.TemplateType().Get(cmpIdx) + _ASSIGN_CMP(cmp, cmpVal, val, _, col, _) + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *_AGG_TYPEAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = _AGG_TYPEAggregator{} +} + +// {{end}} +// {{end}} +// {{end}} + +// getIntervalsGEIdx returns a set of intervals representing all indexes in the +// 'intervals' slice at or after the given index. +func getIntervalsGEIdx(intervals, scratch []windowInterval, idx int) []windowInterval { + scratch = scratch[:0] + for _, interval := range intervals { + if interval.end <= idx { + continue + } + if interval.start >= idx { + scratch = append(scratch, interval) + continue + } + scratch = append(scratch, windowInterval{start: idx, end: interval.end}) + } + return scratch +} diff --git a/pkg/sql/colexec/colexecwindow/window_aggregator.eg.go b/pkg/sql/colexec/colexecwindow/window_aggregator.eg.go index 2674ab374d04..60effe81c329 100644 --- a/pkg/sql/colexec/colexecwindow/window_aggregator.eg.go +++ b/pkg/sql/colexec/colexecwindow/window_aggregator.eg.go @@ -46,6 +46,7 @@ type slidingWindowAggregateFunc interface { // should put its output (if there is no such column, a new column is appended). func NewWindowAggregatorOperator( args *WindowArgs, + aggType execinfrapb.AggregatorSpec_Func, frame *execinfrapb.WindowerSpec_Frame, ordering *execinfrapb.Ordering, argIdxs []int, @@ -81,12 +82,43 @@ func NewWindowAggregatorOperator( closers: closers, vecs: make([]coldata.Vec, len(inputIdxs)), } - agg := aggAlloc.MakeAggregateFuncs()[0] + var agg colexecagg.AggregateFunc + if aggAlloc != nil { + agg = aggAlloc.MakeAggregateFuncs()[0] + } var windower bufferedWindower - if slidingWindowAgg, ok := agg.(slidingWindowAggregateFunc); ok { - windower = &slidingWindowAggregator{windowAggregatorBase: base, agg: slidingWindowAgg} - } else { - windower = &windowAggregator{windowAggregatorBase: base, agg: agg} + switch aggType { + case execinfrapb.Min, execinfrapb.Max: + if WindowFrameCanShrink(frame, ordering) { + // In the case when the window frame for a given row does not necessarily + // include all rows from the previous frame, min and max require a + // specialized implementation that maintains a dequeue of seen values. + if frame.Exclusion != execinfrapb.WindowerSpec_Frame_NO_EXCLUSION { + // TODO(drewk): extend the implementations to work with non-default + // exclusion. For now, we have to use the quadratic-time method. + windower = &windowAggregator{windowAggregatorBase: base, agg: agg} + } else { + switch aggType { + case execinfrapb.Min: + windower = newMinRemovableAggregator(args, framer, buffer, outputType) + case execinfrapb.Max: + windower = newMaxRemovableAggregator(args, framer, buffer, outputType) + } + } + } else { + // When the frame can only grow, the simple sliding window implementation + // is sufficient. + windower = &slidingWindowAggregator{ + windowAggregatorBase: base, + agg: agg.(slidingWindowAggregateFunc), + } + } + default: + if slidingWindowAgg, ok := agg.(slidingWindowAggregateFunc); ok { + windower = &slidingWindowAggregator{windowAggregatorBase: base, agg: slidingWindowAgg} + } else { + windower = &windowAggregator{windowAggregatorBase: base, agg: agg} + } } return newBufferedWindowOperator(args, windower, outputType, mainMemLimit) } diff --git a/pkg/sql/colexec/colexecwindow/window_aggregator_tmpl.go b/pkg/sql/colexec/colexecwindow/window_aggregator_tmpl.go index 0747ad28c2bc..e30451b4d394 100644 --- a/pkg/sql/colexec/colexecwindow/window_aggregator_tmpl.go +++ b/pkg/sql/colexec/colexecwindow/window_aggregator_tmpl.go @@ -49,6 +49,7 @@ type slidingWindowAggregateFunc interface { // should put its output (if there is no such column, a new column is appended). func NewWindowAggregatorOperator( args *WindowArgs, + aggType execinfrapb.AggregatorSpec_Func, frame *execinfrapb.WindowerSpec_Frame, ordering *execinfrapb.Ordering, argIdxs []int, @@ -84,12 +85,43 @@ func NewWindowAggregatorOperator( closers: closers, vecs: make([]coldata.Vec, len(inputIdxs)), } - agg := aggAlloc.MakeAggregateFuncs()[0] + var agg colexecagg.AggregateFunc + if aggAlloc != nil { + agg = aggAlloc.MakeAggregateFuncs()[0] + } var windower bufferedWindower - if slidingWindowAgg, ok := agg.(slidingWindowAggregateFunc); ok { - windower = &slidingWindowAggregator{windowAggregatorBase: base, agg: slidingWindowAgg} - } else { - windower = &windowAggregator{windowAggregatorBase: base, agg: agg} + switch aggType { + case execinfrapb.Min, execinfrapb.Max: + if WindowFrameCanShrink(frame, ordering) { + // In the case when the window frame for a given row does not necessarily + // include all rows from the previous frame, min and max require a + // specialized implementation that maintains a dequeue of seen values. + if frame.Exclusion != execinfrapb.WindowerSpec_Frame_NO_EXCLUSION { + // TODO(drewk): extend the implementations to work with non-default + // exclusion. For now, we have to use the quadratic-time method. + windower = &windowAggregator{windowAggregatorBase: base, agg: agg} + } else { + switch aggType { + case execinfrapb.Min: + windower = newMinRemovableAggregator(args, framer, buffer, outputType) + case execinfrapb.Max: + windower = newMaxRemovableAggregator(args, framer, buffer, outputType) + } + } + } else { + // When the frame can only grow, the simple sliding window implementation + // is sufficient. + windower = &slidingWindowAggregator{ + windowAggregatorBase: base, + agg: agg.(slidingWindowAggregateFunc), + } + } + default: + if slidingWindowAgg, ok := agg.(slidingWindowAggregateFunc); ok { + windower = &slidingWindowAggregator{windowAggregatorBase: base, agg: slidingWindowAgg} + } else { + windower = &windowAggregator{windowAggregatorBase: base, agg: agg} + } } return newBufferedWindowOperator(args, windower, outputType, mainMemLimit) } diff --git a/pkg/sql/colexec/colexecwindow/window_functions_test.go b/pkg/sql/colexec/colexecwindow/window_functions_test.go index 5cb72a1e1dac..3330a71b428b 100644 --- a/pkg/sql/colexec/colexecwindow/window_functions_test.go +++ b/pkg/sql/colexec/colexecwindow/window_functions_test.go @@ -1223,8 +1223,9 @@ func BenchmarkWindowFunctions(b *testing.B) { ) require.NoError(b, err) op = NewWindowAggregatorOperator( - args, NormalizeWindowFrame(nil), &execinfrapb.Ordering{Columns: orderingCols}, - []int{arg1ColIdx}, aggArgs.OutputTypes[0], aggFnsAlloc, toClose) + args, *fun.AggregateFunc, NormalizeWindowFrame(nil), + &execinfrapb.Ordering{Columns: orderingCols}, []int{arg1ColIdx}, + aggArgs.OutputTypes[0], aggFnsAlloc, toClose) } else { require.Fail(b, "expected non-nil window function") } diff --git a/pkg/sql/colexec/colexecwindow/window_functions_util.go b/pkg/sql/colexec/colexecwindow/window_functions_util.go index 8ce125abb24b..1489a6482cae 100644 --- a/pkg/sql/colexec/colexecwindow/window_functions_util.go +++ b/pkg/sql/colexec/colexecwindow/window_functions_util.go @@ -298,3 +298,31 @@ func isWindowFnLinear(fn execinfrapb.WindowerSpec_Func) bool { return false } } + +// WindowFrameCanShrink returns true if a sliding window aggregate function over +// the given frame may need to call Remove, which is the case when the frame for +// a given row may not include all rows that were part of the previous frame. +func WindowFrameCanShrink( + frame *execinfrapb.WindowerSpec_Frame, ordering *execinfrapb.Ordering, +) bool { + if frame.Exclusion != execinfrapb.WindowerSpec_Frame_NO_EXCLUSION { + return true + } + if frame.Bounds.Start.BoundType == execinfrapb.WindowerSpec_Frame_UNBOUNDED_PRECEDING { + return false + } + if len(ordering.Columns) == 0 { + // All rows are part of the same peer group. + if frame.Bounds.Start.BoundType == execinfrapb.WindowerSpec_Frame_CURRENT_ROW && + (frame.Mode == execinfrapb.WindowerSpec_Frame_RANGE || + frame.Mode == execinfrapb.WindowerSpec_Frame_GROUPS) { + return false + } + if frame.Mode == execinfrapb.WindowerSpec_Frame_GROUPS && + frame.Bounds.Start.BoundType == execinfrapb.WindowerSpec_Frame_OFFSET_PRECEDING && + frame.Bounds.Start.IntOffset >= 1 { + return false + } + } + return true +} diff --git a/pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel b/pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel index fb05c3e89673..c5511ac4be2c 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel +++ b/pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel @@ -33,6 +33,7 @@ go_library( "mergejoinbase_gen.go", "mergejoiner_gen.go", "min_max_agg_gen.go", + "min_max_removable_agg_gen.go", "ntile_gen.go", "ordered_synchronizer_gen.go", "overloads_abbr.go", diff --git a/pkg/sql/colexec/execgen/cmd/execgen/min_max_removable_agg_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/min_max_removable_agg_gen.go new file mode 100644 index 000000000000..20c554268021 --- /dev/null +++ b/pkg/sql/colexec/execgen/cmd/execgen/min_max_removable_agg_gen.go @@ -0,0 +1,64 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package main + +import ( + "io" + "strings" + "text/template" + + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +const minMaxRemovableAggTmpl = "pkg/sql/colexec/colexecwindow/min_max_removable_agg_tmpl.go" + +func genMinMaxRemovableAgg(inputFileContents string, wr io.Writer) error { + r := strings.NewReplacer( + "_CANONICAL_TYPE_FAMILY", "{{.CanonicalTypeFamilyStr}}", + "_TYPE_WIDTH", typeWidthReplacement, + "_AGG_TITLE", "{{.AggTitle}}", + "_AGG", "{{$agg}}", + "_GOTYPE", "{{.GoType}}", + "_TYPE", "{{.VecMethod}}", + "TemplateType", "{{.VecMethod}}", + ) + s := r.Replace(inputFileContents) + + assignCmpRe := makeFunctionRegex("_ASSIGN_CMP", 6) + s = assignCmpRe.ReplaceAllString(s, makeTemplateFunctionCall("Assign", 6)) + + s = replaceManipulationFuncs(s) + + tmpl, err := template.New("min_max_agg").Funcs(template.FuncMap{"buildDict": buildDict}).Parse(s) + if err != nil { + return err + } + return tmpl.Execute(wr, []struct { + Agg string + AggTitle string + Overloads []*oneArgOverload + }{ + { + Agg: "min", + AggTitle: "Min", + Overloads: sameTypeComparisonOpToOverloads[tree.LT], + }, + { + Agg: "max", + AggTitle: "Max", + Overloads: sameTypeComparisonOpToOverloads[tree.GT], + }, + }) +} + +func init() { + registerGenerator(genMinMaxRemovableAgg, "min_max_removable_agg.eg.go", minMaxRemovableAggTmpl) +} diff --git a/pkg/sql/memsize/constants.go b/pkg/sql/memsize/constants.go index f6e02d391cca..109bd43ea9ed 100644 --- a/pkg/sql/memsize/constants.go +++ b/pkg/sql/memsize/constants.go @@ -34,6 +34,9 @@ const ( // Int32 is the in-memory size of an int32 in bytes. Int32 = int64(unsafe.Sizeof(int32(0))) + // Uint32 is the in-memory size of a uint32 in bytes. + Uint32 = int64(unsafe.Sizeof(uint32(0))) + // Int64 is the in-memory size of an int64 in bytes. Int64 = int64(unsafe.Sizeof(int64(0)))