github.com/matrixorigin/matrixone@v0.7.0/pkg/sql/compile/scope.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/matrixorigin/matrixone/pkg/catalog" 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/defines" 29 "github.com/matrixorigin/matrixone/pkg/logutil" 30 pbpipeline "github.com/matrixorigin/matrixone/pkg/pb/pipeline" 31 "github.com/matrixorigin/matrixone/pkg/pb/timestamp" 32 "github.com/matrixorigin/matrixone/pkg/sql/colexec" 33 "github.com/matrixorigin/matrixone/pkg/sql/colexec/connector" 34 "github.com/matrixorigin/matrixone/pkg/sql/colexec/group" 35 "github.com/matrixorigin/matrixone/pkg/sql/colexec/limit" 36 "github.com/matrixorigin/matrixone/pkg/sql/colexec/merge" 37 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergegroup" 38 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergelimit" 39 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergeoffset" 40 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergeorder" 41 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergetop" 42 "github.com/matrixorigin/matrixone/pkg/sql/colexec/offset" 43 "github.com/matrixorigin/matrixone/pkg/sql/colexec/order" 44 "github.com/matrixorigin/matrixone/pkg/sql/colexec/top" 45 "github.com/matrixorigin/matrixone/pkg/sql/util" 46 "github.com/matrixorigin/matrixone/pkg/vm" 47 "github.com/matrixorigin/matrixone/pkg/vm/engine" 48 "github.com/matrixorigin/matrixone/pkg/vm/pipeline" 49 "github.com/matrixorigin/matrixone/pkg/vm/process" 50 ) 51 52 func PrintScope(prefix []byte, ss []*Scope) { 53 for _, s := range ss { 54 PrintScope(append(prefix, '\t'), s.PreScopes) 55 p := pipeline.NewMerge(s.Instructions, nil) 56 logutil.Infof("%s:%v %v", prefix, s.Magic, p) 57 } 58 } 59 60 // Run read data from storage engine and run the instructions of scope. 61 func (s *Scope) Run(c *Compile) (err error) { 62 s.Proc.Ctx = context.WithValue(s.Proc.Ctx, defines.EngineKey{}, c.e) 63 p := pipeline.New(s.DataSource.Attributes, s.Instructions, s.Reg) 64 if s.DataSource.Bat != nil { 65 if _, err = p.ConstRun(s.DataSource.Bat, s.Proc); err != nil { 66 return err 67 } 68 } else { 69 if _, err = p.Run(s.DataSource.R, s.Proc); err != nil { 70 return err 71 } 72 } 73 return nil 74 } 75 76 // MergeRun range and run the scope's pre-scopes by go-routine, and finally run itself to do merge work. 77 func (s *Scope) MergeRun(c *Compile) error { 78 s.Proc.Ctx = context.WithValue(s.Proc.Ctx, defines.EngineKey{}, c.e) 79 errChan := make(chan error, len(s.PreScopes)) 80 81 for i := range s.PreScopes { 82 switch s.PreScopes[i].Magic { 83 case Normal: 84 go func(cs *Scope) { 85 var err error 86 defer func() { 87 errChan <- err 88 }() 89 err = cs.Run(c) 90 }(s.PreScopes[i]) 91 case Merge: 92 go func(cs *Scope) { 93 var err error 94 defer func() { 95 errChan <- err 96 }() 97 err = cs.MergeRun(c) 98 }(s.PreScopes[i]) 99 case Remote: 100 go func(cs *Scope) { 101 var err error 102 defer func() { 103 errChan <- err 104 }() 105 err = cs.RemoteRun(c) 106 }(s.PreScopes[i]) 107 case Parallel: 108 go func(cs *Scope) { 109 var err error 110 defer func() { 111 errChan <- err 112 }() 113 err = cs.ParallelRun(c, cs.IsRemote) 114 }(s.PreScopes[i]) 115 case Pushdown: 116 go func(cs *Scope) { 117 var err error 118 defer func() { 119 errChan <- err 120 }() 121 err = cs.PushdownRun(c) 122 }(s.PreScopes[i]) 123 } 124 } 125 var errReceiveChan chan error 126 if len(s.RemoteReceivRegInfos) > 0 { 127 errReceiveChan = make(chan error, len(s.RemoteReceivRegInfos)) 128 s.notifyAndReceiveFromRemote(errReceiveChan) 129 } 130 p := pipeline.NewMerge(s.Instructions, s.Reg) 131 if _, err := p.MergeRun(s.Proc); err != nil { 132 return err 133 } 134 // check sub-goroutine's error 135 if errReceiveChan == nil { 136 // check sub-goroutine's error 137 for i := 0; i < len(s.PreScopes); i++ { 138 if err := <-errChan; err != nil { 139 return err 140 } 141 } 142 return nil 143 } 144 145 slen := len(s.PreScopes) 146 rlen := len(s.RemoteReceivRegInfos) 147 for { 148 select { 149 case err := <-errChan: 150 if err != nil { 151 return err 152 } 153 slen-- 154 case err := <-errReceiveChan: 155 if err != nil { 156 return err 157 } 158 rlen-- 159 } 160 161 if slen == 0 && rlen == 0 { 162 return nil 163 } 164 } 165 } 166 167 // RemoteRun send the scope to a remote node for execution. 168 // if no target node information, just execute it at local. 169 func (s *Scope) RemoteRun(c *Compile) error { 170 // if send to itself, just run it parallel at local. 171 if len(s.NodeInfo.Addr) == 0 || !cnclient.IsCNClientReady() || 172 len(c.addr) == 0 || isCurrentCN(s.NodeInfo.Addr, c.addr) { 173 return s.ParallelRun(c, s.IsRemote) 174 } 175 176 err := s.remoteRun(c) 177 178 // tell connect operator that it's over 179 arg := s.Instructions[len(s.Instructions)-1].Arg.(*connector.Argument) 180 arg.Free(s.Proc, err != nil) 181 return err 182 } 183 184 // ParallelRun try to execute the scope in parallel way. 185 func (s *Scope) ParallelRun(c *Compile, remote bool) error { 186 var rds []engine.Reader 187 188 s.Proc.Ctx = context.WithValue(s.Proc.Ctx, defines.EngineKey{}, c.e) 189 if s.IsJoin { 190 return s.JoinRun(c) 191 } 192 if s.DataSource == nil { 193 return s.MergeRun(c) 194 } 195 mcpu := s.NodeInfo.Mcpu 196 switch { 197 case remote: 198 var err error 199 ctx := c.ctx 200 if util.TableIsClusterTable(s.DataSource.TableDef.GetTableType()) { 201 ctx = context.WithValue(ctx, defines.TenantIDKey{}, catalog.System_Account) 202 } 203 rds, err = c.e.NewBlockReader(ctx, mcpu, s.DataSource.Timestamp, s.DataSource.Expr, 204 s.NodeInfo.Data, s.DataSource.TableDef) 205 if err != nil { 206 return err 207 } 208 s.NodeInfo.Data = nil 209 case s.NodeInfo.Rel != nil: 210 var err error 211 212 if rds, err = s.NodeInfo.Rel.NewReader(c.ctx, mcpu, s.DataSource.Expr, s.NodeInfo.Data); err != nil { 213 return err 214 } 215 s.NodeInfo.Data = nil 216 default: 217 var err error 218 var db engine.Database 219 var rel engine.Relation 220 221 ctx := c.ctx 222 if util.TableIsClusterTable(s.DataSource.TableDef.GetTableType()) { 223 ctx = context.WithValue(ctx, defines.TenantIDKey{}, catalog.System_Account) 224 } 225 db, err = c.e.Database(ctx, s.DataSource.SchemaName, s.Proc.TxnOperator) 226 if err != nil { 227 return err 228 } 229 rel, err = db.Relation(ctx, s.DataSource.RelationName) 230 if err != nil { 231 var e error // avoid contamination of error messages 232 db, e = c.e.Database(c.ctx, defines.TEMPORARY_DBNAME, s.Proc.TxnOperator) 233 if e != nil { 234 return e 235 } 236 rel, e = db.Relation(c.ctx, engine.GetTempTableName(s.DataSource.SchemaName, s.DataSource.RelationName)) 237 if e != nil { 238 return err 239 } 240 } 241 if rds, err = rel.NewReader(ctx, mcpu, s.DataSource.Expr, s.NodeInfo.Data); err != nil { 242 return err 243 } 244 s.NodeInfo.Data = nil 245 } 246 ss := make([]*Scope, mcpu) 247 for i := 0; i < mcpu; i++ { 248 ss[i] = &Scope{ 249 Magic: Normal, 250 DataSource: &Source{ 251 R: rds[i], 252 SchemaName: s.DataSource.SchemaName, 253 RelationName: s.DataSource.RelationName, 254 Attributes: s.DataSource.Attributes, 255 }, 256 NodeInfo: s.NodeInfo, 257 } 258 ss[i].Proc = process.NewWithAnalyze(s.Proc, c.ctx, 0, c.anal.Nodes()) 259 } 260 newScope := newParallelScope(c, s, ss) 261 return newScope.MergeRun(c) 262 } 263 264 func (s *Scope) PushdownRun(c *Compile) error { 265 var end bool // exist flag 266 var err error 267 268 reg := colexec.Srv.GetConnector(s.DataSource.PushdownId) 269 for { 270 bat := <-reg.Ch 271 if bat == nil { 272 s.Proc.Reg.InputBatch = bat 273 _, err = vm.Run(s.Instructions, s.Proc) 274 s.Proc.Cancel() 275 return err 276 } 277 if bat.Length() == 0 { 278 continue 279 } 280 s.Proc.Reg.InputBatch = bat 281 if end, err = vm.Run(s.Instructions, s.Proc); err != nil || end { 282 return err 283 } 284 } 285 } 286 287 func (s *Scope) JoinRun(c *Compile) error { 288 mcpu := s.NodeInfo.Mcpu 289 if mcpu < 1 { 290 mcpu = 1 291 } 292 chp := s.PreScopes 293 for i := range chp { 294 chp[i].IsEnd = true 295 } 296 ss := make([]*Scope, mcpu) 297 for i := 0; i < mcpu; i++ { 298 ss[i] = &Scope{ 299 Magic: Merge, 300 NodeInfo: s.NodeInfo, 301 } 302 ss[i].Proc = process.NewWithAnalyze(s.Proc, c.ctx, 2, c.anal.Nodes()) 303 ss[i].Proc.Reg.MergeReceivers[1].Ch = make(chan *batch.Batch, 10) 304 } 305 left, right := c.newLeftScope(s, ss), c.newRightScope(s, ss) 306 s = newParallelScope(c, s, ss) 307 s.PreScopes = append(s.PreScopes, chp...) 308 s.PreScopes = append(s.PreScopes, left) 309 s.PreScopes = append(s.PreScopes, right) 310 return s.MergeRun(c) 311 } 312 313 func newParallelScope(c *Compile, s *Scope, ss []*Scope) *Scope { 314 var flg bool 315 316 for i, in := range s.Instructions { 317 if flg { 318 break 319 } 320 switch in.Op { 321 case vm.Top: 322 flg = true 323 arg := in.Arg.(*top.Argument) 324 s.Instructions = append(s.Instructions[:1], s.Instructions[i+1:]...) 325 s.Instructions[0] = vm.Instruction{ 326 Op: vm.MergeTop, 327 Idx: in.Idx, 328 Arg: &mergetop.Argument{ 329 Fs: arg.Fs, 330 Limit: arg.Limit, 331 }, 332 } 333 for i := range ss { 334 ss[i].Instructions = append(ss[i].Instructions, vm.Instruction{ 335 Op: vm.Top, 336 Idx: in.Idx, 337 IsFirst: in.IsFirst, 338 Arg: &top.Argument{ 339 Fs: arg.Fs, 340 Limit: arg.Limit, 341 }, 342 }) 343 } 344 case vm.Order: 345 flg = true 346 arg := in.Arg.(*order.Argument) 347 s.Instructions = append(s.Instructions[:1], s.Instructions[i+1:]...) 348 s.Instructions[0] = vm.Instruction{ 349 Op: vm.MergeOrder, 350 Idx: in.Idx, 351 Arg: &mergeorder.Argument{ 352 Fs: arg.Fs, 353 }, 354 } 355 for i := range ss { 356 ss[i].Instructions = append(ss[i].Instructions, vm.Instruction{ 357 Op: vm.Order, 358 Idx: in.Idx, 359 IsFirst: in.IsFirst, 360 Arg: &order.Argument{ 361 Fs: arg.Fs, 362 }, 363 }) 364 } 365 case vm.Limit: 366 flg = true 367 arg := in.Arg.(*limit.Argument) 368 s.Instructions = append(s.Instructions[:1], s.Instructions[i+1:]...) 369 s.Instructions[0] = vm.Instruction{ 370 Op: vm.MergeLimit, 371 Idx: in.Idx, 372 Arg: &mergelimit.Argument{ 373 Limit: arg.Limit, 374 }, 375 } 376 for i := range ss { 377 ss[i].Instructions = append(ss[i].Instructions, vm.Instruction{ 378 Op: vm.Limit, 379 Idx: in.Idx, 380 IsFirst: in.IsFirst, 381 Arg: &limit.Argument{ 382 Limit: arg.Limit, 383 }, 384 }) 385 } 386 case vm.Group: 387 flg = true 388 arg := in.Arg.(*group.Argument) 389 s.Instructions = append(s.Instructions[:1], s.Instructions[i+1:]...) 390 s.Instructions[0] = vm.Instruction{ 391 Op: vm.MergeGroup, 392 Idx: in.Idx, 393 Arg: &mergegroup.Argument{ 394 NeedEval: false, 395 }, 396 } 397 for i := range ss { 398 ss[i].Instructions = append(ss[i].Instructions, vm.Instruction{ 399 Op: vm.Group, 400 Idx: in.Idx, 401 IsFirst: in.IsFirst, 402 Arg: &group.Argument{ 403 Aggs: arg.Aggs, 404 Exprs: arg.Exprs, 405 Types: arg.Types, 406 MultiAggs: arg.MultiAggs, 407 }, 408 }) 409 } 410 case vm.Offset: 411 flg = true 412 arg := in.Arg.(*offset.Argument) 413 s.Instructions = append(s.Instructions[:1], s.Instructions[i+1:]...) 414 s.Instructions[0] = vm.Instruction{ 415 Op: vm.MergeOffset, 416 Idx: in.Idx, 417 Arg: &mergeoffset.Argument{ 418 Offset: arg.Offset, 419 }, 420 } 421 for i := range ss { 422 ss[i].Instructions = append(ss[i].Instructions, vm.Instruction{ 423 Op: vm.Offset, 424 Idx: in.Idx, 425 IsFirst: in.IsFirst, 426 Arg: &offset.Argument{ 427 Offset: arg.Offset, 428 }, 429 }) 430 } 431 default: 432 for i := range ss { 433 ss[i].Instructions = append(ss[i].Instructions, dupInstruction(&in, nil)) 434 } 435 } 436 } 437 if !flg { 438 for i := range ss { 439 ss[i].Instructions = ss[i].Instructions[:len(ss[i].Instructions)-1] 440 } 441 s.Instructions[0] = vm.Instruction{ 442 Op: vm.Merge, 443 Idx: s.Instructions[0].Idx, // TODO: remove it 444 Arg: &merge.Argument{}, 445 } 446 s.Instructions[1] = s.Instructions[len(s.Instructions)-1] 447 s.Instructions = s.Instructions[:2] 448 } 449 s.Magic = Merge 450 s.PreScopes = ss 451 cnt := 0 452 for _, s := range ss { 453 if s.IsEnd { 454 continue 455 } 456 cnt++ 457 } 458 s.Proc.Reg.MergeReceivers = make([]*process.WaitRegister, cnt) 459 { 460 for i := 0; i < cnt; i++ { 461 s.Proc.Reg.MergeReceivers[i] = &process.WaitRegister{ 462 Ctx: s.Proc.Ctx, 463 Ch: make(chan *batch.Batch, 1), 464 } 465 } 466 } 467 j := 0 468 for i := range ss { 469 if !ss[i].IsEnd { 470 ss[i].appendInstruction(vm.Instruction{ 471 Op: vm.Connector, 472 Arg: &connector.Argument{ 473 Reg: s.Proc.Reg.MergeReceivers[j], 474 }, 475 }) 476 j++ 477 } 478 } 479 return s 480 } 481 482 func (s *Scope) appendInstruction(in vm.Instruction) { 483 if !s.IsEnd { 484 s.Instructions = append(s.Instructions, in) 485 } 486 } 487 488 func dupScopeList(ss []*Scope) []*Scope { 489 rs := make([]*Scope, len(ss)) 490 for i := range rs { 491 rs[i] = dupScope(ss[i]) 492 } 493 return rs 494 } 495 496 func dupScope(s *Scope) *Scope { 497 regMap := make(map[*process.WaitRegister]*process.WaitRegister) 498 499 newScope, err := copyScope(s, regMap) 500 if err != nil { 501 return nil 502 } 503 err = fillInstructionsByCopyScope(newScope, s, regMap) 504 if err != nil { 505 return nil 506 } 507 return newScope 508 } 509 510 func copyScope(srcScope *Scope, regMap map[*process.WaitRegister]*process.WaitRegister) (*Scope, error) { 511 var err error 512 newScope := &Scope{ 513 Magic: srcScope.Magic, 514 IsJoin: srcScope.IsJoin, 515 IsEnd: srcScope.IsEnd, 516 IsRemote: srcScope.IsRemote, 517 Plan: srcScope.Plan, 518 PreScopes: make([]*Scope, len(srcScope.PreScopes)), 519 Instructions: make([]vm.Instruction, len(srcScope.Instructions)), 520 NodeInfo: engine.Node{ 521 Rel: srcScope.NodeInfo.Rel, 522 Mcpu: srcScope.NodeInfo.Mcpu, 523 Id: srcScope.NodeInfo.Id, 524 Addr: srcScope.NodeInfo.Addr, 525 Data: make([][]byte, len(srcScope.NodeInfo.Data)), 526 }, 527 RemoteReceivRegInfos: srcScope.RemoteReceivRegInfos, 528 } 529 530 // copy node.Data 531 copy(newScope.NodeInfo.Data, srcScope.NodeInfo.Data) 532 533 if srcScope.DataSource != nil { 534 newScope.DataSource = &Source{ 535 PushdownId: srcScope.DataSource.PushdownId, 536 PushdownAddr: srcScope.DataSource.PushdownAddr, 537 SchemaName: srcScope.DataSource.SchemaName, 538 RelationName: srcScope.DataSource.RelationName, 539 Attributes: srcScope.DataSource.Attributes, 540 Timestamp: timestamp.Timestamp{ 541 PhysicalTime: srcScope.DataSource.Timestamp.PhysicalTime, 542 LogicalTime: srcScope.DataSource.Timestamp.LogicalTime, 543 NodeID: srcScope.DataSource.Timestamp.NodeID, 544 }, 545 // read only. 546 Expr: srcScope.DataSource.Expr, 547 TableDef: srcScope.DataSource.TableDef, 548 } 549 550 // IF const run. 551 if srcScope.DataSource.Bat != nil { 552 newScope.DataSource.Bat, _ = constructValueScanBatch(context.TODO(), nil, nil) 553 } 554 } 555 556 newScope.Proc = process.NewFromProc(srcScope.Proc, srcScope.Proc.Ctx, len(srcScope.PreScopes)) 557 for i := range srcScope.Proc.Reg.MergeReceivers { 558 regMap[srcScope.Proc.Reg.MergeReceivers[i]] = newScope.Proc.Reg.MergeReceivers[i] 559 } 560 561 //copy preScopes. 562 for i := range srcScope.PreScopes { 563 newScope.PreScopes[i], err = copyScope(srcScope.PreScopes[i], regMap) 564 if err != nil { 565 return nil, err 566 } 567 } 568 return newScope, nil 569 } 570 571 func fillInstructionsByCopyScope(targetScope *Scope, srcScope *Scope, 572 regMap map[*process.WaitRegister]*process.WaitRegister) error { 573 var err error 574 575 for i := range srcScope.PreScopes { 576 if err = fillInstructionsByCopyScope(targetScope.PreScopes[i], srcScope.PreScopes[i], regMap); err != nil { 577 return err 578 } 579 } 580 581 for i := range srcScope.Instructions { 582 targetScope.Instructions[i] = dupInstruction(&srcScope.Instructions[i], regMap) 583 } 584 return nil 585 } 586 587 func (s *Scope) notifyAndReceiveFromRemote(errChan chan error) error { 588 for _, rr := range s.RemoteReceivRegInfos { 589 go func(info RemoteReceivRegInfo, reg *process.WaitRegister, mp *mpool.MPool) { 590 streamSender, errStream := cnclient.GetStreamSender(info.FromAddr) 591 if errStream != nil { 592 reg.Ch <- nil 593 errChan <- errStream 594 return 595 } 596 defer func(streamSender morpc.Stream) { 597 close(reg.Ch) 598 _ = streamSender.Close() 599 }(streamSender) 600 601 c, cancel := context.WithTimeout(context.Background(), time.Second*10000) 602 _ = cancel 603 message := cnclient.AcquireMessage() 604 { 605 message.Id = streamSender.ID() 606 message.Cmd = pbpipeline.PrepareDoneNotifyMessage 607 message.Uuid = info.Uuid[:] 608 } 609 if errSend := streamSender.Send(c, message); errSend != nil { 610 reg.Ch <- nil 611 errChan <- errSend 612 return 613 } 614 615 messagesReceive, errReceive := streamSender.Receive() 616 if errReceive != nil { 617 reg.Ch <- nil 618 errChan <- errReceive 619 return 620 } 621 622 if err := receiveMsgAndForward(c, messagesReceive, reg.Ch, mp); err != nil { 623 reg.Ch <- nil 624 errChan <- err 625 return 626 } 627 reg.Ch <- nil 628 errChan <- nil 629 }(rr, s.Proc.Reg.MergeReceivers[rr.Idx], s.Proc.GetMPool()) 630 } 631 632 return nil 633 } 634 635 func receiveMsgAndForward(ctx context.Context, receiveCh chan morpc.Message, forwardCh chan *batch.Batch, mp *mpool.MPool) error { 636 var val morpc.Message 637 var dataBuffer []byte 638 var ok bool 639 for { 640 select { 641 case <-ctx.Done(): 642 return moerr.NewRPCTimeout(ctx) 643 case val, ok = <-receiveCh: 644 } 645 646 if val == nil || !ok { 647 return moerr.NewStreamClosedNoCtx() 648 } 649 650 m, ok := val.(*pbpipeline.Message) 651 if !ok { 652 panic("unexpected message type for cn-server") 653 } 654 655 // receive an end message from remote 656 if err := pbpipeline.GetMessageErrorInfo(m); err != nil { 657 return err 658 } 659 660 // end message 661 if m.IsEndMessage() { 662 return nil 663 } 664 665 // normal receive 666 dataBuffer = append(dataBuffer, m.Data...) 667 switch m.GetSid() { 668 case pbpipeline.BatchWaitingNext: 669 continue 670 case pbpipeline.BatchEnd: 671 if m.Checksum != crc32.ChecksumIEEE(dataBuffer) { 672 return moerr.NewInternalErrorNoCtx("Packages delivered by morpc is broken") 673 } 674 bat, err := decodeBatch(mp, dataBuffer) 675 if err != nil { 676 return err 677 } 678 forwardCh <- bat 679 dataBuffer = nil 680 } 681 } 682 }