github.com/matrixorigin/matrixone@v1.2.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 "errors" 20 "fmt" 21 "time" 22 "unsafe" 23 24 "github.com/matrixorigin/matrixone/pkg/sql/colexec/aggexec" 25 "github.com/matrixorigin/matrixone/pkg/sql/colexec/indexjoin" 26 27 v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" 28 29 "github.com/google/uuid" 30 "github.com/matrixorigin/matrixone/pkg/common/moerr" 31 "github.com/matrixorigin/matrixone/pkg/common/morpc" 32 "github.com/matrixorigin/matrixone/pkg/common/mpool" 33 "github.com/matrixorigin/matrixone/pkg/common/reuse" 34 "github.com/matrixorigin/matrixone/pkg/container/batch" 35 "github.com/matrixorigin/matrixone/pkg/container/types" 36 "github.com/matrixorigin/matrixone/pkg/defines" 37 "github.com/matrixorigin/matrixone/pkg/fileservice" 38 "github.com/matrixorigin/matrixone/pkg/lockservice" 39 "github.com/matrixorigin/matrixone/pkg/logservice" 40 "github.com/matrixorigin/matrixone/pkg/logutil" 41 "github.com/matrixorigin/matrixone/pkg/objectio" 42 "github.com/matrixorigin/matrixone/pkg/pb/pipeline" 43 "github.com/matrixorigin/matrixone/pkg/pb/plan" 44 qclient "github.com/matrixorigin/matrixone/pkg/queryservice/client" 45 "github.com/matrixorigin/matrixone/pkg/sql/colexec" 46 "github.com/matrixorigin/matrixone/pkg/sql/colexec/anti" 47 "github.com/matrixorigin/matrixone/pkg/sql/colexec/connector" 48 "github.com/matrixorigin/matrixone/pkg/sql/colexec/deletion" 49 "github.com/matrixorigin/matrixone/pkg/sql/colexec/dispatch" 50 "github.com/matrixorigin/matrixone/pkg/sql/colexec/external" 51 "github.com/matrixorigin/matrixone/pkg/sql/colexec/fuzzyfilter" 52 "github.com/matrixorigin/matrixone/pkg/sql/colexec/group" 53 "github.com/matrixorigin/matrixone/pkg/sql/colexec/hashbuild" 54 "github.com/matrixorigin/matrixone/pkg/sql/colexec/insert" 55 "github.com/matrixorigin/matrixone/pkg/sql/colexec/intersect" 56 "github.com/matrixorigin/matrixone/pkg/sql/colexec/intersectall" 57 "github.com/matrixorigin/matrixone/pkg/sql/colexec/join" 58 "github.com/matrixorigin/matrixone/pkg/sql/colexec/left" 59 "github.com/matrixorigin/matrixone/pkg/sql/colexec/limit" 60 "github.com/matrixorigin/matrixone/pkg/sql/colexec/lockop" 61 "github.com/matrixorigin/matrixone/pkg/sql/colexec/loopanti" 62 "github.com/matrixorigin/matrixone/pkg/sql/colexec/loopjoin" 63 "github.com/matrixorigin/matrixone/pkg/sql/colexec/loopleft" 64 "github.com/matrixorigin/matrixone/pkg/sql/colexec/loopmark" 65 "github.com/matrixorigin/matrixone/pkg/sql/colexec/loopsemi" 66 "github.com/matrixorigin/matrixone/pkg/sql/colexec/loopsingle" 67 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mark" 68 "github.com/matrixorigin/matrixone/pkg/sql/colexec/merge" 69 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergegroup" 70 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergelimit" 71 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergeoffset" 72 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergeorder" 73 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergerecursive" 74 "github.com/matrixorigin/matrixone/pkg/sql/colexec/mergetop" 75 "github.com/matrixorigin/matrixone/pkg/sql/colexec/minus" 76 "github.com/matrixorigin/matrixone/pkg/sql/colexec/offset" 77 "github.com/matrixorigin/matrixone/pkg/sql/colexec/onduplicatekey" 78 "github.com/matrixorigin/matrixone/pkg/sql/colexec/order" 79 "github.com/matrixorigin/matrixone/pkg/sql/colexec/output" 80 "github.com/matrixorigin/matrixone/pkg/sql/colexec/preinsert" 81 "github.com/matrixorigin/matrixone/pkg/sql/colexec/preinsertsecondaryindex" 82 "github.com/matrixorigin/matrixone/pkg/sql/colexec/preinsertunique" 83 "github.com/matrixorigin/matrixone/pkg/sql/colexec/product" 84 "github.com/matrixorigin/matrixone/pkg/sql/colexec/projection" 85 "github.com/matrixorigin/matrixone/pkg/sql/colexec/restrict" 86 "github.com/matrixorigin/matrixone/pkg/sql/colexec/right" 87 "github.com/matrixorigin/matrixone/pkg/sql/colexec/rightanti" 88 "github.com/matrixorigin/matrixone/pkg/sql/colexec/rightsemi" 89 "github.com/matrixorigin/matrixone/pkg/sql/colexec/sample" 90 "github.com/matrixorigin/matrixone/pkg/sql/colexec/semi" 91 "github.com/matrixorigin/matrixone/pkg/sql/colexec/shuffle" 92 "github.com/matrixorigin/matrixone/pkg/sql/colexec/single" 93 "github.com/matrixorigin/matrixone/pkg/sql/colexec/source" 94 "github.com/matrixorigin/matrixone/pkg/sql/colexec/table_function" 95 "github.com/matrixorigin/matrixone/pkg/sql/colexec/top" 96 "github.com/matrixorigin/matrixone/pkg/sql/colexec/value_scan" 97 plan2 "github.com/matrixorigin/matrixone/pkg/sql/plan" 98 "github.com/matrixorigin/matrixone/pkg/txn/client" 99 "github.com/matrixorigin/matrixone/pkg/udf" 100 "github.com/matrixorigin/matrixone/pkg/vm" 101 "github.com/matrixorigin/matrixone/pkg/vm/engine" 102 "github.com/matrixorigin/matrixone/pkg/vm/process" 103 "go.uber.org/zap" 104 ) 105 106 // CnServerMessageHandler is responsible for processing the cn-client message received at cn-server. 107 // The message is always *pipeline.Message here. It's a byte array encoded by method encodeScope. 108 func CnServerMessageHandler( 109 ctx context.Context, 110 cnAddr string, 111 message morpc.Message, 112 cs morpc.ClientSession, 113 storeEngine engine.Engine, 114 fileService fileservice.FileService, 115 lockService lockservice.LockService, 116 queryClient qclient.QueryClient, 117 hakeeper logservice.CNHAKeeperClient, 118 udfService udf.Service, 119 cli client.TxnClient, 120 aicm *defines.AutoIncrCacheManager, 121 messageAcquirer func() morpc.Message) (err error) { 122 defer func() { 123 if e := recover(); e != nil { 124 err = moerr.ConvertPanicError(ctx, e) 125 getLogger().Error("panic in cn message handler", 126 zap.String("error", err.Error())) 127 err = errors.Join(err, cs.Close()) 128 } 129 }() 130 start := time.Now() 131 defer func() { 132 v2.PipelineServerDurationHistogram.Observe(time.Since(start).Seconds()) 133 }() 134 135 msg, ok := message.(*pipeline.Message) 136 if !ok { 137 logutil.Errorf("cn server should receive *pipeline.Message, but get %v", message) 138 panic("cn server receive a message with unexpected type") 139 } 140 141 receiver := newMessageReceiverOnServer(ctx, cnAddr, msg, 142 cs, messageAcquirer, storeEngine, fileService, lockService, queryClient, hakeeper, udfService, cli, aicm) 143 144 // rebuild pipeline to run and send the query result back. 145 err = cnMessageHandle(&receiver) 146 if err != nil { 147 return receiver.sendError(err) 148 } 149 return receiver.sendEndMessage() 150 } 151 152 // cnMessageHandle deal the received message at cn-server. 153 func cnMessageHandle(receiver *messageReceiverOnServer) error { 154 switch receiver.messageTyp { 155 case pipeline.Method_PrepareDoneNotifyMessage: // notify the dispatch executor 156 dispatchProc, err := receiver.GetProcByUuid(receiver.messageUuid) 157 if err != nil || dispatchProc == nil { 158 return err 159 } 160 161 infoToDispatchOperator := process.WrapCs{ 162 MsgId: receiver.messageId, 163 Uid: receiver.messageUuid, 164 Cs: receiver.clientSession, 165 Err: make(chan error, 1), 166 } 167 168 // todo : the timeout should be removed. 169 // but I keep it here because I don't know whether it will cause hung sometimes. 170 timeLimit, cancel := context.WithTimeout(context.TODO(), HandleNotifyTimeout) 171 172 succeed := false 173 select { 174 case <-timeLimit.Done(): 175 err = moerr.NewInternalError(receiver.ctx, "send notify msg to dispatch operator timeout") 176 case dispatchProc.DispatchNotifyCh <- infoToDispatchOperator: 177 succeed = true 178 case <-receiver.ctx.Done(): 179 case <-dispatchProc.Ctx.Done(): 180 } 181 cancel() 182 183 if err != nil || !succeed { 184 dispatchProc.Cancel() 185 return err 186 } 187 188 select { 189 case <-receiver.ctx.Done(): 190 dispatchProc.Cancel() 191 192 // there is no need to check the dispatchProc.Ctx.Done() here. 193 // because we need to receive the error from dispatchProc.DispatchNotifyCh. 194 case err = <-infoToDispatchOperator.Err: 195 } 196 return err 197 198 case pipeline.Method_PipelineMessage: 199 c := receiver.newCompile() 200 // decode and rewrite the scope. 201 s, err := decodeScope(receiver.scopeData, c.proc, true, c.e) 202 defer func() { 203 c.proc.AnalInfos = nil 204 c.anal.analInfos = nil 205 c.Release() 206 s.release() 207 }() 208 if err != nil { 209 return err 210 } 211 s = appendWriteBackOperator(c, s) 212 s.SetContextRecursively(c.ctx) 213 214 err = s.ParallelRun(c, s.IsRemote) 215 if err == nil { 216 // record the number of s3 requests 217 c.proc.AnalInfos[c.anal.curr].S3IOInputCount += c.counterSet.FileService.S3.Put.Load() 218 c.proc.AnalInfos[c.anal.curr].S3IOInputCount += c.counterSet.FileService.S3.List.Load() 219 c.proc.AnalInfos[c.anal.curr].S3IOOutputCount += c.counterSet.FileService.S3.Head.Load() 220 c.proc.AnalInfos[c.anal.curr].S3IOOutputCount += c.counterSet.FileService.S3.Get.Load() 221 c.proc.AnalInfos[c.anal.curr].S3IOOutputCount += c.counterSet.FileService.S3.Delete.Load() 222 c.proc.AnalInfos[c.anal.curr].S3IOOutputCount += c.counterSet.FileService.S3.DeleteMulti.Load() 223 224 receiver.finalAnalysisInfo = c.proc.AnalInfos 225 } else { 226 // there are 3 situations to release analyzeInfo 227 // 1 is free analyzeInfo of Local CN when release analyze 228 // 2 is free analyzeInfo of remote CN before transfer back 229 // 3 is free analyzeInfo of remote CN when errors happen before transfer back 230 // this is situation 3 231 for i := range c.proc.AnalInfos { 232 reuse.Free[process.AnalyzeInfo](c.proc.AnalInfos[i], nil) 233 } 234 } 235 c.proc.FreeVectors() 236 c.proc.CleanValueScanBatchs() 237 return err 238 239 default: 240 return moerr.NewInternalError(receiver.ctx, "unknown message type") 241 } 242 } 243 244 // receiveMessageFromCnServer deal the back message from cn-server. 245 func receiveMessageFromCnServer(c *Compile, s *Scope, sender *messageSenderOnClient, lastInstruction vm.Instruction) error { 246 var bat *batch.Batch 247 var end bool 248 var err error 249 250 lastAnalyze := c.proc.GetAnalyze(lastInstruction.Idx, -1, false) 251 if sender.receiveCh == nil { 252 sender.receiveCh, err = sender.streamSender.Receive() 253 if err != nil { 254 return err 255 } 256 } 257 258 var lastArg vm.Operator 259 var oldChild []vm.Operator 260 switch arg := lastInstruction.Arg.(type) { 261 case *connector.Argument: 262 lastArg = arg 263 oldChild = arg.Children 264 arg.Children = nil 265 defer func() { 266 arg.Children = oldChild 267 }() 268 case *dispatch.Argument: 269 lastArg = arg 270 oldChild = arg.Children 271 defer func() { 272 arg.Children = oldChild 273 }() 274 default: 275 return moerr.NewInvalidInput(c.ctx, "last operator should only be connector or dispatcher") 276 } 277 278 // can not reuse 279 valueScanOperator := &value_scan.Argument{} 280 info := &vm.OperatorInfo{ 281 Idx: -1, 282 IsFirst: false, 283 IsLast: false, 284 } 285 lastArg.SetInfo(info) 286 lastArg.AppendChild(valueScanOperator) 287 for { 288 bat, end, err = sender.receiveBatch() 289 if err != nil { 290 return err 291 } 292 if end { 293 return nil 294 } 295 296 lastAnalyze.Network(bat) 297 valueScanOperator.Batchs = append(valueScanOperator.Batchs, bat) 298 result, errCall := lastArg.Call(s.Proc) 299 if errCall != nil || result.Status == vm.ExecStop { 300 valueScanOperator.Free(s.Proc, false, errCall) 301 return errCall 302 } 303 valueScanOperator.Free(s.Proc, false, errCall) 304 } 305 } 306 307 // remoteRun sends a scope for remote running and receives the results. 308 // The back result message is always *pipeline.Message contains three cases. 309 // 1. Message with error information 310 // 2. Message with an end flag and analysis result 311 // 3. Batch Message with batch data 312 func (s *Scope) remoteRun(c *Compile) (err error) { 313 // encode the scope but without the last operator. 314 // the last operator will be executed on the current node for receiving the result and send them to the next pipeline. 315 lastIdx := len(s.Instructions) - 1 316 lastInstruction := s.Instructions[lastIdx] 317 318 if lastInstruction.Op == vm.Connector || lastInstruction.Op == vm.Dispatch { 319 if err = lastInstruction.Arg.Prepare(s.Proc); err != nil { 320 return err 321 } 322 } else { 323 return moerr.NewInvalidInput(c.ctx, "last operator should only be connector or dispatcher") 324 } 325 326 for _, ins := range s.Instructions[lastIdx+1:] { 327 ins.Arg.Release() 328 ins.Arg = nil 329 } 330 s.Instructions = s.Instructions[:lastIdx] 331 sData, errEncode := encodeScope(s) 332 if errEncode != nil { 333 return errEncode 334 } 335 s.Instructions = append(s.Instructions, lastInstruction) 336 337 // encode the process related information 338 pData, errEncodeProc := encodeProcessInfo(s.Proc, c.sql) 339 if errEncodeProc != nil { 340 return errEncodeProc 341 } 342 343 // new sender and do send work. 344 sender, err := newMessageSenderOnClient(s.Proc.Ctx, c, s.NodeInfo.Addr) 345 if err != nil { 346 logutil.Errorf("Failed to newMessageSenderOnClient sql=%s, txnID=%s, err=%v", 347 c.sql, c.proc.TxnOperator.Txn().DebugString(), err) 348 return err 349 } 350 defer sender.close() 351 err = sender.send(sData, pData, pipeline.Method_PipelineMessage) 352 if err != nil { 353 return err 354 } 355 356 return receiveMessageFromCnServer(c, s, sender, lastInstruction) 357 } 358 359 // encodeScope generate a pipeline.Pipeline from Scope, encode pipeline, and returns. 360 func encodeScope(s *Scope) ([]byte, error) { 361 p, err := fillPipeline(s) 362 if err != nil { 363 return nil, err 364 } 365 return p.Marshal() 366 } 367 368 // decodeScope decode a pipeline.Pipeline from bytes, and generate a Scope from it. 369 func decodeScope(data []byte, proc *process.Process, isRemote bool, eng engine.Engine) (*Scope, error) { 370 // unmarshal to pipeline 371 p := &pipeline.Pipeline{} 372 err := p.Unmarshal(data) 373 if err != nil { 374 return nil, err 375 } 376 ctx := &scopeContext{ 377 parent: nil, 378 id: p.PipelineId, 379 regs: make(map[*process.WaitRegister]int32), 380 } 381 ctx.root = ctx 382 s, err := generateScope(proc, p, ctx, proc.AnalInfos, isRemote) 383 if err != nil { 384 return nil, err 385 } 386 if err = fillInstructionsForScope(s, ctx, p, eng); err != nil { 387 s.release() 388 return nil, err 389 } 390 391 return s, nil 392 } 393 394 // encodeProcessInfo get needed information from proc, and do serialization work. 395 func encodeProcessInfo(proc *process.Process, sql string) ([]byte, error) { 396 procInfo := &pipeline.ProcessInfo{} 397 if len(proc.AnalInfos) == 0 { 398 getLogger().Error("empty plan", zap.String("sql", sql)) 399 } 400 { 401 procInfo.Id = proc.Id 402 procInfo.Sql = sql 403 procInfo.Lim = convertToPipelineLimitation(proc.Lim) 404 procInfo.UnixTime = proc.UnixTime 405 accountId, err := defines.GetAccountId(proc.Ctx) 406 if err != nil { 407 return nil, err 408 } 409 procInfo.AccountId = accountId 410 snapshot, err := proc.TxnOperator.Snapshot() 411 if err != nil { 412 return nil, err 413 } 414 procInfo.Snapshot = string(snapshot) 415 procInfo.AnalysisNodeList = make([]int32, len(proc.AnalInfos)) 416 for i := range procInfo.AnalysisNodeList { 417 procInfo.AnalysisNodeList[i] = proc.AnalInfos[i].NodeId 418 } 419 } 420 { // session info 421 timeBytes, err := time.Time{}.In(proc.SessionInfo.TimeZone).MarshalBinary() 422 if err != nil { 423 return nil, err 424 } 425 426 procInfo.SessionInfo = &pipeline.SessionInfo{ 427 User: proc.SessionInfo.GetUser(), 428 Host: proc.SessionInfo.GetHost(), 429 Role: proc.SessionInfo.GetRole(), 430 ConnectionId: proc.SessionInfo.GetConnectionID(), 431 Database: proc.SessionInfo.GetDatabase(), 432 Version: proc.SessionInfo.GetVersion(), 433 TimeZone: timeBytes, 434 QueryId: proc.SessionInfo.QueryId, 435 } 436 } 437 return procInfo.Marshal() 438 } 439 440 func appendWriteBackOperator(c *Compile, s *Scope) *Scope { 441 rs := c.newMergeScope([]*Scope{s}) 442 rs.Instructions = append(rs.Instructions, vm.Instruction{ 443 Op: vm.Output, 444 Idx: -1, // useless 445 Arg: output.NewArgument(). 446 WithFunc(c.fill), 447 }) 448 return rs 449 } 450 451 // fillPipeline convert the scope to pipeline.Pipeline structure through 2 iterations. 452 func fillPipeline(s *Scope) (*pipeline.Pipeline, error) { 453 ctx := &scopeContext{ 454 id: 0, 455 parent: nil, 456 regs: make(map[*process.WaitRegister]int32), 457 } 458 ctx.root = ctx 459 p, ctxId, err := generatePipeline(s, ctx, 1) 460 if err != nil { 461 return nil, err 462 } 463 if _, err = fillInstructionsForPipeline(s, ctx, p, ctxId); err != nil { 464 return nil, err 465 } 466 return p, nil 467 } 468 469 // generatePipeline generate a base pipeline.Pipeline structure without instructions 470 // according to source scope. 471 func generatePipeline(s *Scope, ctx *scopeContext, ctxId int32) (*pipeline.Pipeline, int32, error) { 472 var err error 473 474 p := &pipeline.Pipeline{} 475 // Magic and IsEnd 476 p.PipelineType = pipeline.Pipeline_PipelineType(s.Magic) 477 p.PipelineId = ctx.id 478 p.IsEnd = s.IsEnd 479 p.IsJoin = s.IsJoin 480 p.IsLoad = s.IsLoad 481 p.UuidsToRegIdx = convertScopeRemoteReceivInfo(s) 482 p.BuildIdx = int32(s.BuildIdx) 483 p.ShuffleCnt = int32(s.ShuffleCnt) 484 485 // Plan 486 if ctxId == 1 { 487 // encode and decode cost is too large for it. 488 // only encode the first one. 489 p.Qry = s.Plan 490 } 491 p.Node = &pipeline.NodeInfo{ 492 Id: s.NodeInfo.Id, 493 Addr: s.NodeInfo.Addr, 494 Mcpu: int32(s.NodeInfo.Mcpu), 495 Payload: string(s.NodeInfo.Data), 496 Type: objectio.EncodeInfoHeader(objectio.InfoHeader{ 497 Type: objectio.BlockInfoType, 498 Version: objectio.V1}, 499 ), 500 } 501 ctx.pipe = p 502 ctx.scope = s 503 p.ChildrenCount = int32(len(s.Proc.Reg.MergeReceivers)) 504 { 505 for i := range s.Proc.Reg.MergeReceivers { 506 ctx.regs[s.Proc.Reg.MergeReceivers[i]] = int32(i) 507 } 508 } 509 // DataSource 510 if s.DataSource != nil { // if select 1, DataSource is nil 511 p.DataSource = &pipeline.Source{ 512 SchemaName: s.DataSource.SchemaName, 513 TableName: s.DataSource.RelationName, 514 ColList: s.DataSource.Attributes, 515 PushdownId: s.DataSource.PushdownId, 516 PushdownAddr: s.DataSource.PushdownAddr, 517 Expr: s.DataSource.FilterExpr, 518 TableDef: s.DataSource.TableDef, 519 Timestamp: &s.DataSource.Timestamp, 520 RuntimeFilterProbeList: s.DataSource.RuntimeFilterSpecs, 521 } 522 if s.DataSource.Bat != nil { 523 data, err := types.Encode(s.DataSource.Bat) 524 if err != nil { 525 return nil, -1, err 526 } 527 p.DataSource.Block = string(data) 528 } 529 } 530 // PreScope 531 p.Children = make([]*pipeline.Pipeline, len(s.PreScopes)) 532 ctx.children = make([]*scopeContext, len(s.PreScopes)) 533 for i := range s.PreScopes { 534 ctx.children[i] = &scopeContext{ 535 parent: ctx, 536 id: ctxId, 537 root: ctx.root, 538 regs: make(map[*process.WaitRegister]int32), 539 } 540 ctxId++ 541 if p.Children[i], ctxId, err = generatePipeline(s.PreScopes[i], ctx.children[i], ctxId); err != nil { 542 return nil, -1, err 543 } 544 } 545 return p, ctxId, nil 546 } 547 548 // fillInstructionsForPipeline fills pipeline's instructions. 549 func fillInstructionsForPipeline(s *Scope, ctx *scopeContext, p *pipeline.Pipeline, ctxId int32) (int32, error) { 550 var err error 551 552 for i := range s.PreScopes { 553 if ctxId, err = fillInstructionsForPipeline(s.PreScopes[i], ctx.children[i], p.Children[i], ctxId); err != nil { 554 return ctxId, err 555 } 556 } 557 // Instructions 558 p.InstructionList = make([]*pipeline.Instruction, len(s.Instructions)) 559 for i := range p.InstructionList { 560 if ctxId, p.InstructionList[i], err = convertToPipelineInstruction(&s.Instructions[i], ctx, ctxId); err != nil { 561 return ctxId, err 562 } 563 } 564 return ctxId, nil 565 } 566 567 func convertPipelineUuid(p *pipeline.Pipeline, s *Scope) error { 568 s.RemoteReceivRegInfos = make([]RemoteReceivRegInfo, len(p.UuidsToRegIdx)) 569 for i := range p.UuidsToRegIdx { 570 op := p.UuidsToRegIdx[i] 571 uid, err := uuid.FromBytes(op.GetUuid()) 572 if err != nil { 573 return moerr.NewInternalErrorNoCtx("decode uuid failed: %s\n", err) 574 } 575 s.RemoteReceivRegInfos[i] = RemoteReceivRegInfo{ 576 Idx: int(op.GetIdx()), 577 Uuid: uid, 578 FromAddr: op.FromAddr, 579 } 580 } 581 return nil 582 } 583 584 func convertScopeRemoteReceivInfo(s *Scope) (ret []*pipeline.UuidToRegIdx) { 585 ret = make([]*pipeline.UuidToRegIdx, len(s.RemoteReceivRegInfos)) 586 for i := range s.RemoteReceivRegInfos { 587 op := &s.RemoteReceivRegInfos[i] 588 uid, _ := op.Uuid.MarshalBinary() 589 ret[i] = &pipeline.UuidToRegIdx{ 590 Idx: int32(op.Idx), 591 Uuid: uid, 592 FromAddr: op.FromAddr, 593 } 594 } 595 596 return ret 597 } 598 599 // generateScope generate a scope from scope context and pipeline. 600 func generateScope(proc *process.Process, p *pipeline.Pipeline, ctx *scopeContext, 601 analNodes []*process.AnalyzeInfo, isRemote bool) (*Scope, error) { 602 var err error 603 var s *Scope 604 defer func() { 605 if err != nil { 606 s.release() 607 } 608 }() 609 610 if p.Qry != nil { 611 ctx.plan = p.Qry 612 } 613 614 s = newScope(magicType(p.GetPipelineType())) 615 s.IsEnd = p.IsEnd 616 s.IsJoin = p.IsJoin 617 s.IsLoad = p.IsLoad 618 s.IsRemote = isRemote 619 s.BuildIdx = int(p.BuildIdx) 620 s.ShuffleCnt = int(p.ShuffleCnt) 621 if err = convertPipelineUuid(p, s); err != nil { 622 return nil, err 623 } 624 dsc := p.GetDataSource() 625 if dsc != nil { 626 s.DataSource = &Source{ 627 SchemaName: dsc.SchemaName, 628 RelationName: dsc.TableName, 629 Attributes: dsc.ColList, 630 PushdownId: dsc.PushdownId, 631 PushdownAddr: dsc.PushdownAddr, 632 FilterExpr: dsc.Expr, 633 TableDef: dsc.TableDef, 634 Timestamp: *dsc.Timestamp, 635 RuntimeFilterSpecs: dsc.RuntimeFilterProbeList, 636 } 637 if len(dsc.Block) > 0 { 638 bat := new(batch.Batch) 639 if err = types.Decode([]byte(dsc.Block), bat); err != nil { 640 return nil, err 641 } 642 bat.Cnt = 1 643 s.DataSource.Bat = bat 644 } 645 } 646 if p.Node != nil { 647 s.NodeInfo.Id = p.Node.Id 648 s.NodeInfo.Addr = p.Node.Addr 649 s.NodeInfo.Mcpu = int(p.Node.Mcpu) 650 s.NodeInfo.Data = []byte(p.Node.Payload) 651 s.NodeInfo.Header = objectio.DecodeInfoHeader(p.Node.Type) 652 } 653 s.Proc = process.NewWithAnalyze(proc, proc.Ctx, int(p.ChildrenCount), analNodes) 654 { 655 for i := range s.Proc.Reg.MergeReceivers { 656 ctx.regs[s.Proc.Reg.MergeReceivers[i]] = int32(i) 657 } 658 } 659 s.PreScopes = make([]*Scope, len(p.Children)) 660 ctx.children = make([]*scopeContext, len(s.PreScopes)) 661 for i := range s.PreScopes { 662 ctx.children[i] = &scopeContext{ 663 parent: ctx, 664 root: ctx.root, 665 id: p.Children[i].PipelineId, 666 regs: make(map[*process.WaitRegister]int32), 667 } 668 if s.PreScopes[i], err = generateScope(s.Proc, p.Children[i], ctx.children[i], analNodes, isRemote); err != nil { 669 return nil, err 670 } 671 } 672 return s, nil 673 } 674 675 // fillInstructionsForScope fills scope's instructions. 676 func fillInstructionsForScope(s *Scope, ctx *scopeContext, p *pipeline.Pipeline, eng engine.Engine) error { 677 var err error 678 679 for i := range s.PreScopes { 680 if err = fillInstructionsForScope(s.PreScopes[i], ctx.children[i], p.Children[i], eng); err != nil { 681 return err 682 } 683 } 684 s.Instructions = make([]vm.Instruction, len(p.InstructionList)) 685 for i := range s.Instructions { 686 if s.Instructions[i], err = convertToVmInstruction(p.InstructionList[i], ctx, eng); err != nil { 687 return err 688 } 689 } 690 if s.isShuffle() { 691 for _, rr := range s.Proc.Reg.MergeReceivers { 692 rr.Ch = make(chan *batch.Batch, 16) 693 } 694 } 695 return nil 696 } 697 698 // convert vm.Instruction to pipeline.Instruction 699 // todo: bad design, need to be refactored. and please refer to how sample operator do. 700 func convertToPipelineInstruction(opr *vm.Instruction, ctx *scopeContext, ctxId int32) (int32, *pipeline.Instruction, error) { 701 in := &pipeline.Instruction{ 702 Op: int32(opr.Op), 703 Idx: int32(opr.Idx), 704 IsFirst: opr.IsFirst, 705 IsLast: opr.IsLast, 706 707 CnAddr: opr.CnAddr, 708 OperatorId: opr.OperatorID, 709 ParallelId: opr.ParallelID, 710 MaxParallel: opr.MaxParallel, 711 } 712 switch t := opr.Arg.(type) { 713 case *insert.Argument: 714 in.Insert = &pipeline.Insert{ 715 ToWriteS3: t.ToWriteS3, 716 Ref: t.InsertCtx.Ref, 717 Attrs: t.InsertCtx.Attrs, 718 AddAffectedRows: t.InsertCtx.AddAffectedRows, 719 PartitionTableIds: t.InsertCtx.PartitionTableIDs, 720 PartitionTableNames: t.InsertCtx.PartitionTableNames, 721 PartitionIdx: int32(t.InsertCtx.PartitionIndexInBatch), 722 TableDef: t.InsertCtx.TableDef, 723 } 724 case *deletion.Argument: 725 in.Delete = &pipeline.Deletion{ 726 AffectedRows: t.AffectedRows(), 727 RemoteDelete: t.RemoteDelete, 728 SegmentMap: t.SegmentMap, 729 IBucket: t.IBucket, 730 NBucket: t.Nbucket, 731 // deleteCtx 732 RowIdIdx: int32(t.DeleteCtx.RowIdIdx), 733 PartitionTableIds: t.DeleteCtx.PartitionTableIDs, 734 PartitionTableNames: t.DeleteCtx.PartitionTableNames, 735 PartitionIndexInBatch: int32(t.DeleteCtx.PartitionIndexInBatch), 736 AddAffectedRows: t.DeleteCtx.AddAffectedRows, 737 Ref: t.DeleteCtx.Ref, 738 PrimaryKeyIdx: int32(t.DeleteCtx.PrimaryKeyIdx), 739 } 740 case *onduplicatekey.Argument: 741 in.OnDuplicateKey = &pipeline.OnDuplicateKey{ 742 Attrs: t.Attrs, 743 InsertColCount: t.InsertColCount, 744 UniqueColCheckExpr: t.UniqueColCheckExpr, 745 UniqueCols: t.UniqueCols, 746 OnDuplicateIdx: t.OnDuplicateIdx, 747 OnDuplicateExpr: t.OnDuplicateExpr, 748 } 749 case *fuzzyfilter.Argument: 750 in.FuzzyFilter = &pipeline.FuzzyFilter{ 751 N: float32(t.N), 752 PkName: t.PkName, 753 PkTyp: t.PkTyp, 754 } 755 case *preinsert.Argument: 756 in.PreInsert = &pipeline.PreInsert{ 757 SchemaName: t.SchemaName, 758 TableDef: t.TableDef, 759 HasAutoCol: t.HasAutoCol, 760 IsUpdate: t.IsUpdate, 761 Attrs: t.Attrs, 762 EstimatedRowCount: int64(t.EstimatedRowCount), 763 } 764 case *lockop.Argument: 765 in.LockOp = &pipeline.LockOp{ 766 Block: t.Block(), 767 Targets: t.CopyToPipelineTarget(), 768 } 769 case *preinsertunique.Argument: 770 in.PreInsertUnique = &pipeline.PreInsertUnique{ 771 PreInsertUkCtx: t.PreInsertCtx, 772 } 773 case *preinsertsecondaryindex.Argument: 774 in.PreInsertSecondaryIndex = &pipeline.PreInsertSecondaryIndex{ 775 PreInsertSkCtx: t.PreInsertCtx, 776 } 777 case *anti.Argument: 778 in.Anti = &pipeline.AntiJoin{ 779 Expr: t.Cond, 780 Types: convertToPlanTypes(t.Typs), 781 LeftCond: t.Conditions[0], 782 RightCond: t.Conditions[1], 783 Result: t.Result, 784 HashOnPk: t.HashOnPK, 785 IsShuffle: t.IsShuffle, 786 } 787 case *shuffle.Argument: 788 in.Shuffle = &pipeline.Shuffle{} 789 in.Shuffle.ShuffleColIdx = t.ShuffleColIdx 790 in.Shuffle.ShuffleType = t.ShuffleType 791 in.Shuffle.ShuffleColMax = t.ShuffleColMax 792 in.Shuffle.ShuffleColMin = t.ShuffleColMin 793 in.Shuffle.AliveRegCnt = t.AliveRegCnt 794 in.Shuffle.ShuffleRangesUint64 = t.ShuffleRangeUint64 795 in.Shuffle.ShuffleRangesInt64 = t.ShuffleRangeInt64 796 case *dispatch.Argument: 797 in.Dispatch = &pipeline.Dispatch{IsSink: t.IsSink, ShuffleType: t.ShuffleType, RecSink: t.RecSink, FuncId: int32(t.FuncId)} 798 in.Dispatch.ShuffleRegIdxLocal = make([]int32, len(t.ShuffleRegIdxLocal)) 799 for i := range t.ShuffleRegIdxLocal { 800 in.Dispatch.ShuffleRegIdxLocal[i] = int32(t.ShuffleRegIdxLocal[i]) 801 } 802 in.Dispatch.ShuffleRegIdxRemote = make([]int32, len(t.ShuffleRegIdxRemote)) 803 for i := range t.ShuffleRegIdxRemote { 804 in.Dispatch.ShuffleRegIdxRemote[i] = int32(t.ShuffleRegIdxRemote[i]) 805 } 806 807 in.Dispatch.LocalConnector = make([]*pipeline.Connector, len(t.LocalRegs)) 808 for i := range t.LocalRegs { 809 idx, ctx0 := ctx.root.findRegister(t.LocalRegs[i]) 810 in.Dispatch.LocalConnector[i] = &pipeline.Connector{ 811 ConnectorIndex: idx, 812 PipelineId: ctx0.id, 813 } 814 } 815 816 if len(t.RemoteRegs) > 0 { 817 in.Dispatch.RemoteConnector = make([]*pipeline.WrapNode, len(t.RemoteRegs)) 818 for i := range t.RemoteRegs { 819 wn := &pipeline.WrapNode{ 820 NodeAddr: t.RemoteRegs[i].NodeAddr, 821 Uuid: t.RemoteRegs[i].Uuid[:], 822 } 823 in.Dispatch.RemoteConnector[i] = wn 824 } 825 } 826 case *group.Argument: 827 in.Agg = &pipeline.Group{ 828 IsShuffle: t.IsShuffle, 829 PreAllocSize: t.PreAllocSize, 830 NeedEval: t.NeedEval, 831 Ibucket: t.Ibucket, 832 Nbucket: t.Nbucket, 833 Exprs: t.Exprs, 834 Types: convertToPlanTypes(t.Types), 835 Aggs: convertToPipelineAggregates(t.Aggs), 836 } 837 case *sample.Argument: 838 t.ConvertToPipelineOperator(in) 839 840 case *join.Argument: 841 relList, colList := getRelColList(t.Result) 842 in.Join = &pipeline.Join{ 843 RelList: relList, 844 ColList: colList, 845 Expr: t.Cond, 846 Types: convertToPlanTypes(t.Typs), 847 LeftCond: t.Conditions[0], 848 RightCond: t.Conditions[1], 849 RuntimeFilterBuildList: t.RuntimeFilterSpecs, 850 HashOnPk: t.HashOnPK, 851 IsShuffle: t.IsShuffle, 852 } 853 case *left.Argument: 854 relList, colList := getRelColList(t.Result) 855 in.LeftJoin = &pipeline.LeftJoin{ 856 RelList: relList, 857 ColList: colList, 858 Expr: t.Cond, 859 Types: convertToPlanTypes(t.Typs), 860 LeftCond: t.Conditions[0], 861 RightCond: t.Conditions[1], 862 RuntimeFilterBuildList: t.RuntimeFilterSpecs, 863 HashOnPk: t.HashOnPK, 864 IsShuffle: t.IsShuffle, 865 } 866 case *right.Argument: 867 rels, poses := getRelColList(t.Result) 868 in.RightJoin = &pipeline.RightJoin{ 869 RelList: rels, 870 ColList: poses, 871 Expr: t.Cond, 872 LeftTypes: convertToPlanTypes(t.LeftTypes), 873 RightTypes: convertToPlanTypes(t.RightTypes), 874 LeftCond: t.Conditions[0], 875 RightCond: t.Conditions[1], 876 RuntimeFilterBuildList: t.RuntimeFilterSpecs, 877 HashOnPk: t.HashOnPK, 878 IsShuffle: t.IsShuffle, 879 } 880 case *rightsemi.Argument: 881 in.RightSemiJoin = &pipeline.RightSemiJoin{ 882 Result: t.Result, 883 Expr: t.Cond, 884 RightTypes: convertToPlanTypes(t.RightTypes), 885 LeftCond: t.Conditions[0], 886 RightCond: t.Conditions[1], 887 RuntimeFilterBuildList: t.RuntimeFilterSpecs, 888 HashOnPk: t.HashOnPK, 889 IsShuffle: t.IsShuffle, 890 } 891 case *rightanti.Argument: 892 in.RightAntiJoin = &pipeline.RightAntiJoin{ 893 Result: t.Result, 894 Expr: t.Cond, 895 RightTypes: convertToPlanTypes(t.RightTypes), 896 LeftCond: t.Conditions[0], 897 RightCond: t.Conditions[1], 898 RuntimeFilterBuildList: t.RuntimeFilterSpecs, 899 HashOnPk: t.HashOnPK, 900 IsShuffle: t.IsShuffle, 901 } 902 case *limit.Argument: 903 in.Limit = t.Limit 904 case *loopanti.Argument: 905 in.Anti = &pipeline.AntiJoin{ 906 Result: t.Result, 907 Expr: t.Cond, 908 Types: convertToPlanTypes(t.Typs), 909 } 910 case *loopjoin.Argument: 911 relList, colList := getRelColList(t.Result) 912 in.Join = &pipeline.Join{ 913 RelList: relList, 914 ColList: colList, 915 Expr: t.Cond, 916 Types: convertToPlanTypes(t.Typs), 917 } 918 case *loopleft.Argument: 919 relList, colList := getRelColList(t.Result) 920 in.LeftJoin = &pipeline.LeftJoin{ 921 RelList: relList, 922 ColList: colList, 923 Expr: t.Cond, 924 Types: convertToPlanTypes(t.Typs), 925 } 926 case *loopsemi.Argument: 927 in.SemiJoin = &pipeline.SemiJoin{ 928 Result: t.Result, 929 Expr: t.Cond, 930 Types: convertToPlanTypes(t.Typs), 931 } 932 case *loopsingle.Argument: 933 relList, colList := getRelColList(t.Result) 934 in.SingleJoin = &pipeline.SingleJoin{ 935 RelList: relList, 936 ColList: colList, 937 Expr: t.Cond, 938 Types: convertToPlanTypes(t.Typs), 939 } 940 case *loopmark.Argument: 941 in.MarkJoin = &pipeline.MarkJoin{ 942 Expr: t.Cond, 943 Types: convertToPlanTypes(t.Typs), 944 Result: t.Result, 945 } 946 case *offset.Argument: 947 in.Offset = t.Offset 948 case *order.Argument: 949 in.OrderBy = t.OrderBySpec 950 case *product.Argument: 951 relList, colList := getRelColList(t.Result) 952 in.Product = &pipeline.Product{ 953 RelList: relList, 954 ColList: colList, 955 Types: convertToPlanTypes(t.Typs), 956 IsShuffle: t.IsShuffle, 957 } 958 case *projection.Argument: 959 in.ProjectList = t.Es 960 case *restrict.Argument: 961 in.Filter = t.E 962 case *semi.Argument: 963 in.SemiJoin = &pipeline.SemiJoin{ 964 Result: t.Result, 965 Expr: t.Cond, 966 Types: convertToPlanTypes(t.Typs), 967 LeftCond: t.Conditions[0], 968 RightCond: t.Conditions[1], 969 RuntimeFilterBuildList: t.RuntimeFilterSpecs, 970 HashOnPk: t.HashOnPK, 971 IsShuffle: t.IsShuffle, 972 } 973 case *indexjoin.Argument: 974 in.IndexJoin = &pipeline.IndexJoin{ 975 Result: t.Result, 976 Types: convertToPlanTypes(t.Typs), 977 RuntimeFilterBuildList: t.RuntimeFilterSpecs, 978 } 979 case *single.Argument: 980 relList, colList := getRelColList(t.Result) 981 in.SingleJoin = &pipeline.SingleJoin{ 982 RelList: relList, 983 ColList: colList, 984 Expr: t.Cond, 985 Types: convertToPlanTypes(t.Typs), 986 LeftCond: t.Conditions[0], 987 RightCond: t.Conditions[1], 988 RuntimeFilterBuildList: t.RuntimeFilterSpecs, 989 HashOnPk: t.HashOnPK, 990 } 991 case *top.Argument: 992 in.Limit = uint64(t.Limit) 993 in.OrderBy = t.Fs 994 // we reused ANTI to store the information here because of the lack of related structure. 995 case *intersect.Argument: // 1 996 in.Anti = &pipeline.AntiJoin{} 997 case *minus.Argument: // 2 998 in.Anti = &pipeline.AntiJoin{} 999 case *intersectall.Argument: 1000 in.Anti = &pipeline.AntiJoin{} 1001 case *merge.Argument: 1002 in.Merge = &pipeline.Merge{ 1003 SinkScan: t.SinkScan, 1004 } 1005 case *mergerecursive.Argument: 1006 case *mergegroup.Argument: 1007 in.Agg = &pipeline.Group{ 1008 NeedEval: t.NeedEval, 1009 } 1010 EncodeMergeGroup(t, in.Agg) 1011 case *mergelimit.Argument: 1012 in.Limit = t.Limit 1013 case *mergeoffset.Argument: 1014 in.Offset = t.Offset 1015 case *mergetop.Argument: 1016 in.Limit = uint64(t.Limit) 1017 in.OrderBy = t.Fs 1018 case *mergeorder.Argument: 1019 in.OrderBy = t.OrderBySpecs 1020 case *connector.Argument: 1021 idx, ctx0 := ctx.root.findRegister(t.Reg) 1022 in.Connect = &pipeline.Connector{ 1023 PipelineId: ctx0.id, 1024 ConnectorIndex: idx, 1025 } 1026 case *mark.Argument: 1027 in.MarkJoin = &pipeline.MarkJoin{ 1028 Result: t.Result, 1029 LeftCond: t.Conditions[0], 1030 RightCond: t.Conditions[1], 1031 Types: convertToPlanTypes(t.Typs), 1032 Expr: t.Cond, 1033 OnList: t.OnList, 1034 HashOnPk: t.HashOnPK, 1035 } 1036 case *table_function.Argument: 1037 in.TableFunction = &pipeline.TableFunction{ 1038 Attrs: t.Attrs, 1039 Rets: t.Rets, 1040 Args: t.Args, 1041 Params: t.Params, 1042 Name: t.FuncName, 1043 } 1044 case *hashbuild.Argument: 1045 in.HashBuild = &pipeline.HashBuild{ 1046 NeedExpr: t.NeedExpr, 1047 NeedHash: t.NeedHashMap, 1048 Ibucket: t.Ibucket, 1049 Nbucket: t.Nbucket, 1050 Types: convertToPlanTypes(t.Typs), 1051 Conds: t.Conditions, 1052 HashOnPk: t.HashOnPK, 1053 NeedMergedBatch: t.NeedMergedBatch, 1054 NeedAllocateSels: t.NeedAllocateSels, 1055 } 1056 case *external.Argument: 1057 name2ColIndexSlice := make([]*pipeline.ExternalName2ColIndex, len(t.Es.Name2ColIndex)) 1058 i := 0 1059 for k, v := range t.Es.Name2ColIndex { 1060 name2ColIndexSlice[i] = &pipeline.ExternalName2ColIndex{Name: k, Index: v} 1061 i++ 1062 } 1063 in.ExternalScan = &pipeline.ExternalScan{ 1064 Attrs: t.Es.Attrs, 1065 Cols: t.Es.Cols, 1066 FileSize: t.Es.FileSize, 1067 FileOffsetTotal: t.Es.FileOffsetTotal, 1068 Name2ColIndex: name2ColIndexSlice, 1069 CreateSql: t.Es.CreateSql, 1070 FileList: t.Es.FileList, 1071 Filter: t.Es.Filter.FilterExpr, 1072 } 1073 case *source.Argument: 1074 in.StreamScan = &pipeline.StreamScan{ 1075 TblDef: t.TblDef, 1076 Limit: t.Limit, 1077 Offset: t.Offset, 1078 } 1079 default: 1080 return -1, nil, moerr.NewInternalErrorNoCtx(fmt.Sprintf("unexpected operator: %v", opr.Op)) 1081 } 1082 return ctxId, in, nil 1083 } 1084 1085 // convert pipeline.Instruction to vm.Instruction 1086 func convertToVmInstruction(opr *pipeline.Instruction, ctx *scopeContext, eng engine.Engine) (vm.Instruction, error) { 1087 v := vm.Instruction{ 1088 Op: vm.OpType(opr.Op), 1089 Idx: int(opr.Idx), 1090 IsFirst: opr.IsFirst, 1091 IsLast: opr.IsLast, 1092 1093 CnAddr: opr.CnAddr, 1094 OperatorID: opr.OperatorId, 1095 ParallelID: opr.ParallelId, 1096 MaxParallel: opr.MaxParallel, 1097 } 1098 switch v.Op { 1099 case vm.Deletion: 1100 t := opr.GetDelete() 1101 arg := deletion.NewArgument() 1102 arg.RemoteDelete = t.RemoteDelete 1103 arg.SegmentMap = t.SegmentMap 1104 arg.IBucket = t.IBucket 1105 arg.Nbucket = t.NBucket 1106 arg.DeleteCtx = &deletion.DeleteCtx{ 1107 CanTruncate: t.CanTruncate, 1108 RowIdIdx: int(t.RowIdIdx), 1109 PartitionTableIDs: t.PartitionTableIds, 1110 PartitionTableNames: t.PartitionTableNames, 1111 PartitionIndexInBatch: int(t.PartitionIndexInBatch), 1112 Ref: t.Ref, 1113 AddAffectedRows: t.AddAffectedRows, 1114 PrimaryKeyIdx: int(t.PrimaryKeyIdx), 1115 } 1116 v.Arg = arg 1117 case vm.Insert: 1118 t := opr.GetInsert() 1119 arg := insert.NewArgument() 1120 arg.ToWriteS3 = t.ToWriteS3 1121 arg.InsertCtx = &insert.InsertCtx{ 1122 Ref: t.Ref, 1123 AddAffectedRows: t.AddAffectedRows, 1124 Attrs: t.Attrs, 1125 PartitionTableIDs: t.PartitionTableIds, 1126 PartitionTableNames: t.PartitionTableNames, 1127 PartitionIndexInBatch: int(t.PartitionIdx), 1128 TableDef: t.TableDef, 1129 } 1130 v.Arg = arg 1131 case vm.PreInsert: 1132 t := opr.GetPreInsert() 1133 arg := preinsert.NewArgument() 1134 arg.SchemaName = t.GetSchemaName() 1135 arg.TableDef = t.GetTableDef() 1136 arg.Attrs = t.GetAttrs() 1137 arg.HasAutoCol = t.GetHasAutoCol() 1138 arg.IsUpdate = t.GetIsUpdate() 1139 arg.EstimatedRowCount = int64(t.GetEstimatedRowCount()) 1140 v.Arg = arg 1141 case vm.LockOp: 1142 t := opr.GetLockOp() 1143 lockArg := lockop.NewArgumentByEngine(eng) 1144 lockArg.SetBlock(t.Block) 1145 for _, target := range t.Targets { 1146 typ := plan2.MakeTypeByPlan2Type(target.PrimaryColTyp) 1147 lockArg.AddLockTarget(target.GetTableId(), target.GetPrimaryColIdxInBat(), typ, target.GetRefreshTsIdxInBat()) 1148 } 1149 for _, target := range t.Targets { 1150 if target.LockTable { 1151 lockArg.LockTable(target.TableId, target.ChangeDef) 1152 } 1153 } 1154 v.Arg = lockArg 1155 case vm.PreInsertUnique: 1156 t := opr.GetPreInsertUnique() 1157 arg := preinsertunique.NewArgument() 1158 arg.PreInsertCtx = t.GetPreInsertUkCtx() 1159 v.Arg = arg 1160 case vm.PreInsertSecondaryIndex: 1161 t := opr.GetPreInsertSecondaryIndex() 1162 arg := preinsertsecondaryindex.NewArgument() 1163 arg.PreInsertCtx = t.GetPreInsertSkCtx() 1164 v.Arg = arg 1165 case vm.OnDuplicateKey: 1166 t := opr.GetOnDuplicateKey() 1167 arg := onduplicatekey.NewArgument() 1168 arg.Attrs = t.Attrs 1169 arg.InsertColCount = t.InsertColCount 1170 arg.UniqueColCheckExpr = t.UniqueColCheckExpr 1171 arg.UniqueCols = t.UniqueCols 1172 arg.OnDuplicateIdx = t.OnDuplicateIdx 1173 arg.OnDuplicateExpr = t.OnDuplicateExpr 1174 arg.IsIgnore = t.IsIgnore 1175 v.Arg = arg 1176 case vm.FuzzyFilter: 1177 t := opr.GetFuzzyFilter() 1178 arg := fuzzyfilter.NewArgument() 1179 arg.N = float64(t.N) 1180 arg.PkName = t.PkName 1181 arg.PkTyp = t.PkTyp 1182 v.Arg = arg 1183 case vm.Anti: 1184 t := opr.GetAnti() 1185 arg := anti.NewArgument() 1186 arg.Cond = t.Expr 1187 arg.Typs = convertToTypes(t.Types) 1188 arg.Conditions = [][]*plan.Expr{ 1189 t.LeftCond, t.RightCond, 1190 } 1191 arg.Result = t.Result 1192 arg.HashOnPK = t.HashOnPk 1193 arg.IsShuffle = t.IsShuffle 1194 v.Arg = arg 1195 case vm.Shuffle: 1196 t := opr.GetShuffle() 1197 arg := shuffle.NewArgument() 1198 arg.ShuffleColIdx = t.ShuffleColIdx 1199 arg.ShuffleType = t.ShuffleType 1200 arg.ShuffleColMin = t.ShuffleColMin 1201 arg.ShuffleColMax = t.ShuffleColMax 1202 arg.AliveRegCnt = t.AliveRegCnt 1203 arg.ShuffleRangeInt64 = t.ShuffleRangesInt64 1204 arg.ShuffleRangeUint64 = t.ShuffleRangesUint64 1205 v.Arg = arg 1206 case vm.Dispatch: 1207 t := opr.GetDispatch() 1208 regs := make([]*process.WaitRegister, len(t.LocalConnector)) 1209 for i, cp := range t.LocalConnector { 1210 regs[i] = ctx.root.getRegister(cp.PipelineId, cp.ConnectorIndex) 1211 } 1212 rrs := make([]colexec.ReceiveInfo, 0) 1213 if len(t.RemoteConnector) > 0 { 1214 for i := range t.RemoteConnector { 1215 uid, err := uuid.FromBytes(t.RemoteConnector[i].Uuid) 1216 if err != nil { 1217 return v, err 1218 } 1219 n := colexec.ReceiveInfo{ 1220 NodeAddr: t.RemoteConnector[i].NodeAddr, 1221 Uuid: uid, 1222 } 1223 rrs = append(rrs, n) 1224 } 1225 } 1226 shuffleRegIdxLocal := make([]int, len(t.ShuffleRegIdxLocal)) 1227 for i := range t.ShuffleRegIdxLocal { 1228 shuffleRegIdxLocal[i] = int(t.ShuffleRegIdxLocal[i]) 1229 } 1230 shuffleRegIdxRemote := make([]int, len(t.ShuffleRegIdxRemote)) 1231 for i := range t.ShuffleRegIdxRemote { 1232 shuffleRegIdxRemote[i] = int(t.ShuffleRegIdxRemote[i]) 1233 } 1234 1235 arg := dispatch.NewArgument() 1236 arg.IsSink = t.IsSink 1237 arg.RecSink = t.RecSink 1238 arg.FuncId = int(t.FuncId) 1239 arg.LocalRegs = regs 1240 arg.RemoteRegs = rrs 1241 arg.ShuffleType = t.ShuffleType 1242 arg.ShuffleRegIdxLocal = shuffleRegIdxLocal 1243 arg.ShuffleRegIdxRemote = shuffleRegIdxRemote 1244 v.Arg = arg 1245 case vm.Group: 1246 t := opr.GetAgg() 1247 arg := group.NewArgument() 1248 arg.IsShuffle = t.IsShuffle 1249 arg.PreAllocSize = t.PreAllocSize 1250 arg.NeedEval = t.NeedEval 1251 arg.Ibucket = t.Ibucket 1252 arg.Nbucket = t.Nbucket 1253 arg.Exprs = t.Exprs 1254 arg.Types = convertToTypes(t.Types) 1255 arg.Aggs = convertToAggregates(t.Aggs) 1256 v.Arg = arg 1257 case vm.Sample: 1258 v.Arg = sample.GenerateFromPipelineOperator(opr) 1259 1260 case vm.Join: 1261 t := opr.GetJoin() 1262 arg := join.NewArgument() 1263 arg.Cond = t.Expr 1264 arg.Typs = convertToTypes(t.Types) 1265 arg.Result = convertToResultPos(t.RelList, t.ColList) 1266 arg.Conditions = [][]*plan.Expr{t.LeftCond, t.RightCond} 1267 arg.RuntimeFilterSpecs = t.RuntimeFilterBuildList 1268 arg.HashOnPK = t.HashOnPk 1269 arg.IsShuffle = t.IsShuffle 1270 v.Arg = arg 1271 case vm.Left: 1272 t := opr.GetLeftJoin() 1273 arg := left.NewArgument() 1274 arg.Cond = t.Expr 1275 arg.Typs = convertToTypes(t.Types) 1276 arg.Result = convertToResultPos(t.RelList, t.ColList) 1277 arg.Conditions = [][]*plan.Expr{t.LeftCond, t.RightCond} 1278 arg.RuntimeFilterSpecs = t.RuntimeFilterBuildList 1279 arg.HashOnPK = t.HashOnPk 1280 arg.IsShuffle = t.IsShuffle 1281 v.Arg = arg 1282 case vm.Right: 1283 t := opr.GetRightJoin() 1284 arg := right.NewArgument() 1285 arg.Result = convertToResultPos(t.RelList, t.ColList) 1286 arg.LeftTypes = convertToTypes(t.LeftTypes) 1287 arg.RightTypes = convertToTypes(t.RightTypes) 1288 arg.Cond = t.Expr 1289 arg.Conditions = [][]*plan.Expr{t.LeftCond, t.RightCond} 1290 arg.RuntimeFilterSpecs = t.RuntimeFilterBuildList 1291 arg.HashOnPK = t.HashOnPk 1292 arg.IsShuffle = t.IsShuffle 1293 v.Arg = arg 1294 case vm.RightSemi: 1295 t := opr.GetRightSemiJoin() 1296 arg := rightsemi.NewArgument() 1297 arg.Result = t.Result 1298 arg.RightTypes = convertToTypes(t.RightTypes) 1299 arg.Cond = t.Expr 1300 arg.Conditions = [][]*plan.Expr{t.LeftCond, t.RightCond} 1301 arg.RuntimeFilterSpecs = t.RuntimeFilterBuildList 1302 arg.HashOnPK = t.HashOnPk 1303 arg.IsShuffle = t.IsShuffle 1304 v.Arg = arg 1305 case vm.RightAnti: 1306 t := opr.GetRightAntiJoin() 1307 arg := rightanti.NewArgument() 1308 arg.Result = t.Result 1309 arg.RightTypes = convertToTypes(t.RightTypes) 1310 arg.Cond = t.Expr 1311 arg.Conditions = [][]*plan.Expr{t.LeftCond, t.RightCond} 1312 arg.RuntimeFilterSpecs = t.RuntimeFilterBuildList 1313 arg.HashOnPK = t.HashOnPk 1314 arg.IsShuffle = t.IsShuffle 1315 v.Arg = arg 1316 case vm.Limit: 1317 v.Arg = limit.NewArgument().WithLimit(opr.Limit) 1318 case vm.LoopAnti: 1319 t := opr.GetAnti() 1320 arg := loopanti.NewArgument() 1321 arg.Result = t.Result 1322 arg.Cond = t.Expr 1323 arg.Typs = convertToTypes(t.Types) 1324 v.Arg = arg 1325 case vm.LoopJoin: 1326 t := opr.GetJoin() 1327 arg := loopjoin.NewArgument() 1328 arg.Result = convertToResultPos(t.RelList, t.ColList) 1329 arg.Cond = t.Expr 1330 arg.Typs = convertToTypes(t.Types) 1331 v.Arg = arg 1332 case vm.LoopLeft: 1333 t := opr.GetLeftJoin() 1334 arg := loopleft.NewArgument() 1335 arg.Result = convertToResultPos(t.RelList, t.ColList) 1336 arg.Cond = t.Expr 1337 arg.Typs = convertToTypes(t.Types) 1338 v.Arg = arg 1339 case vm.LoopSemi: 1340 t := opr.GetSemiJoin() 1341 arg := loopsemi.NewArgument() 1342 arg.Result = t.Result 1343 arg.Cond = t.Expr 1344 arg.Typs = convertToTypes(t.Types) 1345 v.Arg = arg 1346 case vm.IndexJoin: 1347 t := opr.GetIndexJoin() 1348 arg := indexjoin.NewArgument() 1349 arg.Result = t.Result 1350 arg.Typs = convertToTypes(t.Types) 1351 arg.RuntimeFilterSpecs = t.RuntimeFilterBuildList 1352 v.Arg = arg 1353 case vm.LoopSingle: 1354 t := opr.GetSingleJoin() 1355 arg := loopsingle.NewArgument() 1356 arg.Result = convertToResultPos(t.RelList, t.ColList) 1357 arg.Cond = t.Expr 1358 arg.Typs = convertToTypes(t.Types) 1359 v.Arg = arg 1360 case vm.LoopMark: 1361 t := opr.GetMarkJoin() 1362 arg := loopmark.NewArgument() 1363 arg.Result = t.Result 1364 arg.Cond = t.Expr 1365 arg.Typs = convertToTypes(t.Types) 1366 v.Arg = arg 1367 case vm.Offset: 1368 v.Arg = offset.NewArgument().WithOffset(opr.Offset) 1369 case vm.Order: 1370 arg := order.NewArgument() 1371 arg.OrderBySpec = opr.OrderBy 1372 v.Arg = arg 1373 case vm.Product: 1374 t := opr.GetProduct() 1375 arg := product.NewArgument() 1376 arg.Result = convertToResultPos(t.RelList, t.ColList) 1377 arg.Typs = convertToTypes(t.Types) 1378 arg.IsShuffle = t.IsShuffle 1379 v.Arg = arg 1380 case vm.Projection: 1381 arg := projection.NewArgument() 1382 arg.Es = opr.ProjectList 1383 v.Arg = arg 1384 case vm.Restrict: 1385 arg := restrict.NewArgument() 1386 arg.E = opr.Filter 1387 v.Arg = arg 1388 case vm.Semi: 1389 t := opr.GetSemiJoin() 1390 arg := semi.NewArgument() 1391 arg.Result = t.Result 1392 arg.Cond = t.Expr 1393 arg.Typs = convertToTypes(t.Types) 1394 arg.Conditions = [][]*plan.Expr{t.LeftCond, t.RightCond} 1395 arg.RuntimeFilterSpecs = t.RuntimeFilterBuildList 1396 arg.HashOnPK = t.HashOnPk 1397 arg.IsShuffle = t.IsShuffle 1398 v.Arg = arg 1399 case vm.Single: 1400 t := opr.GetSingleJoin() 1401 arg := single.NewArgument() 1402 arg.Result = convertToResultPos(t.RelList, t.ColList) 1403 arg.Cond = t.Expr 1404 arg.Typs = convertToTypes(t.Types) 1405 arg.Conditions = [][]*plan.Expr{t.LeftCond, t.RightCond} 1406 arg.RuntimeFilterSpecs = t.RuntimeFilterBuildList 1407 arg.HashOnPK = t.HashOnPk 1408 v.Arg = arg 1409 case vm.Mark: 1410 t := opr.GetMarkJoin() 1411 arg := mark.NewArgument() 1412 arg.Result = t.Result 1413 arg.Conditions = [][]*plan.Expr{t.LeftCond, t.RightCond} 1414 arg.Typs = convertToTypes(t.Types) 1415 arg.Cond = t.Expr 1416 arg.OnList = t.OnList 1417 arg.HashOnPK = t.HashOnPk 1418 v.Arg = arg 1419 case vm.Top: 1420 v.Arg = top.NewArgument(). 1421 WithLimit(int64(opr.Limit)). 1422 WithFs(opr.OrderBy) 1423 // should change next day? 1424 case vm.Intersect: 1425 arg := intersect.NewArgument() 1426 v.Arg = arg 1427 case vm.IntersectAll: 1428 arg := intersect.NewArgument() 1429 v.Arg = arg 1430 case vm.Minus: 1431 arg := minus.NewArgument() 1432 v.Arg = arg 1433 case vm.Connector: 1434 t := opr.GetConnect() 1435 v.Arg = connector.NewArgument(). 1436 WithReg(ctx.root.getRegister(t.PipelineId, t.ConnectorIndex)) 1437 case vm.Merge: 1438 v.Arg = merge.NewArgument() 1439 case vm.MergeRecursive: 1440 v.Arg = mergerecursive.NewArgument() 1441 case vm.MergeGroup: 1442 arg := mergegroup.NewArgument() 1443 arg.NeedEval = opr.Agg.NeedEval 1444 v.Arg = arg 1445 DecodeMergeGroup(v.Arg.(*mergegroup.Argument), opr.Agg) 1446 case vm.MergeLimit: 1447 v.Arg = mergelimit.NewArgument().WithLimit(opr.Limit) 1448 case vm.MergeOffset: 1449 v.Arg = mergeoffset.NewArgument().WithOffset(opr.Offset) 1450 case vm.MergeTop: 1451 v.Arg = mergetop.NewArgument(). 1452 WithLimit(int64(opr.Limit)). 1453 WithFs(opr.OrderBy) 1454 case vm.MergeOrder: 1455 arg := mergeorder.NewArgument() 1456 arg.OrderBySpecs = opr.OrderBy 1457 v.Arg = arg 1458 case vm.TableFunction: 1459 arg := table_function.NewArgument() 1460 arg.Attrs = opr.TableFunction.Attrs 1461 arg.Rets = opr.TableFunction.Rets 1462 arg.Args = opr.TableFunction.Args 1463 arg.FuncName = opr.TableFunction.Name 1464 arg.Params = opr.TableFunction.Params 1465 v.Arg = arg 1466 case vm.HashBuild: 1467 t := opr.GetHashBuild() 1468 arg := hashbuild.NewArgument() 1469 arg.Ibucket = t.Ibucket 1470 arg.Nbucket = t.Nbucket 1471 arg.NeedHashMap = t.NeedHash 1472 arg.NeedExpr = t.NeedExpr 1473 arg.Typs = convertToTypes(t.Types) 1474 arg.Conditions = t.Conds 1475 arg.HashOnPK = t.HashOnPk 1476 arg.NeedMergedBatch = t.NeedMergedBatch 1477 arg.NeedAllocateSels = t.NeedAllocateSels 1478 v.Arg = arg 1479 case vm.External: 1480 t := opr.GetExternalScan() 1481 name2ColIndex := make(map[string]int32) 1482 for _, n2i := range t.Name2ColIndex { 1483 name2ColIndex[n2i.Name] = n2i.Index 1484 } 1485 v.Arg = external.NewArgument().WithEs( 1486 &external.ExternalParam{ 1487 ExParamConst: external.ExParamConst{ 1488 Attrs: t.Attrs, 1489 FileSize: t.FileSize, 1490 FileOffsetTotal: t.FileOffsetTotal, 1491 Cols: t.Cols, 1492 CreateSql: t.CreateSql, 1493 Name2ColIndex: name2ColIndex, 1494 FileList: t.FileList, 1495 }, 1496 ExParam: external.ExParam{ 1497 Fileparam: new(external.ExFileparam), 1498 Filter: &external.FilterParam{ 1499 FilterExpr: t.Filter, 1500 }, 1501 }, 1502 }, 1503 ) 1504 case vm.Source: 1505 t := opr.GetStreamScan() 1506 arg := source.NewArgument() 1507 arg.TblDef = t.TblDef 1508 arg.Limit = t.Limit 1509 arg.Offset = t.Offset 1510 v.Arg = arg 1511 default: 1512 return v, moerr.NewInternalErrorNoCtx(fmt.Sprintf("unexpected operator: %v", opr.Op)) 1513 } 1514 return v, nil 1515 } 1516 1517 // convert []types.Type to []*plan.Type 1518 func convertToPlanTypes(ts []types.Type) []plan.Type { 1519 result := make([]plan.Type, len(ts)) 1520 for i, t := range ts { 1521 result[i] = plan.Type{ 1522 Id: int32(t.Oid), 1523 Width: t.Width, 1524 Scale: t.Scale, 1525 } 1526 } 1527 return result 1528 } 1529 1530 // convert []*plan.Type to []types.Type 1531 func convertToTypes(ts []plan.Type) []types.Type { 1532 result := make([]types.Type, len(ts)) 1533 for i, t := range ts { 1534 result[i] = types.New(types.T(t.Id), t.Width, t.Scale) 1535 } 1536 return result 1537 } 1538 1539 // convert []aggexec.AggFuncExecExpression to []*pipeline.Aggregate 1540 func convertToPipelineAggregates(ags []aggexec.AggFuncExecExpression) []*pipeline.Aggregate { 1541 result := make([]*pipeline.Aggregate, len(ags)) 1542 for i, a := range ags { 1543 result[i] = &pipeline.Aggregate{ 1544 Op: a.GetAggID(), 1545 Dist: a.IsDistinct(), 1546 Expr: a.GetArgExpressions(), 1547 Config: a.GetExtraConfig(), 1548 } 1549 } 1550 return result 1551 } 1552 1553 // convert []*pipeline.Aggregate to []aggexec.AggFuncExecExpression 1554 func convertToAggregates(ags []*pipeline.Aggregate) []aggexec.AggFuncExecExpression { 1555 result := make([]aggexec.AggFuncExecExpression, len(ags)) 1556 for i, a := range ags { 1557 result[i] = aggexec.MakeAggFunctionExpression(a.Op, a.Dist, a.Expr, a.Config) 1558 } 1559 return result 1560 } 1561 1562 // get relation list and column list from []colexec.ResultPos 1563 func getRelColList(resultPos []colexec.ResultPos) (relList []int32, colList []int32) { 1564 relList = make([]int32, len(resultPos)) 1565 colList = make([]int32, len(resultPos)) 1566 for i := range resultPos { 1567 relList[i], colList[i] = resultPos[i].Rel, resultPos[i].Pos 1568 } 1569 return 1570 } 1571 1572 // generate []colexec.ResultPos from relation list and column list 1573 func convertToResultPos(relList, colList []int32) []colexec.ResultPos { 1574 res := make([]colexec.ResultPos, len(relList)) 1575 for i := range res { 1576 res[i].Rel, res[i].Pos = relList[i], colList[i] 1577 } 1578 return res 1579 } 1580 1581 // convert process.Limitation to pipeline.ProcessLimitation 1582 func convertToPipelineLimitation(lim process.Limitation) *pipeline.ProcessLimitation { 1583 return &pipeline.ProcessLimitation{ 1584 Size: lim.Size, 1585 BatchRows: lim.BatchRows, 1586 BatchSize: lim.BatchSize, 1587 PartitionRows: lim.PartitionRows, 1588 ReaderSize: lim.ReaderSize, 1589 } 1590 } 1591 1592 // convert pipeline.ProcessLimitation to process.Limitation 1593 func convertToProcessLimitation(lim *pipeline.ProcessLimitation) process.Limitation { 1594 return process.Limitation{ 1595 Size: lim.Size, 1596 BatchRows: lim.BatchRows, 1597 BatchSize: lim.BatchSize, 1598 PartitionRows: lim.PartitionRows, 1599 ReaderSize: lim.ReaderSize, 1600 } 1601 } 1602 1603 // convert pipeline.SessionInfo to process.SessionInfo 1604 func convertToProcessSessionInfo(sei *pipeline.SessionInfo) (process.SessionInfo, error) { 1605 sessionInfo := process.SessionInfo{ 1606 User: sei.User, 1607 Host: sei.Host, 1608 Role: sei.Role, 1609 ConnectionID: sei.ConnectionId, 1610 Database: sei.Database, 1611 Version: sei.Version, 1612 Account: sei.Account, 1613 QueryId: sei.QueryId, 1614 } 1615 t := time.Time{} 1616 err := t.UnmarshalBinary(sei.TimeZone) 1617 if err != nil { 1618 return sessionInfo, nil 1619 } 1620 sessionInfo.TimeZone = t.Location() 1621 return sessionInfo, nil 1622 } 1623 1624 func convertToPlanAnalyzeInfo(info *process.AnalyzeInfo) *plan.AnalyzeInfo { 1625 a := &plan.AnalyzeInfo{ 1626 InputRows: info.InputRows, 1627 OutputRows: info.OutputRows, 1628 InputSize: info.InputSize, 1629 OutputSize: info.OutputSize, 1630 TimeConsumed: info.TimeConsumed, 1631 MemorySize: info.MemorySize, 1632 WaitTimeConsumed: info.WaitTimeConsumed, 1633 DiskIO: info.DiskIO, 1634 S3IOByte: info.S3IOByte, 1635 S3IOInputCount: info.S3IOInputCount, 1636 S3IOOutputCount: info.S3IOOutputCount, 1637 NetworkIO: info.NetworkIO, 1638 ScanTime: info.ScanTime, 1639 InsertTime: info.InsertTime, 1640 } 1641 info.DeepCopyArray(a) 1642 // there are 3 situations to release analyzeInfo 1643 // 1 is free analyzeInfo of Local CN when release analyze 1644 // 2 is free analyzeInfo of remote CN before transfer back 1645 // 3 is free analyzeInfo of remote CN when errors happen before transfer back 1646 // this is situation 2 1647 reuse.Free[process.AnalyzeInfo](info, nil) 1648 return a 1649 } 1650 1651 // func decodeBatch(proc *process.Process, data []byte) (*batch.Batch, error) { 1652 func decodeBatch(mp *mpool.MPool, data []byte) (*batch.Batch, error) { 1653 bat := new(batch.Batch) 1654 if err := bat.UnmarshalBinaryWithCopy(data, mp); err != nil { 1655 bat.Clean(mp) 1656 return nil, err 1657 } 1658 return bat, nil 1659 } 1660 1661 func (ctx *scopeContext) getRegister(id, idx int32) *process.WaitRegister { 1662 if ctx.id == id { 1663 for k, v := range ctx.regs { 1664 if v == idx { 1665 return k 1666 } 1667 } 1668 } 1669 for i := range ctx.children { 1670 if reg := ctx.children[i].getRegister(id, idx); reg != nil { 1671 return reg 1672 } 1673 } 1674 return nil 1675 } 1676 1677 func (ctx *scopeContext) findRegister(reg *process.WaitRegister) (int32, *scopeContext) { 1678 if idx, ok := ctx.regs[reg]; ok { 1679 return idx, ctx 1680 } 1681 for i := range ctx.children { 1682 if idx, ctx := ctx.children[i].findRegister(reg); idx >= 0 { 1683 return idx, ctx 1684 } 1685 } 1686 return -1, nil 1687 } 1688 1689 func EncodeMergeGroup(merge *mergegroup.Argument, pipe *pipeline.Group) { 1690 if !merge.NeedEval || merge.PartialResults == nil { 1691 return 1692 } 1693 pipe.PartialResultTypes = make([]uint32, len(merge.PartialResultTypes)) 1694 pipe.PartialResults = make([]byte, 0) 1695 for i := range pipe.PartialResultTypes { 1696 pipe.PartialResultTypes[i] = uint32(merge.PartialResultTypes[i]) 1697 switch merge.PartialResultTypes[i] { 1698 case types.T_bool: 1699 result := merge.PartialResults[i].(bool) 1700 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1701 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1702 case types.T_bit: 1703 result := merge.PartialResults[i].(uint64) 1704 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1705 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1706 case types.T_int8: 1707 result := merge.PartialResults[i].(int8) 1708 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1709 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1710 case types.T_int16: 1711 result := merge.PartialResults[i].(int16) 1712 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1713 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1714 case types.T_int32: 1715 result := merge.PartialResults[i].(int32) 1716 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1717 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1718 case types.T_int64: 1719 result := merge.PartialResults[i].(int64) 1720 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1721 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1722 case types.T_uint8: 1723 result := merge.PartialResults[i].(uint8) 1724 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1725 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1726 case types.T_uint16: 1727 result := merge.PartialResults[i].(uint16) 1728 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1729 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1730 case types.T_uint32: 1731 result := merge.PartialResults[i].(uint32) 1732 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1733 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1734 case types.T_uint64: 1735 result := merge.PartialResults[i].(uint64) 1736 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1737 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1738 case types.T_float32: 1739 result := merge.PartialResults[i].(float32) 1740 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1741 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1742 case types.T_float64: 1743 result := merge.PartialResults[i].(float64) 1744 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1745 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1746 case types.T_date: 1747 result := merge.PartialResults[i].(types.Date) 1748 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1749 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1750 case types.T_time: 1751 result := merge.PartialResults[i].(types.Time) 1752 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1753 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1754 case types.T_datetime: 1755 result := merge.PartialResults[i].(types.Datetime) 1756 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1757 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1758 case types.T_timestamp: 1759 result := merge.PartialResults[i].(types.Timestamp) 1760 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1761 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1762 case types.T_enum: 1763 result := merge.PartialResults[i].(types.Enum) 1764 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1765 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1766 case types.T_decimal64: 1767 result := merge.PartialResults[i].(types.Decimal64) 1768 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1769 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1770 case types.T_decimal128: 1771 result := merge.PartialResults[i].(types.Decimal128) 1772 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1773 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1774 case types.T_uuid: 1775 result := merge.PartialResults[i].(types.Uuid) 1776 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1777 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1778 case types.T_TS: 1779 result := merge.PartialResults[i].(types.TS) 1780 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1781 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1782 case types.T_Rowid: 1783 result := merge.PartialResults[i].(types.Rowid) 1784 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1785 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1786 case types.T_Blockid: 1787 result := merge.PartialResults[i].(types.Blockid) 1788 bytes := unsafe.Slice((*byte)(unsafe.Pointer(&result)), merge.PartialResultTypes[i].FixedLength()) 1789 pipe.PartialResults = append(pipe.PartialResults, bytes...) 1790 } 1791 } 1792 } 1793 1794 func DecodeMergeGroup(merge *mergegroup.Argument, pipe *pipeline.Group) { 1795 if !pipe.NeedEval || pipe.PartialResults == nil { 1796 return 1797 } 1798 merge.PartialResultTypes = make([]types.T, len(pipe.PartialResultTypes)) 1799 merge.PartialResults = make([]any, 0) 1800 for i := range merge.PartialResultTypes { 1801 merge.PartialResultTypes[i] = types.T(pipe.PartialResultTypes[i]) 1802 switch merge.PartialResultTypes[i] { 1803 case types.T_bool: 1804 result := *(*bool)(unsafe.Pointer(&pipe.PartialResults[0])) 1805 merge.PartialResults = append(merge.PartialResults, result) 1806 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1807 case types.T_bit: 1808 result := *(*uint64)(unsafe.Pointer(&pipe.PartialResults[0])) 1809 merge.PartialResults = append(merge.PartialResults, result) 1810 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1811 case types.T_int8: 1812 result := *(*int8)(unsafe.Pointer(&pipe.PartialResults[0])) 1813 merge.PartialResults = append(merge.PartialResults, result) 1814 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1815 case types.T_int16: 1816 result := *(*int16)(unsafe.Pointer(&pipe.PartialResults[0])) 1817 merge.PartialResults = append(merge.PartialResults, result) 1818 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1819 case types.T_int32: 1820 result := *(*int32)(unsafe.Pointer(&pipe.PartialResults[0])) 1821 merge.PartialResults = append(merge.PartialResults, result) 1822 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1823 case types.T_int64: 1824 result := *(*int64)(unsafe.Pointer(&pipe.PartialResults[0])) 1825 merge.PartialResults = append(merge.PartialResults, result) 1826 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1827 case types.T_uint8: 1828 result := *(*uint8)(unsafe.Pointer(&pipe.PartialResults[0])) 1829 merge.PartialResults = append(merge.PartialResults, result) 1830 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1831 case types.T_uint16: 1832 result := *(*uint16)(unsafe.Pointer(&pipe.PartialResults[0])) 1833 merge.PartialResults = append(merge.PartialResults, result) 1834 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1835 case types.T_uint32: 1836 result := *(*uint32)(unsafe.Pointer(&pipe.PartialResults[0])) 1837 merge.PartialResults = append(merge.PartialResults, result) 1838 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1839 case types.T_uint64: 1840 result := *(*uint64)(unsafe.Pointer(&pipe.PartialResults[0])) 1841 merge.PartialResults = append(merge.PartialResults, result) 1842 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1843 case types.T_float32: 1844 result := *(*float32)(unsafe.Pointer(&pipe.PartialResults[0])) 1845 merge.PartialResults = append(merge.PartialResults, result) 1846 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1847 case types.T_float64: 1848 result := *(*float64)(unsafe.Pointer(&pipe.PartialResults[0])) 1849 merge.PartialResults = append(merge.PartialResults, result) 1850 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1851 case types.T_date: 1852 result := *(*types.Date)(unsafe.Pointer(&pipe.PartialResults[0])) 1853 merge.PartialResults = append(merge.PartialResults, result) 1854 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1855 case types.T_time: 1856 result := *(*types.Time)(unsafe.Pointer(&pipe.PartialResults[0])) 1857 merge.PartialResults = append(merge.PartialResults, result) 1858 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1859 case types.T_datetime: 1860 result := *(*types.Datetime)(unsafe.Pointer(&pipe.PartialResults[0])) 1861 merge.PartialResults = append(merge.PartialResults, result) 1862 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1863 case types.T_timestamp: 1864 result := *(*types.Timestamp)(unsafe.Pointer(&pipe.PartialResults[0])) 1865 merge.PartialResults = append(merge.PartialResults, result) 1866 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1867 case types.T_enum: 1868 result := *(*types.Enum)(unsafe.Pointer(&pipe.PartialResults[0])) 1869 merge.PartialResults = append(merge.PartialResults, result) 1870 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1871 case types.T_decimal64: 1872 result := *(*types.Decimal64)(unsafe.Pointer(&pipe.PartialResults[0])) 1873 merge.PartialResults = append(merge.PartialResults, result) 1874 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1875 case types.T_decimal128: 1876 result := *(*types.Decimal128)(unsafe.Pointer(&pipe.PartialResults[0])) 1877 merge.PartialResults = append(merge.PartialResults, result) 1878 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1879 case types.T_uuid: 1880 result := *(*types.Uuid)(unsafe.Pointer(&pipe.PartialResults[0])) 1881 merge.PartialResults = append(merge.PartialResults, result) 1882 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1883 case types.T_TS: 1884 result := *(*types.TS)(unsafe.Pointer(&pipe.PartialResults[0])) 1885 merge.PartialResults = append(merge.PartialResults, result) 1886 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1887 case types.T_Rowid: 1888 result := *(*types.Rowid)(unsafe.Pointer(&pipe.PartialResults[0])) 1889 merge.PartialResults = append(merge.PartialResults, result) 1890 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1891 case types.T_Blockid: 1892 result := *(*types.Blockid)(unsafe.Pointer(&pipe.PartialResults[0])) 1893 merge.PartialResults = append(merge.PartialResults, result) 1894 pipe.PartialResults = pipe.PartialResults[merge.PartialResultTypes[i].FixedLength():] 1895 } 1896 } 1897 }