github.com/matrixorigin/matrixone@v1.2.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 "fmt" 20 "hash/crc32" 21 goruntime "runtime" 22 "runtime/debug" 23 "sync" 24 25 "github.com/matrixorigin/matrixone/pkg/catalog" 26 "github.com/matrixorigin/matrixone/pkg/cnservice/cnclient" 27 "github.com/matrixorigin/matrixone/pkg/common/bitmap" 28 "github.com/matrixorigin/matrixone/pkg/common/moerr" 29 "github.com/matrixorigin/matrixone/pkg/common/morpc" 30 "github.com/matrixorigin/matrixone/pkg/common/reuse" 31 "github.com/matrixorigin/matrixone/pkg/common/runtime" 32 "github.com/matrixorigin/matrixone/pkg/container/batch" 33 "github.com/matrixorigin/matrixone/pkg/defines" 34 "github.com/matrixorigin/matrixone/pkg/logutil" 35 "github.com/matrixorigin/matrixone/pkg/objectio" 36 pbpipeline "github.com/matrixorigin/matrixone/pkg/pb/pipeline" 37 "github.com/matrixorigin/matrixone/pkg/pb/plan" 38 "github.com/matrixorigin/matrixone/pkg/pb/timestamp" 39 "github.com/matrixorigin/matrixone/pkg/sql/colexec" 40 "github.com/matrixorigin/matrixone/pkg/sql/colexec/connector" 41 "github.com/matrixorigin/matrixone/pkg/sql/colexec/group" 42 "github.com/matrixorigin/matrixone/pkg/sql/colexec/limit" 43 "github.com/matrixorigin/matrixone/pkg/sql/colexec/merge" 44 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergegroup" 45 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergelimit" 46 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergeoffset" 47 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergetop" 48 "github.com/matrixorigin/matrixone/pkg/sql/colexec/offset" 49 "github.com/matrixorigin/matrixone/pkg/sql/colexec/restrict" 50 "github.com/matrixorigin/matrixone/pkg/sql/colexec/right" 51 "github.com/matrixorigin/matrixone/pkg/sql/colexec/rightanti" 52 "github.com/matrixorigin/matrixone/pkg/sql/colexec/rightsemi" 53 "github.com/matrixorigin/matrixone/pkg/sql/colexec/sample" 54 "github.com/matrixorigin/matrixone/pkg/sql/colexec/top" 55 plan2 "github.com/matrixorigin/matrixone/pkg/sql/plan" 56 "github.com/matrixorigin/matrixone/pkg/sql/util" 57 "github.com/matrixorigin/matrixone/pkg/vm" 58 "github.com/matrixorigin/matrixone/pkg/vm/engine" 59 "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae" 60 "github.com/matrixorigin/matrixone/pkg/vm/engine/memoryengine" 61 "github.com/matrixorigin/matrixone/pkg/vm/pipeline" 62 "github.com/matrixorigin/matrixone/pkg/vm/process" 63 "github.com/panjf2000/ants/v2" 64 _ "go.uber.org/automaxprocs" 65 "go.uber.org/zap" 66 ) 67 68 func newScope(magic magicType) *Scope { 69 s := reuse.Alloc[Scope](nil) 70 s.Magic = magic 71 return s 72 } 73 74 func ReleaseScopes(ss []*Scope) { 75 for i := range ss { 76 ss[i].release() 77 } 78 } 79 80 func (s *Scope) withPlan(pn *plan.Plan) *Scope { 81 s.Plan = pn 82 return s 83 } 84 85 func (s *Scope) release() { 86 if s == nil { 87 return 88 } 89 for i := range s.PreScopes { 90 s.PreScopes[i].release() 91 } 92 for i := range s.Instructions { 93 s.Instructions[i].Arg.Release() 94 s.Instructions[i].Arg = nil 95 } 96 reuse.Free[Scope](s, nil) 97 } 98 99 func (s *Scope) initDataSource(c *Compile) (err error) { 100 if s.DataSource == nil { 101 return nil 102 } 103 if s.DataSource.isConst { 104 if s.DataSource.Bat != nil { 105 return 106 } 107 bat, err := constructValueScanBatch(s.Proc.Ctx, c.proc, s.DataSource.node) 108 if err != nil { 109 return err 110 } 111 s.DataSource.Bat = bat 112 } else { 113 if s.DataSource.TableDef != nil { 114 return nil 115 } 116 return c.compileTableScanDataSource(s) 117 } 118 return nil 119 } 120 121 // Run read data from storage engine and run the instructions of scope. 122 func (s *Scope) Run(c *Compile) (err error) { 123 var p *pipeline.Pipeline 124 defer func() { 125 if e := recover(); e != nil { 126 err = moerr.ConvertPanicError(s.Proc.Ctx, e) 127 getLogger().Error("panic in scope run", 128 zap.String("sql", c.sql), 129 zap.String("error", err.Error())) 130 } 131 if p != nil { 132 p.Cleanup(s.Proc, err != nil, err) 133 } 134 }() 135 136 s.Proc.Ctx = context.WithValue(s.Proc.Ctx, defines.EngineKey{}, c.e) 137 // DataSource == nil specify the empty scan 138 if s.DataSource == nil { 139 p = pipeline.New(0, nil, s.Instructions, s.Reg) 140 if _, err = p.ConstRun(nil, s.Proc); err != nil { 141 return err 142 } 143 } else { 144 id := uint64(0) 145 if s.DataSource.TableDef != nil { 146 id = s.DataSource.TableDef.TblId 147 } 148 p = pipeline.New(id, s.DataSource.Attributes, s.Instructions, s.Reg) 149 if s.DataSource.isConst { 150 _, err = p.ConstRun(s.DataSource.Bat, s.Proc) 151 } else { 152 var tag int32 153 if s.DataSource.node != nil && len(s.DataSource.node.RecvMsgList) > 0 { 154 tag = s.DataSource.node.RecvMsgList[0].MsgTag 155 } 156 _, err = p.Run(s.DataSource.R, tag, s.Proc) 157 } 158 } 159 160 select { 161 case <-s.Proc.Ctx.Done(): 162 err = nil 163 default: 164 } 165 return err 166 } 167 168 func (s *Scope) SetContextRecursively(ctx context.Context) { 169 if s.Proc == nil { 170 return 171 } 172 newCtx := s.Proc.ResetContextFromParent(ctx) 173 for _, scope := range s.PreScopes { 174 scope.SetContextRecursively(newCtx) 175 } 176 } 177 178 func (s *Scope) InitAllDataSource(c *Compile) error { 179 err := s.initDataSource(c) 180 if err != nil { 181 return err 182 } 183 for _, scope := range s.PreScopes { 184 err := scope.InitAllDataSource(c) 185 if err != nil { 186 return err 187 } 188 } 189 return nil 190 } 191 192 func (s *Scope) SetOperatorInfoRecursively(cb func() int32) { 193 for i := 0; i < len(s.Instructions); i++ { 194 s.Instructions[i].CnAddr = s.NodeInfo.Addr 195 s.Instructions[i].OperatorID = cb() 196 s.Instructions[i].ParallelID = 0 197 s.Instructions[i].MaxParallel = 1 198 } 199 200 for _, scope := range s.PreScopes { 201 scope.SetOperatorInfoRecursively(cb) 202 } 203 } 204 205 // MergeRun range and run the scope's pre-scopes by go-routine, and finally run itself to do merge work. 206 func (s *Scope) MergeRun(c *Compile) error { 207 var wg sync.WaitGroup 208 209 errChan := make(chan error, len(s.PreScopes)) 210 for i := range s.PreScopes { 211 wg.Add(1) 212 scope := s.PreScopes[i] 213 errSubmit := ants.Submit(func() { 214 defer func() { 215 if e := recover(); e != nil { 216 err := moerr.ConvertPanicError(c.ctx, e) 217 getLogger().Error("panic in merge run run", 218 zap.String("sql", c.sql), 219 zap.String("error", err.Error())) 220 errChan <- err 221 } 222 wg.Done() 223 }() 224 switch scope.Magic { 225 case Normal: 226 errChan <- scope.Run(c) 227 case Merge, MergeInsert: 228 errChan <- scope.MergeRun(c) 229 case Remote: 230 errChan <- scope.RemoteRun(c) 231 case Parallel: 232 errChan <- scope.ParallelRun(c, scope.IsRemote) 233 } 234 }) 235 if errSubmit != nil { 236 errChan <- errSubmit 237 wg.Done() 238 } 239 } 240 241 s.Proc.Ctx = context.WithValue(s.Proc.Ctx, defines.EngineKey{}, c.e) 242 var errReceiveChan chan error 243 if len(s.RemoteReceivRegInfos) > 0 { 244 errReceiveChan = make(chan error, len(s.RemoteReceivRegInfos)) 245 s.notifyAndReceiveFromRemote(&wg, errReceiveChan) 246 } 247 defer wg.Wait() 248 249 p := pipeline.NewMerge(s.Instructions, s.Reg) 250 if _, err := p.MergeRun(s.Proc); err != nil { 251 select { 252 case <-s.Proc.Ctx.Done(): 253 default: 254 p.Cleanup(s.Proc, true, err) 255 return err 256 } 257 } 258 p.Cleanup(s.Proc, false, nil) 259 260 // receive and check error from pre-scopes and remote scopes. 261 preScopeCount := len(s.PreScopes) 262 remoteScopeCount := len(s.RemoteReceivRegInfos) 263 if remoteScopeCount == 0 { 264 for i := 0; i < len(s.PreScopes); i++ { 265 if err := <-errChan; err != nil { 266 return err 267 } 268 } 269 return nil 270 } 271 272 for { 273 select { 274 case err := <-errChan: 275 if err != nil { 276 return err 277 } 278 preScopeCount-- 279 280 case err := <-errReceiveChan: 281 if err != nil { 282 return err 283 } 284 remoteScopeCount-- 285 } 286 287 if preScopeCount == 0 && remoteScopeCount == 0 { 288 return nil 289 } 290 } 291 } 292 293 // RemoteRun send the scope to a remote node for execution. 294 func (s *Scope) RemoteRun(c *Compile) error { 295 if !s.canRemote(c, true) || !cnclient.IsCNClientReady() { 296 return s.ParallelRun(c, s.IsRemote) 297 } 298 299 runtime.ProcessLevelRuntime().Logger(). 300 Debug("remote run pipeline", 301 zap.String("local-address", c.addr), 302 zap.String("remote-address", s.NodeInfo.Addr)) 303 304 p := pipeline.New(0, nil, s.Instructions, s.Reg) 305 err := s.remoteRun(c) 306 select { 307 case <-s.Proc.Ctx.Done(): 308 // this clean-up action shouldn't be called before context check. 309 // because the clean-up action will cancel the context, and error will be suppressed. 310 p.Cleanup(s.Proc, err != nil, err) 311 return nil 312 313 default: 314 p.Cleanup(s.Proc, err != nil, err) 315 return err 316 } 317 } 318 319 func DeterminRuntimeDOP(cpunum, blocks int) int { 320 if cpunum <= 0 || blocks <= 16 { 321 return 1 322 } 323 ret := blocks/16 + 1 324 if ret < cpunum { 325 return ret 326 } 327 return cpunum 328 } 329 330 func (s *Scope) handleRuntimeFilter(c *Compile) error { 331 var err error 332 var inExprList []*plan.Expr 333 exprs := make([]*plan.Expr, 0, len(s.DataSource.RuntimeFilterSpecs)) 334 filters := make([]process.RuntimeFilterMessage, 0, len(exprs)) 335 336 if len(s.DataSource.RuntimeFilterSpecs) > 0 { 337 for _, spec := range s.DataSource.RuntimeFilterSpecs { 338 msgReceiver := c.proc.NewMessageReceiver([]int32{spec.Tag}, process.AddrBroadCastOnCurrentCN()) 339 msgs, ctxDone := msgReceiver.ReceiveMessage(true, s.Proc.Ctx) 340 if ctxDone { 341 return nil 342 } 343 for i := range msgs { 344 msg, ok := msgs[i].(process.RuntimeFilterMessage) 345 if !ok { 346 panic("expect runtime filter message, receive unknown message!") 347 } 348 switch msg.Typ { 349 case process.RuntimeFilter_PASS: 350 continue 351 case process.RuntimeFilter_DROP: 352 // FIXME: Should give an empty "Data" and then early return 353 s.NodeInfo.Data = nil 354 s.NodeInfo.NeedExpandRanges = false 355 s.DataSource.FilterExpr = plan2.MakeFalseExpr() 356 return nil 357 case process.RuntimeFilter_IN: 358 inExpr := plan2.MakeInExpr(c.ctx, spec.Expr, msg.Card, msg.Data, spec.MatchPrefix) 359 inExprList = append(inExprList, inExpr) 360 361 // TODO: implement BETWEEN expression 362 } 363 exprs = append(exprs, spec.Expr) 364 filters = append(filters, msg) 365 } 366 msgReceiver.Free() 367 } 368 } 369 370 for i := range inExprList { 371 fn := inExprList[i].GetF() 372 col := fn.Args[0].GetCol() 373 if col == nil { 374 panic("only support col in runtime filter's left child!") 375 } 376 377 newExpr := plan2.DeepCopyExpr(inExprList[i]) 378 //put expr in reader 379 newExprList := []*plan.Expr{newExpr} 380 if s.DataSource.FilterExpr != nil { 381 newExprList = append(newExprList, s.DataSource.FilterExpr) 382 } 383 s.DataSource.FilterExpr = colexec.RewriteFilterExprList(newExprList) 384 385 isFilterOnPK := s.DataSource.TableDef.Pkey != nil && col.Name == s.DataSource.TableDef.Pkey.PkeyColName 386 if !isFilterOnPK { 387 // put expr in filter instruction 388 ins := s.Instructions[0] 389 arg, ok := ins.Arg.(*restrict.Argument) 390 if !ok { 391 panic("missing instruction for runtime filter!") 392 } 393 newExprList := []*plan.Expr{newExpr} 394 if arg.E != nil { 395 newExprList = append(newExprList, arg.E) 396 } 397 arg.E = colexec.RewriteFilterExprList(newExprList) 398 } 399 } 400 401 if s.NodeInfo.NeedExpandRanges { 402 if s.DataSource.node == nil { 403 panic("can not expand ranges on remote pipeline!") 404 } 405 newExprList := plan2.DeepCopyExprList(inExprList) 406 if len(s.DataSource.node.BlockFilterList) > 0 { 407 newExprList = append(newExprList, s.DataSource.node.BlockFilterList...) 408 } 409 ranges, err := c.expandRanges(s.DataSource.node, s.NodeInfo.Rel, newExprList) 410 if err != nil { 411 return err 412 } 413 s.NodeInfo.Data = append(s.NodeInfo.Data, ranges.GetAllBytes()...) 414 s.NodeInfo.NeedExpandRanges = false 415 } else if len(inExprList) > 0 { 416 s.NodeInfo.Data, err = ApplyRuntimeFilters(c.ctx, s.Proc, s.DataSource.TableDef, s.NodeInfo.Data, exprs, filters) 417 if err != nil { 418 return err 419 } 420 } 421 return nil 422 } 423 424 // ParallelRun try to execute the scope in parallel way. 425 func (s *Scope) ParallelRun(c *Compile, remote bool) error { 426 var rds []engine.Reader 427 var err error 428 429 s.Proc.Ctx = context.WithValue(s.Proc.Ctx, defines.EngineKey{}, c.e) 430 if s.IsJoin { 431 return s.JoinRun(c) 432 } 433 if s.IsLoad { 434 return s.LoadRun(c) 435 } 436 if s.DataSource == nil { 437 return s.MergeRun(c) 438 } 439 440 err = s.handleRuntimeFilter(c) 441 if err != nil { 442 return err 443 } 444 445 numCpu := goruntime.GOMAXPROCS(0) 446 var mcpu int 447 448 switch { 449 case remote: 450 if len(s.DataSource.OrderBy) > 0 { 451 panic("ordered scan can't run on remote CN!") 452 } 453 ctx := c.ctx 454 if util.TableIsClusterTable(s.DataSource.TableDef.GetTableType()) { 455 ctx = defines.AttachAccountId(ctx, catalog.System_Account) 456 457 } 458 if s.DataSource.AccountId != nil { 459 ctx = defines.AttachAccountId(ctx, uint32(s.DataSource.AccountId.GetTenantId())) 460 } 461 blkSlice := objectio.BlockInfoSlice(s.NodeInfo.Data) 462 mcpu = DeterminRuntimeDOP(numCpu, blkSlice.Len()) 463 rds, err = c.e.NewBlockReader(ctx, mcpu, s.DataSource.Timestamp, s.DataSource.FilterExpr, 464 s.NodeInfo.Data, s.DataSource.TableDef, c.proc) 465 if err != nil { 466 return err 467 } 468 s.NodeInfo.Data = nil 469 470 case s.NodeInfo.Rel != nil: 471 switch s.NodeInfo.Rel.GetEngineType() { 472 case engine.Disttae: 473 blkSlice := objectio.BlockInfoSlice(s.NodeInfo.Data) 474 mcpu = DeterminRuntimeDOP(numCpu, blkSlice.Len()) 475 case engine.Memory: 476 idSlice := memoryengine.ShardIdSlice(s.NodeInfo.Data) 477 mcpu = DeterminRuntimeDOP(numCpu, idSlice.Len()) 478 default: 479 mcpu = 1 480 } 481 if len(s.DataSource.OrderBy) > 0 { 482 // ordered scan must run on only one parallel! 483 mcpu = 1 484 } 485 if rds, err = s.NodeInfo.Rel.NewReader(c.ctx, mcpu, s.DataSource.FilterExpr, s.NodeInfo.Data, len(s.DataSource.OrderBy) > 0); err != nil { 486 return err 487 } 488 s.NodeInfo.Data = nil 489 490 // FIXME:: s.NodeInfo.Rel == nil, partition table? 491 default: 492 var db engine.Database 493 var rel engine.Relation 494 495 ctx := c.ctx 496 if util.TableIsClusterTable(s.DataSource.TableDef.GetTableType()) { 497 ctx = defines.AttachAccountId(ctx, catalog.System_Account) 498 } 499 txnOp := s.Proc.TxnOperator 500 if !s.DataSource.Timestamp.Equal(timestamp.Timestamp{LogicalTime: 0, PhysicalTime: 0}) && 501 s.DataSource.Timestamp.Less(s.Proc.TxnOperator.Txn().SnapshotTS) { 502 txnOp = s.Proc.TxnOperator.CloneSnapshotOp(s.DataSource.Timestamp) 503 } 504 db, err = c.e.Database(ctx, s.DataSource.SchemaName, txnOp) 505 if err != nil { 506 return err 507 } 508 rel, err = db.Relation(ctx, s.DataSource.RelationName, c.proc) 509 if err != nil { 510 var e error // avoid contamination of error messages 511 db, e = c.e.Database(c.ctx, defines.TEMPORARY_DBNAME, s.Proc.TxnOperator) 512 if e != nil { 513 return e 514 } 515 rel, e = db.Relation(c.ctx, engine.GetTempTableName(s.DataSource.SchemaName, s.DataSource.RelationName), c.proc) 516 if e != nil { 517 return err 518 } 519 } 520 switch rel.GetEngineType() { 521 case engine.Disttae: 522 blkSlice := objectio.BlockInfoSlice(s.NodeInfo.Data) 523 mcpu = DeterminRuntimeDOP(numCpu, blkSlice.Len()) 524 case engine.Memory: 525 idSlice := memoryengine.ShardIdSlice(s.NodeInfo.Data) 526 mcpu = DeterminRuntimeDOP(numCpu, idSlice.Len()) 527 default: 528 mcpu = 1 529 } 530 if len(s.DataSource.OrderBy) > 0 { 531 // ordered scan must run on only one parallel! 532 mcpu = 1 533 } 534 if rel.GetEngineType() == engine.Memory || 535 s.DataSource.PartitionRelationNames == nil { 536 mainRds, err := rel.NewReader( 537 ctx, 538 mcpu, 539 s.DataSource.FilterExpr, 540 s.NodeInfo.Data, 541 len(s.DataSource.OrderBy) > 0) 542 if err != nil { 543 return err 544 } 545 rds = append(rds, mainRds...) 546 } else { 547 // handle partition table. 548 blkArray := objectio.BlockInfoSlice(s.NodeInfo.Data) 549 dirtyRanges := make(map[int]objectio.BlockInfoSlice, 0) 550 cleanRanges := make(objectio.BlockInfoSlice, 0, blkArray.Len()) 551 ranges := objectio.BlockInfoSlice(blkArray.Slice(1, blkArray.Len())) 552 for i := 0; i < ranges.Len(); i++ { 553 blkInfo := ranges.Get(i) 554 if !blkInfo.CanRemote { 555 if _, ok := dirtyRanges[blkInfo.PartitionNum]; !ok { 556 newRanges := make(objectio.BlockInfoSlice, 0, objectio.BlockInfoSize) 557 newRanges = append(newRanges, objectio.EmptyBlockInfoBytes...) 558 dirtyRanges[blkInfo.PartitionNum] = newRanges 559 } 560 dirtyRanges[blkInfo.PartitionNum] = append(dirtyRanges[blkInfo.PartitionNum], ranges.GetBytes(i)...) 561 continue 562 } 563 cleanRanges = append(cleanRanges, ranges.GetBytes(i)...) 564 } 565 566 if len(cleanRanges) > 0 { 567 // create readers for reading clean blocks from main table. 568 mainRds, err := rel.NewReader( 569 ctx, 570 mcpu, 571 s.DataSource.FilterExpr, 572 cleanRanges, 573 len(s.DataSource.OrderBy) > 0) 574 if err != nil { 575 return err 576 } 577 rds = append(rds, mainRds...) 578 579 } 580 // create readers for reading dirty blocks from partition table. 581 for num, relName := range s.DataSource.PartitionRelationNames { 582 subrel, err := db.Relation(c.ctx, relName, c.proc) 583 if err != nil { 584 return err 585 } 586 memRds, err := subrel.NewReader(c.ctx, mcpu, s.DataSource.FilterExpr, dirtyRanges[num], len(s.DataSource.OrderBy) > 0) 587 if err != nil { 588 return err 589 } 590 rds = append(rds, memRds...) 591 } 592 } 593 s.NodeInfo.Data = nil 594 } 595 596 if len(rds) != mcpu { 597 newRds := make([]engine.Reader, 0, mcpu) 598 step := len(rds) / mcpu 599 for i := 0; i < len(rds); i += step { 600 m := disttae.NewMergeReader(rds[i : i+step]) 601 newRds = append(newRds, m) 602 } 603 rds = newRds 604 } 605 606 if mcpu == 1 { 607 s.Magic = Normal 608 s.DataSource.R = rds[0] // rds's length is equal to mcpu so it is safe to do it 609 s.DataSource.R.SetOrderBy(s.DataSource.OrderBy) 610 return s.Run(c) 611 } 612 613 if len(s.DataSource.OrderBy) > 0 { 614 panic("ordered scan must run on only one parallel!") 615 } 616 ss := make([]*Scope, mcpu) 617 for i := 0; i < mcpu; i++ { 618 ss[i] = newScope(Normal) 619 ss[i].NodeInfo = s.NodeInfo 620 ss[i].DataSource = &Source{ 621 R: rds[i], 622 SchemaName: s.DataSource.SchemaName, 623 RelationName: s.DataSource.RelationName, 624 Attributes: s.DataSource.Attributes, 625 AccountId: s.DataSource.AccountId, 626 } 627 ss[i].Proc = process.NewWithAnalyze(s.Proc, c.ctx, 0, c.anal.Nodes()) 628 } 629 newScope, err := newParallelScope(c, s, ss) 630 if err != nil { 631 ReleaseScopes(ss) 632 return err 633 } 634 newScope.SetContextRecursively(s.Proc.Ctx) 635 return newScope.MergeRun(c) 636 } 637 638 func (s *Scope) JoinRun(c *Compile) error { 639 mcpu := s.NodeInfo.Mcpu 640 if mcpu <= 1 { // no need to parallel 641 buildScope := c.newJoinBuildScope(s, nil) 642 s.PreScopes = append(s.PreScopes, buildScope) 643 if s.BuildIdx > 1 { 644 probeScope := c.newJoinProbeScope(s, nil) 645 s.PreScopes = append(s.PreScopes, probeScope) 646 } 647 return s.MergeRun(c) 648 } 649 650 isRight := s.isRight() 651 652 chp := s.PreScopes 653 for i := range chp { 654 chp[i].IsEnd = true 655 } 656 657 ss := make([]*Scope, mcpu) 658 for i := 0; i < mcpu; i++ { 659 ss[i] = newScope(Merge) 660 ss[i].NodeInfo = s.NodeInfo 661 ss[i].Proc = process.NewWithAnalyze(s.Proc, s.Proc.Ctx, 2, c.anal.Nodes()) 662 ss[i].Proc.Reg.MergeReceivers[1].Ch = make(chan *batch.Batch, 10) 663 } 664 probe_scope, build_scope := c.newJoinProbeScope(s, ss), c.newJoinBuildScope(s, ss) 665 var err error 666 s, err = newParallelScope(c, s, ss) 667 if err != nil { 668 ReleaseScopes(ss) 669 return err 670 } 671 672 if isRight { 673 channel := make(chan *bitmap.Bitmap, mcpu) 674 for i := range s.PreScopes { 675 switch arg := s.PreScopes[i].Instructions[0].Arg.(type) { 676 case *right.Argument: 677 arg.Channel = channel 678 arg.NumCPU = uint64(mcpu) 679 if i == 0 { 680 arg.IsMerger = true 681 } 682 683 case *rightsemi.Argument: 684 arg.Channel = channel 685 arg.NumCPU = uint64(mcpu) 686 if i == 0 { 687 arg.IsMerger = true 688 } 689 690 case *rightanti.Argument: 691 arg.Channel = channel 692 arg.NumCPU = uint64(mcpu) 693 if i == 0 { 694 arg.IsMerger = true 695 } 696 } 697 } 698 } 699 s.PreScopes = append(s.PreScopes, chp...) 700 s.PreScopes = append(s.PreScopes, build_scope) 701 s.PreScopes = append(s.PreScopes, probe_scope) 702 703 return s.MergeRun(c) 704 } 705 706 func (s *Scope) isShuffle() bool { 707 // the pipeline is merge->group->xxx 708 if s != nil && len(s.Instructions) > 1 && (s.Instructions[1].Op == vm.Group) { 709 arg := s.Instructions[1].Arg.(*group.Argument) 710 return arg.IsShuffle 711 } 712 return false 713 } 714 715 func (s *Scope) isRight() bool { 716 return s != nil && (s.Instructions[0].Op == vm.Right || s.Instructions[0].Op == vm.RightSemi || s.Instructions[0].Op == vm.RightAnti) 717 } 718 719 func (s *Scope) LoadRun(c *Compile) error { 720 mcpu := s.NodeInfo.Mcpu 721 ss := make([]*Scope, mcpu) 722 for i := 0; i < mcpu; i++ { 723 ss[i] = newScope(Normal) 724 ss[i].NodeInfo = s.NodeInfo 725 ss[i].DataSource = &Source{ 726 isConst: true, 727 } 728 ss[i].Proc = process.NewWithAnalyze(s.Proc, c.ctx, 0, c.anal.Nodes()) 729 err := ss[i].initDataSource(c) 730 if err != nil { 731 return err 732 } 733 } 734 newScope, err := newParallelScope(c, s, ss) 735 if err != nil { 736 ReleaseScopes(ss) 737 return err 738 } 739 740 return newScope.MergeRun(c) 741 } 742 743 func newParallelScope(c *Compile, s *Scope, ss []*Scope) (*Scope, error) { 744 var flg bool 745 746 idx := 0 747 defer func(ins vm.Instructions) { 748 for i := 0; i < idx; i++ { 749 if ins[i].Arg != nil { 750 ins[i].Arg.Release() 751 } 752 } 753 }(s.Instructions) 754 755 for i, in := range s.Instructions { 756 if flg { 757 break 758 } 759 switch in.Op { 760 case vm.Top: 761 flg = true 762 idx = i 763 arg := in.Arg.(*top.Argument) 764 // release the useless arg 765 s.Instructions = s.Instructions[i:] 766 s.Instructions[0] = vm.Instruction{ 767 Op: vm.MergeTop, 768 Idx: in.Idx, 769 Arg: mergetop.NewArgument(). 770 WithFs(arg.Fs). 771 WithLimit(arg.Limit), 772 773 CnAddr: in.CnAddr, 774 OperatorID: c.allocOperatorID(), 775 ParallelID: 0, 776 MaxParallel: 1, 777 } 778 for j := range ss { 779 newarg := top.NewArgument().WithFs(arg.Fs).WithLimit(arg.Limit) 780 newarg.TopValueTag = arg.TopValueTag 781 ss[j].appendInstruction(vm.Instruction{ 782 Op: vm.Top, 783 Idx: in.Idx, 784 IsFirst: in.IsFirst, 785 Arg: newarg, 786 CnAddr: in.CnAddr, 787 OperatorID: in.OperatorID, 788 MaxParallel: int32(len(ss)), 789 ParallelID: int32(j), 790 }) 791 } 792 arg.Release() 793 // case vm.Order: 794 // there is no need to do special merge for order, because the behavior of order is just sort for each batch. 795 case vm.Limit: 796 flg = true 797 idx = i 798 arg := in.Arg.(*limit.Argument) 799 s.Instructions = s.Instructions[i:] 800 s.Instructions[0] = vm.Instruction{ 801 Op: vm.MergeLimit, 802 Idx: in.Idx, 803 Arg: mergelimit.NewArgument(). 804 WithLimit(arg.Limit), 805 806 CnAddr: in.CnAddr, 807 OperatorID: c.allocOperatorID(), 808 ParallelID: 0, 809 MaxParallel: 1, 810 } 811 for j := range ss { 812 ss[j].appendInstruction(vm.Instruction{ 813 Op: vm.Limit, 814 Idx: in.Idx, 815 IsFirst: in.IsFirst, 816 Arg: limit.NewArgument(). 817 WithLimit(arg.Limit), 818 819 CnAddr: in.CnAddr, 820 OperatorID: in.OperatorID, 821 MaxParallel: int32(len(ss)), 822 ParallelID: int32(j), 823 }) 824 } 825 arg.Release() 826 case vm.Group: 827 flg = true 828 idx = i 829 arg := in.Arg.(*group.Argument) 830 if arg.AnyDistinctAgg() { 831 continue 832 } 833 s.Instructions = s.Instructions[i:] 834 s.Instructions[0] = vm.Instruction{ 835 Op: vm.MergeGroup, 836 Idx: in.Idx, 837 Arg: mergegroup.NewArgument(). 838 WithNeedEval(false), 839 840 CnAddr: in.CnAddr, 841 OperatorID: c.allocOperatorID(), 842 ParallelID: 0, 843 MaxParallel: 1, 844 } 845 for j := range ss { 846 ss[j].appendInstruction(vm.Instruction{ 847 Op: vm.Group, 848 Idx: in.Idx, 849 IsFirst: in.IsFirst, 850 Arg: group.NewArgument(). 851 WithExprs(arg.Exprs). 852 WithTypes(arg.Types). 853 WithAggsNew(arg.Aggs), 854 855 CnAddr: in.CnAddr, 856 OperatorID: in.OperatorID, 857 MaxParallel: int32(len(ss)), 858 ParallelID: int32(j), 859 }) 860 } 861 arg.Release() 862 case vm.Sample: 863 arg := in.Arg.(*sample.Argument) 864 if !arg.IsMergeSampleByRow() { 865 flg = true 866 idx = i 867 // if by percent, there is no need to do merge sample. 868 if arg.IsByPercent() { 869 s.Instructions = s.Instructions[i:] 870 } else { 871 s.Instructions = append(make([]vm.Instruction, 1), s.Instructions[i:]...) 872 s.Instructions[1] = vm.Instruction{ 873 Op: vm.Sample, 874 Idx: in.Idx, 875 IsFirst: false, 876 Arg: sample.NewMergeSample(arg, arg.NeedOutputRowSeen), 877 878 CnAddr: in.CnAddr, 879 OperatorID: c.allocOperatorID(), 880 ParallelID: 0, 881 MaxParallel: 1, 882 } 883 } 884 s.Instructions[0] = vm.Instruction{ 885 Op: vm.Merge, 886 Idx: s.Instructions[0].Idx, 887 Arg: merge.NewArgument(), 888 889 CnAddr: in.CnAddr, 890 OperatorID: c.allocOperatorID(), 891 ParallelID: 0, 892 MaxParallel: 1, 893 } 894 895 for j := range ss { 896 ss[j].appendInstruction(vm.Instruction{ 897 Op: vm.Sample, 898 Idx: in.Idx, 899 IsFirst: in.IsFirst, 900 Arg: arg.SimpleDup(), 901 902 CnAddr: in.CnAddr, 903 OperatorID: in.OperatorID, 904 MaxParallel: int32(len(ss)), 905 ParallelID: int32(j), 906 }) 907 } 908 } 909 arg.Release() 910 case vm.Offset: 911 flg = true 912 idx = i 913 arg := in.Arg.(*offset.Argument) 914 s.Instructions = s.Instructions[i:] 915 s.Instructions[0] = vm.Instruction{ 916 Op: vm.MergeOffset, 917 Idx: in.Idx, 918 Arg: mergeoffset.NewArgument(). 919 WithOffset(arg.Offset), 920 921 CnAddr: in.CnAddr, 922 OperatorID: c.allocOperatorID(), 923 ParallelID: 0, 924 MaxParallel: 1, 925 } 926 for j := range ss { 927 ss[j].appendInstruction(vm.Instruction{ 928 Op: vm.Offset, 929 Idx: in.Idx, 930 IsFirst: in.IsFirst, 931 Arg: offset.NewArgument(). 932 WithOffset(arg.Offset), 933 934 CnAddr: in.CnAddr, 935 OperatorID: in.OperatorID, 936 MaxParallel: int32(len(ss)), 937 ParallelID: int32(j), 938 }) 939 } 940 arg.Release() 941 case vm.Output: 942 default: 943 for j := range ss { 944 ss[j].appendInstruction(dupInstruction(&in, nil, j)) 945 } 946 } 947 } 948 if !flg { 949 for i := range ss { 950 if arg := ss[i].Instructions[len(ss[i].Instructions)-1].Arg; arg != nil { 951 arg.Release() 952 } 953 ss[i].Instructions = ss[i].Instructions[:len(ss[i].Instructions)-1] 954 } 955 if arg := s.Instructions[0].Arg; arg != nil { 956 arg.Release() 957 } 958 s.Instructions[0] = vm.Instruction{ 959 Op: vm.Merge, 960 Idx: s.Instructions[0].Idx, // TODO: remove it 961 Arg: merge.NewArgument(), 962 963 CnAddr: s.Instructions[0].CnAddr, 964 OperatorID: c.allocOperatorID(), 965 ParallelID: 0, 966 MaxParallel: 1, 967 } 968 // Add log for cn panic which reported on issue 10656 969 // If you find this log is printed, please report the repro details 970 if len(s.Instructions) < 2 { 971 logutil.Error("the length of s.Instructions is too short!"+DebugShowScopes([]*Scope{s}), 972 zap.String("stack", string(debug.Stack())), 973 ) 974 return nil, moerr.NewInternalErrorNoCtx("the length of s.Instructions is too short !") 975 } 976 if len(s.Instructions)-1 != 1 && s.Instructions[1].Arg != nil { 977 s.Instructions[1].Arg.Release() 978 } 979 s.Instructions[1] = s.Instructions[len(s.Instructions)-1] 980 for i := 2; i < len(s.Instructions)-1; i++ { 981 if arg := s.Instructions[i].Arg; arg != nil { 982 arg.Release() 983 } 984 } 985 s.Instructions = s.Instructions[:2] 986 } 987 s.Magic = Merge 988 s.PreScopes = ss 989 cnt := 0 990 for _, s := range ss { 991 if s.IsEnd { 992 continue 993 } 994 cnt++ 995 } 996 s.Proc.Reg.MergeReceivers = make([]*process.WaitRegister, cnt) 997 { 998 for i := 0; i < cnt; i++ { 999 s.Proc.Reg.MergeReceivers[i] = &process.WaitRegister{ 1000 Ctx: s.Proc.Ctx, 1001 Ch: make(chan *batch.Batch, 1), 1002 } 1003 } 1004 } 1005 j := 0 1006 for i := range ss { 1007 if !ss[i].IsEnd { 1008 ss[i].appendInstruction(vm.Instruction{ 1009 Op: vm.Connector, 1010 Arg: connector.NewArgument(). 1011 WithReg(s.Proc.Reg.MergeReceivers[j]), 1012 1013 CnAddr: ss[i].Instructions[0].CnAddr, 1014 OperatorID: c.allocOperatorID(), 1015 ParallelID: 0, 1016 MaxParallel: 1, 1017 }) 1018 j++ 1019 } 1020 } 1021 return s, nil 1022 } 1023 1024 func (s *Scope) appendInstruction(in vm.Instruction) { 1025 if !s.IsEnd { 1026 s.Instructions = append(s.Instructions, in) 1027 } 1028 } 1029 1030 func (s *Scope) notifyAndReceiveFromRemote(wg *sync.WaitGroup, errChan chan error) { 1031 // if context has done, it means the user or other part of the pipeline stops this query. 1032 closeWithError := func(err error, reg *process.WaitRegister) { 1033 if reg != nil { 1034 select { 1035 case <-s.Proc.Ctx.Done(): 1036 case reg.Ch <- nil: 1037 } 1038 } 1039 1040 select { 1041 case <-s.Proc.Ctx.Done(): 1042 errChan <- nil 1043 default: 1044 errChan <- err 1045 } 1046 wg.Done() 1047 } 1048 1049 // start N goroutines to send notifications to remote nodes. 1050 // to notify the remote dispatch executor where its remote receivers are. 1051 // dispatch operator will use this stream connection to send data back. 1052 // 1053 // function `cnMessageHandle` at file `scopeRemoteRun.go` will handle the notification. 1054 for i := range s.RemoteReceivRegInfos { 1055 wg.Add(1) 1056 1057 op := &s.RemoteReceivRegInfos[i] 1058 fromAddr := op.FromAddr 1059 receiverIdx := op.Idx 1060 uuid := op.Uuid[:] 1061 1062 errSubmit := ants.Submit( 1063 func() { 1064 streamSender, errStream := cnclient.GetStreamSender(fromAddr) 1065 if errStream != nil { 1066 logutil.Errorf("Failed to get stream sender txnID=%s, err=%v", 1067 s.Proc.TxnOperator.Txn().DebugString(), errStream) 1068 closeWithError(errStream, s.Proc.Reg.MergeReceivers[receiverIdx]) 1069 return 1070 } 1071 defer streamSender.Close(true) 1072 1073 message := cnclient.AcquireMessage() 1074 message.Id = streamSender.ID() 1075 message.Cmd = pbpipeline.Method_PrepareDoneNotifyMessage 1076 message.Sid = pbpipeline.Status_Last 1077 message.Uuid = uuid 1078 1079 if errSend := streamSender.Send(s.Proc.Ctx, message); errSend != nil { 1080 closeWithError(errSend, s.Proc.Reg.MergeReceivers[receiverIdx]) 1081 return 1082 } 1083 1084 messagesReceive, errReceive := streamSender.Receive() 1085 if errReceive != nil { 1086 closeWithError(errReceive, s.Proc.Reg.MergeReceivers[receiverIdx]) 1087 return 1088 } 1089 1090 err := receiveMsgAndForward(s.Proc, messagesReceive, s.Proc.Reg.MergeReceivers[receiverIdx].Ch) 1091 closeWithError(err, s.Proc.Reg.MergeReceivers[receiverIdx]) 1092 }, 1093 ) 1094 1095 if errSubmit != nil { 1096 errChan <- errSubmit 1097 wg.Done() 1098 } 1099 } 1100 } 1101 1102 func receiveMsgAndForward(proc *process.Process, receiveCh chan morpc.Message, forwardCh chan *batch.Batch) error { 1103 var val morpc.Message 1104 var dataBuffer []byte 1105 var ok bool 1106 var m *pbpipeline.Message 1107 1108 for { 1109 select { 1110 case <-proc.Ctx.Done(): 1111 return nil 1112 1113 case val, ok = <-receiveCh: 1114 if val == nil || !ok { 1115 return moerr.NewStreamClosedNoCtx() 1116 } 1117 } 1118 1119 m, ok = val.(*pbpipeline.Message) 1120 if !ok { 1121 panic("unexpected message type for cn-server") 1122 } 1123 1124 // receive an end message from remote 1125 if err := pbpipeline.GetMessageErrorInfo(m); err != nil { 1126 return err 1127 } 1128 1129 // end message 1130 if m.IsEndMessage() { 1131 return nil 1132 } 1133 1134 // normal receive 1135 if dataBuffer == nil { 1136 dataBuffer = m.Data 1137 } else { 1138 dataBuffer = append(dataBuffer, m.Data...) 1139 } 1140 1141 switch m.GetSid() { 1142 case pbpipeline.Status_WaitingNext: 1143 continue 1144 case pbpipeline.Status_Last: 1145 if m.Checksum != crc32.ChecksumIEEE(dataBuffer) { 1146 return moerr.NewInternalError(proc.Ctx, "Packages delivered by morpc is broken") 1147 } 1148 bat, err := decodeBatch(proc.Mp(), dataBuffer) 1149 if err != nil { 1150 return err 1151 } 1152 if forwardCh == nil { 1153 // used for delete 1154 proc.SetInputBatch(bat) 1155 } else { 1156 select { 1157 case <-proc.Ctx.Done(): 1158 bat.Clean(proc.Mp()) 1159 return nil 1160 1161 case forwardCh <- bat: 1162 } 1163 } 1164 dataBuffer = nil 1165 } 1166 } 1167 } 1168 1169 func (s *Scope) replace(c *Compile) error { 1170 tblName := s.Plan.GetQuery().Nodes[0].ReplaceCtx.TableDef.Name 1171 deleteCond := s.Plan.GetQuery().Nodes[0].ReplaceCtx.DeleteCond 1172 1173 delAffectedRows := uint64(0) 1174 if deleteCond != "" { 1175 result, err := c.runSqlWithResult(fmt.Sprintf("delete from %s where %s", tblName, deleteCond)) 1176 if err != nil { 1177 return err 1178 } 1179 delAffectedRows = result.AffectedRows 1180 } 1181 result, err := c.runSqlWithResult("insert " + c.sql[7:]) 1182 if err != nil { 1183 return err 1184 } 1185 c.addAffectedRows(result.AffectedRows + delAffectedRows) 1186 return nil 1187 } 1188 1189 func (s Scope) TypeName() string { 1190 return "compile.Scope" 1191 }