github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/tests/integration_tests/bank/case.go (about) 1 // Copyright 2020 PingCAP, Inc. 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 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package main 15 16 import ( 17 "context" 18 "database/sql" 19 "encoding/json" 20 "fmt" 21 "io" 22 "math/rand" 23 "net/http" 24 "strings" 25 "sync/atomic" 26 "time" 27 28 _ "github.com/go-sql-driver/mysql" // MySQL driver 29 "github.com/pingcap/errors" 30 "github.com/pingcap/log" 31 cerror "github.com/pingcap/tiflow/pkg/errors" 32 "github.com/pingcap/tiflow/pkg/retry" 33 "go.uber.org/zap" 34 "golang.org/x/sync/errgroup" 35 ) 36 37 // -- Create table 38 // CREATE TABLE IF NOT EXISTS accounts%d ( 39 // id BIGINT PRIMARY KEY, 40 // balance BIGINT NOT NULL, 41 // startts BIGINT NOT NULL 42 // ) 43 // CREATE TABLE IF NOT EXISTS accounts_seq%d ( 44 // id BIGINT PRIMARY KEY, 45 // counter BIGINT NOT NULL, 46 // sequence BIGINT NOT NULL, 47 // startts BIGINT NOT NULL 48 // ) 49 // 50 // BEGIN 51 // -- Add sequential update rows. 52 // SELECT counter, sequence FROM accounts_seq%d WHERE id = %d FOR UPDATE 53 // UPDATE accounts_seq%d SET 54 // counter = %d, 55 // sequence = %d, 56 // startts = @@tidb_current_ts 57 // WHERE id IN (%d, %d) 58 // 59 // -- Transaction between accounts. 60 // SELECT id, balance FROM accounts%d WHERE id IN (%d, %d) FOR UPDATE 61 // UPDATE accounts%d SET 62 // balance = CASE id WHEN %d THEN %d WHEN %d THEN %d END, 63 // sequence = %d, 64 // startts = @@tidb_current_ts 65 // WHERE id IN (%d, %d) 66 // COMMIT 67 // 68 // -- Verify sum of balance always be the same. 69 // SELECT SUM(balance) as total FROM accounts%d 70 // -- Verify no missing transaction 71 // SELECT sequence FROM accounts_seq%d ORDER BY sequence 72 73 // testcase ... 74 // testcase.cleanup 75 // testcase.prepare 76 // go { loop { testcase.workload } } 77 // go { loop { testcase.verify } } 78 79 const ( 80 initBalance = 1000 81 ) 82 83 type testcase interface { 84 prepare(ctx context.Context, db *sql.DB, accounts int, tableID int, concurrency int) error 85 workload(ctx context.Context, tx *sql.Tx, accounts int, tableID int) error 86 verify(ctx context.Context, db *sql.DB, accounts, tableID int, tag string, endTs uint64) error 87 cleanup(ctx context.Context, db *sql.DB, accounts, tableID int, force bool) bool 88 } 89 90 type sequenceTest struct{} 91 92 var _ testcase = &sequenceTest{} 93 94 func (*sequenceTest) workload(ctx context.Context, tx *sql.Tx, accounts int, tableID int) error { 95 const sequenceRowID = 0 96 97 getCounterSeq := fmt.Sprintf("SELECT counter, sequence FROM accounts_seq%d WHERE id = %d FOR UPDATE", tableID, sequenceRowID) 98 99 var counter, maxSeq int 100 row := tx.QueryRowContext(ctx, getCounterSeq) 101 if err := row.Scan(&counter, &maxSeq); err != nil { 102 return errors.Trace(err) 103 } 104 105 next := counter % accounts 106 if next == sequenceRowID { 107 next++ 108 counter++ 109 } 110 counter++ 111 112 addSeqCounter := fmt.Sprintf(` 113 UPDATE accounts_seq%d SET 114 counter = %d, 115 sequence = %d, 116 startts = @@tidb_current_ts 117 WHERE id IN (%d, %d)`, tableID, counter, maxSeq+1, sequenceRowID, next) 118 119 if _, err := tx.ExecContext(ctx, addSeqCounter); err != nil { 120 log.Error("sequenceTest workload exec failed", zap.Error(err)) 121 return errors.Trace(err) 122 } 123 return nil 124 } 125 126 func (s *sequenceTest) prepare(ctx context.Context, db *sql.DB, accounts, tableID, concurrency int) error { 127 createTable := fmt.Sprintf(` 128 CREATE TABLE IF NOT EXISTS accounts_seq%d ( 129 id BIGINT PRIMARY KEY, 130 counter BIGINT NOT NULL, 131 sequence BIGINT NOT NULL, 132 startts BIGINT NOT NULL 133 )`, tableID) 134 batchInsertSQLF := func(batchSize, offset int) string { 135 args := make([]string, batchSize) 136 for j := 0; j < batchSize; j++ { 137 args[j] = fmt.Sprintf("(%d, 0, 0, 0)", offset+j) 138 } 139 return fmt.Sprintf("INSERT IGNORE INTO accounts_seq%d (id, counter, sequence, startts) VALUES %s", tableID, strings.Join(args, ",")) 140 } 141 142 prepareImpl(ctx, s, createTable, batchInsertSQLF, db, accounts, tableID, concurrency) 143 return nil 144 } 145 146 func (*sequenceTest) verify(ctx context.Context, db *sql.DB, accounts, tableID int, tag string, endTs uint64) error { 147 return retry.Do(ctx, func() (err error) { 148 defer func() { 149 if err != nil { 150 log.Warn("sequence test verify failed", zap.Error(err)) 151 } 152 }() 153 query := fmt.Sprintf("set @@tidb_snapshot=%d", endTs) 154 // use a single connection to keep the same database session. 155 conn, err := db.Conn(ctx) 156 if err != nil { 157 return errors.Trace(err) 158 } 159 defer conn.Close() 160 if _, err := conn.ExecContext(ctx, query); err != nil { 161 log.Error("sequenceTest set tidb_snapshot failed", zap.String("query", query), zap.Error(err)) 162 return errors.Trace(err) 163 } 164 165 query = fmt.Sprintf("SELECT sequence FROM accounts_seq%d ORDER BY sequence", tableID) 166 rows, err := conn.QueryContext(ctx, query) 167 if err != nil { 168 log.Warn("select sequence err", zap.String("query", query), zap.Error(err), zap.String("tag", tag)) 169 return nil 170 } 171 defer rows.Close() 172 173 var curr, previous int 174 for rows.Next() { 175 if err = rows.Scan(&curr); err != nil { 176 log.Warn("select sequence err", zap.String("query", query), zap.Error(err), zap.String("tag", tag)) 177 return nil 178 } 179 180 if previous != 0 && previous != curr && previous+1 != curr { 181 return errors.Errorf("missing changes sequence account_seq%d, current sequence=%d, previous sequence=%d", tableID, curr, previous) 182 } 183 previous = curr 184 } 185 186 log.Info("sequence verify pass", zap.String("tag", tag)) 187 188 if _, err := conn.ExecContext(ctx, "set @@tidb_snapshot=''"); err != nil { 189 log.Warn("sequenceTest reset tidb_snapshot failed") 190 } 191 192 return nil 193 }, retry.WithBackoffMaxDelay(500), retry.WithBackoffMaxDelay(120*1000), retry.WithMaxTries(10), retry.WithIsRetryableErr(cerror.IsRetryableError)) 194 } 195 196 // tryDropDB will drop table if data incorrect and panic error likes bad connect. 197 func (s *sequenceTest) cleanup(ctx context.Context, db *sql.DB, accounts, tableID int, force bool) bool { 198 return cleanupImpl(ctx, s, fmt.Sprintf("accounts_seq%d", tableID), db, accounts, tableID, force) 199 } 200 201 type bankTest struct{} 202 203 var _ testcase = &bankTest{} 204 205 func (*bankTest) workload(ctx context.Context, tx *sql.Tx, accounts int, tableID int) error { 206 var ( 207 from, fromBalance int 208 to, toBalance int 209 ) 210 211 for { 212 from, to = rand.Intn(accounts), rand.Intn(accounts) 213 if from != to { 214 break 215 } 216 } 217 218 sqlFormat := fmt.Sprintf("SELECT balance FROM accounts%d WHERE id = ? FOR UPDATE", tableID) 219 row := tx.QueryRowContext(ctx, sqlFormat, from) 220 if err := row.Scan(&fromBalance); err != nil { 221 return errors.Trace(err) 222 } 223 row = tx.QueryRowContext(ctx, sqlFormat, to) 224 if err := row.Scan(&toBalance); err != nil { 225 return errors.Trace(err) 226 } 227 228 amount := rand.Intn(fromBalance/2 + 1) 229 fromBalance -= amount 230 toBalance += amount 231 232 sqlFormat = fmt.Sprintf("UPDATE accounts%d SET balance = ? WHERE id = ?", tableID) 233 if _, err := tx.ExecContext(ctx, sqlFormat, fromBalance, from); err != nil { 234 return errors.Trace(err) 235 } 236 if _, err := tx.ExecContext(ctx, sqlFormat, toBalance, to); err != nil { 237 return errors.Trace(err) 238 } 239 240 return nil 241 } 242 243 func (s *bankTest) prepare(ctx context.Context, db *sql.DB, accounts, tableID, concurrency int) error { 244 createTable := fmt.Sprintf(` 245 CREATE TABLE IF NOT EXISTS accounts%d ( 246 id BIGINT PRIMARY KEY, 247 balance BIGINT NOT NULL, 248 startts BIGINT NOT NULL 249 )`, tableID) 250 batchInsertSQLF := func(batchSize, offset int) string { 251 args := make([]string, batchSize) 252 for j := 0; j < batchSize; j++ { 253 args[j] = fmt.Sprintf("(%d, %d, 0)", offset+j, initBalance) 254 } 255 return fmt.Sprintf("INSERT IGNORE INTO accounts%d (id, balance, startts) VALUES %s", tableID, strings.Join(args, ",")) 256 } 257 258 prepareImpl(ctx, s, createTable, batchInsertSQLF, db, accounts, tableID, concurrency) 259 return nil 260 } 261 262 func (*bankTest) verify(ctx context.Context, db *sql.DB, accounts, tableID int, tag string, endTs uint64) error { 263 return retry.Do(ctx, 264 func() (err error) { 265 defer func() { 266 if err != nil { 267 log.Error("bank test verify failed", zap.Error(err)) 268 } 269 }() 270 // use a single connection to keep the same database session. 271 conn, err := db.Conn(ctx) 272 if err != nil { 273 return errors.Trace(err) 274 } 275 defer conn.Close() 276 if _, err := conn.ExecContext(ctx, fmt.Sprintf("set @@tidb_snapshot=%d", endTs)); err != nil { 277 log.Error("bank set tidb_snapshot failed", zap.Uint64("endTs", endTs)) 278 return errors.Trace(err) 279 } 280 281 var obtained, expect int 282 283 query := fmt.Sprintf("SELECT SUM(balance) as total FROM accounts%d", tableID) 284 if err := conn.QueryRowContext(ctx, query).Scan(&obtained); err != nil { 285 log.Warn("query failed", zap.String("query", query), zap.Error(err), zap.String("tag", tag)) 286 return errors.Trace(err) 287 } 288 289 expect = accounts * initBalance 290 if obtained != expect { 291 return errors.Errorf("verify balance failed, accounts%d expect %d, but got %d", tableID, expect, obtained) 292 } 293 294 query = fmt.Sprintf("SELECT COUNT(*) as count FROM accounts%d", tableID) 295 if err := conn.QueryRowContext(ctx, query).Scan(&obtained); err != nil { 296 log.Warn("query failed", zap.String("query", query), zap.Error(err), zap.String("tag", tag)) 297 return errors.Trace(err) 298 } 299 if obtained != accounts { 300 return errors.Errorf("verify count failed, accounts%d expected=%d, obtained=%d", tableID, accounts, obtained) 301 } 302 303 log.Info("bank verify pass", zap.String("tag", tag)) 304 305 if _, err := conn.ExecContext(ctx, "set @@tidb_snapshot=''"); err != nil { 306 log.Warn("bank reset tidb_snapshot failed") 307 } 308 309 return nil 310 }, retry.WithBackoffMaxDelay(500), retry.WithBackoffMaxDelay(120*1000), retry.WithMaxTries(10), retry.WithIsRetryableErr(cerror.IsRetryableError)) 311 } 312 313 // tryDropDB will drop table if data incorrect and panic error likes bad connect. 314 func (s *bankTest) cleanup(ctx context.Context, db *sql.DB, accounts, tableID int, force bool) bool { 315 return cleanupImpl(ctx, s, fmt.Sprintf("accounts%d", tableID), db, accounts, tableID, force) 316 } 317 318 func prepareImpl( 319 ctx context.Context, 320 test testcase, createTable string, batchInsertSQLF func(batchSize, offset int) string, 321 db *sql.DB, accounts, tableID, concurrency int, 322 ) { 323 isDropped := test.cleanup(ctx, db, accounts, tableID, false) 324 if !isDropped { 325 return 326 } 327 328 mustExec(ctx, db, createTable) 329 330 batchSize := 100 331 jobCount := accounts / batchSize 332 if accounts%batchSize != 0 { 333 jobCount++ 334 } 335 336 insertF := func(query string) error { 337 _, err := db.ExecContext(ctx, query) 338 return err 339 } 340 341 g := new(errgroup.Group) 342 ch := make(chan int, jobCount) 343 for i := 0; i < concurrency; i++ { 344 g.Go(func() error { 345 for { 346 startIndex, ok := <-ch 347 if !ok { 348 return nil 349 } 350 351 size := batchSize 352 remained := accounts - startIndex + 1 353 if remained < size { 354 size = remained 355 } 356 357 batchInsertSQL := batchInsertSQLF(size, startIndex) 358 start := time.Now() 359 err := retry.Do(context.Background(), func() error { 360 return insertF(batchInsertSQL) 361 }, retry.WithBackoffBaseDelay(100), retry.WithBackoffMaxDelay(60*100), retry.WithMaxTries(5)) 362 if err != nil { 363 log.Panic("exec batch insert failed", zap.String("query", batchInsertSQL), zap.Error(err)) 364 } 365 log.Info(fmt.Sprintf("insert %d takes %s", batchSize, time.Since(start)), zap.String("query", batchInsertSQL)) 366 } 367 }) 368 } 369 370 for i := 0; i < jobCount; i++ { 371 ch <- i * batchSize 372 } 373 close(ch) 374 _ = g.Wait() 375 } 376 377 func dropDB(ctx context.Context, db *sql.DB) { 378 log.Info("drop database") 379 mustExec(ctx, db, "DROP DATABASES IF EXISTS bank") 380 } 381 382 func dropTable(ctx context.Context, db *sql.DB, table string) { 383 log.Info("drop tables", zap.String("table", table)) 384 mustExec(ctx, db, fmt.Sprintf("DROP TABLE IF EXISTS %s", table)) 385 } 386 387 func cleanupImpl(ctx context.Context, test testcase, tableName string, db *sql.DB, accounts, tableID int, force bool) bool { 388 if force { 389 dropTable(ctx, db, tableName) 390 return true 391 } 392 393 if !isTableExist(ctx, db, tableName) { 394 dropTable(ctx, db, tableName) 395 return true 396 } 397 398 if err := test.verify(ctx, db, accounts, tableID, "tryDropDB", 0); err != nil { 399 dropTable(ctx, db, tableName) 400 return true 401 } 402 403 return false 404 } 405 406 func mustExec(ctx context.Context, db *sql.DB, query string) { 407 execF := func() error { 408 _, err := db.ExecContext(ctx, query) 409 return err 410 } 411 412 err := retry.Do(context.Background(), execF, retry.WithBackoffBaseDelay(100), retry.WithBackoffMaxDelay(60*100), retry.WithMaxTries(5)) 413 if err != nil { 414 log.Panic("exec failed", zap.String("query", query), zap.Error(err)) 415 } 416 } 417 418 func isTableExist(ctx context.Context, db *sql.DB, table string) bool { 419 // if table is not exist, return true directly 420 query := fmt.Sprintf("SHOW TABLES LIKE '%s'", table) 421 var t string 422 err := db.QueryRowContext(ctx, query).Scan(&t) 423 switch { 424 case err == sql.ErrNoRows: 425 return false 426 case err != nil: 427 log.Panic("query failed", zap.String("query", query), zap.Error(err)) 428 } 429 return true 430 } 431 432 func openDB(ctx context.Context, dsn string) *sql.DB { 433 db, err := sql.Open("mysql", dsn) 434 if err != nil { 435 log.Panic("open db failed", zap.String("dsn", dsn), zap.Error(err)) 436 } 437 db.SetMaxOpenConns(10) 438 db.SetMaxIdleConns(10) 439 db.SetConnMaxLifetime(10 * time.Minute) 440 ctx, cancel := context.WithTimeout(ctx, 5*time.Second) 441 defer cancel() 442 if err = db.PingContext(ctx); err != nil { 443 log.Panic("ping db failed", zap.String("dsn", dsn), zap.Error(err)) 444 } 445 log.Info("open db success", zap.String("dsn", dsn)) 446 return db 447 } 448 449 func run( 450 ctx context.Context, upstream, downstream string, downstreamAPIEndpoint string, 451 accounts, tables, concurrency int, 452 interval, testRound int64, cleanupOnly bool, 453 ) { 454 ctx, cancel := context.WithTimeout(ctx, 30*time.Minute) 455 defer cancel() 456 457 upstreamDB := openDB(ctx, upstream) 458 defer upstreamDB.Close() 459 460 downstreamDB := openDB(ctx, downstream) 461 defer downstreamDB.Close() 462 463 tests := []testcase{&sequenceTest{}, &bankTest{}} 464 465 if cleanupOnly { 466 for tableID := 0; tableID < tables; tableID++ { 467 for i := range tests { 468 tests[i].cleanup(ctx, upstreamDB, accounts, tableID, true) 469 tests[i].cleanup(ctx, downstreamDB, accounts, tableID, true) 470 } 471 } 472 473 // a lot of ddl executed at upstream, just drop the db 474 dropDB(ctx, upstreamDB) 475 dropDB(ctx, downstreamDB) 476 log.Info("cleanup done") 477 return 478 } 479 480 // prepare data for upstream db. 481 for _, test := range tests { 482 for tableID := 0; tableID < tables; tableID++ { 483 if err := test.prepare(ctx, upstreamDB, accounts, tableID, concurrency); err != nil { 484 log.Panic("prepare failed", zap.Error(err)) 485 } 486 } 487 } 488 489 // DDL is a strong sync point in TiCDC. Once finishmark table is replicated to downstream 490 // all previous DDL and DML are replicated too. 491 mustExec(ctx, upstreamDB, `CREATE TABLE IF NOT EXISTS finishmark (foo BIGINT PRIMARY KEY)`) 492 waitCtx, waitCancel := context.WithTimeout(ctx, 15*time.Minute) 493 endTs, err := getDownStreamSyncedEndTs(waitCtx, downstreamDB, downstreamAPIEndpoint, "finishmark") 494 waitCancel() 495 if err != nil { 496 log.Panic("wait for table finishmark failed", zap.Error(err)) 497 } 498 log.Info("all tables synced", zap.Uint64("endTs", endTs)) 499 500 var ( 501 counts int64 = 0 502 g = new(errgroup.Group) 503 tblChan = make(chan string, tables) 504 doneCh = make(chan struct{}, 1) 505 valid, tried int64 = 0, 0 506 ) 507 508 for id := 0; id < tables; id++ { 509 tableID := id 510 // Workload 511 g.Go(func() error { 512 workload := func() error { 513 tx, err := upstreamDB.Begin() 514 if err != nil { 515 log.Error("upstream begin tx failed", zap.Error(err)) 516 return errors.Trace(err) 517 } 518 519 for _, test := range tests { 520 if err := test.workload(context.Background(), tx, accounts, tableID); err != nil { 521 _ = tx.Rollback() 522 return errors.Trace(err) 523 } 524 } 525 526 if err := tx.Commit(); err != nil { 527 _ = tx.Rollback() 528 log.Error("upstream tx commit failed", zap.Error(err)) 529 return errors.Trace(err) 530 } 531 532 curr := atomic.AddInt64(&counts, 1) 533 if curr%interval == 0 { 534 tblName := fmt.Sprintf("finishmark%d", curr) 535 ddl := fmt.Sprintf("CREATE TABLE IF NOT EXISTS %s (foo BIGINT PRIMARY KEY)", tblName) 536 mustExec(ctx, upstreamDB, ddl) 537 tblChan <- tblName 538 539 log.Info("upstream create table executed", zap.String("tblName", tblName)) 540 } 541 return nil 542 } 543 544 for { 545 select { 546 case <-ctx.Done(): 547 return ctx.Err() 548 default: 549 err := workload() 550 if err != nil && errors.Cause(err) != context.Canceled { 551 log.Warn("workload failed", zap.Error(err)) 552 } 553 554 curr := atomic.LoadInt64(&counts) 555 if curr >= testRound { 556 log.Info("one upstream workload finished", zap.Int64("round", curr)) 557 doneCh <- struct{}{} 558 return nil 559 } 560 } 561 } 562 }) 563 564 // Verify 565 g.Go(func() error { 566 for { 567 select { 568 case <-ctx.Done(): 569 return ctx.Err() 570 case tblName := <-tblChan: 571 log.Info("downstream start wait for table", zap.String("tblName", tblName)) 572 waitCtx, waitCancel := context.WithTimeout(ctx, 15*time.Minute) 573 endTs, err := getDownStreamSyncedEndTs( 574 waitCtx, downstreamDB, downstreamAPIEndpoint, tblName) 575 waitCancel() 576 log.Info("ddl synced", zap.String("table", tblName)) 577 if err != nil { 578 log.Fatal("[cdc-bank] get ddl end ts error", zap.Error(err)) 579 } 580 581 atomic.AddInt64(&tried, 1) 582 log.Info("downstream sync success", zap.Uint64("endTs", endTs)) 583 584 if endTs == 0 { 585 continue 586 } 587 atomic.AddInt64(&valid, 1) 588 589 for _, test := range tests { 590 verifyCtx, verifyCancel := context.WithTimeout(ctx, 2*time.Minute) 591 if err := test.verify(verifyCtx, upstreamDB, accounts, tableID, upstream, 0); err != nil { 592 log.Panic("upstream verify failed", zap.Error(err)) 593 } 594 verifyCancel() 595 596 verifyCtx, verifyCancel = context.WithTimeout(ctx, 2*time.Minute) 597 if err := test.verify(verifyCtx, downstreamDB, accounts, tableID, downstream, endTs); err != nil { 598 log.Panic("downstream verify failed", zap.Error(err)) 599 } 600 verifyCancel() 601 } 602 case <-doneCh: 603 log.Info("one downstream exit due to receive done") 604 return nil 605 } 606 } 607 }) 608 } 609 610 _ = g.Wait() 611 612 if tried == 0 { 613 log.Warn("bank test finished, but tries is 0") 614 } else { 615 log.Info("bank test finished", zap.Int64("valid", valid), zap.Int64("tries", tried), zap.Float64("ratio", float64(valid)/float64(tried))) 616 } 617 } 618 619 func getDownStreamSyncedEndTs(ctx context.Context, db *sql.DB, tidbAPIEndpoint, tableName string) (result uint64, err error) { 620 for { 621 select { 622 case <-ctx.Done(): 623 log.Error("get downstream sync end ts failed due to timeout", zap.String("table", tableName), zap.Error(ctx.Err())) 624 return 0, ctx.Err() 625 case <-time.After(2 * time.Second): 626 result, ok := tryGetEndTs(db, tidbAPIEndpoint, tableName) 627 if ok { 628 return result, nil 629 } 630 } 631 } 632 } 633 634 func tryGetEndTs(db *sql.DB, tidbAPIEndpoint, tableName string) (result uint64, ok bool) { 635 // Note: We should not use `END_TS` in the table, because it is encoded in 636 // the format `2023-03-16 18:12:51`, it's not precise enough. 637 query := "SELECT JOB_ID FROM information_schema.ddl_jobs WHERE table_name = ?" 638 log.Info("try get end ts", zap.String("query", query), zap.String("tableName", tableName)) 639 var jobID uint64 640 row := db.QueryRow(query, tableName) 641 if err := row.Scan(&jobID); err != nil { 642 if err != sql.ErrNoRows { 643 log.Info("rows scan failed", zap.Error(err)) 644 } 645 return 0, false 646 } 647 ddlJobURL := fmt.Sprintf( 648 "http://%s/ddl/history?start_job_id=%d&limit=1", tidbAPIEndpoint, jobID) 649 ddlJobResp, err := http.Get(ddlJobURL) 650 if err != nil { 651 log.Warn("fail to get DDL history", 652 zap.String("URL", ddlJobURL), zap.Error(err)) 653 return 0, false 654 } 655 defer ddlJobResp.Body.Close() 656 ddlJobJSON, err := io.ReadAll(ddlJobResp.Body) 657 if err != nil { 658 log.Warn("fail to read DDL history", 659 zap.String("URL", ddlJobURL), zap.Error(err)) 660 return 0, false 661 } 662 ddlJob := []struct { 663 Binlog struct { 664 FinishedTS uint64 `json:"FinishedTS"` 665 } `json:"binlog"` 666 }{{}} 667 err = json.Unmarshal(ddlJobJSON, &ddlJob) 668 if err != nil { 669 log.Warn("fail to unmarshal DDL history", 670 zap.String("URL", ddlJobURL), zap.String("resp", string(ddlJobJSON)), zap.Error(err)) 671 return 0, false 672 } 673 log.Info("get end ts", 674 zap.String("tableName", tableName), 675 zap.Uint64("ts", ddlJob[0].Binlog.FinishedTS)) 676 return ddlJob[0].Binlog.FinishedTS, true 677 }