diff --git a/ast/misc.go b/ast/misc.go index baf9bab2d9c7e..0cf7e4d6550b8 100644 --- a/ast/misc.go +++ b/ast/misc.go @@ -591,6 +591,7 @@ const ( AdminCancelDDLJobs AdminCheckIndex AdminRecoverIndex + AdminCleanupIndex AdminCheckIndexRange AdminShowDDLJobQueries AdminChecksumTable diff --git a/executor/admin.go b/executor/admin.go index 9a72c0c80d54a..7d6e0953289b8 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -38,6 +39,7 @@ import ( var ( _ Executor = &CheckIndexRangeExec{} _ Executor = &RecoverIndexExec{} + _ Executor = &CleanupIndexExec{} ) // CheckIndexRangeExec outputs the index values which has handle between begin and end. @@ -304,20 +306,6 @@ func (e *RecoverIndexExec) backfillIndex(ctx context.Context) (int64, int64, err return totalAddedCnt, totalScanCnt, nil } -func (e *RecoverIndexExec) extractIdxVals(row chunk.Row, idxVals []types.Datum) []types.Datum { - if idxVals == nil { - idxVals = make([]types.Datum, 0, row.Len()-1) - } else { - idxVals = idxVals[:0] - } - - for i := 0; i < row.Len()-1; i++ { - colVal := row.GetDatum(i, &e.columns[i].FieldType) - idxVals = append(idxVals, *colVal.Copy()) - } - return idxVals -} - type recoverRows struct { handle int64 idxVals []types.Datum @@ -344,7 +332,7 @@ func (e *RecoverIndexExec) fetchRecoverRows(ctx context.Context, srcResult dists return e.recoverRows, nil } handle := row.GetInt64(handleIdx) - idxVals := e.extractIdxVals(row, e.idxValsBufs[result.scanRowCount]) + idxVals := extractIdxVals(row, e.idxValsBufs[result.scanRowCount], e.colFieldTypes) e.idxValsBufs[result.scanRowCount] = idxVals e.recoverRows = append(e.recoverRows, recoverRows{handle: handle, idxVals: idxVals, skip: false}) result.scanRowCount++ @@ -457,3 +445,244 @@ func (e *RecoverIndexExec) NextChunk(ctx context.Context, chk *chunk.Chunk) erro e.done = true return nil } + +// CleanupIndexExec represents a cleanup index executor. +// It is built from "admin cleanup index" statement, is used to delete +// dangling index data. +type CleanupIndexExec struct { + baseExecutor + + done bool + removeCnt uint64 + + index table.Index + table table.Table + + idxCols []*model.ColumnInfo + idxColFieldTypes []*types.FieldType + idxChunk *chunk.Chunk + + idxValues map[int64][]types.Datum + batchSize uint64 + batchKeys []kv.Key + idxValsBufs [][]types.Datum + lastIdxKey []byte + scanRowCnt uint64 +} + +func (e *CleanupIndexExec) getIdxColTypes() []*types.FieldType { + if e.idxColFieldTypes != nil { + return e.idxColFieldTypes + } + e.idxColFieldTypes = make([]*types.FieldType, 0, len(e.idxCols)) + for _, col := range e.idxCols { + e.idxColFieldTypes = append(e.idxColFieldTypes, &col.FieldType) + } + return e.idxColFieldTypes +} + +func (e *CleanupIndexExec) batchGetRecord(txn kv.Transaction) (map[string][]byte, error) { + for handle := range e.idxValues { + e.batchKeys = append(e.batchKeys, e.table.RecordKey(handle)) + } + values, err := txn.GetSnapshot().BatchGet(e.batchKeys) + if err != nil { + return nil, errors.Trace(err) + } + return values, nil +} + +func (e *CleanupIndexExec) deleteDanglingIdx(txn kv.Transaction, values map[string][]byte) error { + for _, k := range e.batchKeys { + if _, found := values[string(k)]; !found { + _, handle, err := tablecodec.DecodeRecordKey(k) + if err != nil { + return errors.Trace(err) + } + if err := e.index.Delete(e.ctx.GetSessionVars().StmtCtx, txn, e.idxValues[handle], + handle); err != nil { + return errors.Trace(err) + } + e.removeCnt++ + if e.removeCnt%e.batchSize == 0 { + log.Infof("[cleaning up dangling index] table: %v, index: %v, count: %v.", + e.table.Meta().Name.String(), e.index.Meta().Name.String(), e.removeCnt) + } + } + } + return nil +} + +func extractIdxVals(row chunk.Row, idxVals []types.Datum, + fieldTypes []*types.FieldType) []types.Datum { + if idxVals == nil { + idxVals = make([]types.Datum, 0, row.Len()-1) + } else { + idxVals = idxVals[:0] + } + + for i := 0; i < row.Len()-1; i++ { + colVal := row.GetDatum(i, fieldTypes[i]) + idxVals = append(idxVals, *colVal.Copy()) + } + return idxVals +} + +func (e *CleanupIndexExec) fetchIndex(ctx context.Context, txn kv.Transaction) error { + result, err := e.buildIndexScan(ctx, txn) + if err != nil { + return errors.Trace(err) + } + defer terror.Call(result.Close) + + sc := e.ctx.GetSessionVars().StmtCtx + for { + err := result.NextChunk(ctx, e.idxChunk) + if err != nil { + return errors.Trace(err) + } + if e.idxChunk.NumRows() == 0 { + return nil + } + iter := chunk.NewIterator4Chunk(e.idxChunk) + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + handle := row.GetInt64(len(e.idxCols) - 1) + idxVals := extractIdxVals(row, e.idxValsBufs[e.scanRowCnt], e.idxColFieldTypes) + e.idxValsBufs[e.scanRowCnt] = idxVals + e.idxValues[handle] = idxVals + idxKey, _, err := e.index.GenIndexKey(sc, idxVals, handle, nil) + if err != nil { + return errors.Trace(err) + } + e.scanRowCnt++ + e.lastIdxKey = idxKey + if e.scanRowCnt >= e.batchSize { + return nil + } + } + } +} + +// NextChunk implements the Executor NextChunk interface. +func (e *CleanupIndexExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { + chk.Reset() + if e.done { + return nil + } + for { + errInTxn := kv.RunInNewTxn(e.ctx.GetStore(), true, func(txn kv.Transaction) error { + err := e.fetchIndex(ctx, txn) + if err != nil { + return errors.Trace(err) + } + values, err := e.batchGetRecord(txn) + if err != nil { + return errors.Trace(err) + } + err = e.deleteDanglingIdx(txn, values) + if err != nil { + return errors.Trace(err) + } + return nil + }) + if errInTxn != nil { + return errors.Trace(errInTxn) + } + if e.scanRowCnt == 0 { + break + } + e.scanRowCnt = 0 + e.batchKeys = e.batchKeys[:0] + for k := range e.idxValues { + delete(e.idxValues, k) + } + } + e.done = true + chk.AppendUint64(0, e.removeCnt) + return nil +} + +func (e *CleanupIndexExec) buildIndexScan(ctx context.Context, txn kv.Transaction) (distsql.SelectResult, error) { + dagPB, err := e.buildIdxDAGPB(txn) + if err != nil { + return nil, errors.Trace(err) + } + sc := e.ctx.GetSessionVars().StmtCtx + var builder distsql.RequestBuilder + ranges := ranger.FullNewRange() + kvReq, err := builder.SetIndexRanges(sc, e.table.Meta().ID, e.index.Meta().ID, ranges). + SetDAGRequest(dagPB). + SetKeepOrder(true). + SetFromSessionVars(e.ctx.GetSessionVars()). + Build() + kvReq.KeyRanges[0].StartKey = kv.Key(e.lastIdxKey).PrefixNext() + kvReq.Concurrency = 1 + result, err := distsql.Select(ctx, e.ctx, kvReq, e.getIdxColTypes(), statistics.NewQueryFeedback(0, nil, 0, false)) + if err != nil { + return nil, errors.Trace(err) + } + result.Fetch(ctx) + return result, nil +} + +// Open implements the Executor Open interface. +func (e *CleanupIndexExec) Open(ctx context.Context) error { + if err := e.baseExecutor.Open(ctx); err != nil { + return errors.Trace(err) + } + e.idxChunk = chunk.NewChunk(e.getIdxColTypes()) + e.idxValues = make(map[int64][]types.Datum, e.batchSize) + e.batchKeys = make([]kv.Key, 0, e.batchSize) + e.idxValsBufs = make([][]types.Datum, e.batchSize) + sc := e.ctx.GetSessionVars().StmtCtx + idxKey, _, err := e.index.GenIndexKey(sc, []types.Datum{{}}, math.MinInt64, nil) + if err != nil { + return errors.Trace(err) + } + e.lastIdxKey = idxKey + return nil +} + +func (e *CleanupIndexExec) buildIdxDAGPB(txn kv.Transaction) (*tipb.DAGRequest, error) { + dagReq := &tipb.DAGRequest{} + dagReq.StartTs = txn.StartTS() + dagReq.TimeZoneOffset = timeZoneOffset(e.ctx) + sc := e.ctx.GetSessionVars().StmtCtx + dagReq.Flags = statementContextToFlags(sc) + for i := range e.idxCols { + dagReq.OutputOffsets = append(dagReq.OutputOffsets, uint32(i)) + } + + execPB := e.constructIndexScanPB() + dagReq.Executors = append(dagReq.Executors, execPB) + err := plan.SetPBColumnsDefaultValue(e.ctx, dagReq.Executors[0].IdxScan.Columns, e.idxCols) + if err != nil { + return nil, errors.Trace(err) + } + + limitExec := e.constructLimitPB() + dagReq.Executors = append(dagReq.Executors, limitExec) + + return dagReq, nil +} + +func (e *CleanupIndexExec) constructIndexScanPB() *tipb.Executor { + idxExec := &tipb.IndexScan{ + TableId: e.table.Meta().ID, + IndexId: e.index.Meta().ID, + Columns: plan.ColumnsToProto(e.idxCols, e.table.Meta().PKIsHandle), + } + return &tipb.Executor{Tp: tipb.ExecType_TypeIndexScan, IdxScan: idxExec} +} + +func (e *CleanupIndexExec) constructLimitPB() *tipb.Executor { + limitExec := &tipb.Limit{ + Limit: e.batchSize, + } + return &tipb.Executor{Tp: tipb.ExecType_TypeLimit, Limit: limitExec} +} + +// Close implements the Executor Close interface. +func (e *CleanupIndexExec) Close() error { + return nil +} diff --git a/executor/admin_test.go b/executor/admin_test.go index a7a799756f921..f083c1b75bf9a 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -213,6 +213,190 @@ func (s *testSuite) TestAdminRecoverIndex1(c *C) { tk.MustExec("admin check index admin_test `primary`") } +func (s *testSuite) TestAdminCleanupIndex(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + 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), unique key(c2), key (c3))") + tk.MustExec("insert admin_test (c1, c2) values (1, 2), (3, 4), (-5, NULL)") + tk.MustExec("insert admin_test (c1, c3) values (7, 100), (9, 100), (11, NULL)") + + // pk is handle, no need to cleanup + _, err := tk.Exec("admin cleanup index admin_test `primary`") + c.Assert(err, NotNil) + r := tk.MustQuery("admin cleanup index admin_test c2") + r.Check(testkit.Rows("0")) + r = tk.MustQuery("admin cleanup index admin_test c3") + r.Check(testkit.Rows("0")) + + // Make some dangling index. + 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) + + tblInfo := tbl.Meta() + idxInfo2 := findIndexByName("c2", tblInfo.Indices) + indexOpr2 := tables.NewIndex(tblInfo, idxInfo2) + idxInfo3 := findIndexByName("c3", tblInfo.Indices) + indexOpr3 := tables.NewIndex(tblInfo, idxInfo3) + + txn, err := s.store.Begin() + c.Assert(err, IsNil) + _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(1), -100) + c.Assert(err, IsNil) + _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(6), 100) + c.Assert(err, IsNil) + _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(nil), 101) + c.Assert(err, IsNil) + _, err = indexOpr3.Create(s.ctx, txn, types.MakeDatums(6), 200) + c.Assert(err, IsNil) + _, err = indexOpr3.Create(s.ctx, txn, types.MakeDatums(6), -200) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + + _, err = tk.Exec("admin check table admin_test") + c.Assert(err, NotNil) + _, err = tk.Exec("admin check index admin_test c2") + c.Assert(err, NotNil) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") + r.Check(testkit.Rows("9")) + r = tk.MustQuery("admin cleanup index admin_test c2") + r.Check(testkit.Rows("3")) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") + r.Check(testkit.Rows("6")) + tk.MustExec("admin check index admin_test c2") + + _, err = tk.Exec("admin check table admin_test") + c.Assert(err, NotNil) + _, err = tk.Exec("admin check index admin_test c3") + c.Assert(err, NotNil) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c3)") + r.Check(testkit.Rows("8")) + r = tk.MustQuery("admin cleanup index admin_test c3") + r.Check(testkit.Rows("2")) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c3)") + r.Check(testkit.Rows("6")) + tk.MustExec("admin check index admin_test c3") + + tk.MustExec("admin check table admin_test") +} + +func (s *testSuite) TestAdminCleanupIndexPKNotHandle(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists admin_test") + tk.MustExec("create table admin_test (c1 int, c2 int, c3 int, primary key (c1, c2))") + tk.MustExec("insert admin_test (c1, c2) values (1, 2), (3, 4), (-5, 5)") + + r := tk.MustQuery("admin cleanup index admin_test `primary`") + r.Check(testkit.Rows("0")) + + // Make some dangling index. + 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) + + tblInfo := tbl.Meta() + idxInfo := findIndexByName("primary", tblInfo.Indices) + indexOpr := tables.NewIndex(tblInfo, idxInfo) + + txn, err := s.store.Begin() + c.Assert(err, IsNil) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(7, 10), -100) + c.Assert(err, IsNil) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(4, 6), 100) + c.Assert(err, IsNil) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(-7, 4), 101) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + + _, err = tk.Exec("admin check table admin_test") + c.Assert(err, NotNil) + _, err = tk.Exec("admin check index admin_test `primary`") + c.Assert(err, NotNil) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(`primary`)") + r.Check(testkit.Rows("6")) + r = tk.MustQuery("admin cleanup index admin_test `primary`") + r.Check(testkit.Rows("3")) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(`primary`)") + r.Check(testkit.Rows("3")) + tk.MustExec("admin check index admin_test `primary`") + tk.MustExec("admin check table admin_test") +} + +func (s *testSuite) TestAdminCleanupIndexMore(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists admin_test") + tk.MustExec("create table admin_test (c1 int, c2 int, unique key (c1, c2), key (c2))") + tk.MustExec("insert admin_test values (1, 2), (3, 4), (5, 6)") + + tk.MustExec("admin cleanup index admin_test c1") + tk.MustExec("admin cleanup index admin_test c2") + + // Make some dangling index. + 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) + + tblInfo := tbl.Meta() + idxInfo1 := findIndexByName("c1", tblInfo.Indices) + indexOpr1 := tables.NewIndex(tblInfo, idxInfo1) + idxInfo2 := findIndexByName("c2", tblInfo.Indices) + indexOpr2 := tables.NewIndex(tblInfo, idxInfo2) + + txn, err := s.store.Begin() + c.Assert(err, IsNil) + for i := 0; i < 2000; i++ { + c1 := int64(2*i + 7) + c2 := int64(2*i + 8) + _, err = indexOpr1.Create(s.ctx, txn, types.MakeDatums(c1, c2), c1) + c.Assert(err, IsNil) + _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(c2), c1) + c.Assert(err, IsNil) + } + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + + _, err = tk.Exec("admin check table admin_test") + c.Assert(err, NotNil) + _, err = tk.Exec("admin check index admin_test c1") + c.Assert(err, NotNil) + _, err = tk.Exec("admin check index admin_test c2") + c.Assert(err, NotNil) + r := tk.MustQuery("SELECT COUNT(*) FROM admin_test") + r.Check(testkit.Rows("3")) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c1)") + r.Check(testkit.Rows("2003")) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") + r.Check(testkit.Rows("2003")) + r = tk.MustQuery("admin cleanup index admin_test c1") + r.Check(testkit.Rows("2000")) + r = tk.MustQuery("admin cleanup index admin_test c2") + r.Check(testkit.Rows("2000")) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c1)") + r.Check(testkit.Rows("3")) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") + r.Check(testkit.Rows("3")) + tk.MustExec("admin check index admin_test c1") + tk.MustExec("admin check index admin_test c2") + tk.MustExec("admin check table admin_test") +} + func (s *testSuite) TestAdminCheckTable(c *C) { // test NULL value. tk := testkit.NewTestKit(c, s.store) diff --git a/executor/builder.go b/executor/builder.go index d95f596070080..85fc49b163f04 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -74,6 +74,8 @@ func (b *executorBuilder) build(p plan.Plan) Executor { return b.buildCheckIndex(v) case *plan.RecoverIndex: return b.buildRecoverIndex(v) + case *plan.CleanupIndex: + return b.buildCleanupIndex(v) case *plan.CheckIndexRange: return b.buildCheckIndexRange(v) case *plan.ChecksumTable: @@ -294,6 +296,54 @@ func (b *executorBuilder) buildRecoverIndex(v *plan.RecoverIndex) Executor { return e } +func buildCleanupIndexCols(tblInfo *model.TableInfo, indexInfo *model.IndexInfo) []*model.ColumnInfo { + columns := make([]*model.ColumnInfo, 0, len(indexInfo.Columns)+1) + for _, idxCol := range indexInfo.Columns { + columns = append(columns, tblInfo.Columns[idxCol.Offset]) + } + handleColsInfo := &model.ColumnInfo{ + ID: model.ExtraHandleID, + Name: model.ExtraHandleName, + Offset: len(tblInfo.Columns), + } + handleColsInfo.FieldType = *types.NewFieldType(mysql.TypeLonglong) + columns = append(columns, handleColsInfo) + return columns +} + +func (b *executorBuilder) buildCleanupIndex(v *plan.CleanupIndex) Executor { + tblInfo := v.Table.TableInfo + t, err := b.is.TableByName(v.Table.Schema, tblInfo.Name) + if err != nil { + b.err = errors.Trace(err) + return nil + } + idxName := strings.ToLower(v.IndexName) + var index table.Index + for _, idx := range t.Indices() { + if idx.Meta().State != model.StatePublic { + continue + } + if idxName == idx.Meta().Name.L { + index = idx + break + } + } + + if index == nil { + b.err = errors.Errorf("index `%v` is not found in table `%v`.", v.IndexName, v.Table.Name.O) + return nil + } + e := &CleanupIndexExec{ + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), + idxCols: buildCleanupIndexCols(tblInfo, index.Meta()), + index: index, + table: t, + batchSize: 20000, + } + return e +} + func (b *executorBuilder) buildCheckIndexRange(v *plan.CheckIndexRange) Executor { tb, err := b.is.TableByName(v.Table.Schema, v.Table.Name) if err != nil { diff --git a/parser/misc.go b/parser/misc.go index 3e2ce62c28983..1c41bd47a566e 100644 --- a/parser/misc.go +++ b/parser/misc.go @@ -178,6 +178,7 @@ var tokenMap = map[string]int{ "CHARSET": charsetKwd, "CHECK": check, "CHECKSUM": checksum, + "CLEANUP": cleanup, "CLIENT": client, "COALESCE": coalesce, "COLLATE": collate, diff --git a/parser/parser.y b/parser/parser.y index a14b3cccd57e6..a0da3659fc617 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -260,6 +260,7 @@ import ( cascaded "CASCADED" charsetKwd "CHARSET" checksum "CHECKSUM" + cleanup "CLEANUP" client "CLIENT" coalesce "COALESCE" collation "COLLATION" @@ -2532,7 +2533,7 @@ Identifier: identifier | UnReservedKeyword | NotKeywordToken | TiDBKeyword UnReservedKeyword: - "ACTION" | "ASCII" | "AUTO_INCREMENT" | "AFTER" | "ALWAYS" | "AVG" | "BEGIN" | "BIT" | "BOOL" | "BOOLEAN" | "BTREE" | "BYTE" | "CHARSET" + "ACTION" | "ASCII" | "AUTO_INCREMENT" | "AFTER" | "ALWAYS" | "AVG" | "BEGIN" | "BIT" | "BOOL" | "BOOLEAN" | "BTREE" | "BYTE" | "CLEANUP" | "CHARSET" | "COLUMNS" | "COMMIT" | "COMPACT" | "COMPRESSED" | "CONSISTENT" | "DATA" | "DATE" %prec lowerThanStringLitToken| "DATETIME" | "DAY" | "DEALLOCATE" | "DO" | "DUPLICATE" | "DYNAMIC"| "END" | "ENGINE" | "ENGINES" | "ENUM" | "ESCAPE" | "EXECUTE" | "FIELDS" | "FIRST" | "FIXED" | "FLUSH" | "FORMAT" | "FULL" |"GLOBAL" | "HASH" | "HOUR" | "LESS" | "LOCAL" | "NAMES" | "OFFSET" | "PASSWORD" %prec lowerThanEq | "PREPARE" | "QUICK" | "REDUNDANT" @@ -4823,6 +4824,14 @@ AdminStmt: Index: string($5), } } +| "ADMIN" "CLEANUP" "INDEX" TableName Identifier + { + $$ = &ast.AdminStmt{ + Tp: ast.AdminCleanupIndex, + Tables: []*ast.TableName{$4.(*ast.TableName)}, + Index: string($5), + } + } | "ADMIN" "CHECK" "INDEX" TableName Identifier HandleRangeList { $$ = &ast.AdminStmt{ diff --git a/parser/parser_test.go b/parser/parser_test.go index aea081687d92e..4d429bb82edbc 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -416,6 +416,7 @@ func (s *testParserSuite) TestDMLStmt(c *C) { {"admin cancel ddl jobs 1", true}, {"admin cancel ddl jobs 1, 2", true}, {"admin recover index t1 idx_a", true}, + {"admin cleanup index t1 idx_a", true}, // for on duplicate key update {"INSERT INTO t (a,b,c) VALUES (1,2,3),(4,5,6) ON DUPLICATE KEY UPDATE c=VALUES(a)+VALUES(b);", true}, diff --git a/plan/common_plans.go b/plan/common_plans.go index c4f70c98b6ca1..cfefc40c68d9e 100644 --- a/plan/common_plans.go +++ b/plan/common_plans.go @@ -65,6 +65,14 @@ type RecoverIndex struct { IndexName string } +// CleanupIndex is used to delete dangling index data. +type CleanupIndex struct { + baseSchemaProducer + + Table *ast.TableName + IndexName string +} + // CheckIndex is used for checking index data, built from the 'admin check index' statement. type CheckIndex struct { baseSchemaProducer diff --git a/plan/planbuilder.go b/plan/planbuilder.go index a40f4f932ca42..2209b10fad146 100644 --- a/plan/planbuilder.go +++ b/plan/planbuilder.go @@ -491,6 +491,10 @@ func (b *planBuilder) buildAdmin(as *ast.AdminStmt) Plan { p := &RecoverIndex{Table: as.Tables[0], IndexName: as.Index} p.SetSchema(buildRecoverIndexFields()) ret = p + case ast.AdminCleanupIndex: + p := &CleanupIndex{Table: as.Tables[0], IndexName: as.Index} + p.SetSchema(buildCleanupIndexFields()) + ret = p case ast.AdminChecksumTable: p := &ChecksumTable{Tables: as.Tables} p.SetSchema(buildChecksumTableSchema()) @@ -665,6 +669,12 @@ func buildRecoverIndexFields() *expression.Schema { return schema } +func buildCleanupIndexFields() *expression.Schema { + schema := expression.NewSchema(make([]*expression.Column, 0, 1)...) + schema.Append(buildColumn("", "REMOVED_COUNT", mysql.TypeLonglong, 4)) + return schema +} + func buildShowDDLJobsFields() *expression.Schema { schema := expression.NewSchema(make([]*expression.Column, 0, 2)...) schema.Append(buildColumn("", "JOBS", mysql.TypeVarchar, 128))