github.com/matrixorigin/matrixone@v1.2.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 "fmt" 20 "hash/crc32" 21 "runtime" 22 "sync/atomic" 23 "time" 24 25 qclient "github.com/matrixorigin/matrixone/pkg/queryservice/client" 26 27 "github.com/matrixorigin/matrixone/pkg/common/reuse" 28 29 "github.com/matrixorigin/matrixone/pkg/logservice" 30 31 "github.com/google/uuid" 32 "github.com/matrixorigin/matrixone/pkg/cnservice/cnclient" 33 "github.com/matrixorigin/matrixone/pkg/common/moerr" 34 "github.com/matrixorigin/matrixone/pkg/common/morpc" 35 "github.com/matrixorigin/matrixone/pkg/common/mpool" 36 "github.com/matrixorigin/matrixone/pkg/container/batch" 37 "github.com/matrixorigin/matrixone/pkg/defines" 38 "github.com/matrixorigin/matrixone/pkg/fileservice" 39 "github.com/matrixorigin/matrixone/pkg/lockservice" 40 "github.com/matrixorigin/matrixone/pkg/logutil" 41 "github.com/matrixorigin/matrixone/pkg/pb/pipeline" 42 "github.com/matrixorigin/matrixone/pkg/pb/plan" 43 "github.com/matrixorigin/matrixone/pkg/perfcounter" 44 "github.com/matrixorigin/matrixone/pkg/sql/colexec" 45 "github.com/matrixorigin/matrixone/pkg/txn/client" 46 "github.com/matrixorigin/matrixone/pkg/udf" 47 "github.com/matrixorigin/matrixone/pkg/vm/engine" 48 "github.com/matrixorigin/matrixone/pkg/vm/process" 49 ) 50 51 const ( 52 maxMessageSizeToMoRpc = 64 * mpool.MB 53 54 // HandleNotifyTimeout 55 // todo: this is a bad design here. 56 // we should do the waiting work in the prepare stage of the dispatch operator but not in the exec stage. 57 // do the waiting work in the exec stage can save some execution time, but it will cause an unstable waiting time. 58 // (because we cannot control the execution time of the running sql, 59 // and the coming time of the first batch of the result is not a constant time.) 60 // see the codes in pkg/sql/colexec/dispatch/dispatch.go:waitRemoteRegsReady() 61 // 62 // need to fix this in the future. for now, increase it to make tpch1T can run on 3 CN 63 HandleNotifyTimeout = 300 * time.Second 64 ) 65 66 // cnInformation records service information to help handle messages. 67 type cnInformation struct { 68 cnAddr string 69 storeEngine engine.Engine 70 fileService fileservice.FileService 71 lockService lockservice.LockService 72 queryClient qclient.QueryClient 73 hakeeper logservice.CNHAKeeperClient 74 udfService udf.Service 75 aicm *defines.AutoIncrCacheManager 76 } 77 78 // processHelper records source process information to help 79 // rebuild the process at the remote node. 80 type processHelper struct { 81 id string 82 lim process.Limitation 83 unixTime int64 84 accountId uint32 85 txnOperator client.TxnOperator 86 txnClient client.TxnClient 87 sessionInfo process.SessionInfo 88 analysisNodeList []int32 89 } 90 91 // messageSenderOnClient is a structure 92 // for sending message and receiving results on cn-client. 93 type messageSenderOnClient struct { 94 ctx context.Context 95 ctxCancel context.CancelFunc 96 97 streamSender morpc.Stream 98 receiveCh chan morpc.Message 99 100 c *Compile 101 } 102 103 func newMessageSenderOnClient( 104 ctx context.Context, c *Compile, toAddr string) (*messageSenderOnClient, error) { 105 var sender = new(messageSenderOnClient) 106 107 streamSender, err := cnclient.GetStreamSender(toAddr) 108 if err != nil { 109 return sender, err 110 } 111 112 sender.streamSender = streamSender 113 if _, ok := ctx.Deadline(); !ok { 114 sender.ctx, sender.ctxCancel = context.WithTimeout(ctx, time.Second*10000) 115 } else { 116 sender.ctx = ctx 117 } 118 sender.c = c 119 return sender, nil 120 } 121 122 // XXX we can set a scope as argument directly next day. 123 func (sender *messageSenderOnClient) send( 124 scopeData, procData []byte, _ pipeline.Method) error { 125 sdLen := len(scopeData) 126 if sdLen <= maxMessageSizeToMoRpc { 127 message := cnclient.AcquireMessage() 128 message.SetID(sender.streamSender.ID()) 129 message.SetMessageType(pipeline.Method_PipelineMessage) 130 message.SetData(scopeData) 131 message.SetProcData(procData) 132 message.SetSequence(0) 133 message.SetSid(pipeline.Status_Last) 134 return sender.streamSender.Send(sender.ctx, message) 135 } 136 137 start := 0 138 cnt := uint64(0) 139 for start < sdLen { 140 end := start + maxMessageSizeToMoRpc 141 142 message := cnclient.AcquireMessage() 143 message.SetID(sender.streamSender.ID()) 144 message.SetMessageType(pipeline.Method_PipelineMessage) 145 message.SetSequence(cnt) 146 if end >= sdLen { 147 message.SetData(scopeData[start:sdLen]) 148 message.SetProcData(procData) 149 message.SetSid(pipeline.Status_Last) 150 } else { 151 message.SetData(scopeData[start:end]) 152 message.SetSid(pipeline.Status_WaitingNext) 153 } 154 155 if err := sender.streamSender.Send(sender.ctx, message); err != nil { 156 return err 157 } 158 cnt++ 159 start = end 160 } 161 return nil 162 } 163 164 func (sender *messageSenderOnClient) receiveMessage() (morpc.Message, error) { 165 select { 166 case <-sender.ctx.Done(): 167 return nil, nil 168 169 case val, ok := <-sender.receiveCh: 170 if !ok || val == nil { 171 // ch close 172 logutil.Errorf("the stream is closed, ok: %v, val: %v", ok, val) 173 return nil, moerr.NewStreamClosed(sender.ctx) 174 } 175 return val, nil 176 } 177 } 178 179 func (sender *messageSenderOnClient) receiveBatch() (bat *batch.Batch, over bool, err error) { 180 var val morpc.Message 181 var m *pipeline.Message 182 var dataBuffer []byte 183 184 for { 185 val, err = sender.receiveMessage() 186 if err != nil { 187 return nil, false, err 188 } 189 if val == nil { 190 return nil, true, nil 191 } 192 193 m = val.(*pipeline.Message) 194 if info, get := m.TryToGetMoErr(); get { 195 return nil, false, info 196 } 197 if m.IsEndMessage() { 198 anaData := m.GetAnalyse() 199 if len(anaData) > 0 { 200 ana := new(pipeline.AnalysisList) 201 if err = ana.Unmarshal(anaData); err != nil { 202 return nil, false, err 203 } 204 mergeAnalyseInfo(sender.c.anal, ana) 205 } 206 return nil, true, nil 207 } 208 209 if dataBuffer == nil { 210 dataBuffer = m.Data 211 } else { 212 dataBuffer = append(dataBuffer, m.Data...) 213 } 214 215 if m.WaitingNextToMerge() { 216 continue 217 } 218 if m.Checksum != crc32.ChecksumIEEE(dataBuffer) { 219 return nil, false, moerr.NewInternalErrorNoCtx("Packages delivered by morpc is broken") 220 } 221 222 bat, err = decodeBatch(sender.c.proc.Mp(), dataBuffer) 223 return bat, false, err 224 } 225 } 226 227 func mergeAnalyseInfo(target *anaylze, ana *pipeline.AnalysisList) { 228 source := ana.List 229 if len(target.analInfos) != len(source) { 230 return 231 } 232 for i := range target.analInfos { 233 n := source[i] 234 atomic.AddInt64(&target.analInfos[i].OutputSize, n.OutputSize) 235 atomic.AddInt64(&target.analInfos[i].OutputRows, n.OutputRows) 236 atomic.AddInt64(&target.analInfos[i].InputRows, n.InputRows) 237 atomic.AddInt64(&target.analInfos[i].InputSize, n.InputSize) 238 atomic.AddInt64(&target.analInfos[i].MemorySize, n.MemorySize) 239 target.analInfos[i].MergeArray(n) 240 atomic.AddInt64(&target.analInfos[i].TimeConsumed, n.TimeConsumed) 241 atomic.AddInt64(&target.analInfos[i].WaitTimeConsumed, n.WaitTimeConsumed) 242 atomic.AddInt64(&target.analInfos[i].DiskIO, n.DiskIO) 243 atomic.AddInt64(&target.analInfos[i].S3IOByte, n.S3IOByte) 244 atomic.AddInt64(&target.analInfos[i].S3IOInputCount, n.S3IOInputCount) 245 atomic.AddInt64(&target.analInfos[i].S3IOOutputCount, n.S3IOOutputCount) 246 atomic.AddInt64(&target.analInfos[i].NetworkIO, n.NetworkIO) 247 atomic.AddInt64(&target.analInfos[i].ScanTime, n.ScanTime) 248 atomic.AddInt64(&target.analInfos[i].InsertTime, n.InsertTime) 249 } 250 } 251 252 func (sender *messageSenderOnClient) close() { 253 if sender.ctxCancel != nil { 254 sender.ctxCancel() 255 } 256 // XXX not a good way to deal it if close failed. 257 _ = sender.streamSender.Close(true) 258 } 259 260 // messageReceiverOnServer is a structure 261 // for processing received message and writing results back at cn-server. 262 type messageReceiverOnServer struct { 263 ctx context.Context 264 messageId uint64 265 messageTyp pipeline.Method 266 messageUuid uuid.UUID 267 268 cnInformation cnInformation 269 // information to build a process. 270 procBuildHelper processHelper 271 272 clientSession morpc.ClientSession 273 messageAcquirer func() morpc.Message 274 maxMessageSize int 275 scopeData []byte 276 277 // XXX what's that. So confused. 278 sequence uint64 279 280 // result. 281 finalAnalysisInfo []*process.AnalyzeInfo 282 } 283 284 func newMessageReceiverOnServer( 285 ctx context.Context, 286 cnAddr string, 287 m *pipeline.Message, 288 cs morpc.ClientSession, 289 messageAcquirer func() morpc.Message, 290 storeEngine engine.Engine, 291 fileService fileservice.FileService, 292 lockService lockservice.LockService, 293 queryClient qclient.QueryClient, 294 hakeeper logservice.CNHAKeeperClient, 295 udfService udf.Service, 296 txnClient client.TxnClient, 297 aicm *defines.AutoIncrCacheManager) messageReceiverOnServer { 298 299 receiver := messageReceiverOnServer{ 300 ctx: ctx, 301 messageId: m.GetId(), 302 messageTyp: m.GetCmd(), 303 clientSession: cs, 304 messageAcquirer: messageAcquirer, 305 maxMessageSize: maxMessageSizeToMoRpc, 306 sequence: 0, 307 } 308 receiver.cnInformation = cnInformation{ 309 cnAddr: cnAddr, 310 storeEngine: storeEngine, 311 fileService: fileService, 312 lockService: lockService, 313 queryClient: queryClient, 314 hakeeper: hakeeper, 315 udfService: udfService, 316 aicm: aicm, 317 } 318 319 switch m.GetCmd() { 320 case pipeline.Method_PrepareDoneNotifyMessage: 321 opUuid, err := uuid.FromBytes(m.GetUuid()) 322 if err != nil { 323 logutil.Errorf("decode uuid from pipeline.Message failed, bytes are %v", m.GetUuid()) 324 panic("cn receive a message with wrong uuid bytes") 325 } 326 receiver.messageUuid = opUuid 327 328 case pipeline.Method_PipelineMessage: 329 var err error 330 receiver.procBuildHelper, err = generateProcessHelper(m.GetProcInfoData(), txnClient) 331 if err != nil { 332 logutil.Errorf("decode process info from pipeline.Message failed, bytes are %v", m.GetProcInfoData()) 333 panic("cn receive a message with wrong process bytes") 334 } 335 receiver.scopeData = m.Data 336 337 default: 338 logutil.Errorf("unknown cmd %d for pipeline.Message", m.GetCmd()) 339 panic("unknown message type") 340 } 341 342 return receiver 343 } 344 345 func (receiver *messageReceiverOnServer) acquireMessage() (*pipeline.Message, error) { 346 message, ok := receiver.messageAcquirer().(*pipeline.Message) 347 if !ok { 348 return nil, moerr.NewInternalError(receiver.ctx, "get a message with wrong type.") 349 } 350 message.SetID(receiver.messageId) 351 return message, nil 352 } 353 354 // newCompile make and return a new compile to run a pipeline. 355 func (receiver *messageReceiverOnServer) newCompile() *Compile { 356 // compile is almost surely wanting a small or middle pool. Later. 357 mp, err := mpool.NewMPool("compile", 0, mpool.NoFixed) 358 if err != nil { 359 panic(err) 360 } 361 pHelper, cnInfo := receiver.procBuildHelper, receiver.cnInformation 362 proc := process.New( 363 receiver.ctx, 364 mp, 365 pHelper.txnClient, 366 pHelper.txnOperator, 367 cnInfo.fileService, 368 cnInfo.lockService, 369 cnInfo.queryClient, 370 cnInfo.hakeeper, 371 cnInfo.udfService, 372 cnInfo.aicm) 373 proc.UnixTime = pHelper.unixTime 374 proc.Id = pHelper.id 375 proc.Lim = pHelper.lim 376 proc.SessionInfo = pHelper.sessionInfo 377 proc.SessionInfo.StorageEngine = cnInfo.storeEngine 378 proc.AnalInfos = make([]*process.AnalyzeInfo, len(pHelper.analysisNodeList)) 379 for i := range proc.AnalInfos { 380 proc.AnalInfos[i] = reuse.Alloc[process.AnalyzeInfo](nil) 381 proc.AnalInfos[i].NodeId = pHelper.analysisNodeList[i] 382 } 383 proc.DispatchNotifyCh = make(chan process.WrapCs) 384 385 c := reuse.Alloc[Compile](nil) 386 c.proc = proc 387 c.proc.MessageBoard = c.MessageBoard 388 c.e = cnInfo.storeEngine 389 c.anal = newAnaylze() 390 c.anal.analInfos = proc.AnalInfos 391 c.addr = receiver.cnInformation.cnAddr 392 c.proc.Ctx = perfcounter.WithCounterSet(c.proc.Ctx, c.counterSet) 393 c.ctx = defines.AttachAccountId(c.proc.Ctx, pHelper.accountId) 394 395 c.fill = func(b *batch.Batch) error { 396 return receiver.sendBatch(b) 397 } 398 return c 399 } 400 401 func (receiver *messageReceiverOnServer) sendError( 402 errInfo error) error { 403 message, err := receiver.acquireMessage() 404 if err != nil { 405 return err 406 } 407 message.SetID(receiver.messageId) 408 message.SetSid(pipeline.Status_MessageEnd) 409 if errInfo != nil { 410 message.SetMoError(receiver.ctx, errInfo) 411 } 412 return receiver.clientSession.Write(receiver.ctx, message) 413 } 414 415 func (receiver *messageReceiverOnServer) sendBatch( 416 b *batch.Batch) error { 417 // there's no need to send the nil batch. 418 if b == nil { 419 return nil 420 } 421 422 data, err := b.MarshalBinary() 423 if err != nil { 424 return err 425 } 426 427 checksum := crc32.ChecksumIEEE(data) 428 dataLen := len(data) 429 if dataLen <= receiver.maxMessageSize { 430 m, errA := receiver.acquireMessage() 431 if errA != nil { 432 return errA 433 } 434 m.SetMessageType(pipeline.Method_BatchMessage) 435 m.SetData(data) 436 // XXX too bad. 437 m.SetCheckSum(checksum) 438 m.SetSequence(receiver.sequence) 439 m.SetSid(pipeline.Status_Last) 440 receiver.sequence++ 441 return receiver.clientSession.Write(receiver.ctx, m) 442 } 443 // if data is too large, cut and send 444 for start, end := 0, 0; start < dataLen; start = end { 445 m, errA := receiver.acquireMessage() 446 if errA != nil { 447 return errA 448 } 449 end = start + receiver.maxMessageSize 450 if end >= dataLen { 451 end = dataLen 452 m.SetSid(pipeline.Status_Last) 453 m.SetCheckSum(checksum) 454 } else { 455 m.SetSid(pipeline.Status_WaitingNext) 456 } 457 m.SetMessageType(pipeline.Method_BatchMessage) 458 m.SetData(data[start:end]) 459 m.SetSequence(receiver.sequence) 460 receiver.sequence++ 461 462 if errW := receiver.clientSession.Write(receiver.ctx, m); errW != nil { 463 return errW 464 } 465 } 466 return nil 467 } 468 469 func (receiver *messageReceiverOnServer) sendEndMessage() error { 470 message, err := receiver.acquireMessage() 471 if err != nil { 472 return err 473 } 474 message.SetSid(pipeline.Status_MessageEnd) 475 message.SetID(receiver.messageId) 476 message.SetMessageType(receiver.messageTyp) 477 478 analysisInfo := receiver.finalAnalysisInfo 479 if len(analysisInfo) > 0 { 480 anas := &pipeline.AnalysisList{ 481 List: make([]*plan.AnalyzeInfo, len(analysisInfo)), 482 } 483 for i, a := range analysisInfo { 484 anas.List[i] = convertToPlanAnalyzeInfo(a) 485 } 486 data, err := anas.Marshal() 487 if err != nil { 488 return err 489 } 490 message.SetAnalysis(data) 491 } 492 return receiver.clientSession.Write(receiver.ctx, message) 493 } 494 495 func generateProcessHelper(data []byte, cli client.TxnClient) (processHelper, error) { 496 procInfo := &pipeline.ProcessInfo{} 497 err := procInfo.Unmarshal(data) 498 if err != nil { 499 return processHelper{}, err 500 } 501 if len(procInfo.GetAnalysisNodeList()) == 0 { 502 panic(fmt.Sprintf("empty plan: %s", procInfo.Sql)) 503 } 504 505 result := processHelper{ 506 id: procInfo.Id, 507 lim: convertToProcessLimitation(procInfo.Lim), 508 unixTime: procInfo.UnixTime, 509 accountId: procInfo.AccountId, 510 txnClient: cli, 511 analysisNodeList: procInfo.GetAnalysisNodeList(), 512 } 513 result.txnOperator, err = cli.NewWithSnapshot([]byte(procInfo.Snapshot)) 514 if err != nil { 515 return processHelper{}, err 516 } 517 result.sessionInfo, err = convertToProcessSessionInfo(procInfo.SessionInfo) 518 if err != nil { 519 return processHelper{}, err 520 } 521 522 return result, nil 523 } 524 525 func (receiver *messageReceiverOnServer) GetProcByUuid(uid uuid.UUID) (*process.Process, error) { 526 getCtx, getCancel := context.WithTimeout(context.Background(), HandleNotifyTimeout) 527 var opProc *process.Process 528 var ok bool 529 530 for { 531 select { 532 case <-getCtx.Done(): 533 colexec.Get().GetProcByUuid(uid, true) 534 getCancel() 535 return nil, moerr.NewInternalError(receiver.ctx, "get dispatch process by uuid timeout") 536 537 case <-receiver.ctx.Done(): 538 colexec.Get().GetProcByUuid(uid, true) 539 getCancel() 540 return nil, nil 541 542 default: 543 if opProc, ok = colexec.Get().GetProcByUuid(uid, false); !ok { 544 // it's bad to call the Gosched() here. 545 // cut the HandleNotifyTimeout to 1ms, 1ms, 2ms, 3ms, 5ms, 8ms..., and use them as waiting time may be a better way. 546 runtime.Gosched() 547 } else { 548 getCancel() 549 return opProc, nil 550 } 551 } 552 } 553 }