github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/process/types.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 process 16 17 import ( 18 "context" 19 "io" 20 "sync" 21 "sync/atomic" 22 "time" 23 24 "github.com/confluentinc/confluent-kafka-go/v2/kafka" 25 "github.com/google/uuid" 26 "github.com/matrixorigin/matrixone/pkg/common/buffer" 27 "github.com/matrixorigin/matrixone/pkg/common/moerr" 28 "github.com/matrixorigin/matrixone/pkg/common/morpc" 29 "github.com/matrixorigin/matrixone/pkg/common/mpool" 30 "github.com/matrixorigin/matrixone/pkg/container/batch" 31 "github.com/matrixorigin/matrixone/pkg/container/types" 32 "github.com/matrixorigin/matrixone/pkg/container/vector" 33 "github.com/matrixorigin/matrixone/pkg/defines" 34 "github.com/matrixorigin/matrixone/pkg/fileservice" 35 "github.com/matrixorigin/matrixone/pkg/incrservice" 36 "github.com/matrixorigin/matrixone/pkg/lockservice" 37 "github.com/matrixorigin/matrixone/pkg/logservice" 38 "github.com/matrixorigin/matrixone/pkg/pb/lock" 39 "github.com/matrixorigin/matrixone/pkg/pb/plan" 40 qclient "github.com/matrixorigin/matrixone/pkg/queryservice/client" 41 "github.com/matrixorigin/matrixone/pkg/txn/client" 42 "github.com/matrixorigin/matrixone/pkg/udf" 43 "github.com/matrixorigin/matrixone/pkg/vm/engine" 44 ) 45 46 const ( 47 VectorLimit = 32 48 ) 49 50 // Analyze analyzes information for operator 51 type Analyze interface { 52 Stop() 53 ChildrenCallStop(time.Time) 54 Start() 55 Alloc(int64) 56 Input(*batch.Batch, bool) 57 Output(*batch.Batch, bool) 58 WaitStop(time.Time) 59 DiskIO(*batch.Batch) 60 S3IOByte(*batch.Batch) 61 S3IOInputCount(int) 62 S3IOOutputCount(int) 63 Network(*batch.Batch) 64 AddScanTime(t time.Time) 65 AddInsertTime(t time.Time) 66 } 67 68 // WaitRegister channel 69 type WaitRegister struct { 70 Ctx context.Context 71 Ch chan *batch.Batch 72 } 73 74 // Register used in execution pipeline and shared with all operators of the same pipeline. 75 type Register struct { 76 // Ss, temporarily stores the row number list in the execution of operators, 77 // and it can be reused in the future execution. 78 Ss [][]int64 79 // InputBatch, stores the result of the previous operator. 80 InputBatch *batch.Batch 81 // MergeReceivers, receives result of multi previous operators from other pipelines 82 // e.g. merge operator. 83 MergeReceivers []*WaitRegister 84 } 85 86 // Limitation specifies the maximum resources that can be used in one query. 87 type Limitation struct { 88 // Size, memory threshold for operator. 89 Size int64 90 // BatchRows, max rows for batch. 91 BatchRows int64 92 // BatchSize, max size for batch. 93 BatchSize int64 94 // PartitionRows, max rows for partition. 95 PartitionRows int64 96 // ReaderSize, memory threshold for storage's reader 97 ReaderSize int64 98 // MaxMessageSize max size for read messages from dn 99 MaxMsgSize uint64 100 } 101 102 // SessionInfo session information 103 type SessionInfo struct { 104 Account string 105 User string 106 Host string 107 Role string 108 ConnectionID uint64 109 AccountId uint32 110 RoleId uint32 111 UserId uint32 112 LastInsertID uint64 113 Database string 114 Version string 115 TimeZone *time.Location 116 StorageEngine engine.Engine 117 QueryId []string 118 ResultColTypes []types.Type 119 SeqCurValues map[uint64]string 120 SeqDeleteKeys []uint64 121 SeqAddValues map[uint64]string 122 SeqLastValue []string 123 SqlHelper sqlHelper 124 Buf *buffer.Buffer 125 SourceInMemScanBatch []*kafka.Message 126 } 127 128 // AnalyzeInfo analyze information for query 129 type AnalyzeInfo struct { 130 // NodeId, index of query's node list 131 NodeId int32 132 // InputRows, number of rows accepted by node 133 InputRows int64 134 // OutputRows, number of rows output by node 135 OutputRows int64 136 // TimeConsumed, time taken by the node in milliseconds 137 TimeConsumed int64 138 // WaitTimeConsumed, time taken by the node waiting for channel in milliseconds 139 WaitTimeConsumed int64 140 // InputSize, data size accepted by node 141 InputSize int64 142 // OutputSize, data size output by node 143 OutputSize int64 144 // MemorySize, memory alloc by node 145 MemorySize int64 146 // DiskIO, data size read from disk 147 DiskIO int64 148 // S3IOByte, data size read from s3 149 S3IOByte int64 150 // S3IOInputCount, count for PUT, COPY, POST and LIST 151 S3IOInputCount int64 152 // S3IOOutputCount, count for GET, SELECT and other 153 S3IOOutputCount int64 154 // NetworkIO, message size send between CN node 155 NetworkIO int64 156 // ScanTime, scan cost time in external scan 157 ScanTime int64 158 // InsertTime, insert cost time in load flow 159 InsertTime int64 160 161 // time consumed by every single parallel 162 mu *sync.Mutex 163 TimeConsumedArrayMajor []int64 164 TimeConsumedArrayMinor []int64 165 } 166 167 type ExecStatus int 168 169 const ( 170 ExecStop = iota 171 ExecNext 172 ExecHasMore 173 ) 174 175 // StmtProfile will be clear for every statement 176 type StmtProfile struct { 177 mu sync.Mutex 178 // sqlSourceType denotes where the sql 179 sqlSourceType string 180 txnId uuid.UUID 181 stmtId uuid.UUID 182 // stmtType 183 stmtType string 184 // queryType 185 queryType string 186 // queryStart is the time when the query starts. 187 queryStart time.Time 188 //the sql from user may have multiple statements 189 //sqlOfStmt is the text part of one statement in the sql 190 sqlOfStmt string 191 } 192 193 func (sp *StmtProfile) Clear() { 194 sp.mu.Lock() 195 defer sp.mu.Unlock() 196 sp.sqlSourceType = "" 197 sp.txnId = uuid.UUID{} 198 sp.stmtId = uuid.UUID{} 199 sp.stmtType = "" 200 sp.queryType = "" 201 sp.sqlOfStmt = "" 202 } 203 204 func (sp *StmtProfile) SetSqlOfStmt(sot string) { 205 sp.mu.Lock() 206 defer sp.mu.Unlock() 207 sp.sqlOfStmt = sot 208 } 209 210 func (sp *StmtProfile) GetSqlOfStmt() string { 211 sp.mu.Lock() 212 defer sp.mu.Unlock() 213 return sp.sqlOfStmt 214 } 215 216 func (sp *StmtProfile) SetQueryStart(t time.Time) { 217 sp.mu.Lock() 218 defer sp.mu.Unlock() 219 sp.queryStart = t 220 } 221 222 func (sp *StmtProfile) GetQueryStart() time.Time { 223 sp.mu.Lock() 224 defer sp.mu.Unlock() 225 return sp.queryStart 226 } 227 228 func (sp *StmtProfile) SetSqlSourceType(st string) { 229 sp.mu.Lock() 230 defer sp.mu.Unlock() 231 sp.sqlSourceType = st 232 } 233 234 func (sp *StmtProfile) GetSqlSourceType() string { 235 return sp.sqlSourceType 236 } 237 238 func (sp *StmtProfile) SetQueryType(qt string) { 239 sp.mu.Lock() 240 defer sp.mu.Unlock() 241 sp.queryType = qt 242 } 243 244 func (sp *StmtProfile) GetQueryType() string { 245 sp.mu.Lock() 246 defer sp.mu.Unlock() 247 return sp.queryType 248 } 249 250 func (sp *StmtProfile) SetStmtType(st string) { 251 sp.mu.Lock() 252 defer sp.mu.Unlock() 253 sp.stmtType = st 254 } 255 256 func (sp *StmtProfile) GetStmtType() string { 257 sp.mu.Lock() 258 defer sp.mu.Unlock() 259 return sp.stmtType 260 } 261 262 func (sp *StmtProfile) SetTxnId(id []byte) { 263 sp.mu.Lock() 264 defer sp.mu.Unlock() 265 copy(sp.txnId[:], id) 266 } 267 268 func (sp *StmtProfile) GetTxnId() uuid.UUID { 269 sp.mu.Lock() 270 defer sp.mu.Unlock() 271 return sp.txnId 272 } 273 274 func (sp *StmtProfile) SetStmtId(id uuid.UUID) { 275 sp.mu.Lock() 276 defer sp.mu.Unlock() 277 copy(sp.stmtId[:], id[:]) 278 } 279 280 func (sp *StmtProfile) GetStmtId() uuid.UUID { 281 sp.mu.Lock() 282 defer sp.mu.Unlock() 283 return sp.stmtId 284 } 285 286 // Process contains context used in query execution 287 // one or more pipeline will be generated for one query, 288 // and one pipeline has one process instance. 289 type Process struct { 290 StmtProfile *StmtProfile 291 // Id, query id. 292 Id string 293 Reg Register 294 Lim Limitation 295 296 vp *vectorPool 297 mp *mpool.MPool 298 prepareBatch *batch.Batch 299 prepareExprList any 300 301 valueScanBatch map[[16]byte]*batch.Batch 302 303 // unix timestamp 304 UnixTime int64 305 306 TxnClient client.TxnClient 307 308 TxnOperator client.TxnOperator 309 310 AnalInfos []*AnalyzeInfo 311 312 SessionInfo SessionInfo 313 314 Ctx context.Context 315 316 Cancel context.CancelFunc 317 318 FileService fileservice.FileService 319 LockService lockservice.LockService 320 IncrService incrservice.AutoIncrementService 321 322 LoadTag bool 323 324 LastInsertID *uint64 325 326 LoadLocalReader *io.PipeReader 327 328 DispatchNotifyCh chan WrapCs 329 330 Aicm *defines.AutoIncrCacheManager 331 332 resolveVariableFunc func(varName string, isSystemVar, isGlobalVar bool) (interface{}, error) 333 prepareParams *vector.Vector 334 335 QueryClient qclient.QueryClient 336 337 Hakeeper logservice.CNHAKeeperClient 338 339 UdfService udf.Service 340 341 WaitPolicy lock.WaitPolicy 342 343 MessageBoard *MessageBoard 344 } 345 346 type vectorPool struct { 347 sync.Mutex 348 vecs map[uint8][]*vector.Vector 349 350 // max vector count limit for each type in pool. 351 Limit int 352 } 353 354 type sqlHelper interface { 355 GetCompilerContext() any 356 ExecSql(string) ([]interface{}, error) 357 GetSubscriptionMeta(string) (sub *plan.SubscriptionMeta, err error) 358 } 359 360 type WrapCs struct { 361 MsgId uint64 362 Uid uuid.UUID 363 Cs morpc.ClientSession 364 Err chan error 365 } 366 367 func (proc *Process) SetStmtProfile(sp *StmtProfile) { 368 proc.StmtProfile = sp 369 } 370 371 func (proc *Process) GetStmtProfile() *StmtProfile { 372 if proc.StmtProfile != nil { 373 return proc.StmtProfile 374 } 375 return &StmtProfile{} 376 } 377 378 func (proc *Process) InitSeq() { 379 proc.SessionInfo.SeqCurValues = make(map[uint64]string) 380 proc.SessionInfo.SeqLastValue = make([]string, 1) 381 proc.SessionInfo.SeqLastValue[0] = "" 382 proc.SessionInfo.SeqAddValues = make(map[uint64]string) 383 proc.SessionInfo.SeqDeleteKeys = make([]uint64, 0) 384 } 385 386 func (proc *Process) SetValueScanBatch(key uuid.UUID, batch *batch.Batch) { 387 proc.valueScanBatch[key] = batch 388 } 389 390 func (proc *Process) GetValueScanBatch(key uuid.UUID) *batch.Batch { 391 bat, ok := proc.valueScanBatch[key] 392 if ok { 393 bat.SetCnt(1000) // make sure this batch wouldn't be cleaned 394 return bat 395 // delete(proc.valueScanBatch, key) 396 } 397 return bat 398 } 399 400 func (proc *Process) CleanValueScanBatchs() { 401 for k, bat := range proc.valueScanBatch { 402 bat.SetCnt(1) 403 bat.Clean(proc.Mp()) 404 delete(proc.valueScanBatch, k) 405 } 406 } 407 408 func (proc *Process) GetValueScanBatchs() []*batch.Batch { 409 var bats []*batch.Batch 410 411 for k, bat := range proc.valueScanBatch { 412 if bat != nil { 413 bats = append(bats, bat) 414 } 415 delete(proc.valueScanBatch, k) 416 } 417 return bats 418 } 419 420 func (proc *Process) GetPrepareParamsAt(i int) ([]byte, error) { 421 if i < 0 || i >= proc.prepareParams.Length() { 422 return nil, moerr.NewInternalError(proc.Ctx, "get prepare params error, index %d not exists", i) 423 } 424 if proc.prepareParams.IsNull(uint64(i)) { 425 return nil, nil 426 } else { 427 val := proc.prepareParams.GetRawBytesAt(i) 428 return val, nil 429 } 430 } 431 432 func (proc *Process) SetResolveVariableFunc(f func(varName string, isSystemVar, isGlobalVar bool) (interface{}, error)) { 433 proc.resolveVariableFunc = f 434 } 435 436 func (proc *Process) GetResolveVariableFunc() func(varName string, isSystemVar, isGlobalVar bool) (interface{}, error) { 437 return proc.resolveVariableFunc 438 } 439 440 func (proc *Process) SetLastInsertID(num uint64) { 441 if proc.LastInsertID != nil { 442 atomic.StoreUint64(proc.LastInsertID, num) 443 } 444 } 445 446 func (proc *Process) GetSessionInfo() *SessionInfo { 447 return &proc.SessionInfo 448 } 449 450 func (proc *Process) GetLastInsertID() uint64 { 451 if proc.LastInsertID != nil { 452 num := atomic.LoadUint64(proc.LastInsertID) 453 return num 454 } 455 return 0 456 } 457 458 func (proc *Process) SetCacheForAutoCol(name string) { 459 aicm := proc.Aicm 460 aicm.Mu.Lock() 461 defer aicm.Mu.Unlock() 462 aicm.AutoIncrCaches[name] = defines.AutoIncrCache{CurNum: 0, MaxNum: aicm.MaxSize, Step: 1} 463 } 464 465 type analyze struct { 466 parallelMajor bool 467 parallelIdx int 468 start time.Time 469 wait time.Duration 470 analInfo *AnalyzeInfo 471 childrenCallDuration time.Duration 472 } 473 474 func (si *SessionInfo) GetUser() string { 475 return si.User 476 } 477 478 func (si *SessionInfo) GetHost() string { 479 return si.Host 480 } 481 482 func (si *SessionInfo) GetUserHost() string { 483 //currently, the host_name is 'localhost' 484 return si.User + "@localhost" 485 } 486 487 func (si *SessionInfo) GetRole() string { 488 return si.Role 489 } 490 491 func (si *SessionInfo) GetCharset() string { 492 return "utf8mb4" 493 } 494 495 func (si *SessionInfo) GetCollation() string { 496 return "utf8mb4_general_ci" 497 } 498 499 func (si *SessionInfo) GetConnectionID() uint64 { 500 return si.ConnectionID 501 } 502 503 func (si *SessionInfo) GetDatabase() string { 504 return si.Database 505 } 506 507 func (si *SessionInfo) GetVersion() string { 508 return si.Version 509 } 510 511 func (a *AnalyzeInfo) AddNewParallel(major bool) int { 512 a.mu.Lock() 513 defer a.mu.Unlock() 514 if major { 515 a.TimeConsumedArrayMajor = append(a.TimeConsumedArrayMajor, 0) 516 return len(a.TimeConsumedArrayMajor) - 1 517 } else { 518 a.TimeConsumedArrayMinor = append(a.TimeConsumedArrayMinor, 0) 519 return len(a.TimeConsumedArrayMinor) - 1 520 } 521 } 522 523 func (a *AnalyzeInfo) DeepCopyArray(pa *plan.AnalyzeInfo) { 524 a.mu.Lock() 525 defer a.mu.Unlock() 526 pa.TimeConsumedArrayMajor = pa.TimeConsumedArrayMajor[:0] 527 pa.TimeConsumedArrayMajor = append(pa.TimeConsumedArrayMajor, a.TimeConsumedArrayMajor...) 528 pa.TimeConsumedArrayMinor = pa.TimeConsumedArrayMinor[:0] 529 pa.TimeConsumedArrayMinor = append(pa.TimeConsumedArrayMinor, a.TimeConsumedArrayMinor...) 530 } 531 532 func (a *AnalyzeInfo) MergeArray(pa *plan.AnalyzeInfo) { 533 a.mu.Lock() 534 defer a.mu.Unlock() 535 a.TimeConsumedArrayMajor = append(a.TimeConsumedArrayMajor, pa.TimeConsumedArrayMajor...) 536 a.TimeConsumedArrayMinor = append(a.TimeConsumedArrayMinor, pa.TimeConsumedArrayMinor...) 537 } 538 539 func (a *AnalyzeInfo) AddSingleParallelTimeConsumed(major bool, parallelIdx int, t int64) { 540 a.mu.Lock() 541 defer a.mu.Unlock() 542 if major { 543 if parallelIdx >= 0 && parallelIdx < len(a.TimeConsumedArrayMajor) { 544 a.TimeConsumedArrayMajor[parallelIdx] += t 545 } 546 } else { 547 if parallelIdx >= 0 && parallelIdx < len(a.TimeConsumedArrayMinor) { 548 a.TimeConsumedArrayMinor[parallelIdx] += t 549 } 550 } 551 }