Skip to content

Commit

Permalink
Merge branch 'master' into fix_add_ingest_index_as_sub_job
Browse files Browse the repository at this point in the history
  • Loading branch information
Leavrth committed Oct 11, 2023
2 parents bc35a33 + ec2731b commit 695a603
Show file tree
Hide file tree
Showing 98 changed files with 8,493 additions and 3,801 deletions.
2 changes: 1 addition & 1 deletion br/pkg/restore/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -1780,7 +1780,7 @@ func (rc *Client) GoUpdateMetaAndLoadStats(ctx context.Context, inCh <-chan *Cre
zap.Int64("new id", tbl.Table.ID),
)
start := time.Now()
if err := rc.statsHandler.LoadStatsFromJSON(rc.dom.InfoSchema(), oldTable.Stats); err != nil {
if err := rc.statsHandler.LoadStatsFromJSON(ctx, rc.dom.InfoSchema(), oldTable.Stats, 0); err != nil {
log.Error("analyze table failed", zap.Any("table", oldTable.Stats), zap.Error(err))
}
log.Info("restore stat done",
Expand Down
76 changes: 0 additions & 76 deletions ddl/attributes_sql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,28 +56,6 @@ func MockGC(tk *testkit.TestKit) (string, string, string, func()) {
return timeBeforeDrop, timeAfterDrop, safePointSQL, resetGC
}

func TestAlterTableAttributes(t *testing.T) {
store := testkit.CreateMockStore(t)

tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec(`create table alter_t (c int);`)

// normal cases
tk.MustExec(`alter table alter_t attributes="merge_option=allow";`)
tk.MustExec(`alter table alter_t attributes="merge_option=allow,key=value";`)

// space cases
tk.MustExec(`alter table alter_t attributes=" merge_option=allow ";`)
tk.MustExec(`alter table alter_t attributes=" merge_option = allow , key = value ";`)

// without equal
tk.MustExec(`alter table alter_t attributes " merge_option=allow ";`)
tk.MustExec(`alter table alter_t attributes " merge_option=allow , key=value ";`)

tk.MustExec("drop table alter_t")
}

func TestAlterTablePartitionAttributes(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
Expand Down Expand Up @@ -505,57 +483,3 @@ PARTITION BY RANGE (c) (
require.Equal(t, `"key2=value2"`, rows3[1][2])
require.Equal(t, rows2[1][3], rows3[1][3])
}

func TestDropSchema(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)

_, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), keyspace.CodecV1, true)
require.NoError(t, err)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec(`create table drop_s1 (c int)
PARTITION BY RANGE (c) (
PARTITION p0 VALUES LESS THAN (6),
PARTITION p1 VALUES LESS THAN (11)
);`)
tk.MustExec(`create table drop_s2 (c int);`)

// add attributes
tk.MustExec(`alter table drop_s1 attributes="key=value";`)
tk.MustExec(`alter table drop_s1 partition p0 attributes="key1=value1";`)
tk.MustExec(`alter table drop_s2 attributes="key=value";`)
rows := tk.MustQuery(`select * from information_schema.attributes;`).Rows()
require.Len(t, rows, 3)
// drop database
tk.MustExec(`drop database test`)
rows = tk.MustQuery(`select * from information_schema.attributes;`).Rows()
require.Len(t, rows, 0)
}

func TestDefaultKeyword(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)

_, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), keyspace.CodecV1, true)
require.NoError(t, err)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec(`create table def (c int)
PARTITION BY RANGE (c) (
PARTITION p0 VALUES LESS THAN (6),
PARTITION p1 VALUES LESS THAN (11)
);`)

// add attributes
tk.MustExec(`alter table def attributes="key=value";`)
tk.MustExec(`alter table def partition p0 attributes="key1=value1";`)
rows := tk.MustQuery(`select * from information_schema.attributes;`).Rows()
require.Len(t, rows, 2)
// reset the partition p0's attribute
tk.MustExec(`alter table def partition p0 attributes=default;`)
rows = tk.MustQuery(`select * from information_schema.attributes;`).Rows()
require.Len(t, rows, 1)
// reset the table def's attribute
tk.MustExec(`alter table def attributes=default;`)
rows = tk.MustQuery(`select * from information_schema.attributes;`).Rows()
require.Len(t, rows, 0)
}
4 changes: 2 additions & 2 deletions ddl/backfilling_dist_scheduler.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ func NewBackfillSubtaskExecutor(_ context.Context, taskMeta []byte, d *ddl,
if indexInfo == nil {
logutil.BgLogger().Warn("index info not found", zap.String("category", "ddl-ingest"),
zap.Int64("table ID", tbl.Meta().ID), zap.Int64("index ID", eid))
return nil, errors.New("index info not found")
return nil, errors.Errorf("index info not found: %d", eid)
}
indexInfos = append(indexInfos, indexInfo)
}
Expand Down Expand Up @@ -140,7 +140,7 @@ func (s *backfillDistScheduler) Init(ctx context.Context) error {
// we use the first index uniqueness here.
idx := model.FindIndexInfoByID(tbl.Meta().Indices, bgm.EleIDs[0])
if idx == nil {
return errors.Trace(errors.New("index info not found"))
return errors.Trace(errors.Errorf("index info not found: %d", bgm.EleIDs[0]))
}
bc, err := ingest.LitBackCtxMgr.Register(ctx, idx.Unique, job.ID, d.etcdCli, job.ReorgMeta.ResourceGroupName)
if err != nil {
Expand Down
10 changes: 6 additions & 4 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -385,7 +385,7 @@ func needChangeColumnData(oldCol, newCol *model.ColumnInfo) bool {
// cut to eliminate data reorg change for column type change between decimal.
return oldCol.GetFlen() != newCol.GetFlen() || oldCol.GetDecimal() != newCol.GetDecimal() || toUnsigned != originUnsigned
case mysql.TypeEnum, mysql.TypeSet:
return isElemsChangedToModifyColumn(oldCol.GetElems(), newCol.GetElems())
return IsElemsChangedToModifyColumn(oldCol.GetElems(), newCol.GetElems())
case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong:
return toUnsigned != originUnsigned
case mysql.TypeString:
Expand All @@ -398,7 +398,7 @@ func needChangeColumnData(oldCol, newCol *model.ColumnInfo) bool {
return needTruncationOrToggleSign()
}

if convertBetweenCharAndVarchar(oldCol.GetType(), newCol.GetType()) {
if ConvertBetweenCharAndVarchar(oldCol.GetType(), newCol.GetType()) {
return true
}

Expand All @@ -420,12 +420,14 @@ func needChangeColumnData(oldCol, newCol *model.ColumnInfo) bool {
return true
}

// ConvertBetweenCharAndVarchar check whether column converted between char and varchar
// TODO: it is used for plugins. so change plugin's using and remove it.
func convertBetweenCharAndVarchar(oldCol, newCol byte) bool {
func ConvertBetweenCharAndVarchar(oldCol, newCol byte) bool {
return types.ConvertBetweenCharAndVarchar(oldCol, newCol)
}

func isElemsChangedToModifyColumn(oldElems, newElems []string) bool {
// IsElemsChangedToModifyColumn check elems changed
func IsElemsChangedToModifyColumn(oldElems, newElems []string) bool {
if len(newElems) < len(oldElems) {
return true
}
Expand Down
24 changes: 0 additions & 24 deletions ddl/column_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/ddl/util/callback"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/parser/model"
Expand Down Expand Up @@ -438,15 +437,6 @@ func testNewContext(store kv.Storage) sessionctx.Context {
return ctx
}

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

tk.MustExec("CREATE TABLE t40150 (a int) PARTITION BY HASH (a) PARTITIONS 2")
tk.MustContainErrMsg(`alter table t40150 rename column a to c`, "[ddl:3855]Column 'a' has a partitioning function dependency and cannot be dropped or renamed")
}

func TestIssue40135(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
Expand All @@ -470,17 +460,3 @@ func TestIssue40135(t *testing.T) {

require.ErrorContains(t, checkErr, "[ddl:3855]Column 'a' has a partitioning function dependency and cannot be dropped or renamed")
}

func TestIssue38988And24321(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
// For issue https://github.com/pingcap/tidb/issues/38988
tk.MustExec("create table t (a int, b int as (a+3));")
tk.MustGetErrCode("alter table t change a c int not null;", errno.ErrDependentByGeneratedColumn)

// For issue https://github.com/pingcap/tidb/issues/24321
// Note, the result is not the same with MySQL, since the limitation of the current modify column implementation.
tk.MustExec("create table t2(id int, a int, b int generated always as (abs(a)) virtual);")
tk.MustGetErrCode("alter table t2 modify column a bigint;", errno.ErrUnsupportedOnGeneratedColumn)
}
Loading

0 comments on commit 695a603

Please sign in to comment.