Skip to content

Commit

Permalink
Merge remote-tracking branch 'upstream/release-5.0' into err-key-exis…
Browse files Browse the repository at this point in the history
…ts-5.0
  • Loading branch information
sticnarf committed Apr 12, 2021
2 parents 490c9a6 + f3951ec commit 24a964e
Show file tree
Hide file tree
Showing 37 changed files with 860 additions and 244 deletions.
30 changes: 30 additions & 0 deletions cmd/explaintest/r/vitess_hash.result
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
use test;
drop table if exists t;
create table t(
customer_id bigint,
id bigint,
expected_shard bigint unsigned,
computed_shard bigint unsigned null,
primary key (customer_id, id)
);
create index t_vitess_shard on t((vitess_hash(customer_id) >> 56));
explain format = 'brief' select customer_id from t where (vitess_hash(customer_id) >> 56) = x'd6' ORDER BY id;
id estRows task access object operator info
Projection 10.00 root test.t.customer_id
└─Sort 10.00 root test.t.id
└─IndexLookUp 10.00 root
├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:t_vitess_shard(vitess_hash(`customer_id`) >> 56) range:[214,214], keep order:false, stats:pseudo
└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo
explain format = 'brief' select id from t where (vitess_hash(customer_id) >> 56) IN (x'e0', x'e1') AND id BETWEEN 2 AND 5 ORDER BY id;
id estRows task access object operator info
Projection 0.50 root test.t.id
└─Sort 0.50 root test.t.id
└─IndexLookUp 0.50 root
├─IndexRangeScan(Build) 20.00 cop[tikv] table:t, index:t_vitess_shard(vitess_hash(`customer_id`) >> 56) range:[224,224], [225,225], keep order:false, stats:pseudo
└─Selection(Probe) 0.50 cop[tikv] ge(test.t.id, 2), le(test.t.id, 5)
└─TableRowIDScan 20.00 cop[tikv] table:t keep order:false, stats:pseudo
explain format = 'brief' select hex(vitess_hash(1123)) from t;
id estRows task access object operator info
Projection 10000.00 root 31B565D41BDF8CA->Column#7
└─IndexReader 10000.00 root index:IndexFullScan
└─IndexFullScan 10000.00 cop[tikv] table:t, index:t_vitess_shard(vitess_hash(`customer_id`) >> 56) keep order:false, stats:pseudo
13 changes: 13 additions & 0 deletions cmd/explaintest/t/vitess_hash.test
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
use test;
drop table if exists t;
create table t(
customer_id bigint,
id bigint,
expected_shard bigint unsigned,
computed_shard bigint unsigned null,
primary key (customer_id, id)
);
create index t_vitess_shard on t((vitess_hash(customer_id) >> 56));
explain format = 'brief' select customer_id from t where (vitess_hash(customer_id) >> 56) = x'd6' ORDER BY id;
explain format = 'brief' select id from t where (vitess_hash(customer_id) >> 56) IN (x'e0', x'e1') AND id BETWEEN 2 AND 5 ORDER BY id;
explain format = 'brief' select hex(vitess_hash(1123)) from t;
5 changes: 0 additions & 5 deletions config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -294,11 +294,6 @@ max-txn-ttl = 3600000
# the interval duration between two memory profile into global tracker
mem-profile-interval = "1m"

# Index usage sync lease duration, which influences the time of dump index usage information to KV.
# Here we set to 0 to not dump index usage information to KV,
# because we have not completed GC and other functions.
index-usage-sync-lease = "0s"

# The Go GC trigger factor, you can get more information about it at https://golang.org/pkg/runtime.
# If you encounter OOM when executing large query, you can decrease this value to trigger GC earlier.
# If you find the CPU used by GC is too high or GC is too frequent and impact your business you can increase this value.
Expand Down
2 changes: 0 additions & 2 deletions ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1171,8 +1171,6 @@ func (s *testColumnSuite) TestModifyColumn(c *C) {
}{
{"int", "bigint", nil},
{"int", "int unsigned", errUnsupportedModifyColumn.GenWithStackByArgs("can't change unsigned integer to signed or vice versa, and tidb_enable_change_column_type is false")},
{"varchar(10)", "text", nil},
{"varbinary(10)", "blob", nil},
{"text", "blob", errUnsupportedModifyCharset.GenWithStackByArgs("charset from utf8mb4 to binary")},
{"varchar(10)", "varchar(8)", errUnsupportedModifyColumn.GenWithStackByArgs("length 8 is less than origin 10, and tidb_enable_change_column_type is false")},
{"varchar(10)", "varchar(11)", nil},
Expand Down
209 changes: 94 additions & 115 deletions ddl/column_type_change_test.go

Large diffs are not rendered by default.

80 changes: 80 additions & 0 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1510,6 +1510,86 @@ func (s *testIntegrationSuite8) TestCreateTooManyIndexes(c *C) {
tk.MustGetErrCode(alterSQL, errno.ErrTooManyKeys)
}

func (s *testIntegrationSuite8) TestCreateSecondaryIndexInCluster(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")

// test create table with non-unique key
tk.MustGetErrCode(`
CREATE TABLE t (
c01 varchar(255) NOT NULL,
c02 varchar(255) NOT NULL,
c03 varchar(255) NOT NULL,
c04 varchar(255) DEFAULT NULL,
c05 varchar(255) DEFAULT NULL,
c06 varchar(255) DEFAULT NULL,
PRIMARY KEY (c01,c02,c03) clustered,
KEY c04 (c04)
)`, errno.ErrTooLongKey)

// test create long clustered primary key.
tk.MustGetErrCode(`
CREATE TABLE t (
c01 varchar(255) NOT NULL,
c02 varchar(255) NOT NULL,
c03 varchar(255) NOT NULL,
c04 varchar(255) NOT NULL,
c05 varchar(255) DEFAULT NULL,
c06 varchar(255) DEFAULT NULL,
PRIMARY KEY (c01,c02,c03,c04) clustered
)`, errno.ErrTooLongKey)

// test create table with unique key
tk.MustExec(`
CREATE TABLE t (
c01 varchar(255) NOT NULL,
c02 varchar(255) NOT NULL,
c03 varchar(255) NOT NULL,
c04 varchar(255) DEFAULT NULL,
c05 varchar(255) DEFAULT NULL,
c06 varchar(255) DEFAULT NULL,
PRIMARY KEY (c01,c02,c03) clustered,
unique key c04 (c04)
)`)
tk.MustExec("drop table t")

// test create index
tk.MustExec(`
CREATE TABLE t (
c01 varchar(255) NOT NULL,
c02 varchar(255) NOT NULL,
c03 varchar(255) NOT NULL,
c04 varchar(255) DEFAULT NULL,
c05 varchar(255) DEFAULT NULL,
c06 varchar(255) DEFAULT NULL,
PRIMARY KEY (c01,c02) clustered
)`)
tk.MustExec("create index idx1 on t(c03)")
tk.MustGetErrCode("create index idx2 on t(c03, c04)", errno.ErrTooLongKey)
tk.MustExec("create unique index uk2 on t(c03, c04)")
tk.MustExec("drop table t")

// test change/modify column
tk.MustExec(`
CREATE TABLE t (
c01 varchar(255) NOT NULL,
c02 varchar(255) NOT NULL,
c03 varchar(255) NOT NULL,
c04 varchar(255) DEFAULT NULL,
c05 varchar(255) DEFAULT NULL,
c06 varchar(255) DEFAULT NULL,
Index idx1(c03),
PRIMARY KEY (c01,c02) clustered,
unique index uk1(c06)
)`)
tk.MustExec("alter table t change c03 c10 varchar(256) default null")
tk.MustGetErrCode("alter table t change c10 c100 varchar(1024) default null", errno.ErrTooLongKey)
tk.MustGetErrCode("alter table t modify c10 varchar(600) default null", errno.ErrTooLongKey)
tk.MustExec("alter table t modify c06 varchar(600) default null")
tk.MustGetErrCode("alter table t modify c01 varchar(510)", errno.ErrTooLongKey)
tk.MustExec("create table t2 like t")
}

func (s *testIntegrationSuite3) TestAlterColumn(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test_db")
Expand Down
8 changes: 3 additions & 5 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4078,11 +4078,9 @@ func (s *testSerialDBSuite) TestModifyColumnBetweenStringTypes(c *C) {
tk.MustGetErrMsg("alter table tt change a a varchar(4);", "[types:1406]Data Too Long, field len 4, data len 5")
tk.MustExec("alter table tt change a a varchar(100);")

// varchar to char
tk.MustExec("alter table tt change a a char(10);")
c2 = getModifyColumn(c, s.s.(sessionctx.Context), "test", "tt", "a", false)
c.Assert(c2.FieldType.Tp, Equals, mysql.TypeString)
c.Assert(c2.FieldType.Flen, Equals, 10)
tk.MustExec("drop table if exists tt;")
tk.MustExec("create table tt (a char(10));")
tk.MustExec("insert into tt values ('111'),('10000');")
tk.MustQuery("select * from tt").Check(testkit.Rows("111", "10000"))
tk.MustGetErrMsg("alter table tt change a a char(4);", "[types:1406]Data Too Long, field len 4, data len 5")

Expand Down
134 changes: 114 additions & 20 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1505,6 +1505,44 @@ func buildTableInfo(
idxInfo.ID = allocateIndexID(tbInfo)
tbInfo.Indices = append(tbInfo.Indices, idxInfo)
}
if tbInfo.IsCommonHandle {
// Ensure tblInfo's each non-unique secondary-index's len + primary-key's len <= MaxIndexLength for clustered index table.
var pkLen, idxLen int
pkLen, err = indexColumnsLen(tbInfo.Columns, tables.FindPrimaryIndex(tbInfo).Columns)
if err != nil {
return
}
for _, idx := range tbInfo.Indices {
if idx.Unique {
// Only need check for non-unique secondary-index.
continue
}
idxLen, err = indexColumnsLen(tbInfo.Columns, idx.Columns)
if err != nil {
return
}
if pkLen+idxLen > config.GetGlobalConfig().MaxIndexLength {
return nil, errTooLongKey.GenWithStackByArgs(config.GetGlobalConfig().MaxIndexLength)
}
}
}
return
}

func indexColumnsLen(cols []*model.ColumnInfo, idxCols []*model.IndexColumn) (len int, err error) {
for _, idxCol := range idxCols {
col := model.FindColumnInfo(cols, idxCol.Name.L)
if col == nil {
err = errKeyColumnDoesNotExits.GenWithStack("column does not exist: %s", idxCol.Name.L)
return
}
var colLen int
colLen, err = getIndexColumnLength(col, idxCol.Length)
if err != nil {
return
}
len += colLen
}
return
}

Expand Down Expand Up @@ -3555,11 +3593,10 @@ func checkTypeChangeSupported(origin *types.FieldType, to *types.FieldType) bool
func checkModifyTypes(ctx sessionctx.Context, origin *types.FieldType, to *types.FieldType, needRewriteCollationData bool) error {
canReorg, changeColumnErrMsg, err := CheckModifyTypeCompatible(origin, to)
if err != nil {
enableChangeColumnType := ctx.GetSessionVars().EnableChangeColumnType
if !canReorg {
return errors.Trace(err)
}

enableChangeColumnType := ctx.GetSessionVars().EnableChangeColumnType
if !enableChangeColumnType {
msg := fmt.Sprintf("%s, and tidb_enable_change_column_type is false", changeColumnErrMsg)
return errUnsupportedModifyColumn.GenWithStackByArgs(msg)
Expand All @@ -3568,6 +3605,10 @@ func checkModifyTypes(ctx sessionctx.Context, origin *types.FieldType, to *types
return errUnsupportedModifyColumn.GenWithStackByArgs(msg)
}
}
if types.IsTypeVarchar(origin.Tp) != types.IsTypeVarchar(to.Tp) {
unsupportedMsg := "column type conversion between 'varchar' and 'non-varchar' is currently unsupported yet"
return errUnsupportedModifyColumn.GenWithStackByArgs(unsupportedMsg)
}

err = checkModifyCharsetAndCollation(to.Charset, to.Collate, origin.Charset, origin.Collate, needRewriteCollationData)
// column type change can handle the charset change between these two types in the process of the reorg.
Expand Down Expand Up @@ -3878,33 +3919,70 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or
// checkColumnWithIndexConstraint is used to check the related index constraint of the modified column.
// Index has a max-prefix-length constraint. eg: a varchar(100), index idx(a), modifying column a to a varchar(4000)
// will cause index idx to break the max-prefix-length constraint.
//
// For clustered index:
// Change column in pk need recheck all non-unique index, new pk len + index len < maxIndexLength.
// Change column in secondary only need related index, pk len + new index len < maxIndexLength.
func checkColumnWithIndexConstraint(tbInfo *model.TableInfo, originalCol, newCol *model.ColumnInfo) error {
var columns []*model.ColumnInfo
for _, indexInfo := range tbInfo.Indices {
containColumn := false
columns := make([]*model.ColumnInfo, 0, len(tbInfo.Columns))
columns = append(columns, tbInfo.Columns...)
// Replace old column with new column.
for i, col := range columns {
if col.Name.L != originalCol.Name.L {
continue
}
columns[i] = newCol.Clone()
columns[i].Name = originalCol.Name
break
}

pkIndex := tables.FindPrimaryIndex(tbInfo)
var clusteredPkLen int
if tbInfo.IsCommonHandle {
var err error
clusteredPkLen, err = indexColumnsLen(columns, pkIndex.Columns)
if err != nil {
return err
}
}

checkOneIndex := func(indexInfo *model.IndexInfo, pkLenAppendToKey int, skipCheckIfNotModify bool) (modified bool, err error) {
for _, col := range indexInfo.Columns {
if col.Name.L == originalCol.Name.L {
containColumn = true
modified = true
break
}
}
if !containColumn {
if skipCheckIfNotModify && !modified {
return
}
err = checkIndexPrefixLength(columns, indexInfo.Columns, pkLenAppendToKey)
return
}

// Check primary key first and get "does primary key's column has be modified?" info.
var (
pkModified bool
err error
)
if pkIndex != nil {
pkModified, err = checkOneIndex(pkIndex, 0, true)
if err != nil {
return err
}
}

// Check secondary indexes.
for _, indexInfo := range tbInfo.Indices {
if indexInfo.Primary {
continue
}
if columns == nil {
columns = make([]*model.ColumnInfo, 0, len(tbInfo.Columns))
columns = append(columns, tbInfo.Columns...)
// replace old column with new column.
for i, col := range columns {
if col.Name.L != originalCol.Name.L {
continue
}
columns[i] = newCol.Clone()
columns[i].Name = originalCol.Name
break
}
var pkLenAppendToKey int
if !indexInfo.Unique {
pkLenAppendToKey = clusteredPkLen
}
err := checkIndexPrefixLength(columns, indexInfo.Columns)

_, err = checkOneIndex(indexInfo, pkLenAppendToKey, !tbInfo.IsCommonHandle || !pkModified)
if err != nil {
return err
}
Expand Down Expand Up @@ -4978,6 +5056,22 @@ func (d *ddl) CreateIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.Inde
return errors.Trace(err)
}

if !unique && tblInfo.IsCommonHandle {
// Ensure new created non-unique secondary-index's len + primary-key's len <= MaxIndexLength in clustered index table.
var pkLen, idxLen int
pkLen, err = indexColumnsLen(tblInfo.Columns, tables.FindPrimaryIndex(tblInfo).Columns)
if err != nil {
return err
}
idxLen, err = indexColumnsLen(tblInfo.Columns, indexColumns)
if err != nil {
return err
}
if pkLen+idxLen > config.GetGlobalConfig().MaxIndexLength {
return errTooLongKey.GenWithStackByArgs(config.GetGlobalConfig().MaxIndexLength)
}
}

global := false
if unique && tblInfo.GetPartitionInfo() != nil {
ck, err := checkPartitionKeysConstraint(tblInfo.GetPartitionInfo(), indexColumns, tblInfo)
Expand Down
28 changes: 8 additions & 20 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -108,24 +108,13 @@ func checkPKOnGeneratedColumn(tblInfo *model.TableInfo, indexPartSpecifications
return lastCol, nil
}

func checkIndexPrefixLength(columns []*model.ColumnInfo, idxColumns []*model.IndexColumn) error {
// The sum of length of all index columns.
sumLength := 0
for _, ic := range idxColumns {
col := model.FindColumnInfo(columns, ic.Name.L)
if col == nil {
return errKeyColumnDoesNotExits.GenWithStack("column does not exist: %s", ic.Name)
}

indexColumnLength, err := getIndexColumnLength(col, ic.Length)
if err != nil {
return err
}
sumLength += indexColumnLength
// The sum of all lengths must be shorter than the max length for prefix.
if sumLength > config.GetGlobalConfig().MaxIndexLength {
return errTooLongKey.GenWithStackByArgs(config.GetGlobalConfig().MaxIndexLength)
}
func checkIndexPrefixLength(columns []*model.ColumnInfo, idxColumns []*model.IndexColumn, pkLenAppendToKey int) error {
idxLen, err := indexColumnsLen(columns, idxColumns)
if err != nil {
return err
}
if idxLen+pkLenAppendToKey > config.GetGlobalConfig().MaxIndexLength {
return errTooLongKey.GenWithStackByArgs(config.GetGlobalConfig().MaxIndexLength)
}
return nil
}
Expand Down Expand Up @@ -488,7 +477,6 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo
indexInfo.Global = global
indexInfo.ID = allocateIndexID(tblInfo)
tblInfo.Indices = append(tblInfo.Indices, indexInfo)

if err = checkTooManyIndexes(tblInfo.Indices); err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
Expand Down Expand Up @@ -927,7 +915,7 @@ func (w *baseIndexWorker) getIndexRecord(idxInfo *model.IndexInfo, handle kv.Han
idxVal[j] = idxColumnVal
}

rsData := tables.TryGetHandleRestoredDataWrapper(w.table, nil, w.rowMap)
rsData := tables.TryGetHandleRestoredDataWrapper(w.table, nil, w.rowMap, idxInfo)
idxRecord := &indexRecord{handle: handle, key: recordKey, vals: idxVal, rsData: rsData}
return idxRecord, nil
}
Expand Down
Loading

0 comments on commit 24a964e

Please sign in to comment.