github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/colexec/row_number_tmpl.go (about) 1 // Copyright 2019 The Cockroach Authors. 2 // 3 // Use of this software is governed by the Business Source License 4 // included in the file licenses/BSL.txt. 5 // 6 // As of the Change Date specified in that file, in accordance with 7 // the Business Source License, use of this software will be governed 8 // by the Apache License, Version 2.0, included in the file 9 // licenses/APL.txt. 10 11 // {{/* 12 // +build execgen_template 13 // 14 // This file is the execgen template for row_number.eg.go. It's formatted in a 15 // special way, so it's both valid Go and a valid text/template input. This 16 // permits editing this file with editor support. 17 // 18 // */}} 19 20 package colexec 21 22 import ( 23 "context" 24 25 "github.com/cockroachdb/cockroach/pkg/col/coldata" 26 "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" 27 "github.com/cockroachdb/cockroach/pkg/sql/colmem" 28 "github.com/cockroachdb/cockroach/pkg/sql/types" 29 ) 30 31 // TODO(yuzefovich): add benchmarks. 32 33 // NewRowNumberOperator creates a new Operator that computes window function 34 // ROW_NUMBER. outputColIdx specifies in which coldata.Vec the operator should 35 // put its output (if there is no such column, a new column is appended). 36 func NewRowNumberOperator( 37 allocator *colmem.Allocator, input colexecbase.Operator, outputColIdx int, partitionColIdx int, 38 ) colexecbase.Operator { 39 input = newVectorTypeEnforcer(allocator, input, types.Int, outputColIdx) 40 base := rowNumberBase{ 41 OneInputNode: NewOneInputNode(input), 42 allocator: allocator, 43 outputColIdx: outputColIdx, 44 partitionColIdx: partitionColIdx, 45 } 46 if partitionColIdx == -1 { 47 return &rowNumberNoPartitionOp{base} 48 } 49 return &rowNumberWithPartitionOp{base} 50 } 51 52 // rowNumberBase extracts common fields and common initialization of two 53 // variations of row number operators. Note that it is not an operator itself 54 // and should not be used directly. 55 type rowNumberBase struct { 56 OneInputNode 57 allocator *colmem.Allocator 58 outputColIdx int 59 partitionColIdx int 60 61 rowNumber int64 62 } 63 64 func (r *rowNumberBase) Init() { 65 r.Input().Init() 66 } 67 68 // {{/* 69 // _COMPUTE_ROW_NUMBER is a code snippet that computes the row number value 70 // for a single tuple at index i as an increment from the previous tuple's row 71 // number. If a new partition begins, then the running 'rowNumber' variable is 72 // reset. 73 func _COMPUTE_ROW_NUMBER() { // */}} 74 // {{define "computeRowNumber" -}} 75 // {{if $.HasPartition}} 76 if partitionCol[i] { 77 r.rowNumber = 0 78 } 79 // {{end}} 80 r.rowNumber++ 81 rowNumberCol[i] = r.rowNumber 82 // {{end}} 83 // {{/* 84 } // */}} 85 86 // {{range .}} 87 88 type _ROW_NUMBER_STRINGOp struct { 89 rowNumberBase 90 } 91 92 var _ colexecbase.Operator = &_ROW_NUMBER_STRINGOp{} 93 94 func (r *_ROW_NUMBER_STRINGOp) Next(ctx context.Context) coldata.Batch { 95 batch := r.Input().Next(ctx) 96 n := batch.Length() 97 if n == 0 { 98 return coldata.ZeroBatch 99 } 100 101 // {{if .HasPartition}} 102 partitionCol := batch.ColVec(r.partitionColIdx).Bool() 103 // {{end}} 104 rowNumberVec := batch.ColVec(r.outputColIdx) 105 if rowNumberVec.MaybeHasNulls() { 106 // We need to make sure that there are no left over null values in the 107 // output vector. 108 rowNumberVec.Nulls().UnsetNulls() 109 } 110 rowNumberCol := rowNumberVec.Int64() 111 sel := batch.Selection() 112 if sel != nil { 113 for _, i := range sel[:n] { 114 _COMPUTE_ROW_NUMBER() 115 } 116 } else { 117 for i := range rowNumberCol[:n] { 118 _COMPUTE_ROW_NUMBER() 119 } 120 } 121 return batch 122 } 123 124 // {{end}}