github.com/matrixorigin/matrixone@v0.7.0/pkg/sql/compile/scopeRemoteRunTypes.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 compile 16 17 import ( 18 "context" 19 "hash/crc32" 20 "time" 21 22 "github.com/google/uuid" 23 "github.com/matrixorigin/matrixone/pkg/cnservice/cnclient" 24 "github.com/matrixorigin/matrixone/pkg/common/moerr" 25 "github.com/matrixorigin/matrixone/pkg/common/morpc" 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/fileservice" 30 "github.com/matrixorigin/matrixone/pkg/logutil" 31 "github.com/matrixorigin/matrixone/pkg/pb/pipeline" 32 "github.com/matrixorigin/matrixone/pkg/pb/plan" 33 "github.com/matrixorigin/matrixone/pkg/sql/colexec" 34 "github.com/matrixorigin/matrixone/pkg/txn/client" 35 "github.com/matrixorigin/matrixone/pkg/vm/engine" 36 "github.com/matrixorigin/matrixone/pkg/vm/process" 37 ) 38 39 // cnInformation records service information to help handle messages. 40 type cnInformation struct { 41 storeEngine engine.Engine 42 fileService fileservice.FileService 43 getClusterDetails engine.GetClusterDetailsFunc 44 } 45 46 // processHelper records source process information to help 47 // rebuild the process at the remote node. 48 type processHelper struct { 49 id string 50 lim process.Limitation 51 unixTime int64 52 txnOperator client.TxnOperator 53 txnClient client.TxnClient 54 sessionInfo process.SessionInfo 55 analysisNodeList []int32 56 } 57 58 // messageSenderOnClient is a structure 59 // for sending message and receiving results on cn-client. 60 type messageSenderOnClient struct { 61 ctx context.Context 62 ctxCancel context.CancelFunc 63 64 streamSender morpc.Stream 65 receiveCh chan morpc.Message 66 } 67 68 func newMessageSenderOnClient( 69 ctx context.Context, toAddr string) (messageSenderOnClient, error) { 70 var sender = messageSenderOnClient{} 71 72 streamSender, err := cnclient.GetStreamSender(toAddr) 73 if err != nil { 74 return sender, nil 75 } 76 77 sender.streamSender = streamSender 78 if _, ok := ctx.Deadline(); !ok { 79 sender.ctx, sender.ctxCancel = context.WithTimeout(ctx, time.Second*10000) 80 } else { 81 sender.ctx = ctx 82 } 83 return sender, nil 84 } 85 86 // XXX we can set a scope as argument directly next day. 87 func (sender *messageSenderOnClient) send( 88 scopeData, procData []byte, messageType uint64) error { 89 message := cnclient.AcquireMessage() 90 message.SetID(sender.streamSender.ID()) 91 message.SetMessageType(messageType) 92 message.SetData(scopeData) 93 message.SetProcData(procData) 94 return sender.streamSender.Send(sender.ctx, message) 95 } 96 97 func (sender *messageSenderOnClient) receiveMessage() (morpc.Message, error) { 98 var err error 99 if sender.receiveCh == nil { 100 sender.receiveCh, err = sender.streamSender.Receive() 101 if err != nil { 102 return nil, err 103 } 104 } 105 106 select { 107 case <-sender.ctx.Done(): 108 return nil, moerr.NewRPCTimeout(sender.ctx) 109 case val, ok := <-sender.receiveCh: 110 if !ok || val == nil { 111 // ch close 112 return nil, moerr.NewStreamClosed(sender.ctx) 113 } 114 return val, nil 115 } 116 } 117 118 func (sender *messageSenderOnClient) close() { 119 if sender.ctxCancel != nil { 120 sender.ctxCancel() 121 } 122 // XXX not a good way to deal it if close failed. 123 _ = sender.streamSender.Close() 124 } 125 126 // messageReceiverOnServer is a structure 127 // for processing received message and writing results back at cn-server. 128 type messageReceiverOnServer struct { 129 ctx context.Context 130 messageId uint64 131 messageTyp uint64 132 messageUuid uuid.UUID 133 134 cnInformation cnInformation 135 // information to build a process. 136 procBuildHelper processHelper 137 138 clientSession morpc.ClientSession 139 messageAcquirer func() morpc.Message 140 maxMessageSize int 141 scopeData []byte 142 143 // XXX what's that. So confused. 144 sequence uint64 145 146 // result. 147 finalAnalysisInfo []*process.AnalyzeInfo 148 } 149 150 func newMessageReceiverOnServer( 151 ctx context.Context, message morpc.Message, 152 cs morpc.ClientSession, messageAcquirer func() morpc.Message, 153 storeEngine engine.Engine, fileService fileservice.FileService, txnClient client.TxnClient, 154 getClusterDetails engine.GetClusterDetailsFunc, 155 ) messageReceiverOnServer { 156 m, ok := message.(*pipeline.Message) 157 if !ok { 158 logutil.Errorf("cn server should receive *pipeline.Message, but get %v", message) 159 panic("cn server receive a message with unexpected type") 160 } 161 162 receiver := messageReceiverOnServer{ 163 ctx: ctx, 164 messageId: m.GetId(), 165 messageTyp: m.GetCmd(), 166 clientSession: cs, 167 messageAcquirer: messageAcquirer, 168 maxMessageSize: 64 * mpool.MB, 169 sequence: 0, 170 } 171 172 switch m.GetCmd() { 173 case pipeline.PrepareDoneNotifyMessage: 174 opUuid, err := uuid.FromBytes(m.GetUuid()) 175 if err != nil { 176 logutil.Errorf("decode uuid from pipeline.Message failed, bytes are %v", m.GetUuid()) 177 panic("cn receive a message with wrong uuid bytes") 178 } 179 receiver.messageUuid = opUuid 180 181 case pipeline.PipelineMessage: 182 var err error 183 receiver.cnInformation = cnInformation{ 184 storeEngine: storeEngine, 185 fileService: fileService, 186 getClusterDetails: getClusterDetails, 187 } 188 receiver.procBuildHelper, err = generateProcessHelper(m.GetProcInfoData(), txnClient) 189 if err != nil { 190 logutil.Errorf("decode process info from pipeline.Message failed, bytes are %v", m.GetProcInfoData()) 191 panic("cn receive a message with wrong process bytes") 192 } 193 receiver.scopeData = m.Data 194 195 default: 196 logutil.Errorf("unknown cmd %d for pipeline.Message", m.GetCmd()) 197 panic("unknown message type") 198 } 199 200 return receiver 201 } 202 203 func (receiver *messageReceiverOnServer) acquireMessage() (*pipeline.Message, error) { 204 message, ok := receiver.messageAcquirer().(*pipeline.Message) 205 if !ok { 206 return nil, moerr.NewInternalError(receiver.ctx, "get a message with wrong type.") 207 } 208 message.SetID(receiver.messageId) 209 return message, nil 210 } 211 212 // newCompile make and return a new compile to run a pipeline. 213 func (receiver *messageReceiverOnServer) newCompile() *Compile { 214 // compile is almost surely wanting a small or middle pool. Later. 215 mp, err := mpool.NewMPool("compile", 0, mpool.NoFixed) 216 if err != nil { 217 panic(err) 218 } 219 pHelper, cnInfo := receiver.procBuildHelper, receiver.cnInformation 220 proc := process.New( 221 receiver.ctx, mp, 222 pHelper.txnClient, pHelper.txnOperator, 223 cnInfo.fileService, cnInfo.getClusterDetails, 224 ) 225 proc.UnixTime = pHelper.unixTime 226 proc.Id = pHelper.id 227 proc.Lim = pHelper.lim 228 proc.SessionInfo = pHelper.sessionInfo 229 proc.AnalInfos = make([]*process.AnalyzeInfo, len(pHelper.analysisNodeList)) 230 for i := range proc.AnalInfos { 231 proc.AnalInfos[i].NodeId = pHelper.analysisNodeList[i] 232 } 233 proc.DispatchNotifyCh = make(chan process.WrapCs, 1) 234 235 c := &Compile{ 236 ctx: receiver.ctx, 237 proc: proc, 238 e: cnInfo.storeEngine, 239 anal: &anaylze{}, 240 addr: colexec.CnAddr, 241 } 242 243 c.fill = func(_ any, b *batch.Batch) error { 244 return receiver.sendBatch(b) 245 } 246 return c 247 } 248 249 func (receiver *messageReceiverOnServer) sendError( 250 errInfo error) error { 251 message, err := receiver.acquireMessage() 252 if err != nil { 253 return err 254 } 255 message.SetID(receiver.messageId) 256 message.SetSid(pipeline.MessageEnd) 257 if errInfo != nil { 258 message.SetMoError(receiver.ctx, errInfo) 259 } 260 return receiver.clientSession.Write(receiver.ctx, message) 261 } 262 263 func (receiver *messageReceiverOnServer) sendBatch( 264 b *batch.Batch) error { 265 // there's no need to send the nil batch. 266 if b == nil { 267 return nil 268 } 269 data, err := types.Encode(b) 270 if err != nil { 271 return err 272 } 273 274 checksum := crc32.ChecksumIEEE(data) 275 if len(data) <= receiver.maxMessageSize { 276 m, errA := receiver.acquireMessage() 277 if errA != nil { 278 return errA 279 } 280 m.SetData(data) 281 // XXX too bad. 282 m.SetCheckSum(checksum) 283 m.SetSequence(receiver.sequence) 284 receiver.sequence++ 285 return receiver.clientSession.Write(receiver.ctx, m) 286 } 287 // if data is too large, cut and send 288 for start, end := 0, 0; start < len(data); start = end { 289 m, errA := receiver.acquireMessage() 290 if errA != nil { 291 return errA 292 } 293 end = start + receiver.maxMessageSize 294 if end > len(data) { 295 end = len(data) 296 m.SetSid(pipeline.BatchEnd) 297 } else { 298 m.SetSid(pipeline.WaitingNext) 299 } 300 m.SetData(data[start:end]) 301 m.SetCheckSum(checksum) 302 m.SetSequence(receiver.sequence) 303 receiver.sequence++ 304 305 if errW := receiver.clientSession.Write(receiver.ctx, m); errW != nil { 306 return errW 307 } 308 } 309 return nil 310 } 311 312 func (receiver *messageReceiverOnServer) sendEndMessage() error { 313 message, err := receiver.acquireMessage() 314 if err != nil { 315 return err 316 } 317 message.SetSid(pipeline.MessageEnd) 318 message.SetID(receiver.messageId) 319 message.SetMessageType(receiver.messageTyp) 320 321 analysisInfo := receiver.finalAnalysisInfo 322 if len(analysisInfo) > 0 { 323 anas := &pipeline.AnalysisList{ 324 List: make([]*plan.AnalyzeInfo, len(analysisInfo)), 325 } 326 for i, a := range analysisInfo { 327 anas.List[i] = convertToPlanAnalyzeInfo(a) 328 } 329 data, err := anas.Marshal() 330 if err != nil { 331 return err 332 } 333 message.SetAnalysis(data) 334 } 335 return receiver.clientSession.Write(receiver.ctx, message) 336 } 337 338 func generateProcessHelper(data []byte, cli client.TxnClient) (processHelper, error) { 339 procInfo := &pipeline.ProcessInfo{} 340 err := procInfo.Unmarshal(data) 341 if err != nil { 342 return processHelper{}, err 343 } 344 345 result := processHelper{ 346 id: procInfo.Id, 347 lim: convertToProcessLimitation(procInfo.Lim), 348 unixTime: procInfo.UnixTime, 349 txnClient: cli, 350 analysisNodeList: procInfo.GetAnalysisNodeList(), 351 } 352 result.txnOperator, err = cli.NewWithSnapshot([]byte(procInfo.Snapshot)) 353 if err != nil { 354 return processHelper{}, err 355 } 356 result.sessionInfo, err = convertToProcessSessionInfo(procInfo.SessionInfo) 357 if err != nil { 358 return processHelper{}, err 359 } 360 361 return result, nil 362 }