Skip to content

Commit ba791ab

Browse files
authored
ttl: fix the issue that DROP TABLE / ALTER TABLE will keep job running (#57707)
close #57556, close #57702
1 parent d0de86b commit ba791ab

File tree

3 files changed

+100
-31
lines changed

3 files changed

+100
-31
lines changed

pkg/ttl/ttlworker/config.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ const ttlJobTimeout = 6 * time.Hour
3535

3636
const taskManagerLoopTickerInterval = time.Minute
3737
const ttlTaskHeartBeatTickerInterval = time.Minute
38-
const ttlGCInterval = time.Hour
38+
const ttlGCInterval = 10 * time.Minute
3939

4040
func getCheckJobInterval() time.Duration {
4141
failpoint.Inject("check-job-interval", func(val failpoint.Value) time.Duration {

pkg/ttl/ttlworker/job_manager.go

Lines changed: 11 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -64,8 +64,7 @@ const taskGCTemplate = `DELETE task FROM
6464
WHERE job.table_id IS NULL`
6565

6666
const ttlJobHistoryGCTemplate = `DELETE FROM mysql.tidb_ttl_job_history WHERE create_time < CURDATE() - INTERVAL 90 DAY`
67-
const ttlTableStatusGCWithoutIDTemplate = `DELETE FROM mysql.tidb_ttl_table_status WHERE current_job_status IS NULL`
68-
const ttlTableStatusGCWithIDTemplate = ttlTableStatusGCWithoutIDTemplate + ` AND table_id NOT IN (%s)`
67+
const ttlTableStatusGCWithoutIDTemplate = `DELETE FROM mysql.tidb_ttl_table_status WHERE (current_job_status IS NULL OR current_job_owner_hb_time < %?)`
6968

7069
const timeFormat = time.DateTime
7170

@@ -81,15 +80,18 @@ func updateHeartBeatSQL(tableID int64, now time.Time, id string) (string, []any)
8180
return updateHeartBeatTemplate, []any{now.Format(timeFormat), tableID, id}
8281
}
8382

84-
func gcTTLTableStatusGCSQL(existIDs []int64) string {
83+
func gcTTLTableStatusGCSQL(existIDs []int64, now time.Time) (string, []any) {
8584
existIDStrs := make([]string, 0, len(existIDs))
8685
for _, id := range existIDs {
8786
existIDStrs = append(existIDStrs, strconv.Itoa(int(id)))
8887
}
88+
89+
hbExpireTime := now.Add(-jobManagerLoopTickerInterval * 2)
90+
args := []any{hbExpireTime.Format(timeFormat)}
8991
if len(existIDStrs) > 0 {
90-
return fmt.Sprintf(ttlTableStatusGCWithIDTemplate, strings.Join(existIDStrs, ","))
92+
return ttlTableStatusGCWithoutIDTemplate + fmt.Sprintf(` AND table_id NOT IN (%s)`, strings.Join(existIDStrs, ",")), args
9193
}
92-
return ttlTableStatusGCWithoutIDTemplate
94+
return ttlTableStatusGCWithoutIDTemplate, args
9395
}
9496

9597
// JobManager schedules and manages the ttl jobs on this instance
@@ -219,7 +221,7 @@ func (m *JobManager) jobLoop() error {
219221
}
220222
case <-gcTicker:
221223
gcCtx, cancel := context.WithTimeout(m.ctx, ttlInternalSQLTimeout)
222-
m.DoGC(gcCtx, se)
224+
m.DoGC(gcCtx, se, now)
223225
cancel()
224226
// Job Schedule loop:
225227
case <-updateJobHeartBeatTicker:
@@ -1029,7 +1031,7 @@ func summarizeTaskResult(tasks []*cache.TTLTask) (*TTLSummary, error) {
10291031
}
10301032

10311033
// DoGC deletes some old TTL job histories and redundant scan tasks
1032-
func (m *JobManager) DoGC(ctx context.Context, se session.Session) {
1034+
func (m *JobManager) DoGC(ctx context.Context, se session.Session, now time.Time) {
10331035
// Remove the table not exist in info schema cache.
10341036
// Delete the table status before deleting the tasks. Therefore the related tasks
10351037
if err := m.updateInfoSchemaCache(se); err == nil {
@@ -1038,7 +1040,8 @@ func (m *JobManager) DoGC(ctx context.Context, se session.Session) {
10381040
for id := range m.infoSchemaCache.Tables {
10391041
existIDs = append(existIDs, id)
10401042
}
1041-
if _, err := se.ExecuteSQL(ctx, gcTTLTableStatusGCSQL(existIDs)); err != nil {
1043+
sql, args := gcTTLTableStatusGCSQL(existIDs, now)
1044+
if _, err := se.ExecuteSQL(ctx, sql, args...); err != nil {
10421045
logutil.Logger(ctx).Warn("fail to gc ttl table status", zap.Error(err))
10431046
}
10441047
} else {

pkg/ttl/ttlworker/job_manager_integration_test.go

Lines changed: 88 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -612,7 +612,7 @@ func TestRescheduleJobsAfterTableDropped(t *testing.T) {
612612
tk.MustExec(rb.resume)
613613
table, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t"))
614614
require.NoError(t, err)
615-
m.DoGC(context.TODO(), se)
615+
m.DoGC(context.TODO(), se, now)
616616
}
617617
}
618618

@@ -775,7 +775,7 @@ func TestGCScanTasks(t *testing.T) {
775775
return true
776776
})
777777
se := session.NewSession(tk.Session(), tk.Session(), func(_ session.Session) {})
778-
m.DoGC(context.TODO(), se)
778+
m.DoGC(context.TODO(), se, se.Now())
779779
tk.MustQuery("select job_id, scan_id from mysql.tidb_ttl_task order by job_id, scan_id asc").Check(testkit.Rows("1 1", "1 2"))
780780
}
781781

@@ -794,7 +794,7 @@ func TestGCTableStatus(t *testing.T) {
794794
return true
795795
})
796796
se := session.NewSession(tk.Session(), tk.Session(), func(_ session.Session) {})
797-
m.DoGC(context.TODO(), se)
797+
m.DoGC(context.TODO(), se, se.Now())
798798
tk.MustQuery("select * from mysql.tidb_ttl_table_status").Check(nil)
799799

800800
// insert a running table status without corresponding table
@@ -808,7 +808,7 @@ func TestGCTableStatus(t *testing.T) {
808808
current_job_ttl_expire = NOW(),
809809
current_job_owner_hb_time = NOW()
810810
WHERE table_id = ?`, 1, 2024)
811-
m.DoGC(context.TODO(), se)
811+
m.DoGC(context.TODO(), se, se.Now())
812812
// it'll not be removed
813813
tk.MustQuery("select current_job_id from mysql.tidb_ttl_table_status").Check(testkit.Rows("1"))
814814
}
@@ -855,7 +855,7 @@ func TestGCTTLHistory(t *testing.T) {
855855
return true
856856
})
857857
se := session.NewSession(tk.Session(), tk.Session(), func(_ session.Session) {})
858-
m.DoGC(context.TODO(), se)
858+
m.DoGC(context.TODO(), se, se.Now())
859859
tk.MustQuery("select job_id from mysql.tidb_ttl_job_history order by job_id asc").Check(testkit.Rows("1", "2", "3", "4", "5"))
860860
}
861861

@@ -1475,27 +1475,93 @@ func TestDisableTTLAfterLoseHeartbeat(t *testing.T) {
14751475

14761476
ctx := context.Background()
14771477
m1 := ttlworker.NewJobManager("test-ttl-job-manager-1", nil, store, nil, nil)
1478-
require.NoError(t, m1.InfoSchemaCache().Update(se))
1479-
require.NoError(t, m1.TableStatusCache().Update(ctx, se))
1478+
m2 := ttlworker.NewJobManager("test-ttl-job-manager-2", nil, store, nil, nil)
14801479

14811480
now := se.Now()
1482-
_, err = m1.LockJob(context.Background(), se, m1.InfoSchemaCache().Tables[testTable.Meta().ID], now, uuid.NewString(), false)
1483-
require.NoError(t, err)
1484-
tk.MustQuery("select current_job_status from mysql.tidb_ttl_table_status").Check(testkit.Rows("running"))
14851481

1486-
// lose heartbeat. Simulate the situation that m1 doesn't update the hearbeat for 8 hours.
1487-
now = now.Add(time.Hour * 8)
1482+
acquireJob := func(now time.Time) {
1483+
require.NoError(t, m1.InfoSchemaCache().Update(se))
1484+
require.NoError(t, m1.TableStatusCache().Update(ctx, se))
1485+
require.NoError(t, m2.InfoSchemaCache().Update(se))
1486+
require.NoError(t, m2.TableStatusCache().Update(ctx, se))
1487+
_, err = m1.LockJob(context.Background(), se, m1.InfoSchemaCache().Tables[testTable.Meta().ID], now, uuid.NewString(), false)
1488+
require.NoError(t, err)
1489+
tk.MustQuery("select current_job_status from mysql.tidb_ttl_table_status").Check(testkit.Rows("running"))
1490+
}
1491+
t.Run("disable TTL globally after losing heartbeat", func(t *testing.T) {
1492+
// now the TTL job should be scheduled again
1493+
now = now.Add(time.Hour * 8)
1494+
acquireJob(now)
14881495

1489-
// stop the tidb_ttl_job_enable
1490-
tk.MustExec("set global tidb_ttl_job_enable = 'OFF'")
1491-
defer tk.MustExec("set global tidb_ttl_job_enable = 'ON'")
1496+
// lose heartbeat. Simulate the situation that m1 doesn't update the hearbeat for 8 hours.
1497+
now = now.Add(time.Hour * 8)
14921498

1493-
// reschedule and try to get the job
1494-
m2 := ttlworker.NewJobManager("test-ttl-job-manager-2", nil, store, nil, nil)
1495-
require.NoError(t, m2.InfoSchemaCache().Update(se))
1496-
require.NoError(t, m2.TableStatusCache().Update(ctx, se))
1497-
m2.RescheduleJobs(se, now)
1499+
// stop the tidb_ttl_job_enable
1500+
tk.MustExec("set global tidb_ttl_job_enable = 'OFF'")
1501+
defer tk.MustExec("set global tidb_ttl_job_enable = 'ON'")
1502+
1503+
// reschedule and try to get the job
1504+
require.NoError(t, m2.InfoSchemaCache().Update(se))
1505+
require.NoError(t, m2.TableStatusCache().Update(ctx, se))
1506+
m2.RescheduleJobs(se, now)
1507+
1508+
// the job should have been cancelled
1509+
tk.MustQuery("select current_job_status from mysql.tidb_ttl_table_status").Check(testkit.Rows("<nil>"))
1510+
})
1511+
1512+
t.Run("disable TTL for a table after losing heartbeat", func(t *testing.T) {
1513+
// now the TTL job should be scheduled again
1514+
now = now.Add(time.Hour * 8)
1515+
acquireJob(now)
14981516

1499-
// the job should have been cancelled
1500-
tk.MustQuery("select current_job_status from mysql.tidb_ttl_table_status").Check(testkit.Rows("<nil>"))
1517+
// lose heartbeat. Simulate the situation that m1 doesn't update the hearbeat for 8 hours.
1518+
now = now.Add(time.Hour * 8)
1519+
1520+
tk.MustExec("ALTER TABLE t TTL_ENABLE = 'OFF'")
1521+
defer tk.MustExec("ALTER TABLE t TTL_ENABLE = 'ON'")
1522+
1523+
// reschedule and try to get the job
1524+
require.NoError(t, m2.InfoSchemaCache().Update(se))
1525+
require.NoError(t, m2.TableStatusCache().Update(ctx, se))
1526+
m2.RescheduleJobs(se, now)
1527+
1528+
// the job cannot be cancelled, because it doesn't exist in the infoschema cache.
1529+
tk.MustQuery("select current_job_status from mysql.tidb_ttl_table_status").Check(testkit.Rows("running"))
1530+
1531+
// run GC
1532+
m2.DoGC(ctx, se, now)
1533+
1534+
// the job should have been cancelled
1535+
tk.MustQuery("select current_job_status, current_job_owner_hb_time from mysql.tidb_ttl_table_status").Check(testkit.Rows())
1536+
})
1537+
1538+
t.Run("drop a TTL table after losing heartbeat", func(t *testing.T) {
1539+
// now the TTL job should be scheduled again
1540+
now = now.Add(time.Hour * 8)
1541+
acquireJob(now)
1542+
1543+
// lose heartbeat. Simulate the situation that m1 doesn't update the hearbeat for 8 hours.
1544+
now = now.Add(time.Hour * 8)
1545+
1546+
tk.MustExec("DROP TABLE t")
1547+
defer func() {
1548+
tk.MustExec("CREATE TABLE t (id INT PRIMARY KEY, created_at DATETIME) TTL = created_at + INTERVAL 1 HOUR")
1549+
testTable, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t"))
1550+
require.NoError(t, err)
1551+
}()
1552+
1553+
// reschedule and try to get the job
1554+
require.NoError(t, m2.InfoSchemaCache().Update(se))
1555+
require.NoError(t, m2.TableStatusCache().Update(ctx, se))
1556+
m2.RescheduleJobs(se, now)
1557+
1558+
// the job cannot be cancelled, because it doesn't exist in the infoschema cache.
1559+
tk.MustQuery("select current_job_status from mysql.tidb_ttl_table_status").Check(testkit.Rows("running"))
1560+
1561+
// run GC
1562+
m2.DoGC(ctx, se, now)
1563+
1564+
// the job should have been cancelled
1565+
tk.MustQuery("select current_job_status, current_job_owner_hb_time from mysql.tidb_ttl_table_status").Check(testkit.Rows())
1566+
})
15011567
}

0 commit comments

Comments
 (0)