github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/disttae/logtail_consumer.go (about) 1 // Copyright 2022 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 disttae 16 17 import ( 18 "context" 19 "fmt" 20 "strings" 21 "sync" 22 "sync/atomic" 23 "time" 24 25 "github.com/fagongzi/goetty/v2" 26 27 "github.com/matrixorigin/matrixone/pkg/catalog" 28 "github.com/matrixorigin/matrixone/pkg/common/moerr" 29 "github.com/matrixorigin/matrixone/pkg/common/morpc" 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/logutil" 34 "github.com/matrixorigin/matrixone/pkg/pb/api" 35 "github.com/matrixorigin/matrixone/pkg/pb/logtail" 36 "github.com/matrixorigin/matrixone/pkg/pb/timestamp" 37 "github.com/matrixorigin/matrixone/pkg/txn/client" 38 "github.com/matrixorigin/matrixone/pkg/util/address" 39 v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" 40 "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae/logtailreplay" 41 taeLogtail "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logtail" 42 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logtail/service" 43 ) 44 45 const ( 46 // reconnection related constants. 47 // maxTimeToWaitServerResponse : max time to wait for server response. if time exceed, do reconnection. 48 // retryReconnect : if reconnect tn failed. push client will retry after time retryReconnect. 49 maxTimeToWaitServerResponse = 60 * time.Second 50 retryReconnect = 20 * time.Millisecond 51 52 // push client related constants. 53 // maxSubscribeRequestPerSecond : max number of subscribe request we allowed per second. 54 // defaultRequestDeadline : default deadline for every request (subscribe and unsubscribe). 55 maxSubscribeRequestPerSecond = 10000 56 defaultRequestDeadline = 2 * time.Minute 57 58 // subscribe related constants. 59 // periodToCheckTableSubscribeSucceed : check table subscribe status period after push client send a subscribe request. 60 // maxTimeToCheckTableSubscribeSucceed : max time to wait for table subscribe succeed. if time exceed, return error. 61 periodToCheckTableSubscribeSucceed = 1 * time.Millisecond 62 maxTimeToCheckTableSubscribeSucceed = 30 * time.Second 63 maxCheckRangeTableSubscribeSucceed = int(maxTimeToCheckTableSubscribeSucceed / periodToCheckTableSubscribeSucceed) 64 65 // unsubscribe process related constants. 66 // unsubscribe process scan the table every 20 minutes, and unsubscribe table which was unused for 1 hour. 67 unsubscribeProcessTicker = 20 * time.Minute 68 unsubscribeTimer = 1 * time.Hour 69 70 // gc blocks and BlockIndexByTSEntry in partition state 71 gcPartitionStateTicker = 20 * time.Minute 72 gcPartitionStateTimer = 1 * time.Hour 73 74 // log tail consumer related constants. 75 // if buffer is almost full (percent > consumerWarningPercent, we will send a message to log. 76 consumerNumber = 4 77 consumerBufferLength = 8192 78 consumerWarningPercent = 0.9 79 80 logTag = "[logtail-consumer]" 81 ) 82 83 // PushClient is a structure responsible for all operations related to the log tail push model. 84 // It provides the following methods: 85 // 86 // ----------------------------------------------------------------------------------------------------- 87 // 1. checkTxnTimeIsLegal : block the process until we have received enough log tail (T_log >= T_txn) 88 // 2. TryToSubscribeTable : block the process until we subscribed a table succeed. 89 // 3. subscribeTable : send a table subscribe request to service. 90 // 4. subSysTables : subscribe mo_databases, mo_tables, mo_columns 91 // 5. receiveTableLogTailContinuously : start (1 + consumerNumber) routine to receive log tail from service. 92 // 93 // Watch out for the following points: 94 // 1. if we want to lock both subscriber and subscribed, we should lock subscriber first. 95 // ----------------------------------------------------------------------------------------------------- 96 type PushClient struct { 97 serviceID string 98 // Responsible for sending subscription / unsubscription requests to the service 99 // and receiving the log tail from service. 100 subscriber *logTailSubscriber 101 102 // Record the timestamp of last log received by CN. 103 receivedLogTailTime syncLogTailTimestamp 104 105 // Record the subscription status of a tables. 106 subscribed subscribedTable 107 108 // timestampWaiter is used to notify the latest commit timestamp 109 timestampWaiter client.TimestampWaiter 110 111 // connectC is the channel which is used to control the connection 112 // flow. 113 connector *connector 114 115 // initialized is true means that it is not the first time to init push client. 116 initialized bool 117 118 mu struct { 119 sync.Mutex 120 paused bool 121 } 122 // pauseC is the channel used to control whether the receiver is paused. 123 pauseC chan bool 124 resumeC chan struct{} 125 126 consumeErrC chan error 127 receiver []routineController 128 } 129 130 type State struct { 131 LatestTS timestamp.Timestamp 132 SubTables map[SubTableID]SubTableStatus 133 } 134 135 func (c *PushClient) GetState() State { 136 c.subscribed.mutex.Lock() 137 defer c.subscribed.mutex.Unlock() 138 subTables := make(map[SubTableID]SubTableStatus, len(c.subscribed.m)) 139 for k, v := range c.subscribed.m { 140 subTables[k] = v 141 } 142 return State{ 143 LatestTS: c.receivedLogTailTime.getTimestamp(), 144 SubTables: subTables, 145 } 146 } 147 148 type connector struct { 149 first atomic.Bool 150 signal chan struct{} 151 152 client *PushClient 153 engine *Engine 154 } 155 156 func newConnector(c *PushClient, e *Engine) *connector { 157 co := &connector{ 158 signal: make(chan struct{}), 159 client: c, 160 engine: e, 161 } 162 co.first.Store(true) 163 return co 164 } 165 166 func (c *connector) run(ctx context.Context) { 167 for { 168 select { 169 case <-ctx.Done(): 170 logutil.Infof("%s logtail consumer stopped", logTag) 171 return 172 173 case <-c.signal: 174 c.client.connect(ctx, c.engine) 175 } 176 } 177 } 178 179 func (c *PushClient) init( 180 serviceAddr string, 181 timestampWaiter client.TimestampWaiter, 182 serviceID string, 183 e *Engine, 184 ) error { 185 186 c.serviceID = serviceID 187 c.timestampWaiter = timestampWaiter 188 if c.subscriber == nil { 189 c.subscriber = new(logTailSubscriber) 190 } 191 192 // lock all. 193 // release subscribed lock when init finished. 194 // release subscriber lock when we received enough response from service. 195 c.receivedLogTailTime.e = e 196 c.receivedLogTailTime.ready.Store(false) 197 c.subscriber.setNotReady() 198 c.subscribed.mutex.Lock() 199 defer func() { 200 c.subscribed.mutex.Unlock() 201 }() 202 203 c.receivedLogTailTime.initLogTailTimestamp(timestampWaiter) 204 c.subscribed.m = make(map[SubTableID]SubTableStatus) 205 206 if !c.initialized { 207 c.connector = newConnector(c, e) 208 c.receiver = make([]routineController, consumerNumber) 209 c.consumeErrC = make(chan error, consumerNumber) 210 c.pauseC = make(chan bool, 1) 211 c.resumeC = make(chan struct{}) 212 } 213 c.initialized = true 214 215 return c.subscriber.init(serviceAddr) 216 } 217 218 func (c *PushClient) validLogTailMustApplied(snapshotTS timestamp.Timestamp) { 219 // If the client is not ready, do not check. There is another checking logic 220 // before create a new transaction, so we do not need to check here if it 221 // is not ready yet. 222 if !c.receivedLogTailTime.ready.Load() { 223 return 224 } 225 226 // At the time of transaction creation, a ts is obtained as the start timestamp of the transaction. 227 // To ensure that the complete data is visible at the start of the transaction, the logtail of 228 // all < snapshot ts is waited until it is applied when the transaction is created inside the txn client. 229 // 230 // Inside the txn client, there is a waiter waiting for the LogTail to be applied, which will continuously 231 // receive the ts applied by the PushClient, and then the transaction will use the maximum applied LogTail 232 // ts currently received + 1 as the transaction's snapshot ts to ensure that the transaction can see the 233 // log tails corresponding to the max(applied log tail ts in txn client). 234 // 235 // So here we need to use snapshotTS.Prev() to check. 236 recTS := c.receivedLogTailTime.getTimestamp() 237 if snapshotTS.Prev().LessEq(recTS) { 238 return 239 } 240 241 // If reconnect, receivedLogTailTime will reset. But latestAppliedLogTailTS is always keep the latest applied 242 // logtail ts. 243 ts := c.receivedLogTailTime.latestAppliedLogTailTS.Load() 244 if ts != nil && ts.GreaterEq(snapshotTS.Prev()) { 245 return 246 } 247 panic(fmt.Sprintf("BUG: all log tail must be applied before %s, received applied %s, last applied %+v", 248 snapshotTS.Prev().DebugString(), 249 recTS.DebugString(), 250 ts)) 251 } 252 253 // TryToSubscribeTable subscribe a table and block until subscribe succeed. 254 func (c *PushClient) TryToSubscribeTable( 255 ctx context.Context, 256 dbId, tblId uint64) error { 257 if c.subscribed.getTableSubscribe(dbId, tblId) { 258 return nil 259 } 260 if err := c.subscribeTable(ctx, api.TableID{DbId: dbId, TbId: tblId}); err != nil { 261 return err 262 } 263 ticker := time.NewTicker(periodToCheckTableSubscribeSucceed) 264 defer ticker.Stop() 265 266 for i := 0; i < maxCheckRangeTableSubscribeSucceed; i++ { 267 select { 268 case <-ctx.Done(): 269 return ctx.Err() 270 case <-ticker.C: 271 if c.subscribed.getTableSubscribe(dbId, tblId) { 272 return nil 273 } 274 } 275 } 276 logutil.Debugf("%s didn't receive tbl[db: %d, tbl: %d] subscribe response within %s", 277 logTag, dbId, tblId, maxTimeToCheckTableSubscribeSucceed) 278 return moerr.NewInternalError(ctx, "an error has occurred about table subscription, please try again.") 279 } 280 281 // this method will ignore lock check, subscribe a table and block until subscribe succeed. 282 // developer should use this method carefully. 283 // in most time, developer should use TryToSubscribeTable instead. 284 func (c *PushClient) forcedSubscribeTable( 285 ctx context.Context, 286 dbId, tblId uint64) error { 287 s := c.subscriber 288 289 if err := s.doSubscribe(ctx, api.TableID{DbId: dbId, TbId: tblId}); err != nil { 290 return err 291 } 292 ticker := time.NewTicker(periodToCheckTableSubscribeSucceed) 293 defer ticker.Stop() 294 295 for i := 0; i < maxCheckRangeTableSubscribeSucceed; i++ { 296 select { 297 case <-ctx.Done(): 298 return ctx.Err() 299 case <-ticker.C: 300 if c.subscribed.getTableSubscribe(dbId, tblId) { 301 return nil 302 } 303 } 304 } 305 return moerr.NewInternalError(ctx, "forced subscribe table timeout") 306 } 307 308 func (c *PushClient) subscribeTable( 309 ctx context.Context, tblId api.TableID) error { 310 select { 311 case <-ctx.Done(): 312 return ctx.Err() 313 case b := <-c.subscriber.requestLock: 314 err := c.subscriber.doSubscribe(ctx, tblId) 315 c.subscriber.requestLock <- b 316 if err != nil { 317 return err 318 } 319 logutil.Debugf("%s send subscribe tbl[db: %d, tbl: %d] request succeed", logTag, tblId.DbId, tblId.TbId) 320 return nil 321 } 322 } 323 324 func (c *PushClient) subSysTables(ctx context.Context) error { 325 // push subscription to Table `mo_database`, `mo_table`, `mo_column` of mo_catalog. 326 databaseId := uint64(catalog.MO_CATALOG_ID) 327 tableIds := []uint64{catalog.MO_DATABASE_ID, catalog.MO_TABLES_ID, catalog.MO_COLUMNS_ID} 328 329 var err error 330 for _, ti := range tableIds { 331 err = c.forcedSubscribeTable(ctx, databaseId, ti) 332 if err != nil { 333 break 334 } 335 } 336 337 if err != nil { 338 logutil.Errorf("%s %s: connect to tn log tail server failed, err %v", logTag, c.serviceID, err) 339 } 340 return err 341 } 342 343 func (c *PushClient) pause(s bool) { 344 c.mu.Lock() 345 defer c.mu.Unlock() 346 if c.mu.paused { 347 return 348 } 349 select { 350 case c.pauseC <- s: 351 c.mu.paused = true 352 default: 353 logutil.Infof("%s already set to pause", logTag) 354 } 355 } 356 357 func (c *PushClient) resume() { 358 c.mu.Lock() 359 defer c.mu.Unlock() 360 select { 361 case c.resumeC <- struct{}{}: 362 c.mu.paused = false 363 default: 364 logutil.Infof("%s not in pause state", logTag) 365 } 366 } 367 368 func (c *PushClient) receiveOneLogtail(ctx context.Context, e *Engine) error { 369 ctx, cancel := context.WithTimeout(ctx, maxTimeToWaitServerResponse) 370 defer cancel() 371 372 resp := c.subscriber.receiveResponse(ctx) 373 if resp.err != nil { 374 // POSSIBLE ERROR: context deadline exceeded, rpc closed, decode error. 375 logutil.Errorf("%s receive an error from log tail client, err: %s", logTag, resp.err) 376 return resp.err 377 } 378 379 receiveAt := time.Now() 380 v2.LogtailTotalReceivedCounter.Inc() 381 if res := resp.response.GetSubscribeResponse(); res != nil { // consume subscribe response 382 v2.LogtailSubscribeReceivedCounter.Inc() 383 if err := dispatchSubscribeResponse(ctx, e, res, c.receiver, receiveAt); err != nil { 384 logutil.Errorf("%s dispatch subscribe response failed, err: %s", logTag, err) 385 return err 386 } 387 } else if res := resp.response.GetUpdateResponse(); res != nil { // consume update response 388 if len(res.LogtailList) > 0 { 389 v2.LogtailUpdateReceivedCounter.Inc() 390 } else { 391 v2.LogtailHeartbeatReceivedCounter.Inc() 392 } 393 394 if err := dispatchUpdateResponse(ctx, e, res, c.receiver, receiveAt); err != nil { 395 logutil.Errorf("%s dispatch update response failed, err: %s", logTag, err) 396 return err 397 } 398 } else if unResponse := resp.response.GetUnsubscribeResponse(); unResponse != nil { // consume unsubscribe response 399 v2.LogtailUnsubscribeReceivedCounter.Inc() 400 401 if err := dispatchUnSubscribeResponse(ctx, e, unResponse, c.receiver, receiveAt); err != nil { 402 logutil.Errorf("%s dispatch unsubscribe response failed, err: %s", logTag, err) 403 return err 404 } 405 } 406 return nil 407 } 408 409 func (c *PushClient) receiveLogtails(ctx context.Context, e *Engine) { 410 for { 411 select { 412 case <-ctx.Done(): 413 return 414 415 case s := <-c.pauseC: 416 logutil.Infof("%s logtail receiver paused", logTag) 417 if s { 418 c.sendConnectSig() 419 } 420 421 // Wait for resuming logtail receiver. 422 <-c.resumeC 423 logutil.Infof("%s logtail receiver resumed", logTag) 424 425 default: 426 if err := c.receiveOneLogtail(ctx, e); err != nil { 427 logutil.Errorf("%s receive one logtail failed, err: %v", logTag, err) 428 c.pause(!c.connector.first.Load()) 429 } 430 } 431 } 432 } 433 434 func (c *PushClient) startConsumers(ctx context.Context, e *Engine) { 435 // new parallelNums routine to consume log tails. 436 for i := range c.receiver { 437 c.receiver[i] = c.createRoutineToConsumeLogTails(ctx, i, consumerBufferLength, e) 438 } 439 } 440 441 func (c *PushClient) stopConsumers() { 442 for _, r := range c.receiver { 443 r.close() 444 } 445 logutil.Infof("%s %s: logtail consumers stopped", logTag, c.serviceID) 446 } 447 448 func (c *PushClient) sendConnectSig() { 449 if c.connector.first.Load() { 450 c.connector.signal <- struct{}{} 451 return 452 } 453 454 select { 455 case c.connector.signal <- struct{}{}: 456 logutil.Infof("%s reconnect signal is received", logTag) 457 default: 458 logutil.Infof("%s connecting is in progress", logTag) 459 } 460 } 461 462 func (c *PushClient) run(ctx context.Context, e *Engine) { 463 go c.receiveLogtails(ctx, e) 464 465 // for the first time connector. 466 c.sendConnectSig() 467 468 // A dead loop to receive log tail response from log tail service. 469 // if any error happened, we should do reconnection. 470 for { 471 select { 472 case err := <-c.consumeErrC: 473 // receive an error from sub-routine to consume log. 474 logutil.Errorf("%s consume log tail failed, err: %s", logTag, err) 475 c.pause(!c.connector.first.Load()) 476 477 case <-ctx.Done(): 478 logutil.Infof("%s logtail consumer stopped", logTag) 479 return 480 } 481 } 482 } 483 484 func (c *PushClient) waitTimestamp() { 485 timeout := time.NewTimer(time.Second * 10) 486 defer timeout.Stop() 487 488 ticker := time.NewTicker(time.Millisecond * 2) 489 defer ticker.Stop() 490 for { 491 select { 492 case <-ticker.C: 493 // we should always make sure that all the log tail consume 494 // routines have updated its timestamp. 495 if !c.receivedLogTailTime.getTimestamp().IsEmpty() { 496 return 497 } 498 499 case <-timeout.C: 500 panic("cannot receive timestamp") 501 } 502 } 503 } 504 505 func (c *PushClient) connect(ctx context.Context, e *Engine) { 506 if c.connector.first.Load() { 507 c.startConsumers(ctx, e) 508 509 for { 510 err := c.subSysTables(ctx) 511 if err != nil { 512 c.pause(false) 513 time.Sleep(time.Second) 514 515 tnLogTailServerBackend := e.getTNServices()[0].LogTailServiceAddress 516 if err := c.init(tnLogTailServerBackend, c.timestampWaiter, c.serviceID, e); err != nil { 517 logutil.Errorf("%s init push client failed: %v", logTag, err) 518 continue 519 } 520 521 c.resume() 522 continue 523 } 524 c.waitTimestamp() 525 e.setPushClientStatus(true) 526 c.connector.first.Store(false) 527 return 528 } 529 } 530 531 e.setPushClientStatus(false) 532 533 c.stopConsumers() 534 535 logutil.Infof("%s %s: clean finished, start to reconnect to tn log tail service", logTag, c.serviceID) 536 for { 537 if ctx.Err() != nil { 538 logutil.Infof("%s mo context has done, exit log tail receive routine", logTag) 539 return 540 } 541 542 tnLogTailServerBackend := e.getTNServices()[0].LogTailServiceAddress 543 if err := c.init(tnLogTailServerBackend, c.timestampWaiter, c.serviceID, e); err != nil { 544 logutil.Errorf("%s rebuild the cn log tail client failed, reason: %s", logTag, err) 545 time.Sleep(retryReconnect) 546 continue 547 } 548 logutil.Infof("%s %s: client init finished", logTag, c.serviceID) 549 550 // set all the running transaction to be aborted. 551 e.abortAllRunningTxn() 552 logutil.Infof("%s %s: abort all running transactions finished", logTag, c.serviceID) 553 554 // clean memory table. 555 err := e.init(ctx) 556 if err != nil { 557 logutil.Errorf("%s rebuild memory-table failed, err: %s", logTag, err) 558 time.Sleep(retryReconnect) 559 continue 560 } 561 logutil.Infof("%s %s: clean memory table finished", logTag, c.serviceID) 562 563 // After init, start up again. 564 c.startConsumers(ctx, e) 565 566 c.resume() 567 568 err = c.subSysTables(ctx) 569 if err != nil { 570 logutil.Errorf("%s subscribe system tables failed, err %v", logTag, err) 571 continue 572 } 573 574 c.waitTimestamp() 575 e.setPushClientStatus(true) 576 logutil.Infof("%s %s: connected to server", logTag, c.serviceID) 577 578 return 579 } 580 } 581 582 // UnsubscribeTable implements the LogtailEngine interface. 583 func (c *PushClient) UnsubscribeTable(ctx context.Context, dbID, tbID uint64) error { 584 if !c.receivedLogTailTime.ready.Load() { 585 return moerr.NewInternalError(ctx, "%s cannot unsubscribe table %d-%d as logtail client is not ready", logTag, dbID, tbID) 586 } 587 if c.subscriber == nil { 588 return moerr.NewInternalError(ctx, "%s cannot unsubscribe table %d-%d as subscriber not initialized", logTag, dbID, tbID) 589 } 590 if ifShouldNotDistribute(dbID, tbID) { 591 return moerr.NewInternalError(ctx, "%s cannot unsubscribe table %d-%d as table ID is not allowed", logTag, dbID, tbID) 592 } 593 c.subscribed.mutex.Lock() 594 defer c.subscribed.mutex.Unlock() 595 k := SubTableID{DatabaseID: dbID, TableID: tbID} 596 status, ok := c.subscribed.m[k] 597 if !ok { 598 logutil.Infof("%s table %d-%d is not subscribed yet", logTag, dbID, tbID) 599 return nil 600 } 601 if err := c.subscriber.doUnSubscribe(ctx, api.TableID{DbId: dbID, TbId: tbID}); err != nil { 602 logutil.Errorf("%s cannot unsubscribe table %d-%d, err: %v", logTag, dbID, tbID, err) 603 return err 604 } 605 c.subscribed.m[k] = SubTableStatus{ 606 IsDeleting: true, 607 LatestTime: status.LatestTime, 608 } 609 logutil.Infof("%s send unsubscribe table %d-%d request succeed", logTag, dbID, tbID) 610 return nil 611 } 612 613 func (c *PushClient) unusedTableGCTicker(ctx context.Context) { 614 go func() { 615 ticker := time.NewTicker(unsubscribeProcessTicker) 616 for { 617 select { 618 case <-ctx.Done(): 619 logutil.Infof("%s unsubscribe process exit.", logTag) 620 ticker.Stop() 621 return 622 623 case <-ticker.C: 624 if !c.receivedLogTailTime.ready.Load() { 625 continue 626 } 627 if c.subscriber == nil { 628 continue 629 } 630 } 631 shouldClean := time.Now().Add(-unsubscribeTimer) 632 633 // lock the subscriber and subscribed map. 634 b := <-c.subscriber.requestLock 635 c.subscribed.mutex.Lock() 636 logutil.Infof("%s start to unsubscribe unused table", logTag) 637 func() { 638 defer func() { 639 c.subscriber.requestLock <- b 640 c.subscribed.mutex.Unlock() 641 }() 642 643 var err error 644 for k, v := range c.subscribed.m { 645 if ifShouldNotDistribute(k.DatabaseID, k.TableID) { 646 // never unsubscribe the mo_databases, mo_tables, mo_columns. 647 continue 648 } 649 650 if !v.LatestTime.After(shouldClean) { 651 if err = c.subscriber.doUnSubscribe(ctx, api.TableID{DbId: k.DatabaseID, TbId: k.TableID}); err == nil { 652 c.subscribed.m[k] = SubTableStatus{ 653 IsDeleting: true, 654 LatestTime: v.LatestTime, 655 } 656 logutil.Debugf("%s send unsubscribe tbl[db: %d, tbl: %d] request succeed", logTag, k.DatabaseID, k.TableID) 657 continue 658 } 659 logutil.Errorf("%s sign tbl[dbId: %d, tblId: %d] unsubscribing failed, err : %s", logTag, k.DatabaseID, k.TableID, err.Error()) 660 break 661 } 662 } 663 }() 664 665 logutil.Infof("%s unsubscribe unused table finished.", logTag) 666 } 667 }() 668 } 669 670 func (c *PushClient) partitionStateGCTicker(ctx context.Context, e *Engine) { 671 go func() { 672 ticker := time.NewTicker(gcPartitionStateTicker) 673 for { 674 select { 675 case <-ctx.Done(): 676 logutil.Infof("%s GC partition_state process exit.", logTag) 677 ticker.Stop() 678 return 679 680 case <-ticker.C: 681 if !c.receivedLogTailTime.ready.Load() { 682 continue 683 } 684 if c.subscriber == nil { 685 continue 686 } 687 } 688 parts := make(map[[2]uint64]*logtailreplay.Partition) 689 e.Lock() 690 for ids, part := range e.partitions { 691 parts[ids] = part 692 } 693 e.Unlock() 694 ts := types.BuildTS(time.Now().UTC().UnixNano()-gcPartitionStateTimer.Nanoseconds()*5, 0) 695 logutil.Infof("%s GC partition_state %v", logTag, ts.ToString()) 696 for ids, part := range parts { 697 part.Truncate(ctx, ids, ts) 698 part.UpdateStart(ts) 699 } 700 } 701 }() 702 } 703 704 type SubTableID struct { 705 DatabaseID uint64 706 TableID uint64 707 } 708 709 // subscribedTable used to record table subscribed status. 710 // only if m[table T] = true, T has been subscribed. 711 type subscribedTable struct { 712 mutex sync.Mutex 713 714 // value is table's latest use time. 715 m map[SubTableID]SubTableStatus 716 } 717 718 type SubTableStatus struct { 719 IsDeleting bool 720 LatestTime time.Time 721 } 722 723 func (s *subscribedTable) getTableSubscribe(dbId, tblId uint64) bool { 724 s.mutex.Lock() 725 defer s.mutex.Unlock() 726 status, ok := s.m[SubTableID{DatabaseID: dbId, TableID: tblId}] 727 if ok { 728 if status.IsDeleting { 729 ok = false 730 } else { 731 s.m[SubTableID{DatabaseID: dbId, TableID: tblId}] = SubTableStatus{ 732 IsDeleting: false, 733 LatestTime: time.Now(), 734 } 735 } 736 } 737 return ok 738 } 739 740 func (s *subscribedTable) setTableSubscribe(dbId, tblId uint64) { 741 s.mutex.Lock() 742 defer s.mutex.Unlock() 743 s.m[SubTableID{DatabaseID: dbId, TableID: tblId}] = SubTableStatus{ 744 IsDeleting: false, 745 LatestTime: time.Now(), 746 } 747 logutil.Infof("%s subscribe tbl[db: %d, tbl: %d] succeed", logTag, dbId, tblId) 748 } 749 750 func (s *subscribedTable) setTableUnsubscribe(dbId, tblId uint64) { 751 s.mutex.Lock() 752 defer s.mutex.Unlock() 753 delete(s.m, SubTableID{DatabaseID: dbId, TableID: tblId}) 754 logutil.Infof("%s unsubscribe tbl[db: %d, tbl: %d] succeed", logTag, dbId, tblId) 755 } 756 757 // syncLogTailTimestamp is a global log tail timestamp for a cn node. 758 // support `getTimestamp()` method to get time of last received log. 759 type syncLogTailTimestamp struct { 760 timestampWaiter client.TimestampWaiter 761 ready atomic.Bool 762 tList []atomic.Pointer[timestamp.Timestamp] 763 latestAppliedLogTailTS atomic.Pointer[timestamp.Timestamp] 764 e *Engine 765 } 766 767 func (r *syncLogTailTimestamp) initLogTailTimestamp(timestampWaiter client.TimestampWaiter) { 768 ts := r.getTimestamp() 769 if !ts.IsEmpty() { 770 r.latestAppliedLogTailTS.Store(&ts) 771 } 772 773 r.timestampWaiter = timestampWaiter 774 if len(r.tList) == 0 { 775 r.tList = make([]atomic.Pointer[timestamp.Timestamp], consumerNumber+1) 776 } 777 for i := range r.tList { 778 r.tList[i].Store(new(timestamp.Timestamp)) 779 } 780 } 781 782 func (r *syncLogTailTimestamp) getTimestamp() timestamp.Timestamp { 783 var minT timestamp.Timestamp 784 for i := 0; i < len(r.tList); i++ { 785 t := *r.tList[i].Load() 786 if i == 0 { 787 minT = t 788 } else { 789 if t.Less(minT) { 790 minT = t 791 } 792 } 793 } 794 return minT 795 } 796 797 func (r *syncLogTailTimestamp) updateTimestamp( 798 index int, 799 newTimestamp timestamp.Timestamp, 800 receiveAt time.Time) { 801 start := time.Now() 802 v2.LogTailApplyNotifyLatencyDurationHistogram.Observe(start.Sub(receiveAt).Seconds()) 803 defer func() { 804 v2.LogTailApplyNotifyDurationHistogram.Observe(time.Since(start).Seconds()) 805 }() 806 r.tList[index].Store(&newTimestamp) 807 if r.ready.Load() { 808 ts := r.getTimestamp() 809 r.timestampWaiter.NotifyLatestCommitTS(ts) 810 } 811 } 812 813 type logTailSubscriber struct { 814 tnNodeID int 815 rpcClient morpc.RPCClient 816 rpcStream morpc.Stream 817 logTailClient *service.LogtailClient 818 819 ready bool 820 821 requestLock chan bool 822 doSubscribe func(context.Context, api.TableID) error 823 doUnSubscribe func(context.Context, api.TableID) error 824 } 825 826 func clientIsPreparing(context.Context, api.TableID) error { 827 return moerr.NewInternalErrorNoCtx("log tail client is not ready") 828 } 829 830 type logTailSubscriberResponse struct { 831 response *service.LogtailResponse 832 err error 833 } 834 835 // XXX generate a rpc client and new a stream. 836 // we should hide these code into service's NewClient method next day. 837 func (s *logTailSubscriber) newRpcStreamToTnLogTailService(serviceAddr string) error { 838 if s.rpcClient == nil { 839 logger := logutil.GetGlobalLogger().Named("cn-log-tail-client") 840 codec := morpc.NewMessageCodec(func() morpc.Message { 841 return &service.LogtailResponseSegment{} 842 }) 843 factory := morpc.NewGoettyBasedBackendFactory(codec, 844 morpc.WithBackendGoettyOptions( 845 goetty.WithSessionRWBUfferSize(1<<20, 1<<20), 846 ), 847 morpc.WithBackendLogger(logger), 848 ) 849 850 c, err := morpc.NewClient( 851 "logtail-client", 852 factory, 853 morpc.WithClientLogger(logger), 854 ) 855 if err != nil { 856 return err 857 } 858 s.rpcClient = c 859 } 860 861 if s.rpcStream != nil { 862 s.rpcStream.Close(true) 863 s.rpcStream = nil 864 } 865 866 stream, err := s.rpcClient.NewStream(serviceAddr, true) 867 if err != nil { 868 return err 869 } 870 871 s.rpcStream = stream 872 return nil 873 } 874 875 func (s *logTailSubscriber) init(serviceAddr string) (err error) { 876 // XXX we assume that we have only 1 tn now. 877 s.tnNodeID = 0 878 879 // clear the old status. 880 s.doSubscribe = clientIsPreparing 881 s.doUnSubscribe = clientIsPreparing 882 if s.logTailClient != nil { 883 _ = s.logTailClient.Close() 884 s.logTailClient = nil 885 } 886 887 if err := s.newRpcStreamToTnLogTailService(serviceAddr); err != nil { 888 return err 889 } 890 891 // new the log tail client. 892 s.logTailClient, err = service.NewLogtailClient(s.rpcStream, service.WithClientRequestPerSecond(maxSubscribeRequestPerSecond)) 893 if err != nil { 894 return err 895 } 896 897 s.doSubscribe = s.subscribeTable 898 s.doUnSubscribe = s.unSubscribeTable 899 if s.requestLock == nil { 900 s.requestLock = make(chan bool, 1) 901 s.ready = false 902 } 903 return nil 904 } 905 906 func (s *logTailSubscriber) setReady() { 907 if !s.ready && s.requestLock != nil { 908 s.requestLock <- true 909 s.ready = true 910 } 911 } 912 913 func (s *logTailSubscriber) setNotReady() { 914 if s.ready && s.requestLock != nil { 915 <-s.requestLock 916 s.ready = false 917 } 918 } 919 920 // can't call this method directly. 921 func (s *logTailSubscriber) subscribeTable( 922 ctx context.Context, tblId api.TableID) error { 923 // set a default deadline for ctx if it doesn't have. 924 if _, ok := ctx.Deadline(); !ok { 925 newCtx, cancel := context.WithTimeout(ctx, defaultRequestDeadline) 926 _ = cancel 927 return s.logTailClient.Subscribe(newCtx, tblId) 928 } 929 return s.logTailClient.Subscribe(ctx, tblId) 930 } 931 932 // can't call this method directly. 933 func (s *logTailSubscriber) unSubscribeTable( 934 ctx context.Context, tblId api.TableID) error { 935 // set a default deadline for ctx if it doesn't have. 936 if _, ok := ctx.Deadline(); !ok { 937 newCtx, cancel := context.WithTimeout(ctx, defaultRequestDeadline) 938 _ = cancel 939 return s.logTailClient.Unsubscribe(newCtx, tblId) 940 } 941 return s.logTailClient.Unsubscribe(ctx, tblId) 942 } 943 944 func (s *logTailSubscriber) receiveResponse(deadlineCtx context.Context) logTailSubscriberResponse { 945 r, err := s.logTailClient.Receive(deadlineCtx) 946 resp := logTailSubscriberResponse{ 947 response: r, 948 err: err, 949 } 950 return resp 951 } 952 953 func waitServerReady(addr string) { 954 dialTimeout := time.Second * 2 955 // If the logtail server is ready, just return and do not wait. 956 if address.RemoteAddressAvail(addr, dialTimeout) { 957 return 958 } 959 960 // If we still cannot connect to logtail server for serverTimeout, we consider 961 // it has something wrong happened and panic immediately. 962 serverTimeout := time.Minute * 5 963 serverFatal := time.NewTimer(serverTimeout) 964 defer serverFatal.Stop() 965 966 timer := time.NewTimer(time.Second) 967 defer timer.Stop() 968 969 var resetTimout time.Duration 970 started := time.Now() 971 972 for { 973 current := time.Now() 974 // Calculation the proper reset timeout duration. 975 if current.Sub(started) < time.Minute { 976 resetTimout = time.Second 977 } else if current.Sub(started) < time.Minute*3 { 978 resetTimout = time.Second * 10 979 } else { 980 resetTimout = time.Second * 30 981 } 982 983 select { 984 case <-timer.C: 985 if address.RemoteAddressAvail(addr, dialTimeout) { 986 return 987 } 988 timer.Reset(resetTimout) 989 logutil.Warnf("%s logtail server is not ready yet", logTag) 990 991 case <-serverFatal.C: 992 panic(fmt.Sprintf("could not connect to logtail server for %s", serverTimeout)) 993 } 994 } 995 } 996 997 func (e *Engine) InitLogTailPushModel(ctx context.Context, timestampWaiter client.TimestampWaiter) error { 998 tnStores := e.getTNServices() 999 if len(tnStores) == 0 { 1000 return moerr.NewInternalError(ctx, "no TN store found") 1001 } 1002 1003 logTailServerAddr := tnStores[0].LogTailServiceAddress 1004 1005 // Wait for logtail server is ready. 1006 waitServerReady(logTailServerAddr) 1007 1008 // try to init log tail client. if failed, retry. 1009 for { 1010 if err := ctx.Err(); err != nil { 1011 logutil.Infof("%s mo context has done, init log tail client failed.", logTag) 1012 return err 1013 } 1014 1015 // get log tail service address. 1016 if err := e.pClient.init(logTailServerAddr, timestampWaiter, e.ls.GetServiceID(), e); err != nil { 1017 logutil.Errorf("%s client init failed, err is %s", logTag, err) 1018 continue 1019 } 1020 break 1021 } 1022 1023 go e.pClient.connector.run(ctx) 1024 1025 // Start a goroutine that never stops to receive logtail from TN logtail server. 1026 go e.pClient.run(ctx, e) 1027 1028 e.pClient.unusedTableGCTicker(ctx) 1029 e.pClient.partitionStateGCTicker(ctx, e) 1030 return nil 1031 } 1032 1033 func ifShouldNotDistribute(dbId, tblId uint64) bool { 1034 return dbId == catalog.MO_CATALOG_ID && tblId <= catalog.MO_RESERVED_MAX 1035 } 1036 1037 func dispatchSubscribeResponse( 1038 ctx context.Context, 1039 e *Engine, 1040 response *logtail.SubscribeResponse, 1041 recRoutines []routineController, 1042 receiveAt time.Time) error { 1043 lt := response.Logtail 1044 tbl := lt.GetTable() 1045 1046 notDistribute := ifShouldNotDistribute(tbl.DbId, tbl.TbId) 1047 if notDistribute { 1048 // time check for issue #10833. 1049 startTime := time.Now() 1050 defer func() { 1051 tDuration := time.Since(startTime) 1052 if tDuration > time.Millisecond*5 { 1053 logutil.Warnf("%s consume subscribe response for tbl[dbId: %d, tblID: %d] cost %s", 1054 logTag, tbl.DbId, tbl.TbId, tDuration.String()) 1055 } 1056 }() 1057 1058 if err := e.consumeSubscribeResponse(ctx, response, false, receiveAt); err != nil { 1059 return err 1060 } 1061 if len(lt.CkpLocation) == 0 { 1062 p := e.getOrCreateLatestPart(tbl.DbId, tbl.TbId) 1063 p.UpdateDuration(types.TS{}, types.MaxTs()) 1064 c := e.getLatestCatalogCache() 1065 c.UpdateDuration(types.TS{}, types.MaxTs()) 1066 } 1067 e.pClient.subscribed.setTableSubscribe(tbl.DbId, tbl.TbId) 1068 } else { 1069 routineIndex := tbl.TbId % consumerNumber 1070 recRoutines[routineIndex].sendSubscribeResponse(ctx, response, receiveAt) 1071 } 1072 // no matter how we consume the response, should update all timestamp. 1073 e.pClient.receivedLogTailTime.updateTimestamp(consumerNumber, *lt.Ts, receiveAt) 1074 for _, rc := range recRoutines { 1075 rc.updateTimeFromT(*lt.Ts, receiveAt) 1076 } 1077 return nil 1078 } 1079 1080 func dispatchUpdateResponse( 1081 ctx context.Context, 1082 e *Engine, 1083 response *logtail.UpdateResponse, 1084 recRoutines []routineController, 1085 receiveAt time.Time) error { 1086 list := response.GetLogtailList() 1087 1088 // loops for mo_database, mo_tables, mo_columns. 1089 for i := 0; i < len(list); i++ { 1090 table := list[i].Table 1091 if table.TbId == catalog.MO_DATABASE_ID { 1092 if err := e.consumeUpdateLogTail(ctx, list[i], false, receiveAt); err != nil { 1093 return err 1094 } 1095 } 1096 } 1097 for i := 0; i < len(list); i++ { 1098 table := list[i].Table 1099 if table.TbId == catalog.MO_TABLES_ID { 1100 if err := e.consumeUpdateLogTail(ctx, list[i], false, receiveAt); err != nil { 1101 return err 1102 } 1103 } 1104 } 1105 for i := 0; i < len(list); i++ { 1106 table := list[i].Table 1107 if table.TbId == catalog.MO_COLUMNS_ID { 1108 if err := e.consumeUpdateLogTail(ctx, list[i], false, receiveAt); err != nil { 1109 return err 1110 } 1111 } 1112 } 1113 1114 for index := 0; index < len(list); index++ { 1115 table := list[index].Table 1116 if ifShouldNotDistribute(table.DbId, table.TbId) { 1117 continue 1118 } 1119 recIndex := table.TbId % consumerNumber 1120 recRoutines[recIndex].sendTableLogTail(list[index], receiveAt) 1121 } 1122 // should update all the timestamp. 1123 e.pClient.receivedLogTailTime.updateTimestamp(consumerNumber, *response.To, receiveAt) 1124 for _, rc := range recRoutines { 1125 rc.updateTimeFromT(*response.To, receiveAt) 1126 } 1127 1128 n := 0 1129 for _, c := range recRoutines { 1130 n += len(c.signalChan) 1131 } 1132 v2.LogTailApplyQueueSizeGauge.Set(float64(n)) 1133 return nil 1134 } 1135 1136 func dispatchUnSubscribeResponse( 1137 _ context.Context, 1138 _ *Engine, 1139 response *logtail.UnSubscribeResponse, 1140 recRoutines []routineController, 1141 receiveAt time.Time) error { 1142 tbl := response.Table 1143 notDistribute := ifShouldNotDistribute(tbl.DbId, tbl.TbId) 1144 if notDistribute { 1145 logutil.Errorf("%s unexpected unsubscribe response for tbl[dbId: %d, tblID: %d]", 1146 logTag, tbl.DbId, tbl.TbId) 1147 return nil 1148 } 1149 routineIndex := tbl.TbId % consumerNumber 1150 recRoutines[routineIndex].sendUnSubscribeResponse(response, receiveAt) 1151 1152 return nil 1153 } 1154 1155 type routineController struct { 1156 routineId int 1157 closeChan chan bool 1158 signalChan chan routineControlCmd 1159 1160 // monitor the consumption speed of logs. 1161 warningBufferLen int 1162 } 1163 1164 func (rc *routineController) sendSubscribeResponse( 1165 _ context.Context, 1166 r *logtail.SubscribeResponse, 1167 receiveAt time.Time) { 1168 if l := len(rc.signalChan); l > rc.warningBufferLen { 1169 rc.warningBufferLen = l 1170 logutil.Infof("%s consume-routine %d signalChan len is %d, maybe consume is too slow", logTag, rc.routineId, l) 1171 } 1172 1173 rc.signalChan <- cmdToConsumeSub{log: r, receiveAt: receiveAt} 1174 } 1175 1176 func (rc *routineController) sendTableLogTail(r logtail.TableLogtail, receiveAt time.Time) { 1177 if l := len(rc.signalChan); l > rc.warningBufferLen { 1178 rc.warningBufferLen = l 1179 logutil.Infof("%s consume-routine %d signalChan len is %d, maybe consume is too slow", logTag, rc.routineId, l) 1180 } 1181 1182 rc.signalChan <- cmdToConsumeLog{log: r, receiveAt: receiveAt} 1183 } 1184 1185 func (rc *routineController) updateTimeFromT( 1186 t timestamp.Timestamp, 1187 receiveAt time.Time) { 1188 if l := len(rc.signalChan); l > rc.warningBufferLen { 1189 rc.warningBufferLen = l 1190 logutil.Infof("%s consume-routine %d signalChan len is %d, maybe consume is too slow", logTag, rc.routineId, l) 1191 } 1192 1193 rc.signalChan <- cmdToUpdateTime{time: t, receiveAt: receiveAt} 1194 } 1195 1196 func (rc *routineController) sendUnSubscribeResponse(r *logtail.UnSubscribeResponse, receiveAt time.Time) { 1197 // debug for issue #10138. 1198 if l := len(rc.signalChan); l > rc.warningBufferLen { 1199 rc.warningBufferLen = l 1200 logutil.Infof("%s consume-routine %d signalChan len is %d, maybe consume is too slow", logTag, rc.routineId, l) 1201 } 1202 1203 rc.signalChan <- cmdToConsumeUnSub{log: r, receiveAt: receiveAt} 1204 } 1205 1206 func (rc *routineController) close() { 1207 rc.closeChan <- true 1208 } 1209 1210 func (c *PushClient) createRoutineToConsumeLogTails( 1211 ctx context.Context, routineId int, signalBufferLength int, e *Engine, 1212 ) routineController { 1213 1214 singleRoutineToConsumeLogTail := func(ctx context.Context, engine *Engine, receiver *routineController, errRet chan error) { 1215 errHappen := false 1216 for { 1217 select { 1218 case cmd := <-receiver.signalChan: 1219 if errHappen { 1220 continue 1221 } 1222 if err := cmd.action(ctx, engine, receiver); err != nil { 1223 errHappen = true 1224 errRet <- err 1225 } 1226 1227 case <-receiver.closeChan: 1228 close(receiver.closeChan) 1229 close(receiver.signalChan) 1230 return 1231 } 1232 } 1233 } 1234 1235 controller := routineController{ 1236 routineId: routineId, 1237 closeChan: make(chan bool), 1238 signalChan: make(chan routineControlCmd, signalBufferLength), 1239 1240 // Debug for issue #10138. 1241 warningBufferLen: int(float64(signalBufferLength) * consumerWarningPercent), 1242 } 1243 1244 go singleRoutineToConsumeLogTail(ctx, e, &controller, c.consumeErrC) 1245 1246 return controller 1247 } 1248 1249 // a signal to control the routine which is responsible for consuming log tail. 1250 type routineControlCmd interface { 1251 action(ctx context.Context, e *Engine, ctrl *routineController) error 1252 } 1253 1254 type cmdToConsumeSub struct { 1255 log *logtail.SubscribeResponse 1256 receiveAt time.Time 1257 } 1258 type cmdToConsumeLog struct { 1259 log logtail.TableLogtail 1260 receiveAt time.Time 1261 } 1262 type cmdToUpdateTime struct { 1263 time timestamp.Timestamp 1264 receiveAt time.Time 1265 } 1266 type cmdToConsumeUnSub struct { 1267 log *logtail.UnSubscribeResponse 1268 receiveAt time.Time 1269 } 1270 1271 func (cmd cmdToConsumeSub) action(ctx context.Context, e *Engine, ctrl *routineController) error { 1272 response := cmd.log 1273 if err := e.consumeSubscribeResponse(ctx, response, true, cmd.receiveAt); err != nil { 1274 return err 1275 } 1276 lt := response.GetLogtail() 1277 tbl := lt.GetTable() 1278 e.pClient.subscribed.setTableSubscribe(tbl.DbId, tbl.TbId) 1279 return nil 1280 } 1281 1282 func (cmd cmdToConsumeLog) action(ctx context.Context, e *Engine, ctrl *routineController) error { 1283 response := cmd.log 1284 if err := e.consumeUpdateLogTail(ctx, response, true, cmd.receiveAt); err != nil { 1285 return err 1286 } 1287 return nil 1288 } 1289 1290 func (cmd cmdToUpdateTime) action(ctx context.Context, e *Engine, ctrl *routineController) error { 1291 e.pClient.receivedLogTailTime.updateTimestamp(ctrl.routineId, cmd.time, cmd.receiveAt) 1292 return nil 1293 } 1294 1295 func (cmd cmdToConsumeUnSub) action(ctx context.Context, e *Engine, _ *routineController) error { 1296 table := cmd.log.Table 1297 e.cleanMemoryTableWithTable(table.DbId, table.TbId) 1298 e.pClient.subscribed.setTableUnsubscribe(table.DbId, table.TbId) 1299 return nil 1300 } 1301 1302 func (e *Engine) consumeSubscribeResponse( 1303 ctx context.Context, 1304 rp *logtail.SubscribeResponse, 1305 lazyLoad bool, 1306 receiveAt time.Time) error { 1307 lt := rp.GetLogtail() 1308 return updatePartitionOfPush(ctx, e, <, lazyLoad, receiveAt) 1309 } 1310 1311 func (e *Engine) consumeUpdateLogTail( 1312 ctx context.Context, 1313 rp logtail.TableLogtail, 1314 lazyLoad bool, 1315 receiveAt time.Time) error { 1316 return updatePartitionOfPush(ctx, e, &rp, lazyLoad, receiveAt) 1317 } 1318 1319 // updatePartitionOfPush is the partition update method of log tail push model. 1320 func updatePartitionOfPush( 1321 ctx context.Context, 1322 e *Engine, 1323 tl *logtail.TableLogtail, 1324 lazyLoad bool, 1325 receiveAt time.Time) (err error) { 1326 start := time.Now() 1327 v2.LogTailApplyLatencyDurationHistogram.Observe(start.Sub(receiveAt).Seconds()) 1328 defer func() { 1329 v2.LogTailApplyDurationHistogram.Observe(time.Since(start).Seconds()) 1330 }() 1331 1332 // after consume the logtail, enqueue it to global stats. 1333 defer func() { 1334 t0 := time.Now() 1335 e.globalStats.enqueue(tl) 1336 v2.LogtailUpdatePartitonEnqueueGlobalStatsDurationHistogram.Observe(time.Since(t0).Seconds()) 1337 }() 1338 1339 // get table info by table id 1340 dbId, tblId := tl.Table.GetDbId(), tl.Table.GetTbId() 1341 1342 t0 := time.Now() 1343 partition := e.getOrCreateLatestPart(dbId, tblId) 1344 v2.LogtailUpdatePartitonGetPartitionDurationHistogram.Observe(time.Since(t0).Seconds()) 1345 1346 t0 = time.Now() 1347 lockErr := partition.Lock(ctx) 1348 if lockErr != nil { 1349 v2.LogtailUpdatePartitonGetLockDurationHistogram.Observe(time.Since(t0).Seconds()) 1350 return lockErr 1351 } 1352 defer partition.Unlock() 1353 v2.LogtailUpdatePartitonGetLockDurationHistogram.Observe(time.Since(t0).Seconds()) 1354 1355 catalogCache := e.getLatestCatalogCache() 1356 1357 if !partition.TableInfoOK { 1358 t0 = time.Now() 1359 tableInfo := catalogCache.GetTableById(dbId, tblId) 1360 partition.TableInfo.ID = tblId 1361 partition.TableInfo.Name = tableInfo.Name 1362 partition.TableInfo.PrimarySeqnum = tableInfo.PrimarySeqnum 1363 partition.TableInfoOK = true 1364 v2.LogtailUpdatePartitonGetCatalogDurationHistogram.Observe(time.Since(t0).Seconds()) 1365 } 1366 1367 state, doneMutate := partition.MutateState() 1368 1369 var ( 1370 ckpStart types.TS 1371 ckpEnd types.TS 1372 ) 1373 1374 if lazyLoad { 1375 if len(tl.CkpLocation) > 0 { 1376 t0 = time.Now() 1377 //TODO:: 1378 ckpStart, ckpEnd = parseCkpDuration(tl) 1379 if !ckpStart.IsEmpty() && !ckpEnd.IsEmpty() { 1380 state.CacheCkpDuration(ckpStart, ckpEnd, partition) 1381 } 1382 state.AppendCheckpoint(tl.CkpLocation, partition) 1383 v2.LogtailUpdatePartitonHandleCheckpointDurationHistogram.Observe(time.Since(t0).Seconds()) 1384 } 1385 1386 t0 = time.Now() 1387 err = consumeLogTail( 1388 ctx, 1389 partition.TableInfo.PrimarySeqnum, 1390 e, 1391 state, 1392 tl, 1393 ) 1394 v2.LogtailUpdatePartitonConsumeLogtailDurationHistogram.Observe(time.Since(t0).Seconds()) 1395 1396 } else { 1397 if len(tl.CkpLocation) > 0 { 1398 t0 = time.Now() 1399 //TODO:: 1400 ckpStart, ckpEnd = parseCkpDuration(tl) 1401 v2.LogtailUpdatePartitonHandleCheckpointDurationHistogram.Observe(time.Since(t0).Seconds()) 1402 } 1403 t0 = time.Now() 1404 err = consumeCkpsAndLogTail(ctx, partition.TableInfo.PrimarySeqnum, e, state, tl, dbId, tblId, partition.TableInfo.Name) 1405 v2.LogtailUpdatePartitonConsumeLogtailDurationHistogram.Observe(time.Since(t0).Seconds()) 1406 } 1407 1408 if err != nil { 1409 logutil.Errorf("%s consume %d-%s log tail error: %v\n", logTag, tblId, partition.TableInfo.Name, err) 1410 return err 1411 } 1412 1413 //After consume checkpoints finished ,then update the start and end of 1414 //the mo system table's partition and catalog. 1415 if !lazyLoad && len(tl.CkpLocation) != 0 { 1416 if !ckpStart.IsEmpty() || !ckpEnd.IsEmpty() { 1417 t0 = time.Now() 1418 partition.UpdateDuration(ckpStart, types.MaxTs()) 1419 //Notice that the checkpoint duration is same among all mo system tables, 1420 //such as mo_databases, mo_tables, mo_columns. 1421 catalogCache.UpdateDuration(ckpStart, types.MaxTs()) 1422 v2.LogtailUpdatePartitonUpdateTimestampsDurationHistogram.Observe(time.Since(t0).Seconds()) 1423 } 1424 } 1425 1426 doneMutate() 1427 1428 return nil 1429 } 1430 1431 func consumeLogTail( 1432 ctx context.Context, 1433 primarySeqnum int, 1434 engine *Engine, 1435 state *logtailreplay.PartitionState, 1436 lt *logtail.TableLogtail, 1437 ) error { 1438 return hackConsumeLogtail(ctx, primarySeqnum, engine, state, lt) 1439 } 1440 1441 func parseCkpDuration(lt *logtail.TableLogtail) (start types.TS, end types.TS) { 1442 locationsAndVersions := strings.Split(lt.CkpLocation, ";") 1443 //check whether metLoc contains duration: [start, end] 1444 if !strings.Contains(locationsAndVersions[len(locationsAndVersions)-1], "[") { 1445 return 1446 } 1447 1448 newlocs := locationsAndVersions[:len(locationsAndVersions)-1] 1449 lt.CkpLocation = strings.Join(newlocs, ";") 1450 1451 duration := locationsAndVersions[len(locationsAndVersions)-1] 1452 pos1 := strings.Index(duration, "[") 1453 pos2 := strings.Index(duration, "]") 1454 sub := duration[pos1+1 : pos2] 1455 ds := strings.Split(sub, "_") 1456 return types.StringToTS(ds[0]), types.StringToTS(ds[1]) 1457 } 1458 1459 func consumeCkpsAndLogTail( 1460 ctx context.Context, 1461 primarySeqnum int, 1462 engine *Engine, 1463 state *logtailreplay.PartitionState, 1464 lt *logtail.TableLogtail, 1465 databaseId uint64, 1466 tableId uint64, 1467 tableName string, 1468 ) (err error) { 1469 var entries []*api.Entry 1470 var closeCBs []func() 1471 if entries, closeCBs, err = taeLogtail.LoadCheckpointEntries( 1472 ctx, 1473 lt.CkpLocation, 1474 tableId, tableName, 1475 databaseId, "", engine.mp, engine.fs); err != nil { 1476 return 1477 } 1478 defer func() { 1479 for _, cb := range closeCBs { 1480 cb() 1481 } 1482 }() 1483 for _, entry := range entries { 1484 if err = consumeEntry(ctx, primarySeqnum, 1485 engine, engine.getLatestCatalogCache(), state, entry); err != nil { 1486 return 1487 } 1488 } 1489 return hackConsumeLogtail(ctx, primarySeqnum, engine, state, lt) 1490 } 1491 1492 func hackConsumeLogtail( 1493 ctx context.Context, 1494 primarySeqnum int, 1495 engine *Engine, 1496 state *logtailreplay.PartitionState, 1497 lt *logtail.TableLogtail) error { 1498 1499 var packer *types.Packer 1500 put := engine.packerPool.Get(&packer) 1501 defer put.Put() 1502 1503 t0 := time.Now() 1504 switch lt.Table.TbId { 1505 1506 case catalog.MO_TABLES_ID: 1507 primarySeqnum = catalog.MO_TABLES_CATALOG_VERSION_IDX + 1 1508 for i := 0; i < len(lt.Commands); i++ { 1509 if lt.Commands[i].EntryType == api.Entry_Insert { 1510 bat, _ := batch.ProtoBatchToBatch(lt.Commands[i].Bat) 1511 accounts := vector.MustFixedCol[uint32](bat.GetVector(catalog.MO_TABLES_ACCOUNT_ID_IDX + 2)) 1512 names := bat.GetVector(catalog.MO_TABLES_REL_NAME_IDX + 2) 1513 databases := bat.GetVector(catalog.MO_TABLES_RELDATABASE_IDX + 2) 1514 vec := vector.NewVec(types.New(types.T_varchar, 0, 0)) 1515 for i, acc := range accounts { 1516 packer.EncodeUint32(acc) 1517 packer.EncodeStringType(names.GetBytesAt(i)) 1518 packer.EncodeStringType(databases.GetBytesAt(i)) 1519 if err := vector.AppendBytes(vec, packer.Bytes(), false, engine.mp); err != nil { 1520 panic(err) 1521 } 1522 packer.Reset() 1523 } 1524 hackVec, _ := vector.VectorToProtoVector(vec) 1525 lt.Commands[i].Bat.Vecs = append(lt.Commands[i].Bat.Vecs, hackVec) 1526 vec.Free(engine.mp) 1527 } 1528 if lt.Commands[i].EntryType == api.Entry_Delete { 1529 continue 1530 } 1531 if lt.Commands[i].EntryType == api.Entry_SpecialDelete { 1532 lt.Commands[i].EntryType = api.Entry_Delete 1533 } 1534 if err := consumeEntry(ctx, primarySeqnum, 1535 engine, engine.getLatestCatalogCache(), state, <.Commands[i]); err != nil { 1536 return err 1537 } 1538 } 1539 v2.LogtailUpdatePartitonConsumeLogtailCatalogTableDurationHistogram.Observe(time.Since(t0).Seconds()) 1540 return nil 1541 1542 case catalog.MO_DATABASE_ID: 1543 primarySeqnum = catalog.MO_DATABASE_DAT_TYPE_IDX + 1 1544 for i := 0; i < len(lt.Commands); i++ { 1545 if lt.Commands[i].EntryType == api.Entry_Insert { 1546 bat, _ := batch.ProtoBatchToBatch(lt.Commands[i].Bat) 1547 accounts := vector.MustFixedCol[uint32](bat.GetVector(catalog.MO_DATABASE_ACCOUNT_ID_IDX + 2)) 1548 names := bat.GetVector(catalog.MO_DATABASE_DAT_NAME_IDX + 2) 1549 vec := vector.NewVec(types.New(types.T_varchar, 0, 0)) 1550 for i, acc := range accounts { 1551 packer.EncodeUint32(acc) 1552 packer.EncodeStringType(names.GetBytesAt(i)) 1553 if err := vector.AppendBytes(vec, packer.Bytes(), false, engine.mp); err != nil { 1554 panic(err) 1555 } 1556 packer.Reset() 1557 } 1558 hackVec, _ := vector.VectorToProtoVector(vec) 1559 lt.Commands[i].Bat.Vecs = append(lt.Commands[i].Bat.Vecs, hackVec) 1560 vec.Free(engine.mp) 1561 } 1562 if lt.Commands[i].EntryType == api.Entry_Delete { 1563 continue 1564 } 1565 if lt.Commands[i].EntryType == api.Entry_SpecialDelete { 1566 lt.Commands[i].EntryType = api.Entry_Delete 1567 } 1568 if err := consumeEntry(ctx, primarySeqnum, 1569 engine, engine.getLatestCatalogCache(), state, <.Commands[i]); err != nil { 1570 return err 1571 } 1572 } 1573 v2.LogtailUpdatePartitonConsumeLogtailCatalogTableDurationHistogram.Observe(time.Since(t0).Seconds()) 1574 return nil 1575 1576 } 1577 1578 t0 = time.Now() 1579 for i := 0; i < len(lt.Commands); i++ { 1580 if err := consumeEntry(ctx, primarySeqnum, 1581 engine, engine.getLatestCatalogCache(), state, <.Commands[i]); err != nil { 1582 return err 1583 } 1584 } 1585 v2.LogtailUpdatePartitonConsumeLogtailCommandsDurationHistogram.Observe(time.Since(t0).Seconds()) 1586 1587 return nil 1588 }