From b55ccd9d3ccae006e2533c7ab17ec11886e44565 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 20 May 2020 17:10:44 +0800 Subject: [PATCH 1/2] cherry pick #17195 to release-3.0 Signed-off-by: sre-bot --- executor/admin.go | 44 +++++++++++++++++++----- executor/admin_test.go | 78 ++++++++++++++++++++++++++++++++++++++++++ executor/builder.go | 12 ++----- table/tables/tables.go | 11 ++++++ 4 files changed, 127 insertions(+), 18 deletions(-) diff --git a/executor/admin.go b/executor/admin.go index 80c69163ab6bc..ffba1ac8577f0 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -171,9 +171,10 @@ type RecoverIndexExec struct { done bool - index table.Index - table table.Table - batchSize int + index table.Index + table table.Table + physicalID int64 + batchSize int columns []*model.ColumnInfo colFieldTypes []*types.FieldType @@ -214,7 +215,7 @@ func (e *RecoverIndexExec) Open(ctx context.Context) error { func (e *RecoverIndexExec) constructTableScanPB(pbColumnInfos []*tipb.ColumnInfo) *tipb.Executor { tblScan := &tipb.TableScan{ - TableId: e.table.Meta().ID, + TableId: e.physicalID, Columns: pbColumnInfos, } @@ -253,15 +254,23 @@ func (e *RecoverIndexExec) buildDAGPB(txn kv.Transaction, limitCnt uint64) (*tip return dagReq, nil } +<<<<<<< HEAD func (e *RecoverIndexExec) buildTableScan(ctx context.Context, txn kv.Transaction, t table.Table, startHandle int64, limitCnt uint64) (distsql.SelectResult, error) { +======= +func (e *RecoverIndexExec) buildTableScan(ctx context.Context, txn kv.Transaction, startHandle kv.Handle, limitCnt uint64) (distsql.SelectResult, error) { +>>>>>>> 4078eb4... util/admin: support admin recover index on the partition table (#17195) dagPB, err := e.buildDAGPB(txn, limitCnt) if err != nil { return nil, err } +<<<<<<< HEAD tblInfo := e.table.Meta() ranges := []*ranger.Range{{LowVal: []types.Datum{types.NewIntDatum(startHandle)}, HighVal: []types.Datum{types.NewIntDatum(math.MaxInt64)}}} +======= + ranges := []*ranger.Range{{LowVal: []types.Datum{types.NewIntDatum(startHandle.IntValue())}, HighVal: []types.Datum{types.NewIntDatum(math.MaxInt64)}}} +>>>>>>> 4078eb4... util/admin: support admin recover index on the partition table (#17195) var builder distsql.RequestBuilder - kvReq, err := builder.SetTableRanges(tblInfo.ID, ranges, nil). + kvReq, err := builder.SetTableRanges(e.physicalID, ranges, nil). SetDAGRequest(dagPB). SetKeepOrder(true). SetFromSessionVars(e.ctx.GetSessionVars()). @@ -407,7 +416,7 @@ func (e *RecoverIndexExec) batchMarkDup(txn kv.Transaction, rows []recoverRows) func (e *RecoverIndexExec) backfillIndexInTxn(ctx context.Context, txn kv.Transaction, startHandle int64) (result backfillResult, err error) { result.nextHandle = startHandle - srcResult, err := e.buildTableScan(ctx, txn, e.table, startHandle, uint64(e.batchSize)) + srcResult, err := e.buildTableScan(ctx, txn, startHandle, uint64(e.batchSize)) if err != nil { return result, err } @@ -452,9 +461,26 @@ func (e *RecoverIndexExec) Next(ctx context.Context, req *chunk.Chunk) error { return nil } - totalAddedCnt, totalScanCnt, err := e.backfillIndex(ctx) - if err != nil { - return err + var totalAddedCnt, totalScanCnt int64 + var err error + if tbl, ok := e.table.(table.PartitionedTable); ok { + pi := e.table.Meta().GetPartitionInfo() + for _, p := range pi.Definitions { + e.table = tbl.GetPartition(p.ID) + e.index = tables.GetWritableIndexByName(e.index.Meta().Name.L, e.table) + e.physicalID = p.ID + addedCnt, scanCnt, err := e.backfillIndex(ctx) + totalAddedCnt += addedCnt + totalScanCnt += scanCnt + if err != nil { + return err + } + } + } else { + totalAddedCnt, totalScanCnt, err = e.backfillIndex(ctx) + if err != nil { + return err + } } req.AppendInt64(0, totalAddedCnt) diff --git a/executor/admin_test.go b/executor/admin_test.go index c12f1298bf5ed..79b4b27038523 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -21,6 +21,11 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/executor" +<<<<<<< HEAD +======= + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/table" +>>>>>>> 4078eb4... util/admin: support admin recover index on the partition table (#17195) "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" @@ -158,7 +163,80 @@ func (s *testSuite2) TestAdminRecoverIndex(c *C) { tk.MustExec("admin check table admin_test") } +<<<<<<< HEAD func (s *testSuite2) TestAdminRecoverIndex1(c *C) { +======= +func (s *testSuite5) TestAdminRecoverPartitionTableIndex(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + getTable := func() table.Table { + s.ctx = mock.NewContext() + s.ctx.Store = s.store + is := s.domain.InfoSchema() + dbName := model.NewCIStr("test") + tblName := model.NewCIStr("admin_test") + tbl, err := is.TableByName(dbName, tblName) + c.Assert(err, IsNil) + return tbl + } + + checkFunc := func(tbl table.Table, pid int64, idxValue int) { + idxInfo := tbl.Meta().FindIndexByName("c2") + indexOpr := tables.NewIndex(pid, tbl.Meta(), idxInfo) + sc := s.ctx.GetSessionVars().StmtCtx + txn, err := s.store.Begin() + c.Assert(err, IsNil) + err = indexOpr.Delete(sc, txn, types.MakeDatums(idxValue), kv.IntHandle(idxValue)) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + err = tk.ExecToErr("admin check table admin_test") + c.Assert(err, NotNil) + c.Assert(executor.ErrAdminCheckTable.Equal(err), IsTrue) + + r := tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") + r.Check(testkit.Rows("2")) + + r = tk.MustQuery("admin recover index admin_test c2") + r.Check(testkit.Rows("1 3")) + + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") + r.Check(testkit.Rows("3")) + tk.MustExec("admin check table admin_test") + } + + // Test for hash partition table. + tk.MustExec("drop table if exists admin_test") + tk.MustExec("create table admin_test (c1 int, c2 int, c3 int default 1, primary key (c1), index (c2)) partition by hash(c1) partitions 3;") + tk.MustExec("insert admin_test (c1, c2) values (0, 0), (1, 1), (2, 2)") + r := tk.MustQuery("admin recover index admin_test c2") + r.Check(testkit.Rows("0 3")) + tbl := getTable() + pi := tbl.Meta().GetPartitionInfo() + c.Assert(pi, NotNil) + for i, p := range pi.Definitions { + checkFunc(tbl, p.ID, i) + } + + // Test for range partition table. + tk.MustExec("drop table if exists admin_test") + tk.MustExec(`create table admin_test (c1 int, c2 int, c3 int default 1, primary key (c1), index (c2)) PARTITION BY RANGE ( c1 ) ( + PARTITION p0 VALUES LESS THAN (5), + PARTITION p1 VALUES LESS THAN (10), + PARTITION p2 VALUES LESS THAN (MAXVALUE))`) + tk.MustExec("insert admin_test (c1, c2) values (0, 0), (6, 6), (12, 12)") + r = tk.MustQuery("admin recover index admin_test c2") + r.Check(testkit.Rows("0 3")) + tbl = getTable() + pi = tbl.Meta().GetPartitionInfo() + c.Assert(pi, NotNil) + for i, p := range pi.Definitions { + checkFunc(tbl, p.ID, i*6) + } +} + +func (s *testSuite5) TestAdminRecoverIndex1(c *C) { +>>>>>>> 4078eb4... util/admin: support admin recover index on the partition table (#17195) tk := testkit.NewTestKit(c, s.store) s.ctx = mock.NewContext() s.ctx.Store = s.store diff --git a/executor/builder.go b/executor/builder.go index 56da866a3057f..69731d389ee19 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" @@ -405,15 +406,7 @@ func (b *executorBuilder) buildRecoverIndex(v *plannercore.RecoverIndex) Executo return nil } idxName := strings.ToLower(v.IndexName) - indices := t.WritableIndices() - var index table.Index - for _, idx := range indices { - if idxName == idx.Meta().Name.L { - index = idx - break - } - } - + index := tables.GetWritableIndexByName(idxName, t) if index == nil { b.err = errors.Errorf("index `%v` is not found in table `%v`.", v.IndexName, v.Table.Name.O) return nil @@ -423,6 +416,7 @@ func (b *executorBuilder) buildRecoverIndex(v *plannercore.RecoverIndex) Executo columns: buildRecoverIndexCols(tblInfo, index.Meta()), index: index, table: t, + physicalID: t.Meta().ID, } return e } diff --git a/table/tables/tables.go b/table/tables/tables.go index ba9e19193686a..e8a453f4aebba 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -190,6 +190,17 @@ func (t *tableCommon) WritableIndices() []table.Index { return writable } +// GetWritableIndexByName gets the index meta from the table by the index name. +func GetWritableIndexByName(idxName string, t table.Table) table.Index { + indices := t.WritableIndices() + for _, idx := range indices { + if idxName == idx.Meta().Name.L { + return idx + } + } + return nil +} + // DeletableIndices implements table.Table DeletableIndices interface. func (t *tableCommon) DeletableIndices() []table.Index { // All indices are deletable because we don't need to check StateNone. From 819ccb2c39ec27c1e938bb60305b7d6508645142 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Tue, 26 May 2020 15:31:25 +0800 Subject: [PATCH 2/2] fix test Signed-off-by: crazycs520 --- executor/admin.go | 11 +---------- executor/admin_test.go | 14 +++----------- 2 files changed, 4 insertions(+), 21 deletions(-) diff --git a/executor/admin.go b/executor/admin.go index ffba1ac8577f0..21c2c701415fe 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -254,21 +254,12 @@ func (e *RecoverIndexExec) buildDAGPB(txn kv.Transaction, limitCnt uint64) (*tip return dagReq, nil } -<<<<<<< HEAD -func (e *RecoverIndexExec) buildTableScan(ctx context.Context, txn kv.Transaction, t table.Table, startHandle int64, limitCnt uint64) (distsql.SelectResult, error) { -======= -func (e *RecoverIndexExec) buildTableScan(ctx context.Context, txn kv.Transaction, startHandle kv.Handle, limitCnt uint64) (distsql.SelectResult, error) { ->>>>>>> 4078eb4... util/admin: support admin recover index on the partition table (#17195) +func (e *RecoverIndexExec) buildTableScan(ctx context.Context, txn kv.Transaction, startHandle int64, limitCnt uint64) (distsql.SelectResult, error) { dagPB, err := e.buildDAGPB(txn, limitCnt) if err != nil { return nil, err } -<<<<<<< HEAD - tblInfo := e.table.Meta() ranges := []*ranger.Range{{LowVal: []types.Datum{types.NewIntDatum(startHandle)}, HighVal: []types.Datum{types.NewIntDatum(math.MaxInt64)}}} -======= - ranges := []*ranger.Range{{LowVal: []types.Datum{types.NewIntDatum(startHandle.IntValue())}, HighVal: []types.Datum{types.NewIntDatum(math.MaxInt64)}}} ->>>>>>> 4078eb4... util/admin: support admin recover index on the partition table (#17195) var builder distsql.RequestBuilder kvReq, err := builder.SetTableRanges(e.physicalID, ranges, nil). SetDAGRequest(dagPB). diff --git a/executor/admin_test.go b/executor/admin_test.go index 79b4b27038523..b6bd236925231 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -21,11 +21,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/executor" -<<<<<<< HEAD -======= - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/table" ->>>>>>> 4078eb4... util/admin: support admin recover index on the partition table (#17195) "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" @@ -163,10 +159,7 @@ func (s *testSuite2) TestAdminRecoverIndex(c *C) { tk.MustExec("admin check table admin_test") } -<<<<<<< HEAD -func (s *testSuite2) TestAdminRecoverIndex1(c *C) { -======= -func (s *testSuite5) TestAdminRecoverPartitionTableIndex(c *C) { +func (s *testSuite2) TestAdminRecoverPartitionTableIndex(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") getTable := func() table.Table { @@ -186,7 +179,7 @@ func (s *testSuite5) TestAdminRecoverPartitionTableIndex(c *C) { sc := s.ctx.GetSessionVars().StmtCtx txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(idxValue), kv.IntHandle(idxValue)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(idxValue), int64(idxValue), txn) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -235,8 +228,7 @@ func (s *testSuite5) TestAdminRecoverPartitionTableIndex(c *C) { } } -func (s *testSuite5) TestAdminRecoverIndex1(c *C) { ->>>>>>> 4078eb4... util/admin: support admin recover index on the partition table (#17195) +func (s *testSuite2) TestAdminRecoverIndex1(c *C) { tk := testkit.NewTestKit(c, s.store) s.ctx = mock.NewContext() s.ctx.Store = s.store