From 9a13cc21e2046121828522968b31cbe18ef1dd93 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Wed, 11 Jan 2023 11:20:11 +0800 Subject: [PATCH 1/5] ttl: forbid create/alter a table with TTL options when pk contains float/double column --- ddl/ttl.go | 30 ++++++ errno/errcode.go | 1 + errno/errname.go | 203 ++++++++++++++++++------------------ errors.toml | 15 +-- executor/ddl_test.go | 37 +++++++ ttl/sqlbuilder/sql_test.go | 59 +++++++---- util/dbterror/ddl_terror.go | 2 + 7 files changed, 216 insertions(+), 131 deletions(-) diff --git a/ddl/ttl.go b/ddl/ttl.go index e707b61ea9fbd..307034b41b1a9 100644 --- a/ddl/ttl.go +++ b/ddl/ttl.go @@ -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" @@ -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 { @@ -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`. diff --git a/errno/errcode.go b/errno/errcode.go index 56461a884a150..502d09dd0c562 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -1043,6 +1043,7 @@ const ( ErrSetTTLOptionForNonTTLTable = 8150 ErrTempTableNotAllowedWithTTL = 8151 ErrUnsupportedTTLReferencedByFK = 8152 + ErrUnsupportedPrimaryKeyTypeWithTTL = 8153 // Error codes used by TiDB ddl package ErrUnsupportedDDLOperation = 8200 diff --git a/errno/errname.go b/errno/errname.go index 871a33fbfcb29..01c8e768a873e 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -937,107 +937,108 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrSequenceInvalidTableStructure: mysql.Message("Sequence '%-.64s.%-.64s' table structure is invalid (%s)", nil), // TiDB errors. - ErrMemExceedThreshold: mysql.Message("%s holds %dB memory, exceeds threshold %dB.%s", nil), - ErrForUpdateCantRetry: mysql.Message("[%d] can not retry select for update statement", nil), - ErrAdminCheckTable: mysql.Message("TiDB admin check table failed.", nil), - ErrOptOnTemporaryTable: mysql.Message("`%s` is unsupported on temporary tables.", nil), - ErrDropTableOnTemporaryTable: mysql.Message("`drop global temporary table` can only drop global temporary table", nil), - ErrTxnTooLarge: mysql.Message("Transaction is too large, size: %d", nil), - ErrWriteConflictInTiDB: mysql.Message("Write conflict, txnStartTS %d is stale", nil), - ErrInvalidPluginID: mysql.Message("Wrong plugin id: %s, valid plugin id is [name]-[version], both name and version should not contain '-'", nil), - ErrInvalidPluginManifest: mysql.Message("Cannot read plugin %s's manifest", nil), - ErrInvalidPluginName: mysql.Message("Plugin load with %s but got wrong name %s", nil), - ErrInvalidPluginVersion: mysql.Message("Plugin load with %s but got %s", nil), - ErrDuplicatePlugin: mysql.Message("Plugin [%s] is redeclared", nil), - ErrInvalidPluginSysVarName: mysql.Message("Plugin %s's sysVar %s must start with its plugin name %s", nil), - ErrRequireVersionCheckFail: mysql.Message("Plugin %s require %s be %v but got %v", nil), - ErrUnsupportedReloadPlugin: mysql.Message("Plugin %s isn't loaded so cannot be reloaded", nil), - ErrUnsupportedReloadPluginVar: mysql.Message("Reload plugin with different sysVar is unsupported %v", nil), - ErrTableLocked: mysql.Message("Table '%s' was locked in %s by %v", nil), - ErrNotExist: mysql.Message("Error: key not exist", nil), - ErrTxnRetryable: mysql.Message("Error: KV error safe to retry %s ", []int{0}), - ErrCannotSetNilValue: mysql.Message("can not set nil value", nil), - ErrInvalidTxn: mysql.Message("invalid transaction", nil), - ErrEntryTooLarge: mysql.Message("entry too large, the max entry size is %d, the size of data is %d", nil), - ErrNotImplemented: mysql.Message("not implemented", nil), - ErrInfoSchemaExpired: mysql.Message("Information schema is out of date: schema failed to update in 1 lease, please make sure TiDB can connect to TiKV", nil), - ErrInfoSchemaChanged: mysql.Message("Information schema is changed during the execution of the statement(for example, table definition may be updated by other DDL ran in parallel). If you see this error often, try increasing `tidb_max_delta_schema_count`", nil), - ErrBadNumber: mysql.Message("Bad Number", nil), - ErrCastAsSignedOverflow: mysql.Message("Cast to signed converted positive out-of-range integer to it's negative complement", nil), - ErrCastNegIntAsUnsigned: mysql.Message("Cast to unsigned converted negative integer to it's positive complement", nil), - ErrInvalidYearFormat: mysql.Message("invalid year format", nil), - ErrInvalidYear: mysql.Message("invalid year", nil), - ErrIncorrectDatetimeValue: mysql.Message("Incorrect datetime value: '%s'", []int{0}), - ErrInvalidTimeFormat: mysql.Message("invalid time format: '%v'", []int{0}), - ErrInvalidWeekModeFormat: mysql.Message("invalid week mode format: '%v'", nil), - ErrFieldGetDefaultFailed: mysql.Message("Field '%s' get default value fail", nil), - ErrIndexOutBound: mysql.Message("Index column %s offset out of bound, offset: %d, row: %v", []int{2}), - ErrUnsupportedOp: mysql.Message("operation not supported", nil), - ErrRowNotFound: mysql.Message("can not find the row: %s", []int{0}), - ErrTableStateCantNone: mysql.Message("table %s can't be in none state", nil), - ErrColumnStateCantNone: mysql.Message("column %s can't be in none state", nil), - ErrColumnStateNonPublic: mysql.Message("can not use non-public column", nil), - ErrIndexStateCantNone: mysql.Message("index %s can't be in none state", nil), - ErrInvalidRecordKey: mysql.Message("invalid record key", nil), - ErrUnsupportedValueForVar: mysql.Message("variable '%s' does not yet support value: %s", nil), - ErrUnsupportedIsolationLevel: mysql.Message("The isolation level '%s' is not supported. Set tidb_skip_isolation_level_check=1 to skip this error", nil), - ErrInvalidDDLWorker: mysql.Message("Invalid DDL worker", nil), - ErrUnsupportedDDLOperation: mysql.Message("Unsupported %s", nil), - ErrNotOwner: mysql.Message("TiDB server is not a DDL owner", nil), - ErrCantDecodeRecord: mysql.Message("Cannot decode %s value, because %v", nil), - ErrInvalidDDLJob: mysql.Message("Invalid DDL job", nil), - ErrInvalidDDLJobFlag: mysql.Message("Invalid DDL job flag", nil), - ErrWaitReorgTimeout: mysql.Message("Timeout waiting for data reorganization", nil), - ErrInvalidStoreVersion: mysql.Message("Invalid storage current version: %d", nil), - ErrUnknownTypeLength: mysql.Message("Unknown length for type %d", nil), - ErrUnknownFractionLength: mysql.Message("Unknown length for type %d and fraction %d", nil), - ErrInvalidDDLJobVersion: mysql.Message("Version %d of DDL job is greater than current one: %d", nil), - ErrInvalidSplitRegionRanges: mysql.Message("Failed to split region ranges: %s", nil), - ErrReorgPanic: mysql.Message("Reorg worker panic", nil), - ErrInvalidDDLState: mysql.Message("Invalid %s state: %v", nil), - ErrCancelledDDLJob: mysql.Message("Cancelled DDL job", nil), - ErrRepairTable: mysql.Message("Failed to repair table: %s", nil), - ErrLoadPrivilege: mysql.Message("Load privilege table fail: %s", nil), - ErrInvalidPrivilegeType: mysql.Message("unknown privilege type %s", nil), - ErrUnknownFieldType: mysql.Message("unknown field type", nil), - ErrInvalidSequence: mysql.Message("invalid sequence", nil), - ErrInvalidType: mysql.Message("invalid type", nil), - ErrCantGetValidID: mysql.Message("Cannot get a valid auto-ID when retrying the statement", nil), - ErrCantSetToNull: mysql.Message("cannot set variable to null", nil), - ErrSnapshotTooOld: mysql.Message("snapshot is older than GC safe point %s", nil), - ErrInvalidTableID: mysql.Message("invalid TableID", nil), - ErrInvalidAutoRandom: mysql.Message("Invalid auto random: %s", nil), - ErrInvalidHashKeyFlag: mysql.Message("invalid encoded hash key flag", nil), - ErrInvalidListIndex: mysql.Message("invalid list index", nil), - ErrInvalidListMetaData: mysql.Message("invalid list meta data", nil), - ErrWriteOnSnapshot: mysql.Message("write on snapshot", nil), - ErrInvalidKey: mysql.Message("invalid key", nil), - ErrInvalidIndexKey: mysql.Message("invalid index key", nil), - ErrDataInconsistent: mysql.Message("data inconsistency in table: %s, index: %s, handle: %s, index-values:%#v != record-values:%#v", []int{2, 3, 4}), - ErrDDLReorgElementNotExist: mysql.Message("DDL reorg element does not exist", nil), - ErrDDLJobNotFound: mysql.Message("DDL Job:%v not found", nil), - ErrCancelFinishedDDLJob: mysql.Message("This job:%v is finished, so can't be cancelled", nil), - ErrCannotCancelDDLJob: mysql.Message("This job:%v is almost finished, can't be cancelled now", nil), - ErrUnknownAllocatorType: mysql.Message("Invalid allocator type", nil), - ErrAutoRandReadFailed: mysql.Message("Failed to read auto-random value from storage engine", nil), - ErrInvalidIncrementAndOffset: mysql.Message("Invalid auto_increment settings: auto_increment_increment: %d, auto_increment_offset: %d, both of them must be in range [1..65535]", nil), - ErrDataInconsistentMismatchCount: mysql.Message("data inconsistency in table: %s, index: %s, index-count:%d != record-count:%d", nil), - ErrDataInconsistentMismatchIndex: mysql.Message("data inconsistency in table: %s, index: %s, col: %s, handle: %#v, index-values:%#v != record-values:%#v, compare err:%#v", []int{3, 4, 5, 6}), - ErrInconsistentRowValue: mysql.Message("writing inconsistent data in table: %s, expected-values:{%s} != record-values:{%s}", []int{1, 2}), - ErrInconsistentHandle: mysql.Message("writing inconsistent data in table: %s, index: %s, index-handle:%#v != record-handle:%#v, index: %#v, record: %#v", []int{2, 3, 4, 5}), - ErrInconsistentIndexedValue: mysql.Message("writing inconsistent data in table: %s, index: %s, col: %s, indexed-value:{%s} != record-value:{%s}", []int{3, 4}), - ErrAssertionFailed: mysql.Message("assertion failed: key: %s, assertion: %s, start_ts: %v, existing start ts: %v, existing commit ts: %v", []int{0}), - ErrInstanceScope: mysql.Message("modifying %s will require SET GLOBAL in a future version of TiDB", nil), - ErrNonTransactionalJobFailure: mysql.Message("non-transactional job failed, job id: %d, total jobs: %d. job range: [%s, %s], job sql: %s, err: %v", []int{2, 3, 4}), - ErrSettingNoopVariable: mysql.Message("setting %s has no effect in TiDB", nil), - ErrGettingNoopVariable: mysql.Message("variable %s has no effect in TiDB", nil), - ErrCannotMigrateSession: mysql.Message("cannot migrate the current session: %s", nil), - ErrLazyUniquenessCheckFailure: mysql.Message("transaction aborted because lazy uniqueness check is enabled and an error occurred: %s", nil), - ErrUnsupportedColumnInTTLConfig: mysql.Message("Field '%-.192s' is of a not supported type for TTL config, expect DATETIME, DATE or TIMESTAMP", nil), - ErrTTLColumnCannotDrop: mysql.Message("Cannot drop column '%-.192s': needed in TTL config", nil), - ErrSetTTLOptionForNonTTLTable: mysql.Message("Cannot set %s on a table without TTL config", nil), - ErrTempTableNotAllowedWithTTL: mysql.Message("Set TTL for temporary table is not allowed", nil), - ErrUnsupportedTTLReferencedByFK: mysql.Message("Set TTL for a table referenced by foreign key is not allowed", nil), + ErrMemExceedThreshold: mysql.Message("%s holds %dB memory, exceeds threshold %dB.%s", nil), + ErrForUpdateCantRetry: mysql.Message("[%d] can not retry select for update statement", nil), + ErrAdminCheckTable: mysql.Message("TiDB admin check table failed.", nil), + ErrOptOnTemporaryTable: mysql.Message("`%s` is unsupported on temporary tables.", nil), + ErrDropTableOnTemporaryTable: mysql.Message("`drop global temporary table` can only drop global temporary table", nil), + ErrTxnTooLarge: mysql.Message("Transaction is too large, size: %d", nil), + ErrWriteConflictInTiDB: mysql.Message("Write conflict, txnStartTS %d is stale", nil), + ErrInvalidPluginID: mysql.Message("Wrong plugin id: %s, valid plugin id is [name]-[version], both name and version should not contain '-'", nil), + ErrInvalidPluginManifest: mysql.Message("Cannot read plugin %s's manifest", nil), + ErrInvalidPluginName: mysql.Message("Plugin load with %s but got wrong name %s", nil), + ErrInvalidPluginVersion: mysql.Message("Plugin load with %s but got %s", nil), + ErrDuplicatePlugin: mysql.Message("Plugin [%s] is redeclared", nil), + ErrInvalidPluginSysVarName: mysql.Message("Plugin %s's sysVar %s must start with its plugin name %s", nil), + ErrRequireVersionCheckFail: mysql.Message("Plugin %s require %s be %v but got %v", nil), + ErrUnsupportedReloadPlugin: mysql.Message("Plugin %s isn't loaded so cannot be reloaded", nil), + ErrUnsupportedReloadPluginVar: mysql.Message("Reload plugin with different sysVar is unsupported %v", nil), + ErrTableLocked: mysql.Message("Table '%s' was locked in %s by %v", nil), + ErrNotExist: mysql.Message("Error: key not exist", nil), + ErrTxnRetryable: mysql.Message("Error: KV error safe to retry %s ", []int{0}), + ErrCannotSetNilValue: mysql.Message("can not set nil value", nil), + ErrInvalidTxn: mysql.Message("invalid transaction", nil), + ErrEntryTooLarge: mysql.Message("entry too large, the max entry size is %d, the size of data is %d", nil), + ErrNotImplemented: mysql.Message("not implemented", nil), + ErrInfoSchemaExpired: mysql.Message("Information schema is out of date: schema failed to update in 1 lease, please make sure TiDB can connect to TiKV", nil), + ErrInfoSchemaChanged: mysql.Message("Information schema is changed during the execution of the statement(for example, table definition may be updated by other DDL ran in parallel). If you see this error often, try increasing `tidb_max_delta_schema_count`", nil), + ErrBadNumber: mysql.Message("Bad Number", nil), + ErrCastAsSignedOverflow: mysql.Message("Cast to signed converted positive out-of-range integer to it's negative complement", nil), + ErrCastNegIntAsUnsigned: mysql.Message("Cast to unsigned converted negative integer to it's positive complement", nil), + ErrInvalidYearFormat: mysql.Message("invalid year format", nil), + ErrInvalidYear: mysql.Message("invalid year", nil), + ErrIncorrectDatetimeValue: mysql.Message("Incorrect datetime value: '%s'", []int{0}), + ErrInvalidTimeFormat: mysql.Message("invalid time format: '%v'", []int{0}), + ErrInvalidWeekModeFormat: mysql.Message("invalid week mode format: '%v'", nil), + ErrFieldGetDefaultFailed: mysql.Message("Field '%s' get default value fail", nil), + ErrIndexOutBound: mysql.Message("Index column %s offset out of bound, offset: %d, row: %v", []int{2}), + ErrUnsupportedOp: mysql.Message("operation not supported", nil), + ErrRowNotFound: mysql.Message("can not find the row: %s", []int{0}), + ErrTableStateCantNone: mysql.Message("table %s can't be in none state", nil), + ErrColumnStateCantNone: mysql.Message("column %s can't be in none state", nil), + ErrColumnStateNonPublic: mysql.Message("can not use non-public column", nil), + ErrIndexStateCantNone: mysql.Message("index %s can't be in none state", nil), + ErrInvalidRecordKey: mysql.Message("invalid record key", nil), + ErrUnsupportedValueForVar: mysql.Message("variable '%s' does not yet support value: %s", nil), + ErrUnsupportedIsolationLevel: mysql.Message("The isolation level '%s' is not supported. Set tidb_skip_isolation_level_check=1 to skip this error", nil), + ErrInvalidDDLWorker: mysql.Message("Invalid DDL worker", nil), + ErrUnsupportedDDLOperation: mysql.Message("Unsupported %s", nil), + ErrNotOwner: mysql.Message("TiDB server is not a DDL owner", nil), + ErrCantDecodeRecord: mysql.Message("Cannot decode %s value, because %v", nil), + ErrInvalidDDLJob: mysql.Message("Invalid DDL job", nil), + ErrInvalidDDLJobFlag: mysql.Message("Invalid DDL job flag", nil), + ErrWaitReorgTimeout: mysql.Message("Timeout waiting for data reorganization", nil), + ErrInvalidStoreVersion: mysql.Message("Invalid storage current version: %d", nil), + ErrUnknownTypeLength: mysql.Message("Unknown length for type %d", nil), + ErrUnknownFractionLength: mysql.Message("Unknown length for type %d and fraction %d", nil), + ErrInvalidDDLJobVersion: mysql.Message("Version %d of DDL job is greater than current one: %d", nil), + ErrInvalidSplitRegionRanges: mysql.Message("Failed to split region ranges: %s", nil), + ErrReorgPanic: mysql.Message("Reorg worker panic", nil), + ErrInvalidDDLState: mysql.Message("Invalid %s state: %v", nil), + ErrCancelledDDLJob: mysql.Message("Cancelled DDL job", nil), + ErrRepairTable: mysql.Message("Failed to repair table: %s", nil), + ErrLoadPrivilege: mysql.Message("Load privilege table fail: %s", nil), + ErrInvalidPrivilegeType: mysql.Message("unknown privilege type %s", nil), + ErrUnknownFieldType: mysql.Message("unknown field type", nil), + ErrInvalidSequence: mysql.Message("invalid sequence", nil), + ErrInvalidType: mysql.Message("invalid type", nil), + ErrCantGetValidID: mysql.Message("Cannot get a valid auto-ID when retrying the statement", nil), + ErrCantSetToNull: mysql.Message("cannot set variable to null", nil), + ErrSnapshotTooOld: mysql.Message("snapshot is older than GC safe point %s", nil), + ErrInvalidTableID: mysql.Message("invalid TableID", nil), + ErrInvalidAutoRandom: mysql.Message("Invalid auto random: %s", nil), + ErrInvalidHashKeyFlag: mysql.Message("invalid encoded hash key flag", nil), + ErrInvalidListIndex: mysql.Message("invalid list index", nil), + ErrInvalidListMetaData: mysql.Message("invalid list meta data", nil), + ErrWriteOnSnapshot: mysql.Message("write on snapshot", nil), + ErrInvalidKey: mysql.Message("invalid key", nil), + ErrInvalidIndexKey: mysql.Message("invalid index key", nil), + ErrDataInconsistent: mysql.Message("data inconsistency in table: %s, index: %s, handle: %s, index-values:%#v != record-values:%#v", []int{2, 3, 4}), + ErrDDLReorgElementNotExist: mysql.Message("DDL reorg element does not exist", nil), + ErrDDLJobNotFound: mysql.Message("DDL Job:%v not found", nil), + ErrCancelFinishedDDLJob: mysql.Message("This job:%v is finished, so can't be cancelled", nil), + ErrCannotCancelDDLJob: mysql.Message("This job:%v is almost finished, can't be cancelled now", nil), + ErrUnknownAllocatorType: mysql.Message("Invalid allocator type", nil), + ErrAutoRandReadFailed: mysql.Message("Failed to read auto-random value from storage engine", nil), + ErrInvalidIncrementAndOffset: mysql.Message("Invalid auto_increment settings: auto_increment_increment: %d, auto_increment_offset: %d, both of them must be in range [1..65535]", nil), + ErrDataInconsistentMismatchCount: mysql.Message("data inconsistency in table: %s, index: %s, index-count:%d != record-count:%d", nil), + ErrDataInconsistentMismatchIndex: mysql.Message("data inconsistency in table: %s, index: %s, col: %s, handle: %#v, index-values:%#v != record-values:%#v, compare err:%#v", []int{3, 4, 5, 6}), + ErrInconsistentRowValue: mysql.Message("writing inconsistent data in table: %s, expected-values:{%s} != record-values:{%s}", []int{1, 2}), + ErrInconsistentHandle: mysql.Message("writing inconsistent data in table: %s, index: %s, index-handle:%#v != record-handle:%#v, index: %#v, record: %#v", []int{2, 3, 4, 5}), + ErrInconsistentIndexedValue: mysql.Message("writing inconsistent data in table: %s, index: %s, col: %s, indexed-value:{%s} != record-value:{%s}", []int{3, 4}), + ErrAssertionFailed: mysql.Message("assertion failed: key: %s, assertion: %s, start_ts: %v, existing start ts: %v, existing commit ts: %v", []int{0}), + ErrInstanceScope: mysql.Message("modifying %s will require SET GLOBAL in a future version of TiDB", nil), + ErrNonTransactionalJobFailure: mysql.Message("non-transactional job failed, job id: %d, total jobs: %d. job range: [%s, %s], job sql: %s, err: %v", []int{2, 3, 4}), + ErrSettingNoopVariable: mysql.Message("setting %s has no effect in TiDB", nil), + ErrGettingNoopVariable: mysql.Message("variable %s has no effect in TiDB", nil), + ErrCannotMigrateSession: mysql.Message("cannot migrate the current session: %s", nil), + ErrLazyUniquenessCheckFailure: mysql.Message("transaction aborted because lazy uniqueness check is enabled and an error occurred: %s", nil), + ErrUnsupportedColumnInTTLConfig: mysql.Message("Field '%-.192s' is of a not supported type for TTL config, expect DATETIME, DATE or TIMESTAMP", nil), + ErrTTLColumnCannotDrop: mysql.Message("Cannot drop column '%-.192s': needed in TTL config", nil), + ErrSetTTLOptionForNonTTLTable: mysql.Message("Cannot set %s on a table without TTL config", nil), + ErrTempTableNotAllowedWithTTL: mysql.Message("Set TTL for temporary table is not allowed", nil), + ErrUnsupportedTTLReferencedByFK: mysql.Message("Set TTL for a table referenced by foreign key is not allowed", nil), + ErrUnsupportedPrimaryKeyTypeWithTTL: mysql.Message("Unsupported clustered primary key type FLOAT/DOUBLE for TTL", nil), ErrWarnOptimizerHintInvalidInteger: mysql.Message("integer value is out of range in '%s'", nil), ErrWarnOptimizerHintUnsupportedHint: mysql.Message("Optimizer hint %s is not supported by TiDB and is ignored", nil), diff --git a/errors.toml b/errors.toml index ea9ed7231df55..0a7a6a88a215f 100644 --- a/errors.toml +++ b/errors.toml @@ -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 @@ -1386,16 +1391,6 @@ error = ''' Ingest failed: %s ''' -["domain:8027"] -error = ''' -Information schema is out of date: schema failed to update in 1 lease, please make sure TiDB can connect to TiKV -''' - -["domain:8028"] -error = ''' -Information schema is changed during the execution of the statement(for example, table definition may be updated by other DDL ran in parallel). If you see this error often, try increasing `tidb_max_delta_schema_count`. [try again later] -''' - ["domain:8243"] error = ''' HTTP request failed with status %s diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 59e8b1c719620..097e17b193267 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -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") +} diff --git a/ttl/sqlbuilder/sql_test.go b/ttl/sqlbuilder/sql_test.go index 67b9d282ed94b..8c00f4b7b3bc8 100644 --- a/ttl/sqlbuilder/sql_test.go +++ b/ttl/sqlbuilder/sql_test.go @@ -21,6 +21,10 @@ import ( "testing" "time" + "github.com/pingcap/tidb/parser/terror" + + "github.com/pingcap/tidb/util/dbterror" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" @@ -162,40 +166,56 @@ func TestFormatSQLDatum(t *testing.T) { // invalid pk types contains the types that should not exist in primary keys of a TTL table. // We do not need to check sqlbuilder.FormatSQLDatum for these types invalidPKTypes := []struct { - types []string - errMsg string + types []string + err *terror.Error }{ { - types: []string{"json"}, - errMsg: "[ddl:3152]JSON column 'pk0' cannot be used in key specification.", + types: []string{"json"}, + err: dbterror.ErrJSONUsedAsKey, + }, + { + types: []string{"blob"}, + err: dbterror.ErrBlobKeyWithoutLength, + }, + { + types: []string{"blob(8)"}, + err: dbterror.ErrBlobKeyWithoutLength, + }, + { + types: []string{"text"}, + err: dbterror.ErrBlobKeyWithoutLength, + }, + { + types: []string{"text(8)"}, + err: dbterror.ErrBlobKeyWithoutLength, }, { - types: []string{"blob"}, - errMsg: "[ddl:1170]BLOB/TEXT column 'pk0' used in key specification without a key length", + types: []string{"int", "json"}, + err: dbterror.ErrJSONUsedAsKey, }, { - types: []string{"blob(8)"}, - errMsg: "[ddl:1170]BLOB/TEXT column 'pk0' used in key specification without a key length", + types: []string{"int", "blob"}, + err: dbterror.ErrBlobKeyWithoutLength, }, { - types: []string{"text"}, - errMsg: "[ddl:1170]BLOB/TEXT column 'pk0' used in key specification without a key length", + types: []string{"int", "text"}, + err: dbterror.ErrBlobKeyWithoutLength, }, { - types: []string{"text(8)"}, - errMsg: "[ddl:1170]BLOB/TEXT column 'pk0' used in key specification without a key length", + types: []string{"float"}, + err: dbterror.ErrUnsupportedPrimaryKeyTypeWithTTL, }, { - types: []string{"int", "json"}, - errMsg: "[ddl:3152]JSON column 'pk1' cannot be used in key specification.", + types: []string{"double"}, + err: dbterror.ErrUnsupportedPrimaryKeyTypeWithTTL, }, { - types: []string{"int", "blob"}, - errMsg: "[ddl:1170]BLOB/TEXT column 'pk1' used in key specification without a key length", + types: []string{"int", "float"}, + err: dbterror.ErrUnsupportedPrimaryKeyTypeWithTTL, }, { - types: []string{"int", "text"}, - errMsg: "[ddl:1170]BLOB/TEXT column 'pk1' used in key specification without a key length", + types: []string{"int", "double"}, + err: dbterror.ErrUnsupportedPrimaryKeyTypeWithTTL, }, } @@ -321,8 +341,7 @@ func TestFormatSQLDatum(t *testing.T) { sb.WriteString("primary key (") sb.WriteString(strings.Join(cols, ", ")) sb.WriteString(")) TTL=`t` + INTERVAL 1 DAY") - err := tk.ExecToErr(sb.String()) - require.Equal(t, c.errMsg, err.Error(), sb.String()) + tk.MustGetDBError(sb.String(), c.err) } // create a table with n columns diff --git a/util/dbterror/ddl_terror.go b/util/dbterror/ddl_terror.go index f975a421fae6a..ddacf77c025ef 100644 --- a/util/dbterror/ddl_terror.go +++ b/util/dbterror/ddl_terror.go @@ -431,6 +431,8 @@ var ( ErrTempTableNotAllowedWithTTL = ClassDDL.NewStd(mysql.ErrTempTableNotAllowedWithTTL) // ErrUnsupportedTTLReferencedByFK returns when the TTL config is set for a table referenced by foreign key ErrUnsupportedTTLReferencedByFK = ClassDDL.NewStd(mysql.ErrUnsupportedTTLReferencedByFK) + // ErrUnsupportedPrimaryKeyTypeWithTTL returns when create or alter a table with TTL options but the primary key is not supported + ErrUnsupportedPrimaryKeyTypeWithTTL = ClassDDL.NewStd(mysql.ErrUnsupportedPrimaryKeyTypeWithTTL) // ErrNotSupportedYet returns when tidb does not support this feature. ErrNotSupportedYet = ClassDDL.NewStd(mysql.ErrNotSupportedYet) From 7d30e4c264b301ccccf1e27dd3f4853c932363bd Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Wed, 11 Jan 2023 11:29:58 +0800 Subject: [PATCH 2/5] format --- ttl/sqlbuilder/sql_test.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/ttl/sqlbuilder/sql_test.go b/ttl/sqlbuilder/sql_test.go index 8c00f4b7b3bc8..76e42e6c5ca18 100644 --- a/ttl/sqlbuilder/sql_test.go +++ b/ttl/sqlbuilder/sql_test.go @@ -21,19 +21,17 @@ import ( "testing" "time" - "github.com/pingcap/tidb/parser/terror" - - "github.com/pingcap/tidb/util/dbterror" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/ttl/cache" "github.com/pingcap/tidb/ttl/sqlbuilder" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/sqlexec" "github.com/stretchr/testify/require" ) From bc266b8178f4b4b11c1fd92e1c9f0c2ae3c26ac9 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Wed, 11 Jan 2023 11:32:30 +0800 Subject: [PATCH 3/5] update --- ttl/sqlbuilder/BUILD.bazel | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ttl/sqlbuilder/BUILD.bazel b/ttl/sqlbuilder/BUILD.bazel index 95d0371243158..505e9ffcb3576 100644 --- a/ttl/sqlbuilder/BUILD.bazel +++ b/ttl/sqlbuilder/BUILD.bazel @@ -31,10 +31,12 @@ go_test( "//parser/ast", "//parser/model", "//parser/mysql", + "//parser/terror", "//testkit", "//testkit/testsetup", "//ttl/cache", "//types", + "//util/dbterror", "//util/sqlexec", "@com_github_stretchr_testify//require", "@org_uber_go_goleak//:goleak", From 706deb43f52242796c29bdbf7fdd60d6ea38f109 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Wed, 11 Jan 2023 11:43:22 +0800 Subject: [PATCH 4/5] update --- errors.toml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/errors.toml b/errors.toml index 0a7a6a88a215f..36768c7311e2b 100644 --- a/errors.toml +++ b/errors.toml @@ -1391,6 +1391,16 @@ error = ''' Ingest failed: %s ''' +["domain:8027"] +error = ''' +Information schema is out of date: schema failed to update in 1 lease, please make sure TiDB can connect to TiKV +''' + +["domain:8028"] +error = ''' +Information schema is changed during the execution of the statement(for example, table definition may be updated by other DDL ran in parallel). If you see this error often, try increasing `tidb_max_delta_schema_count`. [try again later] +''' + ["domain:8243"] error = ''' HTTP request failed with status %s From 06d7fa46aee5118213bf084216e47b059b085ca5 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Wed, 11 Jan 2023 12:33:39 +0800 Subject: [PATCH 5/5] update --- ttl/cache/split_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/ttl/cache/split_test.go b/ttl/cache/split_test.go index 1d2279eb8d0f9..35638f1d1f409 100644 --- a/ttl/cache/split_test.go +++ b/ttl/cache/split_test.go @@ -451,7 +451,6 @@ func TestNoTTLSplitSupportTables(t *testing.T) { tbls := []*cache.PhysicalTable{ createTTLTable(t, tk, "t1", "char(32) CHARACTER SET UTF8MB4"), createTTLTable(t, tk, "t2", "varchar(32) CHARACTER SET UTF8MB4"), - createTTLTable(t, tk, "t3", "double"), createTTLTable(t, tk, "t4", "decimal(32, 2)"), create2PKTTLTable(t, tk, "t5", "char(32) CHARACTER SET UTF8MB4"), }