github.com/matrixorigin/matrixone@v1.2.0/pkg/sql/colexec/receiver_operator.go (about) 1 // Copyright 2023 Matrix Origin 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 15 package colexec 16 17 import ( 18 "reflect" 19 "time" 20 21 "github.com/matrixorigin/matrixone/pkg/container/batch" 22 "github.com/matrixorigin/matrixone/pkg/vm/process" 23 ) 24 25 // isMergeType means the receiver operator receive batch from all regs or single by some order 26 // Merge/MergeGroup/MergeLimit ... are Merge-Type 27 // while Join/Intersect/Minus ... are not 28 func (r *ReceiverOperator) InitReceiver(proc *process.Process, isMergeType bool) { 29 r.proc = proc 30 if isMergeType { 31 r.aliveMergeReceiver = len(proc.Reg.MergeReceivers) 32 r.chs = make([]chan *batch.Batch, r.aliveMergeReceiver) 33 r.receiverListener = make([]reflect.SelectCase, r.aliveMergeReceiver+1) 34 r.receiverListener[0] = reflect.SelectCase{Dir: reflect.SelectRecv, Chan: reflect.ValueOf(r.proc.Ctx.Done())} 35 for i, mr := range proc.Reg.MergeReceivers { 36 r.chs[i] = mr.Ch 37 r.receiverListener[i+1] = reflect.SelectCase{ 38 Dir: reflect.SelectRecv, 39 Chan: reflect.ValueOf(mr.Ch), 40 } 41 } 42 } 43 } 44 45 func (r *ReceiverOperator) ReceiveFromSingleReg(regIdx int, analyze process.Analyze) (*batch.Batch, bool, error) { 46 start := time.Now() 47 defer analyze.WaitStop(start) 48 select { 49 case <-r.proc.Ctx.Done(): 50 return nil, true, nil 51 case bat, ok := <-r.proc.Reg.MergeReceivers[regIdx].Ch: 52 if !ok { 53 return nil, true, nil 54 } 55 return bat, false, nil 56 } 57 } 58 59 func (r *ReceiverOperator) ReceiveFromSingleRegNonBlock(regIdx int, analyze process.Analyze) (*batch.Batch, bool, error) { 60 start := time.Now() 61 defer analyze.WaitStop(start) 62 select { 63 case <-r.proc.Ctx.Done(): 64 return nil, true, nil 65 case bat, ok := <-r.proc.Reg.MergeReceivers[regIdx].Ch: 66 if !ok || bat == nil { 67 return nil, true, nil 68 } 69 return bat, false, nil 70 default: 71 return nil, false, nil 72 } 73 } 74 75 func (r *ReceiverOperator) FreeAllReg() { 76 for i := range r.proc.Reg.MergeReceivers { 77 r.FreeSingleReg(i) 78 } 79 } 80 81 func (r *ReceiverOperator) FreeSingleReg(regIdx int) { 82 w := r.proc.Reg.MergeReceivers[regIdx] 83 w.CleanChannel(r.proc.GetMPool()) 84 } 85 86 // You MUST Init ReceiverOperator with Merge-Type 87 // if you want to use this function 88 func (r *ReceiverOperator) ReceiveFromAllRegs(analyze process.Analyze) (*batch.Batch, bool, error) { 89 for { 90 if r.aliveMergeReceiver == 0 { 91 return nil, true, nil 92 } 93 94 start := time.Now() 95 chosen, bat, ok := r.selectFromAllReg() 96 analyze.WaitStop(start) 97 98 // chosen == 0 means the info comes from proc context.Done 99 if chosen == 0 { 100 return nil, true, nil 101 } 102 103 if !ok { 104 return nil, true, nil 105 } 106 107 if bat == nil { 108 continue 109 } 110 111 if bat.IsEmpty() { 112 r.proc.PutBatch(bat) 113 continue 114 } 115 116 return bat, false, nil 117 } 118 } 119 120 func (r *ReceiverOperator) FreeMergeTypeOperator(failed bool) { 121 if len(r.receiverListener) > 0 { 122 // Remove the proc context.Done waiter because it MUST BE done 123 // when called this function 124 r.receiverListener = r.receiverListener[1:] 125 } 126 127 mp := r.proc.Mp() 128 // Senders will never send more because the context is done. 129 for _, ch := range r.chs { 130 for len(ch) > 0 { 131 bat := <-ch 132 if bat != nil { 133 bat.Clean(mp) 134 } 135 } 136 } 137 } 138 139 func (r *ReceiverOperator) RemoveChosen(idx int) { 140 if idx == 0 { 141 return 142 } 143 r.receiverListener = append(r.receiverListener[:idx], r.receiverListener[idx+1:]...) 144 //remove idx-1 from chs 145 r.chs = append(r.chs[:idx-1], r.chs[idx:]...) 146 r.aliveMergeReceiver-- 147 } 148 149 func (r *ReceiverOperator) DisableChosen(idx int) { 150 if idx == 0 { 151 return 152 } 153 //disable idx-1 from chs 154 r.chs[idx-1] = nil 155 r.aliveMergeReceiver-- 156 } 157 158 func (r *ReceiverOperator) selectFromAllReg() (int, *batch.Batch, bool) { 159 var bat *batch.Batch 160 chosen := 0 161 var ok bool 162 switch len(r.chs) { 163 case 1: 164 chosen, bat, ok = r.selectFrom1Reg() 165 case 2: 166 chosen, bat, ok = r.selectFrom2Reg() 167 case 3: 168 chosen, bat, ok = r.selectFrom3Reg() 169 case 4: 170 chosen, bat, ok = r.selectFrom4Reg() 171 case 5: 172 chosen, bat, ok = r.selectFrom5Reg() 173 case 6: 174 chosen, bat, ok = r.selectFrom6Reg() 175 case 7: 176 chosen, bat, ok = r.selectFrom7Reg() 177 case 8: 178 chosen, bat, ok = r.selectFrom8Reg() 179 case 9: 180 chosen, bat, ok = r.selectFrom9Reg() 181 case 10: 182 chosen, bat, ok = r.selectFrom10Reg() 183 case 11: 184 chosen, bat, ok = r.selectFrom11Reg() 185 case 12: 186 chosen, bat, ok = r.selectFrom12Reg() 187 case 13: 188 chosen, bat, ok = r.selectFrom13Reg() 189 case 14: 190 chosen, bat, ok = r.selectFrom14Reg() 191 case 15: 192 chosen, bat, ok = r.selectFrom15Reg() 193 case 16: 194 chosen, bat, ok = r.selectFrom16Reg() 195 case 32: 196 chosen, bat, ok = r.selectFrom32Reg() 197 case 48: 198 chosen, bat, ok = r.selectFrom48Reg() 199 case 64: 200 chosen, bat, ok = r.selectFrom64Reg() 201 case 80: 202 chosen, bat, ok = r.selectFrom80Reg() 203 default: 204 var value reflect.Value 205 chosen, value, ok = reflect.Select(r.receiverListener) 206 if chosen != 0 && ok { 207 bat = (*batch.Batch)(value.UnsafePointer()) 208 } 209 if !ok || bat == nil { 210 r.RemoveChosen(chosen) 211 } 212 return chosen, bat, ok 213 } 214 215 if !ok || bat == nil { 216 r.DisableChosen(chosen) 217 } 218 return chosen, bat, ok 219 }