github.com/matrixorigin/matrixone@v0.7.0/pkg/sql/compile/scopeRemoteRun.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 "time" 23 24 "github.com/google/uuid" 25 "github.com/matrixorigin/matrixone/pkg/common/moerr" 26 "github.com/matrixorigin/matrixone/pkg/common/morpc" 27 "github.com/matrixorigin/matrixone/pkg/common/mpool" 28 "github.com/matrixorigin/matrixone/pkg/container/batch" 29 "github.com/matrixorigin/matrixone/pkg/container/types" 30 "github.com/matrixorigin/matrixone/pkg/container/vector" 31 "github.com/matrixorigin/matrixone/pkg/fileservice" 32 "github.com/matrixorigin/matrixone/pkg/pb/pipeline" 33 "github.com/matrixorigin/matrixone/pkg/pb/plan" 34 "github.com/matrixorigin/matrixone/pkg/sql/colexec" 35 "github.com/matrixorigin/matrixone/pkg/sql/colexec/agg" 36 "github.com/matrixorigin/matrixone/pkg/sql/colexec/anti" 37 "github.com/matrixorigin/matrixone/pkg/sql/colexec/connector" 38 "github.com/matrixorigin/matrixone/pkg/sql/colexec/dispatch" 39 "github.com/matrixorigin/matrixone/pkg/sql/colexec/external" 40 "github.com/matrixorigin/matrixone/pkg/sql/colexec/group" 41 "github.com/matrixorigin/matrixone/pkg/sql/colexec/hashbuild" 42 "github.com/matrixorigin/matrixone/pkg/sql/colexec/insert" 43 "github.com/matrixorigin/matrixone/pkg/sql/colexec/intersect" 44 "github.com/matrixorigin/matrixone/pkg/sql/colexec/intersectall" 45 "github.com/matrixorigin/matrixone/pkg/sql/colexec/join" 46 "github.com/matrixorigin/matrixone/pkg/sql/colexec/left" 47 "github.com/matrixorigin/matrixone/pkg/sql/colexec/limit" 48 "github.com/matrixorigin/matrixone/pkg/sql/colexec/loopanti" 49 "github.com/matrixorigin/matrixone/pkg/sql/colexec/loopjoin" 50 "github.com/matrixorigin/matrixone/pkg/sql/colexec/loopleft" 51 "github.com/matrixorigin/matrixone/pkg/sql/colexec/loopmark" 52 "github.com/matrixorigin/matrixone/pkg/sql/colexec/loopsemi" 53 "github.com/matrixorigin/matrixone/pkg/sql/colexec/loopsingle" 54 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mark" 55 "github.com/matrixorigin/matrixone/pkg/sql/colexec/merge" 56 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergegroup" 57 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergelimit" 58 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergeoffset" 59 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergeorder" 60 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergetop" 61 "github.com/matrixorigin/matrixone/pkg/sql/colexec/minus" 62 "github.com/matrixorigin/matrixone/pkg/sql/colexec/offset" 63 "github.com/matrixorigin/matrixone/pkg/sql/colexec/order" 64 "github.com/matrixorigin/matrixone/pkg/sql/colexec/output" 65 "github.com/matrixorigin/matrixone/pkg/sql/colexec/product" 66 "github.com/matrixorigin/matrixone/pkg/sql/colexec/projection" 67 "github.com/matrixorigin/matrixone/pkg/sql/colexec/restrict" 68 "github.com/matrixorigin/matrixone/pkg/sql/colexec/semi" 69 "github.com/matrixorigin/matrixone/pkg/sql/colexec/single" 70 "github.com/matrixorigin/matrixone/pkg/sql/colexec/table_function" 71 "github.com/matrixorigin/matrixone/pkg/sql/colexec/top" 72 "github.com/matrixorigin/matrixone/pkg/txn/client" 73 "github.com/matrixorigin/matrixone/pkg/vm" 74 "github.com/matrixorigin/matrixone/pkg/vm/engine" 75 "github.com/matrixorigin/matrixone/pkg/vm/process" 76 ) 77 78 // CnServerMessageHandler is responsible for processing the cn-client message received at cn-server. 79 // the message is always *pipeline.Message here. It's a byte array which encoded by method encodeScope. 80 func CnServerMessageHandler( 81 ctx context.Context, 82 message morpc.Message, 83 cs morpc.ClientSession, 84 storeEngine engine.Engine, fileService fileservice.FileService, cli client.TxnClient, messageAcquirer func() morpc.Message, 85 getClusterDetails engine.GetClusterDetailsFunc) error { 86 // new a receiver to receive message and write back result. 87 receiver := newMessageReceiverOnServer(ctx, message, 88 cs, messageAcquirer, storeEngine, fileService, cli, getClusterDetails) 89 90 // rebuild pipeline to run and send query result back. 91 err := cnMessageHandle(receiver) 92 if err != nil { 93 return receiver.sendError(err) 94 } 95 return receiver.sendEndMessage() 96 } 97 98 func fillEngineForInsert(s *Scope, e engine.Engine) { 99 for i := range s.Instructions { 100 if s.Instructions[i].Op == vm.Insert { 101 s.Instructions[i].Arg.(*insert.Argument).Engine = e 102 } 103 } 104 for i := range s.PreScopes { 105 fillEngineForInsert(s.PreScopes[i], e) 106 } 107 } 108 109 // cnMessageHandle deal the received message at cn-server. 110 func cnMessageHandle(receiver messageReceiverOnServer) error { 111 switch receiver.messageTyp { 112 case pipeline.PrepareDoneNotifyMessage: // notify the dispatch executor 113 var ch chan process.WrapCs 114 var ok bool 115 opUuid := receiver.messageUuid 116 for { 117 if ch, ok = colexec.Srv.GetNotifyChByUuid(opUuid); !ok { 118 runtime.Gosched() 119 } else { 120 break 121 } 122 } 123 124 doneCh := make(chan struct{}) 125 info := process.WrapCs{ 126 MsgId: receiver.messageId, 127 Uid: opUuid, 128 Cs: receiver.clientSession, 129 DoneCh: doneCh, 130 } 131 ch <- info 132 <-doneCh 133 return nil 134 135 case pipeline.PipelineMessage: 136 c := receiver.newCompile() 137 138 // decode and rewrite the scope. 139 // insert operator needs to fill the engine info. 140 s, err := decodeScope(receiver.scopeData, c.proc, true) 141 if err != nil { 142 return err 143 } 144 fillEngineForInsert(s, c.e) 145 s = refactorScope(c, c.ctx, s) 146 147 err = s.ParallelRun(c, s.IsRemote) 148 if err != nil { 149 return err 150 } 151 receiver.finalAnalysisInfo = c.proc.AnalInfos 152 return nil 153 154 default: 155 return moerr.NewInternalError(receiver.ctx, "unknown message type") 156 } 157 } 158 159 // receiveMessageFromCnServer deal the back message from cn-server. 160 func receiveMessageFromCnServer(c *Compile, sender messageSenderOnClient, nextAnalyze process.Analyze, nextOperator *connector.Argument) error { 161 var val morpc.Message 162 var err error 163 var dataBuffer []byte 164 var sequence uint64 165 for { 166 val, err = sender.receiveMessage() 167 if err != nil { 168 return err 169 } 170 171 m := val.(*pipeline.Message) 172 173 if errInfo, get := m.TryToGetMoErr(); get { 174 return errInfo 175 } 176 if m.IsEndMessage() { 177 anaData := m.GetAnalyse() 178 if len(anaData) > 0 { 179 ana := new(pipeline.AnalysisList) 180 if err = ana.Unmarshal(anaData); err != nil { 181 return err 182 } 183 mergeAnalyseInfo(c.anal, ana) 184 } 185 return nil 186 } 187 // XXX some order check just for safety ? 188 if sequence != m.Sequence { 189 return moerr.NewInternalErrorNoCtx("Packages passed by morpc are out of order") 190 } 191 sequence++ 192 193 dataBuffer = append(dataBuffer, m.Data...) 194 if m.WaitingNextToMerge() { 195 continue 196 } 197 if m.Checksum != crc32.ChecksumIEEE(dataBuffer) { 198 return moerr.NewInternalErrorNoCtx("Packages delivered by morpc is broken") 199 } 200 201 bat, err := decodeBatch(c.proc.Mp(), dataBuffer) 202 if err != nil { 203 return err 204 } 205 nextAnalyze.Network(bat) 206 sendToConnectOperator(nextOperator, bat) 207 // XXX maybe we can use dataBuffer = dataBuffer[:0] to do memory reuse. 208 // but it seems that decode batch will do some memory reflect. but not copy. 209 dataBuffer = nil 210 } 211 } 212 213 // remoteRun sends a scope for remote running and receive the results. 214 // the back result message is always *pipeline.Message contains three cases. 215 // 1. Message with error information 216 // 2. Message with end flag and analysis result 217 // 3. Batch Message with batch data 218 func (s *Scope) remoteRun(c *Compile) error { 219 // encode the scope. shouldn't encode the `connector` operator which used to receive the back batch. 220 n := len(s.Instructions) - 1 221 con := s.Instructions[n] 222 s.Instructions = s.Instructions[:n] 223 sData, errEncode := encodeScope(s) 224 if errEncode != nil { 225 return errEncode 226 } 227 s.Instructions = append(s.Instructions, con) 228 229 // encode the process related information 230 pData, errEncodeProc := encodeProcessInfo(c.proc) 231 if errEncodeProc != nil { 232 return errEncodeProc 233 } 234 235 // new sender and do send work. 236 sender, err := newMessageSenderOnClient(c.ctx, c.addr) 237 if err != nil { 238 return err 239 } 240 err = sender.send(sData, pData, pipeline.PipelineMessage) 241 if err != nil { 242 sender.close() 243 return err 244 } 245 246 nextInstruction := s.Instructions[len(s.Instructions)-1] 247 nextAnalyze := c.proc.GetAnalyze(nextInstruction.Idx) 248 nextArg := nextInstruction.Arg.(*connector.Argument) 249 err = receiveMessageFromCnServer(c, sender, nextAnalyze, nextArg) 250 sender.close() 251 return err 252 } 253 254 // encodeScope generate a pipeline.Pipeline from Scope, encode pipeline, and returns. 255 func encodeScope(s *Scope) ([]byte, error) { 256 p, err := fillPipeline(s) 257 if err != nil { 258 return nil, err 259 } 260 return p.Marshal() 261 } 262 263 // decodeScope decode a pipeline.Pipeline from bytes, and generate a Scope from it. 264 func decodeScope(data []byte, proc *process.Process, isRemote bool) (*Scope, error) { 265 // unmarshal to pipeline 266 p := &pipeline.Pipeline{} 267 err := p.Unmarshal(data) 268 if err != nil { 269 return nil, err 270 } 271 ctx := &scopeContext{ 272 parent: nil, 273 id: p.PipelineId, 274 regs: make(map[*process.WaitRegister]int32), 275 } 276 ctx.root = ctx 277 s, err := generateScope(proc, p, ctx, nil, isRemote) 278 if err != nil { 279 return nil, err 280 } 281 if err := fillInstructionsForScope(s, ctx, p); err != nil { 282 return nil, err 283 } 284 285 return s, nil 286 } 287 288 // encodeProcessInfo get needed information from proc, and do serialization work. 289 func encodeProcessInfo(proc *process.Process) ([]byte, error) { 290 procInfo := &pipeline.ProcessInfo{} 291 { 292 procInfo.Id = proc.Id 293 procInfo.Lim = convertToPipelineLimitation(proc.Lim) 294 procInfo.UnixTime = proc.UnixTime 295 snapshot, err := proc.TxnOperator.Snapshot() 296 if err != nil { 297 return nil, err 298 } 299 procInfo.Snapshot = string(snapshot) 300 procInfo.AnalysisNodeList = make([]int32, len(proc.AnalInfos)) 301 for i := range procInfo.AnalysisNodeList { 302 procInfo.AnalysisNodeList[i] = proc.AnalInfos[i].NodeId 303 } 304 } 305 { // session info 306 timeBytes, err := time.Time{}.In(proc.SessionInfo.TimeZone).MarshalBinary() 307 if err != nil { 308 return nil, err 309 } 310 311 procInfo.SessionInfo = &pipeline.SessionInfo{ 312 User: proc.SessionInfo.GetUser(), 313 Host: proc.SessionInfo.GetHost(), 314 Role: proc.SessionInfo.GetRole(), 315 ConnectionId: proc.SessionInfo.GetConnectionID(), 316 Database: proc.SessionInfo.GetDatabase(), 317 Version: proc.SessionInfo.GetVersion(), 318 TimeZone: timeBytes, 319 } 320 } 321 return procInfo.Marshal() 322 } 323 324 func refactorScope(c *Compile, _ context.Context, s *Scope) *Scope { 325 rs := c.newMergeScope([]*Scope{s}) 326 rs.Instructions = append(rs.Instructions, vm.Instruction{ 327 Op: vm.Output, 328 Idx: -1, // useless 329 Arg: &output.Argument{Data: nil, Func: c.fill}, 330 }) 331 return rs 332 } 333 334 // fillPipeline convert the scope to pipeline.Pipeline structure through 2 iterations. 335 func fillPipeline(s *Scope) (*pipeline.Pipeline, error) { 336 ctx := &scopeContext{ 337 id: 0, 338 parent: nil, 339 regs: make(map[*process.WaitRegister]int32), 340 } 341 ctx.root = ctx 342 p, ctxId, err := generatePipeline(s, ctx, 1) 343 if err != nil { 344 return nil, err 345 } 346 if _, err = fillInstructionsForPipeline(s, ctx, p, ctxId); err != nil { 347 return nil, err 348 } 349 return p, nil 350 } 351 352 // generatePipeline generate a base pipeline.Pipeline structure without instructions 353 // according to source scope. 354 func generatePipeline(s *Scope, ctx *scopeContext, ctxId int32) (*pipeline.Pipeline, int32, error) { 355 var err error 356 357 p := &pipeline.Pipeline{} 358 // Magic and IsEnd 359 p.PipelineType = pipeline.Pipeline_PipelineType(s.Magic) 360 p.PipelineId = ctx.id 361 p.IsEnd = s.IsEnd 362 p.IsJoin = s.IsJoin 363 p.UuidsToRegIdx = convertScopeRemoteReceivInfo(s) 364 365 // Plan 366 if ctxId == 1 { 367 // encode and decode cost is too large for it. 368 // only encode the first one. 369 p.Qry = s.Plan 370 } 371 p.Node = &pipeline.NodeInfo{ 372 Id: s.NodeInfo.Id, 373 Addr: s.NodeInfo.Addr, 374 Mcpu: int32(s.NodeInfo.Mcpu), 375 Payload: make([]string, len(s.NodeInfo.Data)), 376 } 377 ctx.pipe = p 378 ctx.scope = s 379 { 380 for i := range s.NodeInfo.Data { 381 p.Node.Payload[i] = string(s.NodeInfo.Data[i]) 382 } 383 } 384 p.ChildrenCount = int32(len(s.Proc.Reg.MergeReceivers)) 385 { 386 for i := range s.Proc.Reg.MergeReceivers { 387 ctx.regs[s.Proc.Reg.MergeReceivers[i]] = int32(i) 388 } 389 } 390 // DataSource 391 if s.DataSource != nil { // if select 1, DataSource is nil 392 p.DataSource = &pipeline.Source{ 393 SchemaName: s.DataSource.SchemaName, 394 TableName: s.DataSource.RelationName, 395 ColList: s.DataSource.Attributes, 396 PushdownId: s.DataSource.PushdownId, 397 PushdownAddr: s.DataSource.PushdownAddr, 398 Expr: s.DataSource.Expr, 399 TableDef: s.DataSource.TableDef, 400 Timestamp: &s.DataSource.Timestamp, 401 } 402 if s.DataSource.Bat != nil { 403 data, err := types.Encode(s.DataSource.Bat) 404 if err != nil { 405 return nil, -1, err 406 } 407 p.DataSource.Block = string(data) 408 } 409 } 410 // PreScope 411 p.Children = make([]*pipeline.Pipeline, len(s.PreScopes)) 412 ctx.children = make([]*scopeContext, len(s.PreScopes)) 413 for i := range s.PreScopes { 414 ctx.children[i] = &scopeContext{ 415 parent: ctx, 416 id: ctxId, 417 root: ctx.root, 418 regs: make(map[*process.WaitRegister]int32), 419 } 420 ctxId++ 421 if p.Children[i], ctxId, err = generatePipeline(s.PreScopes[i], ctx.children[i], ctxId); err != nil { 422 return nil, -1, err 423 } 424 } 425 return p, ctxId, nil 426 } 427 428 // fillInstructionsForPipeline fills pipeline's instructions. 429 func fillInstructionsForPipeline(s *Scope, ctx *scopeContext, p *pipeline.Pipeline, ctxId int32) (int32, error) { 430 var err error 431 432 for i := range s.PreScopes { 433 if ctxId, err = fillInstructionsForPipeline(s.PreScopes[i], ctx.children[i], p.Children[i], ctxId); err != nil { 434 return ctxId, err 435 } 436 } 437 // Instructions 438 p.InstructionList = make([]*pipeline.Instruction, len(s.Instructions)) 439 for i := range p.InstructionList { 440 if ctxId, p.InstructionList[i], err = convertToPipelineInstruction(&s.Instructions[i], ctx, ctxId); err != nil { 441 return ctxId, err 442 } 443 } 444 return ctxId, nil 445 } 446 447 func convertPipelineUuid(p *pipeline.Pipeline, s *Scope) error { 448 s.RemoteReceivRegInfos = make([]RemoteReceivRegInfo, len(p.UuidsToRegIdx)) 449 for i, u := range p.UuidsToRegIdx { 450 uid, err := uuid.FromBytes(u.GetUuid()) 451 if err != nil { 452 return moerr.NewInvalidInputNoCtx("decode uuid failed: %s\n", err) 453 } 454 s.RemoteReceivRegInfos[i] = RemoteReceivRegInfo{ 455 Idx: int(u.GetIdx()), 456 Uuid: uid, 457 FromAddr: u.FromAddr, 458 } 459 } 460 return nil 461 } 462 463 func convertScopeRemoteReceivInfo(s *Scope) (ret []*pipeline.UuidToRegIdx) { 464 ret = make([]*pipeline.UuidToRegIdx, len(s.RemoteReceivRegInfos)) 465 for i, u := range s.RemoteReceivRegInfos { 466 ret[i] = &pipeline.UuidToRegIdx{ 467 Idx: int32(u.Idx), 468 Uuid: u.Uuid[:], 469 FromAddr: u.FromAddr, 470 } 471 } 472 return ret 473 } 474 475 // generateScope generate a scope from scope context and pipeline. 476 func generateScope(proc *process.Process, p *pipeline.Pipeline, ctx *scopeContext, 477 analNodes []*process.AnalyzeInfo, isRemote bool) (*Scope, error) { 478 var err error 479 if p.Qry != nil { 480 ctx.plan = p.Qry 481 } 482 483 s := &Scope{ 484 Magic: int(p.GetPipelineType()), 485 IsEnd: p.IsEnd, 486 IsJoin: p.IsJoin, 487 Plan: ctx.plan, 488 IsRemote: isRemote, 489 } 490 if err := convertPipelineUuid(p, s); err != nil { 491 return s, err 492 } 493 dsc := p.GetDataSource() 494 if dsc != nil { 495 s.DataSource = &Source{ 496 SchemaName: dsc.SchemaName, 497 RelationName: dsc.TableName, 498 Attributes: dsc.ColList, 499 PushdownId: dsc.PushdownId, 500 PushdownAddr: dsc.PushdownAddr, 501 Expr: dsc.Expr, 502 TableDef: dsc.TableDef, 503 Timestamp: *dsc.Timestamp, 504 } 505 if len(dsc.Block) > 0 { 506 bat := new(batch.Batch) 507 if err := types.Decode([]byte(dsc.Block), bat); err != nil { 508 return nil, err 509 } 510 s.DataSource.Bat = bat 511 } 512 } 513 if p.Node != nil { 514 s.NodeInfo.Id = p.Node.Id 515 s.NodeInfo.Addr = p.Node.Addr 516 s.NodeInfo.Mcpu = int(p.Node.Mcpu) 517 s.NodeInfo.Data = make([][]byte, len(p.Node.Payload)) 518 for i := range p.Node.Payload { 519 s.NodeInfo.Data[i] = []byte(p.Node.Payload[i]) 520 } 521 } 522 s.Proc = process.NewWithAnalyze(proc, proc.Ctx, int(p.ChildrenCount), analNodes) 523 { 524 for i := range s.Proc.Reg.MergeReceivers { 525 ctx.regs[s.Proc.Reg.MergeReceivers[i]] = int32(i) 526 } 527 } 528 s.PreScopes = make([]*Scope, len(p.Children)) 529 ctx.children = make([]*scopeContext, len(s.PreScopes)) 530 for i := range s.PreScopes { 531 ctx.children[i] = &scopeContext{ 532 parent: ctx, 533 root: ctx.root, 534 id: p.Children[i].PipelineId, 535 regs: make(map[*process.WaitRegister]int32), 536 } 537 if s.PreScopes[i], err = generateScope(s.Proc, p.Children[i], ctx.children[i], analNodes, isRemote); err != nil { 538 return nil, err 539 } 540 } 541 return s, nil 542 } 543 544 // fillInstructionsForScope fills scope's instructions. 545 func fillInstructionsForScope(s *Scope, ctx *scopeContext, p *pipeline.Pipeline) error { 546 var err error 547 548 for i := range s.PreScopes { 549 if err = fillInstructionsForScope(s.PreScopes[i], ctx.children[i], p.Children[i]); err != nil { 550 return err 551 } 552 } 553 s.Instructions = make([]vm.Instruction, len(p.InstructionList)) 554 for i := range s.Instructions { 555 if s.Instructions[i], err = convertToVmInstruction(p.InstructionList[i], ctx); err != nil { 556 return err 557 } 558 } 559 return nil 560 } 561 562 // convert vm.Instruction to pipeline.Instruction 563 func convertToPipelineInstruction(opr *vm.Instruction, ctx *scopeContext, ctxId int32) (int32, *pipeline.Instruction, error) { 564 var err error 565 566 in := &pipeline.Instruction{Op: int32(opr.Op), Idx: int32(opr.Idx), IsFirst: opr.IsFirst, IsLast: opr.IsLast} 567 switch t := opr.Arg.(type) { 568 case *insert.Argument: 569 in.Insert = &pipeline.Insert{ 570 IsRemote: t.IsRemote, 571 Affected: t.Affected, 572 // TargetColDefs: t.TargetColDefs, 573 // TableID: t.TableID, 574 // CPkeyColDef: t.CPkeyColDef, 575 // DBName: t.DBName, 576 // TableName: t.TableName, 577 // ClusterTable: t.ClusterTable, 578 // ClusterByDef: t.ClusterByDef, 579 // UniqueIndexDef: t.UniqueIndexDef, 580 // HasAutoCol: t.HasAutoCol, 581 Ref: t.InsertCtx.Ref, 582 TableDef: t.InsertCtx.TableDef, 583 Idx: t.InsertCtx.Idx, 584 ClusterTable: t.InsertCtx.ClusterTable, 585 ParentIdx: t.InsertCtx.ParentIdx, 586 } 587 case *anti.Argument: 588 in.Anti = &pipeline.AntiJoin{ 589 Ibucket: t.Ibucket, 590 Nbucket: t.Nbucket, 591 Expr: t.Cond, 592 Types: convertToPlanTypes(t.Typs), 593 LeftCond: t.Conditions[0], 594 RightCond: t.Conditions[1], 595 Result: t.Result, 596 } 597 case *dispatch.Argument: 598 in.Dispatch = &pipeline.Dispatch{FuncId: int32(t.FuncId)} 599 in.Dispatch.LocalConnector = make([]*pipeline.Connector, len(t.LocalRegs)) 600 for i := range t.LocalRegs { 601 idx, ctx0 := ctx.root.findRegister(t.LocalRegs[i]) 602 if ctx0.root.isRemote(ctx0, 0) && !ctx0.isDescendant(ctx) { 603 id := colexec.Srv.RegistConnector(t.LocalRegs[i]) 604 if ctxId, err = ctx0.addSubPipeline(id, idx, ctxId); err != nil { 605 return ctxId, nil, err 606 } 607 } 608 in.Dispatch.LocalConnector[i] = &pipeline.Connector{ 609 ConnectorIndex: idx, 610 PipelineId: ctx0.id, 611 } 612 } 613 614 if len(t.RemoteRegs) > 0 { 615 in.Dispatch.RemoteConnector = make([]*pipeline.WrapNode, len(t.RemoteRegs)) 616 for i, r := range t.RemoteRegs { 617 wn := &pipeline.WrapNode{ 618 NodeAddr: r.NodeAddr, 619 Uuid: r.Uuid[:], 620 } 621 in.Dispatch.RemoteConnector[i] = wn 622 } 623 } 624 case *group.Argument: 625 in.Agg = &pipeline.Group{ 626 NeedEval: t.NeedEval, 627 Ibucket: t.Ibucket, 628 Nbucket: t.Nbucket, 629 Exprs: t.Exprs, 630 Types: convertToPlanTypes(t.Types), 631 Aggs: convertToPipelineAggregates(t.Aggs), 632 } 633 case *join.Argument: 634 relList, colList := getRelColList(t.Result) 635 in.Join = &pipeline.Join{ 636 Ibucket: t.Ibucket, 637 Nbucket: t.Nbucket, 638 RelList: relList, 639 ColList: colList, 640 Expr: t.Cond, 641 Types: convertToPlanTypes(t.Typs), 642 LeftCond: t.Conditions[0], 643 RightCond: t.Conditions[1], 644 } 645 case *left.Argument: 646 relList, colList := getRelColList(t.Result) 647 in.LeftJoin = &pipeline.LeftJoin{ 648 Ibucket: t.Ibucket, 649 Nbucket: t.Nbucket, 650 RelList: relList, 651 ColList: colList, 652 Expr: t.Cond, 653 Types: convertToPlanTypes(t.Typs), 654 LeftCond: t.Conditions[0], 655 RightCond: t.Conditions[1], 656 } 657 case *limit.Argument: 658 in.Limit = t.Limit 659 case *loopanti.Argument: 660 in.Anti = &pipeline.AntiJoin{ 661 Result: t.Result, 662 Expr: t.Cond, 663 Types: convertToPlanTypes(t.Typs), 664 } 665 case *loopjoin.Argument: 666 relList, colList := getRelColList(t.Result) 667 in.Join = &pipeline.Join{ 668 RelList: relList, 669 ColList: colList, 670 Expr: t.Cond, 671 Types: convertToPlanTypes(t.Typs), 672 } 673 case *loopleft.Argument: 674 relList, colList := getRelColList(t.Result) 675 in.LeftJoin = &pipeline.LeftJoin{ 676 RelList: relList, 677 ColList: colList, 678 Expr: t.Cond, 679 Types: convertToPlanTypes(t.Typs), 680 } 681 case *loopsemi.Argument: 682 in.SemiJoin = &pipeline.SemiJoin{ 683 Result: t.Result, 684 Expr: t.Cond, 685 Types: convertToPlanTypes(t.Typs), 686 } 687 case *loopsingle.Argument: 688 relList, colList := getRelColList(t.Result) 689 in.SingleJoin = &pipeline.SingleJoin{ 690 RelList: relList, 691 ColList: colList, 692 Expr: t.Cond, 693 Types: convertToPlanTypes(t.Typs), 694 } 695 case *offset.Argument: 696 in.Offset = t.Offset 697 case *order.Argument: 698 in.OrderBy = t.Fs 699 case *product.Argument: 700 relList, colList := getRelColList(t.Result) 701 in.Product = &pipeline.Product{ 702 RelList: relList, 703 ColList: colList, 704 Types: convertToPlanTypes(t.Typs), 705 } 706 case *projection.Argument: 707 in.ProjectList = t.Es 708 case *restrict.Argument: 709 in.Filter = t.E 710 case *semi.Argument: 711 in.SemiJoin = &pipeline.SemiJoin{ 712 Ibucket: t.Ibucket, 713 Nbucket: t.Nbucket, 714 Result: t.Result, 715 Expr: t.Cond, 716 Types: convertToPlanTypes(t.Typs), 717 LeftCond: t.Conditions[0], 718 RightCond: t.Conditions[1], 719 } 720 case *single.Argument: 721 relList, colList := getRelColList(t.Result) 722 in.SingleJoin = &pipeline.SingleJoin{ 723 Ibucket: t.Ibucket, 724 Nbucket: t.Nbucket, 725 RelList: relList, 726 ColList: colList, 727 Expr: t.Cond, 728 Types: convertToPlanTypes(t.Typs), 729 LeftCond: t.Conditions[0], 730 RightCond: t.Conditions[1], 731 } 732 case *top.Argument: 733 in.Limit = uint64(t.Limit) 734 in.OrderBy = t.Fs 735 // we reused ANTI to store the information here because of the lack of related structure. 736 case *intersect.Argument: // 1 737 in.Anti = &pipeline.AntiJoin{ 738 Ibucket: t.IBucket, 739 Nbucket: t.NBucket, 740 } 741 case *minus.Argument: // 2 742 in.Anti = &pipeline.AntiJoin{ 743 Ibucket: t.IBucket, 744 Nbucket: t.NBucket, 745 } 746 case *intersectall.Argument: 747 in.Anti = &pipeline.AntiJoin{ 748 Ibucket: t.IBucket, 749 Nbucket: t.NBucket, 750 } 751 case *merge.Argument: 752 case *mergegroup.Argument: 753 in.Agg = &pipeline.Group{ 754 NeedEval: t.NeedEval, 755 } 756 case *mergelimit.Argument: 757 in.Limit = t.Limit 758 case *mergeoffset.Argument: 759 in.Offset = t.Offset 760 case *mergetop.Argument: 761 in.Limit = uint64(t.Limit) 762 in.OrderBy = t.Fs 763 case *mergeorder.Argument: 764 in.OrderBy = t.Fs 765 case *connector.Argument: 766 idx, ctx0 := ctx.root.findRegister(t.Reg) 767 if ctx0.root.isRemote(ctx0, 0) && !ctx0.isDescendant(ctx) { 768 id := colexec.Srv.RegistConnector(t.Reg) 769 if ctxId, err = ctx0.addSubPipeline(id, idx, ctxId); err != nil { 770 return ctxId, nil, err 771 } 772 } 773 in.Connect = &pipeline.Connector{ 774 PipelineId: ctx0.id, 775 ConnectorIndex: idx, 776 } 777 case *mark.Argument: 778 in.MarkJoin = &pipeline.MarkJoin{ 779 Ibucket: t.Ibucket, 780 Nbucket: t.Nbucket, 781 Result: t.Result, 782 LeftCond: t.Conditions[0], 783 RightCond: t.Conditions[1], 784 Types: convertToPlanTypes(t.Typs), 785 Expr: t.Cond, 786 OnList: t.OnList, 787 } 788 case *table_function.Argument: 789 in.TableFunction = &pipeline.TableFunction{ 790 Attrs: t.Attrs, 791 Rets: t.Rets, 792 Args: t.Args, 793 Params: t.Params, 794 Name: t.Name, 795 } 796 case *hashbuild.Argument: 797 in.HashBuild = &pipeline.HashBuild{ 798 NeedExpr: t.NeedExpr, 799 NeedHash: t.NeedHashMap, 800 Ibucket: t.Ibucket, 801 Nbucket: t.Nbucket, 802 Types: convertToPlanTypes(t.Typs), 803 Conds: t.Conditions, 804 } 805 case *external.Argument: 806 name2ColIndexSlice := make([]*pipeline.ExternalName2ColIndex, len(t.Es.Name2ColIndex)) 807 i := 0 808 for k, v := range t.Es.Name2ColIndex { 809 name2ColIndexSlice[i] = &pipeline.ExternalName2ColIndex{Name: k, Index: v} 810 i++ 811 } 812 in.ExternalScan = &pipeline.ExternalScan{ 813 Attrs: t.Es.Attrs, 814 Cols: t.Es.Cols, 815 Name2ColIndex: name2ColIndexSlice, 816 CreateSql: t.Es.CreateSql, 817 FileList: t.Es.FileList, 818 } 819 default: 820 return -1, nil, moerr.NewInternalErrorNoCtx(fmt.Sprintf("unexpected operator: %v", opr.Op)) 821 } 822 return ctxId, in, nil 823 } 824 825 // convert pipeline.Instruction to vm.Instruction 826 func convertToVmInstruction(opr *pipeline.Instruction, ctx *scopeContext) (vm.Instruction, error) { 827 v := vm.Instruction{Op: int(opr.Op), Idx: int(opr.Idx), IsFirst: opr.IsFirst, IsLast: opr.IsLast} 828 switch opr.Op { 829 case vm.Insert: 830 t := opr.GetInsert() 831 v.Arg = &insert.Argument{ 832 Affected: t.Affected, 833 IsRemote: t.IsRemote, 834 InsertCtx: &insert.InsertCtx{ 835 Idx: t.Idx, 836 Ref: t.Ref, 837 TableDef: t.TableDef, 838 ParentIdx: t.ParentIdx, 839 ClusterTable: t.ClusterTable, 840 }, 841 } 842 case vm.Anti: 843 t := opr.GetAnti() 844 v.Arg = &anti.Argument{ 845 Ibucket: t.Ibucket, 846 Nbucket: t.Nbucket, 847 Cond: t.Expr, 848 Typs: convertToTypes(t.Types), 849 Conditions: [][]*plan.Expr{ 850 t.LeftCond, t.RightCond, 851 }, 852 Result: t.Result, 853 } 854 case vm.Dispatch: 855 t := opr.GetDispatch() 856 regs := make([]*process.WaitRegister, len(t.LocalConnector)) 857 for i, cp := range t.LocalConnector { 858 regs[i] = ctx.root.getRegister(cp.PipelineId, cp.ConnectorIndex) 859 } 860 rrs := make([]colexec.ReceiveInfo, 0) 861 if len(t.RemoteConnector) > 0 { 862 for _, rc := range t.RemoteConnector { 863 uid, err := uuid.FromBytes(rc.Uuid) 864 if err != nil { 865 return v, err 866 } 867 n := colexec.ReceiveInfo{ 868 NodeAddr: rc.NodeAddr, 869 Uuid: uid, 870 } 871 rrs = append(rrs, n) 872 } 873 } 874 v.Arg = &dispatch.Argument{ 875 FuncId: int(t.FuncId), 876 LocalRegs: regs, 877 RemoteRegs: rrs, 878 } 879 case vm.Group: 880 t := opr.GetAgg() 881 v.Arg = &group.Argument{ 882 NeedEval: t.NeedEval, 883 Ibucket: t.Ibucket, 884 Nbucket: t.Nbucket, 885 Exprs: t.Exprs, 886 Types: convertToTypes(t.Types), 887 Aggs: convertToAggregates(t.Aggs), 888 } 889 case vm.Join: 890 t := opr.GetJoin() 891 v.Arg = &join.Argument{ 892 Ibucket: t.Ibucket, 893 Nbucket: t.Nbucket, 894 Cond: t.Expr, 895 Typs: convertToTypes(t.Types), 896 Result: convertToResultPos(t.RelList, t.ColList), 897 Conditions: [][]*plan.Expr{t.LeftCond, t.RightCond}, 898 } 899 case vm.Left: 900 t := opr.GetLeftJoin() 901 v.Arg = &left.Argument{ 902 Ibucket: t.Ibucket, 903 Nbucket: t.Nbucket, 904 Cond: t.Expr, 905 Typs: convertToTypes(t.Types), 906 Result: convertToResultPos(t.RelList, t.ColList), 907 Conditions: [][]*plan.Expr{t.LeftCond, t.RightCond}, 908 } 909 910 case vm.Limit: 911 v.Arg = &limit.Argument{Limit: opr.Limit} 912 case vm.LoopAnti: 913 t := opr.GetAnti() 914 v.Arg = &loopanti.Argument{ 915 Result: t.Result, 916 Cond: t.Expr, 917 Typs: convertToTypes(t.Types), 918 } 919 case vm.LoopJoin: 920 t := opr.GetJoin() 921 v.Arg = &loopjoin.Argument{ 922 Result: convertToResultPos(t.RelList, t.ColList), 923 Cond: t.Expr, 924 Typs: convertToTypes(t.Types), 925 } 926 case vm.LoopLeft: 927 t := opr.GetLeftJoin() 928 v.Arg = &loopleft.Argument{ 929 Result: convertToResultPos(t.RelList, t.ColList), 930 Cond: t.Expr, 931 Typs: convertToTypes(t.Types), 932 } 933 case vm.LoopSemi: 934 t := opr.GetSemiJoin() 935 v.Arg = &loopsemi.Argument{ 936 Result: t.Result, 937 Cond: t.Expr, 938 Typs: convertToTypes(t.Types), 939 } 940 case vm.LoopSingle: 941 t := opr.GetSingleJoin() 942 v.Arg = &loopsingle.Argument{ 943 Result: convertToResultPos(t.RelList, t.ColList), 944 Cond: t.Expr, 945 Typs: convertToTypes(t.Types), 946 } 947 case vm.LoopMark: 948 t := opr.GetMarkJoin() 949 v.Arg = &loopmark.Argument{ 950 Result: t.Result, 951 Cond: t.Expr, 952 Typs: convertToTypes(t.Types), 953 } 954 case vm.Offset: 955 v.Arg = &offset.Argument{Offset: opr.Offset} 956 case vm.Order: 957 v.Arg = &order.Argument{Fs: opr.OrderBy} 958 case vm.Product: 959 t := opr.GetProduct() 960 v.Arg = &product.Argument{ 961 Result: convertToResultPos(t.RelList, t.ColList), 962 Typs: convertToTypes(t.Types), 963 } 964 case vm.Projection: 965 v.Arg = &projection.Argument{Es: opr.ProjectList} 966 case vm.Restrict: 967 v.Arg = &restrict.Argument{E: opr.Filter} 968 case vm.Semi: 969 t := opr.GetSemiJoin() 970 v.Arg = &semi.Argument{ 971 Ibucket: t.Ibucket, 972 Nbucket: t.Nbucket, 973 Result: t.Result, 974 Cond: t.Expr, 975 Typs: convertToTypes(t.Types), 976 Conditions: [][]*plan.Expr{t.LeftCond, t.RightCond}, 977 } 978 case vm.Single: 979 t := opr.GetSingleJoin() 980 v.Arg = &single.Argument{ 981 Ibucket: t.Ibucket, 982 Nbucket: t.Nbucket, 983 Result: convertToResultPos(t.RelList, t.ColList), 984 Cond: t.Expr, 985 Typs: convertToTypes(t.Types), 986 Conditions: [][]*plan.Expr{t.LeftCond, t.RightCond}, 987 } 988 case vm.Mark: 989 t := opr.GetMarkJoin() 990 v.Arg = &mark.Argument{ 991 Ibucket: t.Ibucket, 992 Nbucket: t.Nbucket, 993 Result: t.Result, 994 Conditions: [][]*plan.Expr{t.LeftCond, t.RightCond}, 995 Typs: convertToTypes(t.Types), 996 Cond: t.Expr, 997 OnList: t.OnList, 998 } 999 case vm.Top: 1000 v.Arg = &top.Argument{ 1001 Limit: int64(opr.Limit), 1002 Fs: opr.OrderBy, 1003 } 1004 // should change next day? 1005 case vm.Intersect: 1006 t := opr.GetAnti() 1007 v.Arg = &intersect.Argument{ 1008 IBucket: t.Ibucket, 1009 NBucket: t.Nbucket, 1010 } 1011 case vm.IntersectAll: 1012 t := opr.GetAnti() 1013 v.Arg = &intersectall.Argument{ 1014 IBucket: t.Ibucket, 1015 NBucket: t.Nbucket, 1016 } 1017 case vm.Minus: 1018 t := opr.GetAnti() 1019 v.Arg = &minus.Argument{ 1020 IBucket: t.Ibucket, 1021 NBucket: t.Nbucket, 1022 } 1023 case vm.Connector: 1024 t := opr.GetConnect() 1025 v.Arg = &connector.Argument{ 1026 Reg: ctx.root.getRegister(t.PipelineId, t.ConnectorIndex), 1027 } 1028 case vm.Merge: 1029 v.Arg = &merge.Argument{} 1030 case vm.MergeGroup: 1031 v.Arg = &mergegroup.Argument{ 1032 NeedEval: opr.Agg.NeedEval, 1033 } 1034 case vm.MergeLimit: 1035 v.Arg = &mergelimit.Argument{ 1036 Limit: opr.Limit, 1037 } 1038 case vm.MergeOffset: 1039 v.Arg = &mergeoffset.Argument{ 1040 Offset: opr.Offset, 1041 } 1042 case vm.MergeTop: 1043 v.Arg = &mergetop.Argument{ 1044 Limit: int64(opr.Limit), 1045 Fs: opr.OrderBy, 1046 } 1047 case vm.MergeOrder: 1048 v.Arg = &mergeorder.Argument{ 1049 Fs: opr.OrderBy, 1050 } 1051 case vm.TableFunction: 1052 v.Arg = &table_function.Argument{ 1053 Attrs: opr.TableFunction.Attrs, 1054 Rets: opr.TableFunction.Rets, 1055 Args: opr.TableFunction.Args, 1056 Name: opr.TableFunction.Name, 1057 Params: opr.TableFunction.Params, 1058 } 1059 case vm.HashBuild: 1060 t := opr.GetHashBuild() 1061 v.Arg = &hashbuild.Argument{ 1062 Ibucket: t.Ibucket, 1063 Nbucket: t.Nbucket, 1064 NeedHashMap: t.NeedHash, 1065 NeedExpr: t.NeedExpr, 1066 Typs: convertToTypes(t.Types), 1067 Conditions: t.Conds, 1068 } 1069 case vm.External: 1070 t := opr.GetExternalScan() 1071 name2ColIndex := make(map[string]int32) 1072 for _, n2i := range t.Name2ColIndex { 1073 name2ColIndex[n2i.Name] = n2i.Index 1074 } 1075 v.Arg = &external.Argument{ 1076 Es: &external.ExternalParam{ 1077 ExParamConst: external.ExParamConst{ 1078 Attrs: t.Attrs, 1079 Cols: t.Cols, 1080 CreateSql: t.CreateSql, 1081 Name2ColIndex: name2ColIndex, 1082 FileList: t.FileList, 1083 OriginCols: t.OriginCols, 1084 }, 1085 ExParam: external.ExParam{ 1086 Fileparam: new(external.ExFileparam), 1087 }, 1088 }, 1089 } 1090 default: 1091 return v, moerr.NewInternalErrorNoCtx(fmt.Sprintf("unexpected operator: %v", opr.Op)) 1092 } 1093 return v, nil 1094 } 1095 1096 func mergeAnalyseInfo(target *anaylze, ana *pipeline.AnalysisList) { 1097 source := ana.List 1098 if len(target.analInfos) != len(source) { 1099 return 1100 } 1101 for i := range target.analInfos { 1102 n := source[i] 1103 target.analInfos[i].OutputSize += n.OutputSize 1104 target.analInfos[i].OutputRows += n.OutputRows 1105 target.analInfos[i].InputRows += n.InputRows 1106 target.analInfos[i].InputSize += n.InputSize 1107 target.analInfos[i].MemorySize += n.MemorySize 1108 target.analInfos[i].TimeConsumed += n.TimeConsumed 1109 target.analInfos[i].WaitTimeConsumed += n.WaitTimeConsumed 1110 target.analInfos[i].DiskIO += n.DiskIO 1111 target.analInfos[i].S3IOByte += n.S3IOByte 1112 target.analInfos[i].S3IOCount += n.S3IOCount 1113 target.analInfos[i].NetworkIO += n.NetworkIO 1114 target.analInfos[i].ScanTime += n.ScanTime 1115 target.analInfos[i].InsertTime += n.InsertTime 1116 } 1117 } 1118 1119 // convert []types.Type to []*plan.Type 1120 func convertToPlanTypes(ts []types.Type) []*plan.Type { 1121 result := make([]*plan.Type, len(ts)) 1122 for i, t := range ts { 1123 result[i] = &plan.Type{ 1124 Id: int32(t.Oid), 1125 Width: t.Width, 1126 Precision: t.Precision, 1127 Size: t.Size, 1128 Scale: t.Scale, 1129 } 1130 } 1131 return result 1132 } 1133 1134 // convert []*plan.Type to []types.Type 1135 func convertToTypes(ts []*plan.Type) []types.Type { 1136 result := make([]types.Type, len(ts)) 1137 for i, t := range ts { 1138 result[i] = types.Type{ 1139 Oid: types.T(t.Id), 1140 Width: t.Width, 1141 Precision: t.Precision, 1142 Size: t.Size, 1143 Scale: t.Scale, 1144 } 1145 } 1146 return result 1147 } 1148 1149 // convert []agg.Aggregate to []*pipeline.Aggregate 1150 func convertToPipelineAggregates(ags []agg.Aggregate) []*pipeline.Aggregate { 1151 result := make([]*pipeline.Aggregate, len(ags)) 1152 for i, a := range ags { 1153 result[i] = &pipeline.Aggregate{ 1154 Op: int32(a.Op), 1155 Dist: a.Dist, 1156 Expr: a.E, 1157 } 1158 } 1159 return result 1160 } 1161 1162 // convert []*pipeline.Aggregate to []agg.Aggregate 1163 func convertToAggregates(ags []*pipeline.Aggregate) []agg.Aggregate { 1164 result := make([]agg.Aggregate, len(ags)) 1165 for i, a := range ags { 1166 result[i] = agg.Aggregate{ 1167 Op: int(a.Op), 1168 Dist: a.Dist, 1169 E: a.Expr, 1170 } 1171 } 1172 return result 1173 } 1174 1175 // get relation list and column list from []colexec.ResultPos 1176 func getRelColList(resultPos []colexec.ResultPos) (relList []int32, colList []int32) { 1177 relList = make([]int32, len(resultPos)) 1178 colList = make([]int32, len(resultPos)) 1179 for i := range resultPos { 1180 relList[i], colList[i] = resultPos[i].Rel, resultPos[i].Pos 1181 } 1182 return 1183 } 1184 1185 // generate []colexec.ResultPos from relation list and column list 1186 func convertToResultPos(relList, colList []int32) []colexec.ResultPos { 1187 res := make([]colexec.ResultPos, len(relList)) 1188 for i := range res { 1189 res[i].Rel, res[i].Pos = relList[i], colList[i] 1190 } 1191 return res 1192 } 1193 1194 // convert process.Limitation to pipeline.ProcessLimitation 1195 func convertToPipelineLimitation(lim process.Limitation) *pipeline.ProcessLimitation { 1196 return &pipeline.ProcessLimitation{ 1197 Size: lim.Size, 1198 BatchRows: lim.BatchRows, 1199 BatchSize: lim.BatchSize, 1200 PartitionRows: lim.PartitionRows, 1201 ReaderSize: lim.ReaderSize, 1202 } 1203 } 1204 1205 // convert pipeline.ProcessLimitation to process.Limitation 1206 func convertToProcessLimitation(lim *pipeline.ProcessLimitation) process.Limitation { 1207 return process.Limitation{ 1208 Size: lim.Size, 1209 BatchRows: lim.BatchRows, 1210 BatchSize: lim.BatchSize, 1211 PartitionRows: lim.PartitionRows, 1212 ReaderSize: lim.ReaderSize, 1213 } 1214 } 1215 1216 // convert pipeline.SessionInfo to process.SessionInfo 1217 func convertToProcessSessionInfo(sei *pipeline.SessionInfo) (process.SessionInfo, error) { 1218 sessionInfo := process.SessionInfo{ 1219 User: sei.User, 1220 Host: sei.Host, 1221 Role: sei.Role, 1222 ConnectionID: sei.ConnectionId, 1223 Database: sei.Database, 1224 Version: sei.Version, 1225 Account: sei.Account, 1226 } 1227 t := time.Time{} 1228 err := t.UnmarshalBinary(sei.TimeZone) 1229 if err != nil { 1230 return sessionInfo, nil 1231 } 1232 sessionInfo.TimeZone = t.Location() 1233 return sessionInfo, nil 1234 } 1235 1236 func convertToPlanAnalyzeInfo(info *process.AnalyzeInfo) *plan.AnalyzeInfo { 1237 return &plan.AnalyzeInfo{ 1238 InputRows: info.InputRows, 1239 OutputRows: info.OutputRows, 1240 InputSize: info.InputSize, 1241 OutputSize: info.OutputSize, 1242 TimeConsumed: info.TimeConsumed, 1243 MemorySize: info.MemorySize, 1244 WaitTimeConsumed: info.WaitTimeConsumed, 1245 DiskIO: info.DiskIO, 1246 S3IOByte: info.S3IOByte, 1247 S3IOCount: info.S3IOCount, 1248 NetworkIO: info.NetworkIO, 1249 ScanTime: info.ScanTime, 1250 InsertTime: info.InsertTime, 1251 } 1252 } 1253 1254 // func decodeBatch(proc *process.Process, data []byte) (*batch.Batch, error) { 1255 func decodeBatch(mp *mpool.MPool, data []byte) (*batch.Batch, error) { 1256 bat := new(batch.Batch) 1257 //mp := proc.Mp() 1258 err := types.Decode(data, bat) 1259 // allocated memory of vec from mPool. 1260 for i := range bat.Vecs { 1261 bat.Vecs[i], err = vector.Dup(bat.Vecs[i], mp) 1262 if err != nil { 1263 for j := 0; j < i; j++ { 1264 bat.Vecs[j].Free(mp) 1265 } 1266 return nil, err 1267 } 1268 } 1269 // allocated memory of aggVec from mPool. 1270 for i, ag := range bat.Aggs { 1271 err = ag.WildAggReAlloc(mp) 1272 if err != nil { 1273 for j := 0; j < i; j++ { 1274 bat.Aggs[j].Free(mp) 1275 } 1276 for j := range bat.Vecs { 1277 bat.Vecs[j].Free(mp) 1278 } 1279 return nil, err 1280 } 1281 } 1282 return bat, err 1283 } 1284 1285 func sendToConnectOperator(arg *connector.Argument, bat *batch.Batch) { 1286 select { 1287 case <-arg.Reg.Ctx.Done(): 1288 case arg.Reg.Ch <- bat: 1289 } 1290 } 1291 1292 func (ctx *scopeContext) getRegister(id, idx int32) *process.WaitRegister { 1293 if ctx.id == id { 1294 for k, v := range ctx.regs { 1295 if v == idx { 1296 return k 1297 } 1298 } 1299 } 1300 for i := range ctx.children { 1301 if reg := ctx.children[i].getRegister(id, idx); reg != nil { 1302 return reg 1303 } 1304 } 1305 return nil 1306 } 1307 1308 func (ctx *scopeContext) findRegister(reg *process.WaitRegister) (int32, *scopeContext) { 1309 if idx, ok := ctx.regs[reg]; ok { 1310 return idx, ctx 1311 } 1312 for i := range ctx.children { 1313 if idx, ctx := ctx.children[i].findRegister(reg); idx >= 0 { 1314 return idx, ctx 1315 } 1316 } 1317 return -1, nil 1318 } 1319 1320 func (ctx *scopeContext) addSubPipeline(id uint64, idx int32, ctxId int32) (int32, error) { 1321 ds := &Scope{Magic: Pushdown} 1322 ds.Proc = process.NewWithAnalyze(ctx.scope.Proc, ctx.scope.Proc.Ctx, 0, nil) 1323 ds.DataSource = &Source{ 1324 PushdownId: id, 1325 PushdownAddr: colexec.CnAddr, 1326 } 1327 ds.appendInstruction(vm.Instruction{ 1328 Op: vm.Connector, 1329 Arg: &connector.Argument{ 1330 Reg: ctx.scope.Proc.Reg.MergeReceivers[idx], 1331 }, 1332 }) 1333 ctx.scope.PreScopes = append(ctx.scope.PreScopes, ds) 1334 p := &pipeline.Pipeline{} 1335 p.PipelineId = ctxId 1336 p.PipelineType = Pushdown 1337 ctxId++ 1338 p.DataSource = &pipeline.Source{ 1339 PushdownId: id, 1340 PushdownAddr: colexec.CnAddr, 1341 } 1342 p.InstructionList = append(p.InstructionList, &pipeline.Instruction{ 1343 Op: vm.Connector, 1344 Connect: &pipeline.Connector{ 1345 ConnectorIndex: idx, 1346 PipelineId: ctx.id, 1347 }, 1348 }) 1349 ctx.pipe.Children = append(ctx.pipe.Children, p) 1350 return ctxId, nil 1351 } 1352 1353 func (ctx *scopeContext) isDescendant(dsc *scopeContext) bool { 1354 if ctx.id == dsc.id { 1355 return true 1356 } 1357 for i := range ctx.children { 1358 if ctx.children[i].isDescendant(dsc) { 1359 return true 1360 } 1361 } 1362 return false 1363 } 1364 1365 func (ctx *scopeContext) isRemote(targetContext *scopeContext, depth int) bool { 1366 if targetContext.scope.Magic != Remote { 1367 return false 1368 } 1369 if ctx.id == targetContext.id && depth == 0 { 1370 return true 1371 } 1372 for i := range ctx.children { 1373 if ctx.children[i].scope.Magic == Remote { 1374 if ctx.children[i].isRemote(targetContext, depth+1) { 1375 return true 1376 } 1377 } else { 1378 if ctx.children[i].isRemote(targetContext, depth) { 1379 return true 1380 } 1381 } 1382 } 1383 return false 1384 }