ddl: refine the criterion for modify column (#64834)
close pingcap/tiflash#10509, ref pingcap/tidb#63366
This commit is contained in:
@ -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)
|
||||
|
||||
@ -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")
|
||||
}
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user