diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 6d017b9528f5..eb81e53f7585 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -810,31 +810,6 @@ func NewColOperator( return r, err } aggSpec := core.Aggregator - if len(aggSpec.Aggregations) == 0 { - // We can get an aggregator when no aggregate functions are - // present if HAVING clause is present, for example, with a - // query as follows: SELECT 1 FROM t HAVING true. In this case, - // we plan a special operator that outputs a batch of length 1 - // or 0 (depending on whether the aggregate is in scalar context - // or not) without actual columns once and then zero-length - // batches. The actual "data" will be added by projections - // below. - // TODO(solon): The distsql plan for this case includes a - // TableReader, so we end up creating an orphaned colBatchScan. - // We should avoid that. Ideally the optimizer would not plan a - // scan in this unusual case. - numTuples := 0 - if aggSpec.IsScalar() { - numTuples = 1 - } - result.Root, err = colexecutils.NewFixedNumTuplesNoInputOp( - getStreamingAllocator(ctx, args), numTuples, inputs[0].Root, - ), nil - // We make ColumnTypes non-nil so that sanity check doesn't - // panic. - result.ColumnTypes = []*types.T{} - break - } if aggSpec.IsRowCount() { result.Root, err = colexec.NewCountOp(getStreamingAllocator(ctx, args), inputs[0].Root), nil result.ColumnTypes = []*types.T{types.Int} diff --git a/pkg/sql/colexec/colexecbase/distinct.go b/pkg/sql/colexec/colexecbase/distinct.go index 998aadb59f1b..ba0510f4cba6 100644 --- a/pkg/sql/colexec/colexecbase/distinct.go +++ b/pkg/sql/colexec/colexecbase/distinct.go @@ -35,8 +35,19 @@ func OrderedDistinctColsToOperators( OneInputHelper: colexecop.MakeOneInputHelper(input), fn: func() { copy(distinctCol, colexecutils.ZeroBoolColumn) }, } - for i := range distinctCols { - input = newSingleDistinct(input, int(distinctCols[i]), distinctCol, typs[distinctCols[i]], nullsAreDistinct) + if len(distinctCols) > 0 { + for i := range distinctCols { + input = newSingleDistinct(input, int(distinctCols[i]), distinctCol, typs[distinctCols[i]], nullsAreDistinct) + } + } else { + // If there are no distinct columns, we have to mark the very first + // tuple as distinct ourselves. + firstTuple := true + input.(*fnOp).fn = func() { + copy(distinctCol, colexecutils.ZeroBoolColumn) + distinctCol[0] = firstTuple + firstTuple = false + } } r, ok := input.(colexecop.ResettableOperator) if !ok { diff --git a/pkg/sql/colexec/ordered_aggregator.go b/pkg/sql/colexec/ordered_aggregator.go index 76bef5d09007..83f81dce9e1c 100644 --- a/pkg/sql/colexec/ordered_aggregator.go +++ b/pkg/sql/colexec/ordered_aggregator.go @@ -255,7 +255,18 @@ func (a *orderedAggregator) Next() coldata.Batch { } }) } - a.scratch.resumeIdx = a.bucket.fns[0].CurrentOutputIndex() + if len(a.bucket.fns) > 0 { + a.scratch.resumeIdx = a.bucket.fns[0].CurrentOutputIndex() + } else { + // When there are no aggregate functions to compute, we + // simply need to output the same number of empty rows as + // the number of groups. + for _, newGroup := range a.groupCol[:batchLength] { + if newGroup { + a.scratch.resumeIdx++ + } + } + } } if batchLength == 0 { a.state = orderedAggregatorOutputting