Skip to content

Commit

Permalink
Merge branch 'master' into fk-fix10
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 authored Jan 11, 2023
2 parents 12b7532 + caffd8d commit 542d7f5
Show file tree
Hide file tree
Showing 16 changed files with 542 additions and 162 deletions.
30 changes: 30 additions & 0 deletions ddl/ttl.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/tidb/parser/duration"
"github.com/pingcap/tidb/parser/format"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessiontxn"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -143,6 +144,10 @@ func checkTTLTableSuitable(ctx sessionctx.Context, schema model.CIStr, tblInfo *
return dbterror.ErrTempTableNotAllowedWithTTL
}

if err := checkPrimaryKeyForTTLTable(tblInfo); err != nil {
return err
}

// checks even when the foreign key check is not enabled, to keep safe
is := sessiontxn.GetTxnManager(ctx).GetTxnInfoSchema()
if referredFK := checkTableHasForeignKeyReferred(is, schema.L, tblInfo.Name.L, nil, true); referredFK != nil {
Expand All @@ -162,6 +167,31 @@ func checkDropColumnWithTTLConfig(tblInfo *model.TableInfo, colName string) erro
return nil
}

// We should forbid creating a TTL table with clustered primary key that contains a column with type float/double.
// This is because currently we are using SQL to delete expired rows and when the primary key contains float/double column,
// it is hard to use condition `WHERE PK in (...)` to delete specified rows because some precision will be lost when comparing.
func checkPrimaryKeyForTTLTable(tblInfo *model.TableInfo) error {
if !tblInfo.IsCommonHandle {
// only check the primary keys when it is common handle
return nil
}

pk := tblInfo.GetPrimaryKey()
if pk == nil {
return nil
}

for _, colDef := range pk.Columns {
col := tblInfo.Columns[colDef.Offset]
switch col.GetType() {
case mysql.TypeFloat, mysql.TypeDouble:
return dbterror.ErrUnsupportedPrimaryKeyTypeWithTTL
}
}

return nil
}

// getTTLInfoInOptions returns the aggregated ttlInfo, the ttlEnable, or an error.
// if TTL, TTL_ENABLE or TTL_JOB_INTERVAL is not set in the config, the corresponding return value will be nil.
// if both of TTL and TTL_ENABLE are set, the `ttlInfo.Enable` will be equal with `ttlEnable`.
Expand Down
7 changes: 6 additions & 1 deletion domain/historical_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,5 +77,10 @@ func (w *HistoricalStatsWorker) DumpHistoricalStats(tableID int64, statsHandle *

// GetOneHistoricalStatsTable gets one tableID from channel, only used for test
func (w *HistoricalStatsWorker) GetOneHistoricalStatsTable() int64 {
return <-w.tblCH
select {
case tblID := <-w.tblCH:
return tblID
default:
return -1
}
}
1 change: 1 addition & 0 deletions errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -1043,6 +1043,7 @@ const (
ErrSetTTLOptionForNonTTLTable = 8150
ErrTempTableNotAllowedWithTTL = 8151
ErrUnsupportedTTLReferencedByFK = 8152
ErrUnsupportedPrimaryKeyTypeWithTTL = 8153

// Error codes used by TiDB ddl package
ErrUnsupportedDDLOperation = 8200
Expand Down
203 changes: 102 additions & 101 deletions errno/errname.go

Large diffs are not rendered by default.

5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -1236,6 +1236,11 @@ error = '''
Set TTL for a table referenced by foreign key is not allowed
'''

["ddl:8153"]
error = '''
Unsupported clustered primary key type FLOAT/DOUBLE for TTL
'''

["ddl:8200"]
error = '''
Unsupported shard_row_id_bits for table with primary key as row id
Expand Down
37 changes: 37 additions & 0 deletions executor/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1646,3 +1646,40 @@ func TestDisableTTLForFKParentTable(t *testing.T) {
tk.MustGetDBError("ALTER TABLE t_1 ADD FOREIGN KEY fk_t_id(t_id) references t(id)", dbterror.ErrUnsupportedTTLReferencedByFK)
tk.MustExec("drop table t,t_1")
}

func TestCheckPrimaryKeyForTTLTable(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

// create table should fail when pk contains double/float
tk.MustGetDBError("create table t1(id float primary key, t timestamp) TTL=`t`+INTERVAL 1 DAY", dbterror.ErrUnsupportedPrimaryKeyTypeWithTTL)
tk.MustGetDBError("create table t1(id float(10,2) primary key, t timestamp) TTL=`t`+INTERVAL 1 DAY", dbterror.ErrUnsupportedPrimaryKeyTypeWithTTL)
tk.MustGetDBError("create table t1(id double primary key, t timestamp) TTL=`t`+INTERVAL 1 DAY", dbterror.ErrUnsupportedPrimaryKeyTypeWithTTL)
tk.MustGetDBError("create table t1(id float(10,2) primary key, t timestamp) TTL=`t`+INTERVAL 1 DAY", dbterror.ErrUnsupportedPrimaryKeyTypeWithTTL)
tk.MustGetDBError("create table t1(id1 int, id2 float, t timestamp, primary key(id1, id2)) TTL=`t`+INTERVAL 1 DAY", dbterror.ErrUnsupportedPrimaryKeyTypeWithTTL)
tk.MustGetDBError("create table t1(id1 int, id2 double, t timestamp, primary key(id1, id2)) TTL=`t`+INTERVAL 1 DAY", dbterror.ErrUnsupportedPrimaryKeyTypeWithTTL)

// alter table should fail when pk contains double/float
tk.MustExec("create table t1(id float primary key, t timestamp)")
tk.MustExec("create table t2(id double primary key, t timestamp)")
tk.MustExec("create table t3(id1 int, id2 float, primary key(id1, id2), t timestamp)")
tk.MustExec("create table t4(id1 int, id2 double, primary key(id1, id2), t timestamp)")
tk.MustGetDBError("alter table t1 TTL=`t`+INTERVAL 1 DAY", dbterror.ErrUnsupportedPrimaryKeyTypeWithTTL)
tk.MustGetDBError("alter table t2 TTL=`t`+INTERVAL 1 DAY", dbterror.ErrUnsupportedPrimaryKeyTypeWithTTL)
tk.MustGetDBError("alter table t3 TTL=`t`+INTERVAL 1 DAY", dbterror.ErrUnsupportedPrimaryKeyTypeWithTTL)
tk.MustGetDBError("alter table t4 TTL=`t`+INTERVAL 1 DAY", dbterror.ErrUnsupportedPrimaryKeyTypeWithTTL)

// create table should not fail when the pk is not clustered
tk.MustExec("create table t11(id float primary key nonclustered, t timestamp) TTL=`t`+INTERVAL 1 DAY")
tk.MustExec("create table t12(id double primary key nonclustered, t timestamp) TTL=`t`+INTERVAL 1 DAY")
tk.MustExec("create table t13(id1 int, id2 float, t timestamp, primary key(id1, id2) nonclustered) TTL=`t`+INTERVAL 1 DAY")

// alter table should not fail when the pk is not clustered
tk.MustExec("create table t21(id float primary key nonclustered, t timestamp)")
tk.MustExec("create table t22(id double primary key nonclustered, t timestamp)")
tk.MustExec("create table t23(id1 int, id2 float, t timestamp, primary key(id1, id2) nonclustered)")
tk.MustExec("alter table t21 TTL=`t`+INTERVAL 1 DAY")
tk.MustExec("alter table t22 TTL=`t`+INTERVAL 1 DAY")
tk.MustExec("alter table t23 TTL=`t`+INTERVAL 1 DAY")
}
63 changes: 63 additions & 0 deletions executor/historical_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/testkit"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/oracle"
)

func TestRecordHistoryStatsAfterAnalyze(t *testing.T) {
Expand Down Expand Up @@ -243,3 +244,65 @@ PARTITION p0 VALUES LESS THAN (6)
require.NoError(t, err)
tk.MustQuery("select count(*) from mysql.stats_history").Check(testkit.Rows("2"))
}

func TestDumpHistoricalStatsByTable(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set global tidb_enable_historical_stats = 1")
tk.MustExec("set @@tidb_partition_prune_mode='static'")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec(`CREATE TABLE t (a int, b int, index idx(b))
PARTITION BY RANGE ( a ) (
PARTITION p0 VALUES LESS THAN (6)
)`)
// dump historical stats
h := dom.StatsHandle()

tk.MustExec("analyze table t")
is := dom.InfoSchema()
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
require.NotNil(t, tbl)

// dump historical stats
hsWorker := dom.GetHistoricalStatsWorker()
// only partition p0 stats will be dumped in static mode
tblID := hsWorker.GetOneHistoricalStatsTable()
require.NotEqual(t, tblID, -1)
err = hsWorker.DumpHistoricalStats(tblID, h)
require.NoError(t, err)
tblID = hsWorker.GetOneHistoricalStatsTable()
require.Equal(t, tblID, int64(-1))

time.Sleep(1 * time.Second)
snapshot := oracle.GoTimeToTS(time.Now())
jsTable, err := h.DumpHistoricalStatsBySnapshot("test", tbl.Meta(), snapshot)
require.NoError(t, err)
require.NotNil(t, jsTable)
// only has p0 stats
require.NotNil(t, jsTable.Partitions["p0"])
require.Nil(t, jsTable.Partitions["global"])

// change static to dynamic then assert
tk.MustExec("set @@tidb_partition_prune_mode='dynamic'")
tk.MustExec("analyze table t")
require.NoError(t, err)
// global and p0's stats will be dumped
tblID = hsWorker.GetOneHistoricalStatsTable()
require.NotEqual(t, tblID, -1)
err = hsWorker.DumpHistoricalStats(tblID, h)
require.NoError(t, err)
tblID = hsWorker.GetOneHistoricalStatsTable()
require.NotEqual(t, tblID, -1)
err = hsWorker.DumpHistoricalStats(tblID, h)
require.NoError(t, err)
time.Sleep(1 * time.Second)
snapshot = oracle.GoTimeToTS(time.Now())
jsTable, err = h.DumpHistoricalStatsBySnapshot("test", tbl.Meta(), snapshot)
require.NoError(t, err)
require.NotNil(t, jsTable)
// has both global and p0 stats
require.NotNil(t, jsTable.Partitions["p0"])
require.NotNil(t, jsTable.Partitions["global"])
}
Loading

0 comments on commit 542d7f5

Please sign in to comment.