Skip to content

Commit

Permalink
util/admin: support admin recover index on the partition table (#17195)…
Browse files Browse the repository at this point in the history
… (#17317)

* cherry pick #17195 to release-4.0

Signed-off-by: sre-bot <sre-bot@pingcap.com>

* fix test

Signed-off-by: crazycs520 <crazycs520@gmail.com>

Co-authored-by: crazycs <crazycs520@gmail.com>
  • Loading branch information
sre-bot and crazycs520 authored May 21, 2020
1 parent 489d4ea commit d514a17
Show file tree
Hide file tree
Showing 4 changed files with 112 additions and 20 deletions.
39 changes: 28 additions & 11 deletions executor/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -173,9 +173,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
Expand Down Expand Up @@ -216,7 +217,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,
}

Expand Down Expand Up @@ -254,15 +255,14 @@ func (e *RecoverIndexExec) buildDAGPB(txn kv.Transaction, limitCnt uint64) (*tip
return dagReq, nil
}

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 int64, limitCnt uint64) (distsql.SelectResult, error) {
dagPB, err := e.buildDAGPB(txn, limitCnt)
if err != nil {
return nil, err
}
tblInfo := e.table.Meta()
ranges := []*ranger.Range{{LowVal: []types.Datum{types.NewIntDatum(startHandle)}, HighVal: []types.Datum{types.NewIntDatum(math.MaxInt64)}}}
var builder distsql.RequestBuilder
kvReq, err := builder.SetTableRanges(tblInfo.ID, ranges, nil).
kvReq, err := builder.SetTableRanges(e.physicalID, ranges, nil).
SetDAGRequest(dagPB).
SetStartTS(txn.StartTS()).
SetKeepOrder(true).
Expand Down Expand Up @@ -409,7 +409,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
}
Expand Down Expand Up @@ -454,9 +454,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)
Expand Down
70 changes: 70 additions & 0 deletions executor/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
. "github.com/pingcap/check"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/mock"
Expand Down Expand Up @@ -158,6 +159,75 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) {
tk.MustExec("admin check table admin_test")
}

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), int64(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) {
tk := testkit.NewTestKit(c, s.store)
s.ctx = mock.NewContext()
Expand Down
12 changes: 3 additions & 9 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,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"
"github.com/pingcap/tidb/util/admin"
Expand Down Expand Up @@ -429,15 +430,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
Expand All @@ -447,6 +440,7 @@ func (b *executorBuilder) buildRecoverIndex(v *plannercore.RecoverIndex) Executo
columns: buildRecoverIndexCols(tblInfo, index.Meta()),
index: index,
table: t,
physicalID: t.Meta().ID,
}
return e
}
Expand Down
11 changes: 11 additions & 0 deletions table/tables/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -204,6 +204,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.
Expand Down

0 comments on commit d514a17

Please sign in to comment.