ddl: refine the criterion for modify column (#64834)

close pingcap/tiflash#10509, ref pingcap/tidb#63366
This commit is contained in:
Ruihao Chen
2025-12-18 19:41:59 +08:00
committed by GitHub
parent 2bde4659c8
commit bf83436418
2 changed files with 182 additions and 40 deletions

View File

@ -46,6 +46,7 @@ import (
"github.com/pingcap/tidb/pkg/table/tables"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/collate"
"github.com/pingcap/tidb/pkg/util/dbterror"
"github.com/pingcap/tidb/pkg/util/filter"
"github.com/pingcap/tidb/pkg/util/intest"
@ -60,6 +61,14 @@ func isNullToNotNullChange(oldCol, newCol *model.ColumnInfo) bool {
return !mysql.HasNotNullFlag(oldCol.GetFlag()) && mysql.HasNotNullFlag(newCol.GetFlag())
}
func isIntegerChange(from, to *model.ColumnInfo) bool {
return mysql.IsIntegerType(from.GetType()) && mysql.IsIntegerType(to.GetType())
}
func isCharChange(from, to *model.ColumnInfo) bool {
return types.IsTypeChar(from.GetType()) && types.IsTypeChar(to.GetType())
}
// getModifyColumnType gets the modify column type.
// 1. ModifyTypeNoReorg: The range of new type is a superset of the old type
// 2. ModifyTypeNoReorgWithCheck: The range of new type is a subset of the old type, but we are running in strict SQL mode.
@ -88,8 +97,8 @@ func getModifyColumnType(
return model.ModifyTypeReorg
}
// FIXME(joechenrh): handle partition table case
if tblInfo.Partition != nil {
// FIXME(joechenrh): handle partition and TiFlash replica case
if tblInfo.Partition != nil || (tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Count > 0) {
return model.ModifyTypeReorg
}
@ -99,6 +108,16 @@ func getModifyColumnType(
}
})
// FIXME(joechenrh): remove this when stats correctness is resolved.
// Since stats may store bytes encoded by codec.EncodeKey, we should disable the optimization
// if the same data produces different encoded bytes for the old and new types.
if (isIntegerChange(oldCol, args.Column) &&
mysql.HasUnsignedFlag(oldCol.GetFlag()) != mysql.HasUnsignedFlag(args.Column.GetFlag())) ||
(isCharChange(oldCol, args.Column) &&
!collate.CompatibleCollate(oldCol.GetCollate(), args.Column.GetCollate())) {
return model.ModifyTypeReorg
}
if !sqlMode.HasStrictMode() {
return model.ModifyTypeReorg
}
@ -752,17 +771,14 @@ func adjustForeignKeyChildTableInfoAfterModifyColumn(infoCache *infoschema.InfoC
}
func needIndexReorg(oldCol, changingCol *model.ColumnInfo) bool {
if mysql.IsIntegerType(oldCol.GetType()) && mysql.IsIntegerType(changingCol.GetType()) {
if isIntegerChange(oldCol, changingCol) {
return mysql.HasUnsignedFlag(oldCol.GetFlag()) != mysql.HasUnsignedFlag(changingCol.GetFlag())
}
// CHAR/VARCHAR
if !types.IsTypeChar(oldCol.GetType()) || !types.IsTypeChar(changingCol.GetType()) {
return true
}
intest.Assert(isCharChange(oldCol, changingCol))
// Check index key part, ref tablecodec.GenIndexKey
if oldCol.GetCollate() != changingCol.GetCollate() {
if !collate.CompatibleCollate(oldCol.GetCollate(), changingCol.GetCollate()) {
return true
}
@ -773,19 +789,18 @@ func needIndexReorg(oldCol, changingCol *model.ColumnInfo) bool {
}
func needRowReorg(oldCol, changingCol *model.ColumnInfo) bool {
oldTp := oldCol.GetType()
changingTp := changingCol.GetType()
if mysql.IsIntegerType(oldTp) && mysql.IsIntegerType(changingTp) {
// Integer changes can skip reorg
if isIntegerChange(oldCol, changingCol) {
return false
}
// _bin collation has padding, it must need reorg.
if types.IsBinaryStr(&oldCol.FieldType) || types.IsBinaryStr(&changingCol.FieldType) {
// Other changes except char changes need row reorg.
if !isCharChange(oldCol, changingCol) {
return true
}
return !types.IsTypeChar(oldTp) || !types.IsTypeChar(changingTp)
// We have checked charset before, only need to check binary string, which needs padding.
return types.IsBinaryStr(&oldCol.FieldType) || types.IsBinaryStr(&changingCol.FieldType)
}
// checkModifyColumnData checks the values of the old column data
@ -1102,7 +1117,7 @@ func (w *worker) doModifyColumnTypeWithData(
default:
errMsg := fmt.Sprintf("unexpected column state %s in modify column job", oldCol.State)
intest.Assert(false, errMsg)
return ver, errors.New(errMsg)
return ver, errors.Errorf("%s", errMsg)
}
default:
err = dbterror.ErrInvalidDDLState.GenWithStackByArgs("column", changingCol.State)
@ -1296,7 +1311,7 @@ func (w *worker) doModifyColumnIndexReorg(
default:
errMsg := fmt.Sprintf("unexpected column state %s in modify column job", oldCol.State)
intest.Assert(false, errMsg)
return ver, errors.New(errMsg)
return ver, errors.Errorf("%s", errMsg)
}
default:
err = dbterror.ErrInvalidDDLState.GenWithStackByArgs("column", oldIdxInfos[0].State)

View File

@ -716,24 +716,24 @@ func TestGetModifyColumnType(t *testing.T) {
{
beforeType: "bigint",
afterType: "bigint unsigned",
tp: model.ModifyTypeNoReorgWithCheck,
tp: model.ModifyTypeReorg,
},
{
beforeType: "bigint",
afterType: "bigint unsigned",
index: true,
tp: model.ModifyTypeIndexReorg,
tp: model.ModifyTypeReorg,
},
{
beforeType: "int unsigned",
afterType: "bigint",
tp: model.ModifyTypeNoReorgWithCheck,
tp: model.ModifyTypeReorg,
},
{
beforeType: "int unsigned",
afterType: "bigint",
index: true,
tp: model.ModifyTypeIndexReorg,
tp: model.ModifyTypeReorg,
},
// string
{
@ -829,25 +829,25 @@ func TestGetModifyColumnType(t *testing.T) {
beforeType: "char(20) collate utf8mb4_bin",
afterType: "varchar(10) collate utf8_unicode_ci",
index: true,
tp: model.ModifyTypeIndexReorg,
tp: model.ModifyTypeReorg,
},
{
beforeType: "char(20) collate utf8_unicode_ci",
afterType: "varchar(10) collate utf8mb4_bin",
index: true,
tp: model.ModifyTypeIndexReorg,
tp: model.ModifyTypeReorg,
},
{
beforeType: "varchar(20) collate utf8mb4_bin",
afterType: "char(10) collate utf8_unicode_ci",
index: true,
tp: model.ModifyTypeIndexReorg,
tp: model.ModifyTypeReorg,
},
{
beforeType: "varchar(20) collate utf8_unicode_ci",
afterType: "char(10) collate utf8mb4_bin",
index: true,
tp: model.ModifyTypeIndexReorg,
tp: model.ModifyTypeReorg,
},
}
@ -1070,18 +1070,18 @@ func TestModifyIntegerColumn(t *testing.T) {
for _, val := range insertVal {
tk.MustExec(fmt.Sprintf("insert into t values(%s)", val))
err := tk.ExecToErr(fmt.Sprintf("alter table t modify column a %s", newColTp))
require.Contains(t, err.Error(), "Data truncated for column 'a'")
require.True(t, strings.Contains(err.Error(), "Data truncated for column 'a'") || strings.Contains(err.Error(), "overflow"))
tk.MustExec("delete from t")
}
}
successValue := func(insertVal string, newColTp string) {
successValue := func(insertVal string, newColTp string, expectReorgTp byte) {
tk.MustExec(fmt.Sprintf("insert into t values %s", insertVal))
tk.MustExec(fmt.Sprintf("alter table t modify column a %s", newColTp))
require.Equal(t, model.ModifyTypeNoReorgWithCheck, reorgType)
require.Equal(t, expectReorgTp, reorgType)
}
signed2Signed := func(oldColTp, newColTp string, t *testing.T, expectReorgTp byte) {
signed2Signed := func(oldColTp, newColTp string, expectReorgTp byte) {
maxValOfNewCol, minValOfNewCol := maxMinSignedVal[newColTp][0], maxMinSignedVal[newColTp][1]
maxValOfOldCol, minValOfOldCol := maxMinSignedVal[oldColTp][0], maxMinSignedVal[oldColTp][1]
tk.MustExec("drop table if exists t")
@ -1100,10 +1100,10 @@ func TestModifyIntegerColumn(t *testing.T) {
}, newColTp)
// [maxValOfNewCol, minValOfNewCol] pass
successValue(fmt.Sprintf("(%d), (%d), (0)", maxValOfNewCol, minValOfNewCol), newColTp)
successValue(fmt.Sprintf("(%d), (%d), (0)", maxValOfNewCol, minValOfNewCol), newColTp, expectReorgTp)
}
unsigned2Unsigned := func(oldColTp, newColTp string, t *testing.T, expectReorgTp byte) {
unsigned2Unsigned := func(oldColTp, newColTp string, expectReorgTp byte) {
maxValOfNewCol, minValOfNewCol := maxMinUnsignedVal[newColTp][0], maxMinUnsignedVal[newColTp][1]
maxValOfOldCol := maxMinUnsignedVal[oldColTp][0]
tk.MustExec("drop table if exists t")
@ -1116,10 +1116,10 @@ func TestModifyIntegerColumn(t *testing.T) {
}, newColTp)
// [0, maxValOfNewCol] pass
successValue(fmt.Sprintf("(%d), (%d), (1)", maxValOfNewCol, minValOfNewCol), newColTp)
successValue(fmt.Sprintf("(%d), (%d), (1)", maxValOfNewCol, minValOfNewCol), newColTp, expectReorgTp)
}
signed2Unsigned := func(oldColTp, newColTp string, t *testing.T, expectReorgTp byte, oldColIdx, newColIdx int) {
signed2Unsigned := func(oldColTp, newColTp string, expectReorgTp byte, oldColIdx, newColIdx int) {
maxValOfOldCol, minValOfOldCol := maxMinSignedVal[oldColTp][0], maxMinSignedVal[oldColTp][1]
maxValOfNewCol := maxMinUnsignedVal[newColTp][0]
tk.MustExec("drop table if exists t")
@ -1140,10 +1140,10 @@ func TestModifyIntegerColumn(t *testing.T) {
}
// [0, min(maxValOfOldCol, maxValOfNewCol)] pass
successValue(fmt.Sprintf("(%d), (1), (0)", min(uint(maxValOfOldCol), maxValOfNewCol)), newColTp)
successValue(fmt.Sprintf("(%d), (1), (0)", min(uint(maxValOfOldCol), maxValOfNewCol)), newColTp, expectReorgTp)
}
unsigned2Signed := func(oldColTp, newColTp string, t *testing.T, expectReorgTp byte) {
unsigned2Signed := func(oldColTp, newColTp string, expectReorgTp byte) {
maxValOfNewCol := maxMinSignedVal[newColTp][0]
maxValOfOldCol := maxMinUnsignedVal[oldColTp][0]
tk.MustExec("drop table if exists t")
@ -1156,7 +1156,7 @@ func TestModifyIntegerColumn(t *testing.T) {
}, newColTp)
// [0, maxValOfNewCol] pass
successValue(fmt.Sprintf("(%d), (1), (0)", maxValOfNewCol), newColTp)
successValue(fmt.Sprintf("(%d), (1), (0)", maxValOfNewCol), newColTp, expectReorgTp)
}
signedTp := []string{"bigint", "int", "mediumint", "smallint", "tinyint"}
@ -1165,24 +1165,24 @@ func TestModifyIntegerColumn(t *testing.T) {
// 1. signed -> signed
// bigint -> int, mediumint, smallint, tinyint; int -> mediumint, smallint, tinyint; ...
for newColIdx := oldColIdx + 1; newColIdx < len(signedTp); newColIdx++ {
signed2Signed(signedTp[oldColIdx], signedTp[newColIdx], t, model.ModifyTypeNoReorgWithCheck)
signed2Signed(signedTp[oldColIdx], signedTp[newColIdx], model.ModifyTypeNoReorgWithCheck)
}
// 2. signed -> unsigned
// bigint -> bigint unsigned, int unsigned, mediumint unsigned, smallint unsigned, tinyint unsigned; int -> int unsigned, mediumint unsigned, smallint unsigned, tinyint unsigned; ...
for newColIdx := range unsignedTp {
signed2Unsigned(signedTp[oldColIdx], unsignedTp[newColIdx], t, model.ModifyTypeNoReorgWithCheck, oldColIdx, newColIdx)
signed2Unsigned(signedTp[oldColIdx], unsignedTp[newColIdx], model.ModifyTypeReorg, oldColIdx, newColIdx)
}
}
for oldColIdx := range unsignedTp {
// 3. unsigned -> unsigned
// bigint unsigned -> int unsigned, mediumint unsigned, smallint unsigned, tinyint unsigned; int unsigned -> mediumint unsigned, smallint unsigned, tinyint unsigned; ...
for newColIdx := oldColIdx + 1; newColIdx < len(unsignedTp); newColIdx++ {
unsigned2Unsigned(unsignedTp[oldColIdx], unsignedTp[newColIdx], t, model.ModifyTypeNoReorgWithCheck)
unsigned2Unsigned(unsignedTp[oldColIdx], unsignedTp[newColIdx], model.ModifyTypeNoReorgWithCheck)
}
// 4. unsigned -> signed
// bigint unsigned -> bigint, int, mediumint, smallint, tinyint; int unsigned -> int, mediumint, smallint, tinyint; ...
for newColIdx := oldColIdx; newColIdx < len(signedTp); newColIdx++ {
unsigned2Signed(unsignedTp[oldColIdx], signedTp[newColIdx], t, model.ModifyTypeNoReorgWithCheck)
unsigned2Signed(unsignedTp[oldColIdx], signedTp[newColIdx], model.ModifyTypeReorg)
}
}
}
@ -1363,3 +1363,130 @@ func TestModifyColumnWithDifferentCollation(t *testing.T) {
}
}
}
func TestStatsAfterModifyColumn(t *testing.T) {
type query struct {
pred string
idx string
}
type testCase struct {
caseName string
createTableSQL string
modifySQL string
embeddedAnalyze bool
checkResult bool
queries []query
}
tcs := []testCase{
{
// Check stats correctness after modifying column without any reorg
// We don't add index on b, because these indexes need reorg due to NeedRestoreData changes.
caseName: "no reorg without analyze",
createTableSQL: "create table t (a bigint, b char(16) collate utf8mb4_bin, index i1(a))",
modifySQL: "alter table t modify column a int, modify column b varchar(16) collate utf8mb4_bin",
embeddedAnalyze: false,
checkResult: true,
queries: []query{
{"a < 10", "i1"},
{"a <= 10", ""},
{"a > 10", "i1"},
{"a >= 10", ""},
{"a = 10", "i1"},
{"a = -1", ""},
{"b < '10'", ""},
{"b <= '10'", ""},
{"b > '10'", ""},
{"b >= '10'", ""},
{"b = '10'", ""},
{"b = 'non-exist'", ""},
},
},
{
// Only indexes are rewritten.
// The row data remains the same, so the stats are still valid.
caseName: "row and index reorg with analyze",
createTableSQL: "create table t (a bigint, b char(16) collate utf8mb4_bin, index i1(a), index i2(b))",
modifySQL: "alter table t modify column a int, modify column b varchar(16) collate utf8mb4_bin",
embeddedAnalyze: true,
checkResult: true,
queries: []query{
{"a < 10", "i1"},
{"a <= 10", ""},
{"a > 10", "i1"},
{"a >= 10", ""},
{"a = 10", "i1"},
{"a = -1", ""},
{"b < '10'", ""},
{"b <= '10'", "i2"},
{"b > '10'", ""},
{"b >= '10'", "i2"},
{"b = '10'", ""},
{"b = 'non-exist'", "i2"},
},
},
{
// Both row and index reorg happen, but with no embedded analyze.
// All the stats become invalid, so don't check the results.
caseName: "row and index reorg without analyze",
createTableSQL: "create table t (a bigint, b char(16) collate utf8mb4_bin, index i1(a), index i2(b))",
modifySQL: "alter table t modify column a int unsigned, modify column b varchar(16) collate utf8mb4_general_ci",
embeddedAnalyze: false,
checkResult: false,
queries: []query{
{"a < 10", "i1"},
{"a <= 10", ""},
{"a > 10", "i1"},
{"a >= 10", ""},
{"a = 10", "i1"},
{"a = -1", ""},
{"b < '10'", ""},
{"b <= '10'", "i2"},
{"b > '10'", ""},
{"b >= '10'", "i2"},
{"b = '10'", ""},
{"b = 'non-exist'", "i2"},
},
},
}
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set @@tidb_stats_update_during_ddl = true;")
for _, tc := range tcs {
t.Run(tc.caseName, func(t *testing.T) {
tk.MustExec("drop table if exists t")
tk.MustExec(tc.createTableSQL)
tk.MustExec(fmt.Sprintf("set @@tidb_stats_update_during_ddl = %t", tc.embeddedAnalyze))
for i := range 128 {
tk.MustExec(fmt.Sprintf("insert into t values (%d, '%d')", i, i))
}
tk.MustExec("analyze table t columns a, b")
oldRs := make([]string, 0, len(tc.queries))
for _, q := range tc.queries {
rs := tk.MustQuery(fmt.Sprintf("explain select * from t use index(%s) where %s", q.idx, q.pred)).Rows()
oldRs = append(oldRs, rs[0][1].(string))
}
tk.MustExec(tc.modifySQL)
for i, q := range tc.queries {
rs := tk.MustQuery(fmt.Sprintf("explain select * from t use index(%s) where %s", q.idx, q.pred)).Rows()
if tc.checkResult {
require.Equal(t, oldRs[i], rs[0][1].(string), "predicate: %s", tc.queries[i].pred)
} else {
// For index selectivity, the stats is missing here.
if q.idx != "" {
require.Contains(t, rs[len(rs)-1][len(rs[0])-1], "missing")
}
}
}
})
}
}