github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/colexec/rank_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 rank.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/colexecbase/colexecerror" 28 "github.com/cockroachdb/cockroach/pkg/sql/colmem" 29 "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" 30 "github.com/cockroachdb/cockroach/pkg/sql/types" 31 "github.com/cockroachdb/errors" 32 ) 33 34 // Remove unused warning. 35 var _ = colexecerror.InternalError 36 37 // TODO(yuzefovich): add benchmarks. 38 39 // NewRankOperator creates a new Operator that computes window functions RANK 40 // or DENSE_RANK (depending on the passed in windowFn). 41 // outputColIdx specifies in which coldata.Vec the operator should put its 42 // output (if there is no such column, a new column is appended). 43 func NewRankOperator( 44 allocator *colmem.Allocator, 45 input colexecbase.Operator, 46 windowFn execinfrapb.WindowerSpec_WindowFunc, 47 orderingCols []execinfrapb.Ordering_Column, 48 outputColIdx int, 49 partitionColIdx int, 50 peersColIdx int, 51 ) (colexecbase.Operator, error) { 52 if len(orderingCols) == 0 { 53 return NewConstOp(allocator, input, types.Int, int64(1), outputColIdx) 54 } 55 input = newVectorTypeEnforcer(allocator, input, types.Int, outputColIdx) 56 initFields := rankInitFields{ 57 OneInputNode: NewOneInputNode(input), 58 allocator: allocator, 59 outputColIdx: outputColIdx, 60 partitionColIdx: partitionColIdx, 61 peersColIdx: peersColIdx, 62 } 63 switch windowFn { 64 case execinfrapb.WindowerSpec_RANK: 65 if partitionColIdx != columnOmitted { 66 return &rankWithPartitionOp{rankInitFields: initFields}, nil 67 } 68 return &rankNoPartitionOp{rankInitFields: initFields}, nil 69 case execinfrapb.WindowerSpec_DENSE_RANK: 70 if partitionColIdx != columnOmitted { 71 return &denseRankWithPartitionOp{rankInitFields: initFields}, nil 72 } 73 return &denseRankNoPartitionOp{rankInitFields: initFields}, nil 74 default: 75 return nil, errors.Errorf("unsupported rank type %s", windowFn) 76 } 77 } 78 79 // {{/* 80 81 // _UPDATE_RANK is the template function for updating the state of rank 82 // operators. 83 func _UPDATE_RANK() { 84 colexecerror.InternalError("") 85 } 86 87 // _UPDATE_RANK_INCREMENT is the template function for updating the state of 88 // rank operators. 89 func _UPDATE_RANK_INCREMENT() { 90 colexecerror.InternalError("") 91 } 92 93 // */}} 94 95 type rankInitFields struct { 96 OneInputNode 97 98 allocator *colmem.Allocator 99 outputColIdx int 100 partitionColIdx int 101 peersColIdx int 102 } 103 104 // {{/* 105 // _COMPUTE_RANK is a code snippet that computes the rank for a single tuple at 106 // index i. 107 func _COMPUTE_RANK() { // */}} 108 // {{define "computeRank" -}} 109 // {{if $.HasPartition}} 110 if partitionCol[i] { 111 // We need to reset the internal state because of the new partition. 112 // Note that the beginning of new partition necessarily starts a new 113 // peer group, so peersCol[i] *must* be true, and we will correctly 114 // update the rank before setting it to rankCol. 115 r.rank = 0 116 r.rankIncrement = 1 117 } 118 // {{end}} 119 if peersCol[i] { 120 _UPDATE_RANK() 121 rankCol[i] = r.rank 122 } else { 123 rankCol[i] = r.rank 124 _UPDATE_RANK_INCREMENT() 125 } 126 // {{end}} 127 // {{/* 128 } // */}} 129 130 // {{range .}} 131 132 type _RANK_STRINGOp struct { 133 rankInitFields 134 135 // rank indicates which rank should be assigned to the next tuple. 136 rank int64 137 // rankIncrement indicates by how much rank should be incremented when a 138 // tuple distinct from the previous one on the ordering columns is seen. 139 rankIncrement int64 140 } 141 142 var _ colexecbase.Operator = &_RANK_STRINGOp{} 143 144 func (r *_RANK_STRINGOp) Init() { 145 r.Input().Init() 146 // All rank functions start counting from 1. Before we assign the rank to a 147 // tuple in the batch, we first increment r.rank, so setting this 148 // rankIncrement to 1 will update r.rank to 1 on the very first tuple (as 149 // desired). 150 r.rankIncrement = 1 151 } 152 153 func (r *_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { 154 batch := r.Input().Next(ctx) 155 n := batch.Length() 156 if n == 0 { 157 return coldata.ZeroBatch 158 } 159 // {{if .HasPartition}} 160 partitionCol := batch.ColVec(r.partitionColIdx).Bool() 161 // {{end}} 162 peersCol := batch.ColVec(r.peersColIdx).Bool() 163 rankVec := batch.ColVec(r.outputColIdx) 164 if rankVec.MaybeHasNulls() { 165 // We need to make sure that there are no left over null values in the 166 // output vector. 167 rankVec.Nulls().UnsetNulls() 168 } 169 rankCol := rankVec.Int64() 170 sel := batch.Selection() 171 if sel != nil { 172 for _, i := range sel[:n] { 173 _COMPUTE_RANK() 174 } 175 } else { 176 for i := range rankCol[:n] { 177 _COMPUTE_RANK() 178 } 179 } 180 return batch 181 } 182 183 // {{end}}