github.com/matrixorigin/matrixone@v0.7.0/pkg/sql/colexec/dispatch/sendfunc.go (about) 1 // Copyright 2021 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 dispatch 16 17 import ( 18 "context" 19 "hash/crc32" 20 "sync/atomic" 21 "time" 22 23 "github.com/matrixorigin/matrixone/pkg/cnservice/cnclient" 24 "github.com/matrixorigin/matrixone/pkg/common/hashmap" 25 "github.com/matrixorigin/matrixone/pkg/common/moerr" 26 "github.com/matrixorigin/matrixone/pkg/common/mpool" 27 "github.com/matrixorigin/matrixone/pkg/container/batch" 28 "github.com/matrixorigin/matrixone/pkg/container/types" 29 "github.com/matrixorigin/matrixone/pkg/pb/pipeline" 30 "github.com/matrixorigin/matrixone/pkg/vm/process" 31 ) 32 33 const ( 34 maxMessageSizeToMoRpc = 64 * mpool.MB 35 36 SendToAllFunc = iota 37 SendToAllLocalFunc 38 SendToAnyLocalFunc 39 ) 40 41 // common sender: send to any LocalReceiver 42 func sendToAllLocalFunc(bat *batch.Batch, ap *Argument, proc *process.Process) error { 43 refCountAdd := int64(len(ap.LocalRegs) - 1) 44 atomic.AddInt64(&bat.Cnt, refCountAdd) 45 if jm, ok := bat.Ht.(*hashmap.JoinMap); ok { 46 jm.IncRef(refCountAdd) 47 jm.SetDupCount(int64(len(ap.LocalRegs))) 48 } 49 50 for _, reg := range ap.LocalRegs { 51 select { 52 case <-reg.Ctx.Done(): 53 return moerr.NewInternalError(proc.Ctx, "pipeline context has done.") 54 case reg.Ch <- bat: 55 } 56 } 57 58 return nil 59 } 60 61 // common sender: send to any LocalReceiver 62 func sendToAnyLocalFunc(bat *batch.Batch, ap *Argument, proc *process.Process) error { 63 // send to local receiver 64 sendto := ap.sendCnt % len(ap.LocalRegs) 65 reg := ap.LocalRegs[sendto] 66 select { 67 case <-reg.Ctx.Done(): 68 for len(reg.Ch) > 0 { // free memory 69 bat := <-reg.Ch 70 if bat == nil { 71 break 72 } 73 bat.Clean(proc.Mp()) 74 } 75 ap.LocalRegs = append(ap.LocalRegs[:sendto], ap.LocalRegs[sendto+1:]...) 76 return nil 77 case reg.Ch <- bat: 78 ap.sendCnt++ 79 } 80 81 return nil 82 } 83 84 // common sender: send to all receiver 85 func sendToAllFunc(bat *batch.Batch, ap *Argument, proc *process.Process) error { 86 if !ap.prepared { 87 cnt := len(ap.RemoteRegs) 88 for cnt > 0 { 89 csinfo := <-proc.DispatchNotifyCh 90 ap.ctr.remoteReceivers = append(ap.ctr.remoteReceivers, &WrapperClientSession{ 91 msgId: csinfo.MsgId, 92 cs: csinfo.Cs, 93 uuid: csinfo.Uid, 94 doneCh: csinfo.DoneCh, 95 }) 96 cnt-- 97 } 98 ap.prepared = true 99 } 100 101 if ap.ctr.remoteReceivers != nil { 102 encodeData, errEncode := types.Encode(bat) 103 if errEncode != nil { 104 return errEncode 105 } 106 for _, r := range ap.ctr.remoteReceivers { 107 if err := sendBatchToClientSession(encodeData, r); err != nil { 108 return err 109 } 110 } 111 } 112 113 refCountAdd := int64(len(ap.LocalRegs) - 1) 114 atomic.AddInt64(&bat.Cnt, refCountAdd) 115 if jm, ok := bat.Ht.(*hashmap.JoinMap); ok { 116 jm.IncRef(refCountAdd) 117 jm.SetDupCount(int64(len(ap.LocalRegs))) 118 } 119 120 for _, reg := range ap.LocalRegs { 121 select { 122 case <-reg.Ctx.Done(): 123 return moerr.NewInternalError(proc.Ctx, "pipeline context has done.") 124 case reg.Ch <- bat: 125 } 126 } 127 128 return nil 129 } 130 131 func sendBatchToClientSession(encodeBatData []byte, wcs *WrapperClientSession) error { 132 checksum := crc32.ChecksumIEEE(encodeBatData) 133 if len(encodeBatData) <= maxMessageSizeToMoRpc { 134 timeoutCtx, cancel := context.WithTimeout(context.Background(), time.Second*10000) 135 _ = cancel 136 msg := cnclient.AcquireMessage() 137 { 138 msg.Id = wcs.msgId 139 msg.Data = encodeBatData 140 msg.Cmd = pipeline.BatchMessage 141 msg.Sid = pipeline.BatchEnd 142 msg.Checksum = checksum 143 } 144 if err := wcs.cs.Write(timeoutCtx, msg); err != nil { 145 return err 146 } 147 return nil 148 } 149 150 start := 0 151 for start < len(encodeBatData) { 152 end := start + maxMessageSizeToMoRpc 153 sid := pipeline.BatchWaitingNext 154 if end > len(encodeBatData) { 155 end = len(encodeBatData) 156 sid = pipeline.BatchEnd 157 } 158 timeoutCtx, cancel := context.WithTimeout(context.Background(), time.Second*10000) 159 _ = cancel 160 msg := cnclient.AcquireMessage() 161 { 162 msg.Id = wcs.msgId 163 msg.Data = encodeBatData[start:end] 164 msg.Cmd = pipeline.BatchMessage 165 msg.Sid = uint64(sid) 166 msg.Checksum = checksum 167 } 168 169 if err := wcs.cs.Write(timeoutCtx, msg); err != nil { 170 return err 171 } 172 start = end 173 } 174 return nil 175 }