From 48d831fc5781787c8005562cf140b71b3691c500 Mon Sep 17 00:00:00 2001 From: Lynn Date: Tue, 11 Sep 2018 17:09:51 +0800 Subject: [PATCH 01/23] *: speed up "admin check table" --- executor/builder.go | 19 +++++- executor/distsql.go | 6 ++ executor/executor.go | 124 +++++++++++++++++++++++++--------- executor/executor_test.go | 2 +- plan/common_plans.go | 5 +- plan/planbuilder.go | 136 ++++++++++++++++++++++++++++++-------- util/admin/admin.go | 28 ++++++-- util/admin/admin_test.go | 8 +-- 8 files changed, 253 insertions(+), 75 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index aed3694fdde3a..59233917d098d 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -246,10 +246,27 @@ func (b *executorBuilder) buildCheckIndex(v *plan.CheckIndex) Executor { } func (b *executorBuilder) buildCheckTable(v *plan.CheckTable) Executor { + readerExecs := make([]*IndexLookUpExecutor, 0, len(v.IndexLookUpReaders)) + for _, readerPlan := range v.IndexLookUpReaders { + readerExec, err := buildNoRangeIndexLookUpReader(b, readerPlan) + if err != nil { + b.err = errors.Trace(err) + return nil + } + readerExec.ranges = ranger.FullRange() + readerExec.isCheckOp = true + readerExecs = append(readerExecs, readerExec) + } + e := &CheckTableExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), - tables: v.Tables, + dbName: v.DBName, + tblInfo: v.TblInfo, + indices: v.Indices, is: b.is, + srcs: readerExecs, + exitCh: make(chan struct{}), + errCh: make(chan error, 1), genExprs: v.GenExprs, } return e diff --git a/executor/distsql.go b/executor/distsql.go index b16c04aad1901..78b8a29004de9 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -14,6 +14,7 @@ package executor import ( + //"fmt" "math" "runtime" "sort" @@ -760,6 +761,11 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er sort.Sort(task) } + // var rowsStr string + // for i, r := range task.rows { + // rowsStr += fmt.Sprintf("no.%d row: %v, ", i, r.GetInt64(w.handleIdx)) + // } + // log.Warnf("----------------------------------- handles %v, rows %v, cnt %d, len %d", task.handles, rowsStr, handleCnt, len(task.rows)) if w.isCheckOp && handleCnt != len(task.rows) { obtainedHandlesMap := make(map[int64]struct{}, len(task.rows)) for _, row := range task.rows { diff --git a/executor/executor.go b/executor/executor.go index 7932c0a90a2aa..ba007acec16a7 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -349,9 +349,14 @@ func getTableName(is infoschema.InfoSchema, id int64) string { type CheckTableExec struct { baseExecutor - tables []*ast.TableName - done bool - is infoschema.InfoSchema + dbName string + tblInfo *model.TableInfo + indices []table.Index + srcs []*IndexLookUpExecutor + done bool + is infoschema.InfoSchema + exitCh chan struct{} + errCh chan error genExprs map[string]expression.Expression } @@ -361,56 +366,111 @@ func (e *CheckTableExec) Open(ctx context.Context) error { if err := e.baseExecutor.Open(ctx); err != nil { return errors.Trace(err) } + for _, src := range e.srcs { + if err := src.Open(ctx); err != nil { + return errors.Trace(err) + } + } e.done = false return nil } -// Next implements the Executor Next interface. -func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { +// Close implements the Executor Close interface. +func (e *CheckTableExec) Close() error { if e.done { return nil } - defer func() { e.done = true }() - for _, t := range e.tables { - dbName := t.DBInfo.Name - tb, err := e.is.TableByName(dbName, t.Name) + + var err error + for _, src := range e.srcs { + err = src.Close() if err != nil { return errors.Trace(err) } - if tb.Meta().GetPartitionInfo() != nil { - err = e.doCheckPartitionedTable(tb.(table.PartitionedTable)) - } else { - err = e.doCheckTable(tb) - } + } + close(e.exitCh) + return nil +} + +func (e *CheckTableExec) checkIndex(ctx context.Context, src *IndexLookUpExecutor) error { + chk := src.newChunk() + for { + err := src.Next(ctx, chk) if err != nil { - log.Warnf("%v error:%v", t.Name, errors.ErrorStack(err)) - if admin.ErrDataInConsistent.Equal(err) { - return ErrAdminCheckTable.Gen("%v err:%v", t.Name, err) + select { + case e.errCh <- errors.Trace(err): + log.Warnf(".................. err %v", err) + return errors.Trace(err) + default: } + } + if chk.NumRows() == 0 { + break + } - return errors.Errorf("%v err:%v", t.Name, err) + select { + case <-e.exitCh: + return nil + default: } } return nil } -func (e *CheckTableExec) doCheckPartitionedTable(tbl table.PartitionedTable) error { - info := tbl.Meta().GetPartitionInfo() - for _, def := range info.Definitions { - pid := def.ID - partition := tbl.GetPartition(pid) - if err := e.doCheckTable(partition); err != nil { - return errors.Trace(err) +// Next implements the Executor Next interface. +func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.done || len(e.srcs) == 0 { + return nil + } + defer func() { e.done = true }() + + idxNames := make([]string, 0, len(e.indices)) + for _, idx := range e.indices { + idxNames = append(idxNames, idx.Meta().Name.O) + } + greater, idxOffset, err := admin.CheckIndicesCount(e.ctx, e.dbName, e.tblInfo.Name.O, idxNames) + if err != nil { + log.Warnf("check table, greater %v index %s error %v", greater, idxNames[idxOffset], errors.ErrorStack(err)) + t := e.srcs[idxOffset].table + txn := e.ctx.Txn() + if greater == admin.IdxCntGreater { + err = e.checkIndex(ctx, e.srcs[idxOffset]) + } else { + if t.Meta().GetPartitionInfo() != nil { + err = e.doCheckPartitionedTable(txn, t.(table.PartitionedTable), e.indices[idxOffset]) + } else { + err = admin.CheckRecordAndIndex(e.ctx, txn, t, e.indices[idxOffset], e.genExprs) + } + } + if err != nil && admin.ErrDataInConsistent.Equal(err) { + return ErrAdminCheckTable.Gen("%v err:%v", t.Meta().Name, err) } + return errors.Trace(err) } - return nil + + wg := sync.WaitGroup{} + for _, src := range e.srcs { + wg.Add(1) + go func() { + defer wg.Done() + e.checkIndex(ctx, src) + }() + } + wg.Wait() + + select { + case err = <-e.errCh: + default: + } + return errors.Trace(err) } -func (e *CheckTableExec) doCheckTable(tbl table.Table) error { - for _, idx := range tbl.Indices() { - txn := e.ctx.Txn() - err := admin.CompareIndexData(e.ctx, txn, tbl, idx, e.genExprs) - if err != nil { +func (e *CheckTableExec) doCheckPartitionedTable(txn kv.Transaction, tbl table.PartitionedTable, idx table.Index) error { + info := tbl.Meta().GetPartitionInfo() + for _, def := range info.Definitions { + pid := def.ID + partition := tbl.GetPartition(pid) + if err := admin.CheckRecordAndIndex(e.ctx, txn, partition, idx, e.genExprs); err != nil { return errors.Trace(err) } } @@ -455,7 +515,7 @@ func (e *CheckIndexExec) Next(ctx context.Context, chk *chunk.Chunk) error { } defer func() { e.done = true }() - err := admin.CheckIndicesCount(e.ctx, e.dbName, e.tableName, []string{e.idxName}) + _, _, err := admin.CheckIndicesCount(e.ctx, e.dbName, e.tableName, []string{e.idxName}) if err != nil { return errors.Trace(err) } diff --git a/executor/executor_test.go b/executor/executor_test.go index 926a3e3f40afc..3fa070cde9247 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -230,7 +230,7 @@ func (s *testSuite) TestAdmin(c *C) { tb, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("admin_test")) c.Assert(err, IsNil) c.Assert(tb.Indices(), HasLen, 1) - _, err = tb.Indices()[0].Create(mock.NewContext(), txn, types.MakeDatums(int64(10)), 1) + _, err = tb.Indices()[0].Create(mock.NewContext(), txn, types.MakeDatums(int64(10)), 10) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) diff --git a/plan/common_plans.go b/plan/common_plans.go index fafd419a3b62d..456616b1b2d12 100644 --- a/plan/common_plans.go +++ b/plan/common_plans.go @@ -55,7 +55,10 @@ type ShowDDLJobQueries struct { type CheckTable struct { baseSchemaProducer - Tables []*ast.TableName + DBName string + TblInfo *model.TableInfo + Indices []table.Index + IndexLookUpReaders []*PhysicalIndexLookUpReader GenExprs map[string]expression.Expression } diff --git a/plan/planbuilder.go b/plan/planbuilder.go index 3b64ffcb3f164..ede49fb27316d 100644 --- a/plan/planbuilder.go +++ b/plan/planbuilder.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" + log "github.com/sirupsen/logrus" ) type visitInfo struct { @@ -508,44 +509,121 @@ func (b *planBuilder) buildAdmin(as *ast.AdminStmt) (Plan, error) { return ret, nil } +func (b *planBuilder) buildPhysicalIndexScan(dbName model.CIStr, tblInfo *model.TableInfo, idx *model.IndexInfo, id int) Plan { + columns := make([]*model.ColumnInfo, 0, len(idx.Columns)) + schema := expression.NewSchema(make([]*expression.Column, 0, len(idx.Columns))...) + for _, idxCol := range idx.Columns { + for _, col := range tblInfo.Columns { + if idxCol.Name.L == col.Name.L { + columns = append(columns, col) + schema.Append(&expression.Column{ + ColName: col.Name, + UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(), + RetType: &col.FieldType, + }) + } + } + } + is := PhysicalIndexScan{ + Table: tblInfo, + TableAsName: &tblInfo.Name, + DBName: dbName, + Columns: columns, + Index: idx, + dataSourceSchema: schema, + Ranges: ranger.FullRange(), + KeepOrder: false, + }.init(b.ctx) + is.stats = &statsInfo{} + cop := &copTask{indexPlan: is} + // It's double read case. + ts := PhysicalTableScan{Columns: columns, Table: is.Table}.init(b.ctx) + ts.SetSchema(is.dataSourceSchema) + cop.tablePlan = ts + is.initSchema(id, idx, true) + t := finishCopTask(b.ctx, cop) + rootT := t.(*rootTask) + return rootT.p +} + +func (b *planBuilder) buildPhysicalIndexScans(dbName model.CIStr, as *ast.AdminStmt) ([]Plan, []table.Index, error) { + tblName := as.Tables[0] + tbl, err := b.is.TableByName(dbName, tblName.Name) + if err != nil { + return nil, nil, errors.Trace(err) + } + tblInfo := tbl.Meta() + + // get index information + indices := make([]table.Index, 0, len(tblInfo.Indices)) + indexLookUpReaders := make([]Plan, 0, len(tblInfo.Indices)) + for i, idx := range tbl.Indices() { + idxInfo := idx.Meta() + if idxInfo.State != model.StatePublic { + log.Warnf("index %s state %s isn't public", idxInfo.Name.O, idxInfo.State) + } else { + indices = append(indices, idx) + reader := b.buildPhysicalIndexScan(dbName, tblInfo, idxInfo, i) + indexLookUpReaders = append(indexLookUpReaders, reader) + } + } + if len(indexLookUpReaders) == 0 { + return nil, nil, nil + } + return indexLookUpReaders, indices, nil +} + func (b *planBuilder) buildAdminCheckTable(as *ast.AdminStmt) (*CheckTable, error) { - p := &CheckTable{Tables: as.Tables} - p.GenExprs = make(map[string]expression.Expression) + tbl := as.Tables[0] + p := &CheckTable{ + DBName: tbl.Schema.O, + TblInfo: tbl.TableInfo, + GenExprs: make(map[string]expression.Expression), + } mockTablePlan := LogicalTableDual{}.init(b.ctx) - for _, tbl := range p.Tables { - tableInfo := tbl.TableInfo - schema := expression.TableInfo2SchemaWithDBName(b.ctx, tbl.Schema, tableInfo) - table, ok := b.is.TableByID(tableInfo.ID) - if !ok { - return nil, infoschema.ErrTableNotExists.GenByArgs(tbl.DBInfo.Name.O, tableInfo.Name.O) - } + tableInfo := as.Tables[0].TableInfo + schema := expression.TableInfo2SchemaWithDBName(b.ctx, tbl.Schema, tableInfo) + table, ok := b.is.TableByID(tableInfo.ID) + if !ok { + return nil, infoschema.ErrTableNotExists.GenByArgs(tbl.DBInfo.Name.O, tableInfo.Name.O) + } - mockTablePlan.SetSchema(schema) + mockTablePlan.SetSchema(schema) - // Calculate generated columns. - columns := table.Cols() - for _, column := range columns { - if !column.IsGenerated() { - continue - } - columnName := &ast.ColumnName{Name: column.Name} - columnName.SetText(column.Name.O) + // Calculate generated columns. + columns := table.Cols() + for _, column := range columns { + if !column.IsGenerated() { + continue + } + columnName := &ast.ColumnName{Name: column.Name} + columnName.SetText(column.Name.O) - colExpr, _, err := mockTablePlan.findColumn(columnName) - if err != nil { - return nil, errors.Trace(err) - } + colExpr, _, err := mockTablePlan.findColumn(columnName) + if err != nil { + return nil, errors.Trace(err) + } - expr, _, err := b.rewrite(column.GeneratedExpr, mockTablePlan, nil, true) - if err != nil { - return nil, errors.Trace(err) - } - expr = expression.BuildCastFunction(b.ctx, expr, colExpr.GetType()) - genColumnName := model.GetTableColumnID(tableInfo, column.ColumnInfo) - p.GenExprs[genColumnName] = expr + expr, _, err := b.rewrite(column.GeneratedExpr, mockTablePlan, nil, true) + if err != nil { + return nil, errors.Trace(err) } + expr = expression.BuildCastFunction(b.ctx, expr, colExpr.GetType()) + genColumnName := model.GetTableColumnID(tableInfo, column.ColumnInfo) + p.GenExprs[genColumnName] = expr + } + + readerPlans, indices, err := b.buildPhysicalIndexScans(tbl.Schema, as) + if err != nil { + return nil, errors.Trace(err) + } + readers := make([]*PhysicalIndexLookUpReader, 0, len(readerPlans)) + for _, plan := range readerPlans { + readers = append(readers, plan.(*PhysicalIndexLookUpReader)) } + p.Indices = indices + p.IndexLookUpReaders = readers return p, nil } diff --git a/util/admin/admin.go b/util/admin/admin.go index 97a0e6d2dac3d..4586bc98c3a4d 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -231,28 +231,42 @@ func getCount(ctx sessionctx.Context, sql string) (int64, error) { return rows[0].GetInt64(0), nil } +const ( + TblCntGreater byte = 1 + IdxCntGreater byte = 2 +) + // CheckIndicesCount compares indices count with table count. // It returns nil if the count from the index is equal to the count from the table columns, // otherwise it returns an error with a different information. -func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices []string) error { +func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices []string) (byte, int, error) { // Add `` for some names like `table name`. sql := fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s`", dbName, tableName) tblCnt, err := getCount(ctx, sql) if err != nil { - return errors.Trace(err) + return 0, 0, errors.Trace(err) } - for _, idx := range indices { + for i, idx := range indices { sql = fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s` USE INDEX(`%s`)", dbName, tableName, idx) idxCnt, err := getCount(ctx, sql) if err != nil { - return errors.Trace(err) + return 0, 0, errors.Trace(err) } - if tblCnt != idxCnt { - return errors.Errorf("table count %d != index(%s) count %d", tblCnt, idx, idxCnt) + + if tblCnt == idxCnt { + continue } + + var ret byte + if tblCnt > idxCnt { + ret = TblCntGreater + } else if idxCnt > tblCnt { + ret = IdxCntGreater + } + return ret, i, errors.Errorf("table count %d != index(%s) count %d", tblCnt, idx, idxCnt) } - return nil + return 0, 0, nil } // ScanIndexData scans the index handles and values in a limited number, according to the index information. diff --git a/util/admin/admin_test.go b/util/admin/admin_test.go index 1e88fe08a67bc..70139b73a7836 100644 --- a/util/admin/admin_test.go +++ b/util/admin/admin_test.go @@ -418,7 +418,7 @@ func (s *testSuite) testIndex(c *C, ctx sessionctx.Context, dbName string, tb ta c.Assert(err, IsNil) idxNames := []string{idx.Meta().Name.L} - err = CheckIndicesCount(ctx, dbName, tb.Meta().Name.L, idxNames) + _, _, err = CheckIndicesCount(ctx, dbName, tb.Meta().Name.L, idxNames) c.Assert(err, IsNil) mockCtx := mock.NewContext() @@ -440,7 +440,7 @@ func (s *testSuite) testIndex(c *C, ctx sessionctx.Context, dbName string, tb ta diffMsg := newDiffRetError("index", record1, nil) c.Assert(err.Error(), DeepEquals, diffMsg) - err = CheckIndicesCount(ctx, dbName, tb.Meta().Name.L, idxNames) + _, _, err = CheckIndicesCount(ctx, dbName, tb.Meta().Name.L, idxNames) c.Assert(err, IsNil) // set data to: @@ -499,7 +499,7 @@ func (s *testSuite) testIndex(c *C, ctx sessionctx.Context, dbName string, tb ta diffMsg = newDiffRetError("index", record1, nil) c.Assert(err.Error(), DeepEquals, diffMsg) - err = CheckIndicesCount(ctx, dbName, tb.Meta().Name.L, idxNames) + _, _, err = CheckIndicesCount(ctx, dbName, tb.Meta().Name.L, idxNames) c.Assert(err.Error(), Equals, "table count 3 != index(c) count 4") // set data to: @@ -519,7 +519,7 @@ func (s *testSuite) testIndex(c *C, ctx sessionctx.Context, dbName string, tb ta diffMsg = newDiffRetError("index", nil, record1) c.Assert(err.Error(), DeepEquals, diffMsg) - err = CheckIndicesCount(ctx, dbName, tb.Meta().Name.L, idxNames) + _, _, err = CheckIndicesCount(ctx, dbName, tb.Meta().Name.L, idxNames) c.Assert(err.Error(), Equals, "table count 4 != index(c) count 3") } From 25f85a2b5adb1ca395c20878db38f9f7c582203e Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 26 Nov 2018 15:56:57 +0800 Subject: [PATCH 02/23] *: update --- executor/builder.go | 2 +- executor/executor.go | 12 +++++++++--- planner/core/planbuilder.go | 8 ++++---- 3 files changed, 14 insertions(+), 8 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 086cc8eaf26ed..c50f85a14e1c1 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -268,7 +268,7 @@ func (b *executorBuilder) buildCheckIndex(v *plannercore.CheckIndex) Executor { return e } -func (b *executorBuilder) buildCheckTable(v *plan.CheckTable) Executor { +func (b *executorBuilder) buildCheckTable(v *plannercore.CheckTable) Executor { readerExecs := make([]*IndexLookUpExecutor, 0, len(v.IndexLookUpReaders)) for _, readerPlan := range v.IndexLookUpReaders { readerExec, err := buildNoRangeIndexLookUpReader(b, readerPlan) diff --git a/executor/executor.go b/executor/executor.go index 9ab51b1f12eea..e17d242023d9c 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -451,7 +451,13 @@ func (e *CheckTableExec) Close() error { } func (e *CheckTableExec) checkIndex(ctx context.Context, src *IndexLookUpExecutor) error { - chk := src.newChunk() + cols := src.schema.Columns + retFieldTypes := make([]*types.FieldType, len(cols)) + for i := range cols { + retFieldTypes[i] = cols[i].RetType + } + chk := chunk.New(retFieldTypes, e.initCap, e.maxChunkSize) + log.Warnf(".................. types %v, cols %v", retFieldTypes, cols) for { err := src.Next(ctx, chk) if err != nil { @@ -490,7 +496,7 @@ func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { if err != nil { log.Warnf("check table, greater %v index %s error %v", greater, idxNames[idxOffset], errors.ErrorStack(err)) t := e.srcs[idxOffset].table - txn := e.ctx.Txn() + txn := e.ctx.Txn(true) if greater == admin.IdxCntGreater { err = e.checkIndex(ctx, e.srcs[idxOffset]) } else { @@ -501,7 +507,7 @@ func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { } } if err != nil && admin.ErrDataInConsistent.Equal(err) { - return ErrAdminCheckTable.Gen("%v err:%v", t.Meta().Name, err) + return ErrAdminCheckTable.GenWithStack("%v err:%v", t.Meta().Name, err) } return errors.Trace(err) } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 288eb8265b301..dce9f895d8ab6 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -458,7 +458,7 @@ func (b *PlanBuilder) buildCheckIndex(dbName model.CIStr, as *ast.AdminStmt) (Pl Ranges: ranger.FullRange(), KeepOrder: false, }.Init(b.ctx) - is.stats = &property.StatsInfo{} + is.stats = property.NewSimpleStats(0) cop := &copTask{indexPlan: is} // It's double read case. ts := PhysicalTableScan{Columns: columns, Table: is.Table}.Init(b.ctx) @@ -571,7 +571,7 @@ func (b *PlanBuilder) buildPhysicalIndexScan(dbName model.CIStr, tblInfo *model. Ranges: ranger.FullRange(), KeepOrder: false, }.Init(b.ctx) - is.stats = &statsInfo{} + is.stats = property.NewSimpleStats(0) cop := &copTask{indexPlan: is} // It's double read case. ts := PhysicalTableScan{Columns: columns, Table: is.Table}.Init(b.ctx) @@ -647,8 +647,8 @@ func (b *PlanBuilder) buildAdminCheckTable(as *ast.AdminStmt) (*CheckTable, erro return nil, errors.Trace(err) } expr = expression.BuildCastFunction(b.ctx, expr, colExpr.GetType()) - genColumnName := model.GetTableColumnID(tableInfo, column.ColumnInfo) - p.GenExprs[genColumnName] = expr + genColumnID := model.TableColumnID{TableID: tableInfo.ID, ColumnID: column.ColumnInfo.ID} + p.GenExprs[genColumnID] = expr } readerPlans, indices, err := b.buildPhysicalIndexScans(tbl.Schema, as) From 7e2bf6f412ff0a58fd28e686f9f6a1c3e19eb2c3 Mon Sep 17 00:00:00 2001 From: Lynn Date: Wed, 28 Nov 2018 14:43:51 +0800 Subject: [PATCH 03/23] *: add logs --- executor/executor.go | 10 ++++++---- util/admin/admin.go | 2 ++ 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index e17d242023d9c..c8d299c25dc65 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -457,7 +457,7 @@ func (e *CheckTableExec) checkIndex(ctx context.Context, src *IndexLookUpExecuto retFieldTypes[i] = cols[i].RetType } chk := chunk.New(retFieldTypes, e.initCap, e.maxChunkSize) - log.Warnf(".................. types %v, cols %v", retFieldTypes, cols) + // log.Warnf(".................. types %v, cols %v", retFieldTypes, cols) for { err := src.Next(ctx, chk) if err != nil { @@ -513,12 +513,14 @@ func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { } wg := sync.WaitGroup{} - for _, src := range e.srcs { + for i, src := range e.srcs { wg.Add(1) - go func() { + go func(num int) { defer wg.Done() + startTime := time.Now() e.checkIndex(ctx, src) - }() + log.Warnf(".................. no.%d sub %v", num, time.Since(startTime)) + }(i) } wg.Wait() diff --git a/util/admin/admin.go b/util/admin/admin.go index 35aafb45e55af..7ba3b2a2c9077 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -247,6 +247,7 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices return 0, 0, errors.Trace(err) } for i, idx := range indices { + startTime := time.Now() sql = fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s` USE INDEX(`%s`)", dbName, tableName, idx) idxCnt, err := getCount(ctx, sql) if err != nil { @@ -254,6 +255,7 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices } if tblCnt == idxCnt { + log.Warnf("---............... no.%d, name %v sub %v", i, idx, time.Since(startTime)) continue } From a7785e82de3f5124faaa79babe7506efb2997edb Mon Sep 17 00:00:00 2001 From: Lynn Date: Wed, 28 Nov 2018 15:32:47 +0800 Subject: [PATCH 04/23] *: add logs --- executor/executor.go | 11 ++++++-- util/admin/admin.go | 67 +++++++++++++++++++++++++++++++------------- 2 files changed, 56 insertions(+), 22 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index c8d299c25dc65..4c67e9cc352a3 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -497,14 +497,16 @@ func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { log.Warnf("check table, greater %v index %s error %v", greater, idxNames[idxOffset], errors.ErrorStack(err)) t := e.srcs[idxOffset].table txn := e.ctx.Txn(true) - if greater == admin.IdxCntGreater { + switch greater { + case admin.IdxCntGreater: err = e.checkIndex(ctx, e.srcs[idxOffset]) - } else { + case admin.TblCntGreater: if t.Meta().GetPartitionInfo() != nil { err = e.doCheckPartitionedTable(txn, t.(table.PartitionedTable), e.indices[idxOffset]) } else { err = admin.CheckRecordAndIndex(e.ctx, txn, t, e.indices[idxOffset], e.genExprs) } + default: } if err != nil && admin.ErrDataInConsistent.Equal(err) { return ErrAdminCheckTable.GenWithStack("%v err:%v", t.Meta().Name, err) @@ -513,21 +515,24 @@ func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { } wg := sync.WaitGroup{} + startT := time.Now() for i, src := range e.srcs { wg.Add(1) go func(num int) { defer wg.Done() startTime := time.Now() e.checkIndex(ctx, src) - log.Warnf(".................. no.%d sub %v", num, time.Since(startTime)) + log.Warnf(".................. no.%d index look up, sub %v", num, time.Since(startTime)) }(i) } wg.Wait() + log.Warnf("finish 111 .................. sub %v", time.Since(startT)) select { case err = <-e.errCh: default: } + log.Warnf("finish 222 .................. sub %v", time.Since(startT)) return errors.Trace(err) } diff --git a/util/admin/admin.go b/util/admin/admin.go index 7ba3b2a2c9077..a3273dcec2947 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -17,6 +17,7 @@ import ( "fmt" "io" "sort" + "sync" "time" "github.com/pingcap/errors" @@ -232,8 +233,9 @@ func getCount(ctx sessionctx.Context, sql string) (int64, error) { } const ( - TblCntGreater byte = 1 - IdxCntGreater byte = 2 + InvalidGreater byte = 0 + TblCntGreater byte = 1 + IdxCntGreater byte = 2 ) // CheckIndicesCount compares indices count with table count. @@ -246,28 +248,55 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices if err != nil { return 0, 0, errors.Trace(err) } + + wg := sync.WaitGroup{} + type result struct { + greater byte + offsert int + err error + } + retCh := make(chan result, len(indices)) + startT := time.Now() for i, idx := range indices { - startTime := time.Now() - sql = fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s` USE INDEX(`%s`)", dbName, tableName, idx) - idxCnt, err := getCount(ctx, sql) - if err != nil { - return 0, 0, errors.Trace(err) - } + wg.Add(1) + go func(num int, idx string, ch chan result) { + defer wg.Done() + startTime := time.Now() + sql = fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s` USE INDEX(`%s`)", dbName, tableName, idx) + idxCnt, err := getCount(ctx, sql) + if err != nil { + ch <- result{ + greater: InvalidGreater, + offsert: i, + err: err, + } + } - if tblCnt == idxCnt { - log.Warnf("---............... no.%d, name %v sub %v", i, idx, time.Since(startTime)) - continue - } + if tblCnt == idxCnt { + log.Warnf("count ............... no.%d name %v, sub %v", num, idx, time.Since(startTime)) + return + } - var ret byte - if tblCnt > idxCnt { - ret = TblCntGreater - } else if idxCnt > tblCnt { - ret = IdxCntGreater - } - return ret, i, errors.Errorf("table count %d != index(%s) count %d", tblCnt, idx, idxCnt) + var ret byte + if tblCnt > idxCnt { + ret = TblCntGreater + } else if idxCnt > tblCnt { + ret = IdxCntGreater + } + ch <- result{ + greater: ret, + offsert: i, + err: errors.Errorf("table count %d != index(%s) count %d", tblCnt, idx, idxCnt), + } + }(i, idx, retCh) } + wg.Wait() + log.Warnf("finish count .................. sub %v", time.Since(startT)) + if len(retCh) > 0 { + ret := <-retCh + return ret.greater, ret.offsert, err + } return 0, 0, nil } From e7c6392baceee38c81e8291e613f3eac6774463b Mon Sep 17 00:00:00 2001 From: xia Date: Fri, 30 Nov 2018 15:03:52 +0800 Subject: [PATCH 05/23] executor: tiny update --- executor/distsql.go | 5 +++++ executor/executor.go | 3 +-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/executor/distsql.go b/executor/distsql.go index ebe760e44e2fd..15bdac0def532 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -354,6 +354,11 @@ type IndexLookUpExecutor struct { // Open implements the Executor Open interface. func (e *IndexLookUpExecutor) Open(ctx context.Context) error { + log.Warnf("index look up, open ---------------------------------------------------------------------------- table %v, index %v", e.table.Meta().Name, e.index.Name) + if e.table.Meta().Name.L == "x" { + return nil + } + var err error if e.corColInAccess { e.ranges, err = rebuildIndexRanges(e.ctx, e.idxPlans[0].(*plannercore.PhysicalIndexScan), e.idxCols, e.colLens) diff --git a/executor/executor.go b/executor/executor.go index 4c67e9cc352a3..b1db24c89276e 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -527,12 +527,11 @@ func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { } wg.Wait() - log.Warnf("finish 111 .................. sub %v", time.Since(startT)) select { case err = <-e.errCh: default: } - log.Warnf("finish 222 .................. sub %v", time.Since(startT)) + log.Warnf("finish .................. sub %v", time.Since(startT)) return errors.Trace(err) } From 9c324e96d6facd5c11617adf4c61132d1a7df337 Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 3 Dec 2018 23:05:53 +0800 Subject: [PATCH 06/23] *: update --- executor/distsql.go | 5 ----- executor/executor.go | 14 ++++++-------- util/admin/admin.go | 10 ++++------ 3 files changed, 10 insertions(+), 19 deletions(-) diff --git a/executor/distsql.go b/executor/distsql.go index 15bdac0def532..ebe760e44e2fd 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -354,11 +354,6 @@ type IndexLookUpExecutor struct { // Open implements the Executor Open interface. func (e *IndexLookUpExecutor) Open(ctx context.Context) error { - log.Warnf("index look up, open ---------------------------------------------------------------------------- table %v, index %v", e.table.Meta().Name, e.index.Name) - if e.table.Meta().Name.L == "x" { - return nil - } - var err error if e.corColInAccess { e.ranges, err = rebuildIndexRanges(e.ctx, e.idxPlans[0].(*plannercore.PhysicalIndexScan), e.idxCols, e.colLens) diff --git a/executor/executor.go b/executor/executor.go index b1db24c89276e..ffbf9e41c87d5 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -439,15 +439,15 @@ func (e *CheckTableExec) Close() error { return nil } + close(e.exitCh) var err error for _, src := range e.srcs { - err = src.Close() - if err != nil { - return errors.Trace(err) + err1 := src.Close() + if err == nil && err1 != nil { + err = err1 } } - close(e.exitCh) - return nil + return errors.Trace(err) } func (e *CheckTableExec) checkIndex(ctx context.Context, src *IndexLookUpExecutor) error { @@ -457,13 +457,11 @@ func (e *CheckTableExec) checkIndex(ctx context.Context, src *IndexLookUpExecuto retFieldTypes[i] = cols[i].RetType } chk := chunk.New(retFieldTypes, e.initCap, e.maxChunkSize) - // log.Warnf(".................. types %v, cols %v", retFieldTypes, cols) for { err := src.Next(ctx, chk) if err != nil { select { case e.errCh <- errors.Trace(err): - log.Warnf(".................. err %v", err) return errors.Trace(err) default: } @@ -494,7 +492,7 @@ func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { } greater, idxOffset, err := admin.CheckIndicesCount(e.ctx, e.dbName, e.tblInfo.Name.O, idxNames) if err != nil { - log.Warnf("check table, greater %v index %s error %v", greater, idxNames[idxOffset], errors.ErrorStack(err)) + log.Warnf("check table %v, greater %v index %s error %v", e.tblInfo.Name, greater, idxNames[idxOffset], errors.ErrorStack(err)) t := e.srcs[idxOffset].table txn := e.ctx.Txn(true) switch greater { diff --git a/util/admin/admin.go b/util/admin/admin.go index a3273dcec2947..b7b4ce277c4b6 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -261,19 +261,17 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices wg.Add(1) go func(num int, idx string, ch chan result) { defer wg.Done() - startTime := time.Now() sql = fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s` USE INDEX(`%s`)", dbName, tableName, idx) idxCnt, err := getCount(ctx, sql) if err != nil { ch <- result{ greater: InvalidGreater, offsert: i, - err: err, + err: errors.Trace(err), } } - + log.Infof("check indices count, table %s cnt %d, index %s cnt %d", tableName, tblCnt, idx, idxCnt) if tblCnt == idxCnt { - log.Warnf("count ............... no.%d name %v, sub %v", num, idx, time.Since(startTime)) return } @@ -292,11 +290,11 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices } wg.Wait() - log.Warnf("finish count .................. sub %v", time.Since(startT)) if len(retCh) > 0 { ret := <-retCh - return ret.greater, ret.offsert, err + return ret.greater, ret.offsert, errors.Trace(ret.err) } + log.Warnf("finish count .................. sub %v", time.Since(startT)) return 0, 0, nil } From 0a590abb1406d62a6562aa93e46c8794dd28ac78 Mon Sep 17 00:00:00 2001 From: Lynn Date: Tue, 4 Dec 2018 12:53:46 +0800 Subject: [PATCH 07/23] *: add tests --- executor/admin_test.go | 50 +++++++++++++++++++++++++++++++++++++ executor/distsql.go | 5 ---- executor/executor.go | 5 +--- planner/core/planbuilder.go | 12 +++------ util/admin/admin.go | 10 +++----- 5 files changed, 58 insertions(+), 24 deletions(-) diff --git a/executor/admin_test.go b/executor/admin_test.go index 20ad7b47d80ca..981bf3e293e38 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -409,6 +409,56 @@ func (s *testSuite) TestAdminCleanupIndexMore(c *C) { tk.MustExec("admin check table admin_test") } +func (s *testSuite) TestAdminCheckTableFailed(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), key(c3), unique key(c2))") + tk.MustExec("insert admin_test (c1, c2) values (1, 1), (2, 2), (3, 3), (10, 10), (20, 20)") + + // Make some corrupted index. Build the index information. + 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("c2", tblInfo.Indices) + indexOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) + sc := s.ctx.GetSessionVars().StmtCtx + + // Reduce one row of index. + // Table count > index count. + txn, err := s.store.Begin() + c.Assert(err, IsNil) + err = indexOpr.Delete(sc, txn, types.MakeDatums(1), 1) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + _, err = tk.Exec("admin check table admin_test") + c.Assert(err.Error(), Equals, + "[executor:8003]admin_test err:[admin:1]index: != record:&admin.RecordData{Handle:1, Values:[]types.Datum{types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:1, b:[]uint8(nil), x:interface {}(nil)}}}") + c.Assert(executor.ErrAdminCheckTable.Equal(err), IsTrue) + r := tk.MustQuery("admin recover index admin_test c2") + r.Check(testkit.Rows("1 5")) + tk.MustExec("admin check table admin_test") + + // Add one row of index. + // Table count < index count. + txn, err = s.store.Begin() + c.Assert(err, IsNil) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(11), 11) + c.Assert(err, IsNil) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(22), 22) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + _, err = tk.Exec("admin check table admin_test") + c.Assert(err.Error(), Equals, "handle count 7 isn't equal to value count 5, missing handles [11 22] in a batch") +} + func (s *testSuite) TestAdminCheckTable(c *C) { // test NULL value. tk := testkit.NewTestKit(c, s.store) diff --git a/executor/distsql.go b/executor/distsql.go index ebe760e44e2fd..55e5566ca7b2e 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -762,11 +762,6 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er sort.Sort(task) } - // var rowsStr string - // for i, r := range task.rows { - // rowsStr += fmt.Sprintf("no.%d row: %v, ", i, r.GetInt64(w.handleIdx)) - // } - // log.Warnf("----------------------------------- handles %v, rows %v, cnt %d, len %d", task.handles, rowsStr, handleCnt, len(task.rows)) if w.isCheckOp && handleCnt != len(task.rows) { obtainedHandlesMap := make(map[int64]struct{}, len(task.rows)) for _, row := range task.rows { diff --git a/executor/executor.go b/executor/executor.go index ffbf9e41c87d5..c68ad82764b91 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -512,15 +512,13 @@ func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { return errors.Trace(err) } + // The number of table rows is equal to the number of index rows. wg := sync.WaitGroup{} - startT := time.Now() for i, src := range e.srcs { wg.Add(1) go func(num int) { defer wg.Done() - startTime := time.Now() e.checkIndex(ctx, src) - log.Warnf(".................. no.%d index look up, sub %v", num, time.Since(startTime)) }(i) } wg.Wait() @@ -529,7 +527,6 @@ func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { case err = <-e.errCh: default: } - log.Warnf("finish .................. sub %v", time.Since(startT)) return errors.Trace(err) } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index dce9f895d8ab6..a83b685205a97 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -583,21 +583,15 @@ func (b *PlanBuilder) buildPhysicalIndexScan(dbName model.CIStr, tblInfo *model. return rootT.p } -func (b *PlanBuilder) buildPhysicalIndexScans(dbName model.CIStr, as *ast.AdminStmt) ([]Plan, []table.Index, error) { - tblName := as.Tables[0] - tbl, err := b.is.TableByName(dbName, tblName.Name) - if err != nil { - return nil, nil, errors.Trace(err) - } +func (b *PlanBuilder) buildPhysicalIndexScans(dbName model.CIStr, tbl table.Table) ([]Plan, []table.Index, error) { tblInfo := tbl.Meta() - // get index information indices := make([]table.Index, 0, len(tblInfo.Indices)) indexLookUpReaders := make([]Plan, 0, len(tblInfo.Indices)) for i, idx := range tbl.Indices() { idxInfo := idx.Meta() if idxInfo.State != model.StatePublic { - log.Warnf("index %s state %s isn't public", idxInfo.Name.O, idxInfo.State) + log.Warnf("index %s state %s isn't public in table %s", idxInfo.Name, idxInfo.State, tblInfo.Name) } else { indices = append(indices, idx) reader := b.buildPhysicalIndexScan(dbName, tblInfo, idxInfo, i) @@ -651,7 +645,7 @@ func (b *PlanBuilder) buildAdminCheckTable(as *ast.AdminStmt) (*CheckTable, erro p.GenExprs[genColumnID] = expr } - readerPlans, indices, err := b.buildPhysicalIndexScans(tbl.Schema, as) + readerPlans, indices, err := b.buildPhysicalIndexScans(tbl.Schema, table) if err != nil { return nil, errors.Trace(err) } diff --git a/util/admin/admin.go b/util/admin/admin.go index b7b4ce277c4b6..9b0d32b0e6ba4 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -252,11 +252,10 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices wg := sync.WaitGroup{} type result struct { greater byte - offsert int + offset int err error } retCh := make(chan result, len(indices)) - startT := time.Now() for i, idx := range indices { wg.Add(1) go func(num int, idx string, ch chan result) { @@ -266,7 +265,7 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices if err != nil { ch <- result{ greater: InvalidGreater, - offsert: i, + offset: i, err: errors.Trace(err), } } @@ -283,7 +282,7 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices } ch <- result{ greater: ret, - offsert: i, + offset: i, err: errors.Errorf("table count %d != index(%s) count %d", tblCnt, idx, idxCnt), } }(i, idx, retCh) @@ -292,9 +291,8 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices if len(retCh) > 0 { ret := <-retCh - return ret.greater, ret.offsert, errors.Trace(ret.err) + return ret.greater, ret.offset, errors.Trace(ret.err) } - log.Warnf("finish count .................. sub %v", time.Since(startT)) return 0, 0, nil } From dc8fbd6e81b10762d424539def66fd225ca86fcd Mon Sep 17 00:00:00 2001 From: Lynn Date: Tue, 4 Dec 2018 13:08:35 +0800 Subject: [PATCH 08/23] executor: tiny update tests --- executor/admin_test.go | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/executor/admin_test.go b/executor/admin_test.go index 981bf3e293e38..bdea454b3dbfc 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -457,6 +457,17 @@ func (s *testSuite) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) _, err = tk.Exec("admin check table admin_test") c.Assert(err.Error(), Equals, "handle count 7 isn't equal to value count 5, missing handles [11 22] in a batch") + + // Table count = index count. + txn, err = s.store.Begin() + c.Assert(err, IsNil) + err = indexOpr.Delete(sc, txn, types.MakeDatums(11), 11) + c.Assert(err, IsNil) + err = indexOpr.Delete(sc, txn, types.MakeDatums(22), 22) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + tk.MustExec("admin check table admin_test") } func (s *testSuite) TestAdminCheckTable(c *C) { From 271c12b3e05e0f54a5bb993057347922442aa25d Mon Sep 17 00:00:00 2001 From: Lynn Date: Tue, 4 Dec 2018 19:37:49 +0800 Subject: [PATCH 09/23] *: update --- executor/admin_test.go | 32 ++++++++++++----- executor/builder.go | 2 +- executor/distsql.go | 1 - executor/executor.go | 70 +++++++++++++++++++++---------------- executor/executor_test.go | 2 +- planner/core/planbuilder.go | 26 +++++++------- util/admin/admin.go | 4 +-- 7 files changed, 79 insertions(+), 58 deletions(-) diff --git a/executor/admin_test.go b/executor/admin_test.go index bdea454b3dbfc..5eb528927bdcb 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -413,8 +413,8 @@ func (s *testSuite) TestAdminCheckTableFailed(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), key(c3), unique key(c2))") - tk.MustExec("insert admin_test (c1, c2) values (1, 1), (2, 2), (3, 3), (10, 10), (20, 20)") + tk.MustExec("create table admin_test (c1 int, c2 int, c3 int default 1, primary key(c1), key(c3), unique key(c2), key(c2, c3))") + tk.MustExec("insert admin_test (c1, c2, c3) values (1, 11, 21), (2, 12, 22), (5, 15, 23), (10, 20, 30), (20, 30, 40)") // Make some corrupted index. Build the index information. s.ctx = mock.NewContext() @@ -428,18 +428,20 @@ func (s *testSuite) TestAdminCheckTableFailed(c *C) { idxInfo := findIndexByName("c2", tblInfo.Indices) indexOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) sc := s.ctx.GetSessionVars().StmtCtx + tk.Se.GetSessionVars().IndexLookupSize = 3 + tk.Se.GetSessionVars().MaxChunkSize = 3 // Reduce one row of index. // Table count > index count. txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(1), 1) + err = indexOpr.Delete(sc, txn, types.MakeDatums(11), 1) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) _, err = tk.Exec("admin check table admin_test") c.Assert(err.Error(), Equals, - "[executor:8003]admin_test err:[admin:1]index: != record:&admin.RecordData{Handle:1, Values:[]types.Datum{types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:1, b:[]uint8(nil), x:interface {}(nil)}}}") + "[executor:8003]admin_test err:[admin:1]index: != record:&admin.RecordData{Handle:1, Values:[]types.Datum{types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:11, b:[]uint8(nil), x:interface {}(nil)}}}") c.Assert(executor.ErrAdminCheckTable.Equal(err), IsTrue) r := tk.MustQuery("admin recover index admin_test c2") r.Check(testkit.Rows("1 5")) @@ -449,21 +451,33 @@ func (s *testSuite) TestAdminCheckTableFailed(c *C) { // Table count < index count. txn, err = s.store.Begin() c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(11), 11) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(21), 1) c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(22), 22) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(13), 2) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) _, err = tk.Exec("admin check table admin_test") - c.Assert(err.Error(), Equals, "handle count 7 isn't equal to value count 5, missing handles [11 22] in a batch") + c.Assert(err.Error(), Equals, "admin check table admin_test, index c2 handle 2 more than one") // Table count = index count. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(11), 11) + err = indexOpr.Delete(sc, txn, types.MakeDatums(13), 2) + c.Assert(err, IsNil) + err = indexOpr.Delete(sc, txn, types.MakeDatums(12), 2) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + _, err = tk.Exec("admin check table admin_test") + c.Assert(err.Error(), Equals, "admin check table admin_test, index c2 handle 1 more than one") + + // Recover records. + txn, err = s.store.Begin() + c.Assert(err, IsNil) + err = indexOpr.Delete(sc, txn, types.MakeDatums(21), 1) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(22), 22) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(12), 2) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) diff --git a/executor/builder.go b/executor/builder.go index 6da244402ea28..5253226d8bd92 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -289,7 +289,7 @@ func (b *executorBuilder) buildCheckTable(v *plannercore.CheckTable) Executor { is: b.is, srcs: readerExecs, exitCh: make(chan struct{}), - errCh: make(chan error, 1), + retCh: make(chan error, len(v.Indices)), genExprs: v.GenExprs, } return e diff --git a/executor/distsql.go b/executor/distsql.go index 92e4a32073d21..d31a5bdbd4451 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -14,7 +14,6 @@ package executor import ( - //"fmt" "math" "runtime" "sort" diff --git a/executor/executor.go b/executor/executor.go index c68ad82764b91..8a6db59966306 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -414,7 +414,7 @@ type CheckTableExec struct { done bool is infoschema.InfoSchema exitCh chan struct{} - errCh chan error + retCh chan error genExprs map[model.TableColumnID]expression.Expression } @@ -450,33 +450,45 @@ func (e *CheckTableExec) Close() error { return errors.Trace(err) } -func (e *CheckTableExec) checkIndex(ctx context.Context, src *IndexLookUpExecutor) error { +func (e *CheckTableExec) checkIndexHandle(ctx context.Context, num int, src *IndexLookUpExecutor) error { cols := src.schema.Columns retFieldTypes := make([]*types.FieldType, len(cols)) for i := range cols { retFieldTypes[i] = cols[i].RetType } chk := chunk.New(retFieldTypes, e.initCap, e.maxChunkSize) + iter := chunk.NewIterator4Chunk(chk) + + var err error + handles := make(map[int64]struct{}, 1024) for { - err := src.Next(ctx, chk) + err = src.Next(ctx, chk) if err != nil { - select { - case e.errCh <- errors.Trace(err): - return errors.Trace(err) - default: - } + break } if chk.NumRows() == 0 { break } + for chunkRow := iter.Begin(); chunkRow != iter.End(); chunkRow = iter.Next() { + datum := chunkRow.GetDatum(0, retFieldTypes[0]) + h := datum.GetInt64() + if _, ok := handles[h]; ok { + err = errors.Errorf("admin check table %s, index %s handle %d more than one", e.tblInfo.Name, e.indices[num].Meta().Name, h) + e.retCh <- errors.Trace(err) + return errors.Trace(err) + } + handles[h] = struct{}{} + } + select { case <-e.exitCh: return nil default: } } - return nil + e.retCh <- errors.Trace(err) + return errors.Trace(err) } // Next implements the Executor Next interface. @@ -492,49 +504,47 @@ func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { } greater, idxOffset, err := admin.CheckIndicesCount(e.ctx, e.dbName, e.tblInfo.Name.O, idxNames) if err != nil { - log.Warnf("check table %v, greater %v index %s error %v", e.tblInfo.Name, greater, idxNames[idxOffset], errors.ErrorStack(err)) - t := e.srcs[idxOffset].table + log.Warnf("check table %v, greater %v index %s err: %v", e.tblInfo.Name, greater, idxNames[idxOffset], errors.ErrorStack(err)) + tbl := e.srcs[idxOffset].table txn := e.ctx.Txn(true) switch greater { case admin.IdxCntGreater: - err = e.checkIndex(ctx, e.srcs[idxOffset]) + err = e.checkIndexHandle(ctx, idxOffset, e.srcs[idxOffset]) case admin.TblCntGreater: - if t.Meta().GetPartitionInfo() != nil { - err = e.doCheckPartitionedTable(txn, t.(table.PartitionedTable), e.indices[idxOffset]) - } else { - err = admin.CheckRecordAndIndex(e.ctx, txn, t, e.indices[idxOffset], e.genExprs) - } + err = e.checkTableRecord(txn, tbl, e.indices[idxOffset]) default: } if err != nil && admin.ErrDataInConsistent.Equal(err) { - return ErrAdminCheckTable.GenWithStack("%v err:%v", t.Meta().Name, err) + return ErrAdminCheckTable.GenWithStack("%v err:%v", tbl.Meta().Name, err) } return errors.Trace(err) } // The number of table rows is equal to the number of index rows. - wg := sync.WaitGroup{} - for i, src := range e.srcs { - wg.Add(1) + for i, _ := range e.srcs { go func(num int) { - defer wg.Done() - e.checkIndex(ctx, src) + e.checkIndexHandle(ctx, num, e.srcs[num]) }(i) } - wg.Wait() - select { - case err = <-e.errCh: - default: + for i := 0; i < len(e.srcs); i++ { + err = <-e.retCh + if err != nil { + return errors.Trace(err) + } } - return errors.Trace(err) + return nil } -func (e *CheckTableExec) doCheckPartitionedTable(txn kv.Transaction, tbl table.PartitionedTable, idx table.Index) error { +func (e *CheckTableExec) checkTableRecord(txn kv.Transaction, tbl table.Table, idx table.Index) error { + if tbl.Meta().GetPartitionInfo() == nil { + return admin.CheckRecordAndIndex(e.ctx, txn, tbl, idx, e.genExprs) + } + info := tbl.Meta().GetPartitionInfo() for _, def := range info.Definitions { pid := def.ID - partition := tbl.GetPartition(pid) + partition := tbl.(table.PartitionedTable).GetPartition(pid) if err := admin.CheckRecordAndIndex(e.ctx, txn, partition, idx, e.genExprs); err != nil { return errors.Trace(err) } diff --git a/executor/executor_test.go b/executor/executor_test.go index 6813dc4c75d79..66c58f12411b0 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -235,7 +235,7 @@ func (s *testSuite) TestAdmin(c *C) { tb, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("admin_test")) c.Assert(err, IsNil) c.Assert(tb.Indices(), HasLen, 1) - _, err = tb.Indices()[0].Create(mock.NewContext(), txn, types.MakeDatums(int64(10)), 10) + _, err = tb.Indices()[0].Create(mock.NewContext(), txn, types.MakeDatums(int64(10)), 1) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 1d0d977348204..9fbb77bdad492 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -552,20 +552,18 @@ func (b *PlanBuilder) buildAdmin(as *ast.AdminStmt) (Plan, error) { } func (b *PlanBuilder) buildPhysicalIndexScan(dbName model.CIStr, tblInfo *model.TableInfo, idx *model.IndexInfo, id int) Plan { - columns := make([]*model.ColumnInfo, 0, len(idx.Columns)) - schema := expression.NewSchema(make([]*expression.Column, 0, len(idx.Columns))...) - for _, idxCol := range idx.Columns { - for _, col := range tblInfo.Columns { - if idxCol.Name.L == col.Name.L { - columns = append(columns, col) - schema.Append(&expression.Column{ - ColName: col.Name, - UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(), - RetType: &col.FieldType, - }) - } - } - } + columns := make([]*model.ColumnInfo, 0, 1) + schema := expression.NewSchema(make([]*expression.Column, 0, 1)...) + columns = append(columns, model.NewExtraHandleColInfo()) + handleCol := &expression.Column{ + DBName: dbName, + TblName: tblInfo.Name, + ColName: model.ExtraHandleName, + RetType: types.NewFieldType(mysql.TypeLonglong), + UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(), + ID: model.ExtraHandleID, + } + schema.Append(handleCol) is := PhysicalIndexScan{ Table: tblInfo, TableAsName: &tblInfo.Name, diff --git a/util/admin/admin.go b/util/admin/admin.go index 3a77207711235..777267f40f886 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -265,7 +265,7 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices if err != nil { ch <- result{ greater: InvalidGreater, - offset: i, + offset: num, err: errors.Trace(err), } } @@ -282,7 +282,7 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices } ch <- result{ greater: ret, - offset: i, + offset: num, err: errors.Errorf("table count %d != index(%s) count %d", tblCnt, idx, idxCnt), } }(i, idx, retCh) From 41c753ce67eb9cbc990c9584715cf1d2b3bad285 Mon Sep 17 00:00:00 2001 From: Lynn Date: Wed, 5 Dec 2018 14:00:43 +0800 Subject: [PATCH 10/23] executor: make fmt happy --- executor/executor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executor/executor.go b/executor/executor.go index 8a6db59966306..121366cd68763 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -521,7 +521,7 @@ func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { } // The number of table rows is equal to the number of index rows. - for i, _ := range e.srcs { + for i := range e.srcs { go func(num int) { e.checkIndexHandle(ctx, num, e.srcs[num]) }(i) From bb42b3d9d9048dc702b8513c87d6b1eb912d657c Mon Sep 17 00:00:00 2001 From: Lynn Date: Wed, 5 Dec 2018 15:45:17 +0800 Subject: [PATCH 11/23] util/admin: make lint happy --- util/admin/admin.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/util/admin/admin.go b/util/admin/admin.go index 777267f40f886..0a19c932cc0e0 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -232,10 +232,13 @@ func getCount(ctx sessionctx.Context, sql string) (int64, error) { return rows[0].GetInt64(0), nil } +// Count greater Types const ( InvalidGreater byte = 0 - TblCntGreater byte = 1 - IdxCntGreater byte = 2 + // TblCntGreater means that the number of table rows is more than the number of index rows. + TblCntGreater byte = 1 + // IdxCntGreater means that the number of index rows is more than the number of table rows. + IdxCntGreater byte = 2 ) // CheckIndicesCount compares indices count with table count. From 83e019ce4e470a5a47053d331a648558bfaf4449 Mon Sep 17 00:00:00 2001 From: Lynn Date: Tue, 11 Dec 2018 16:04:48 +0800 Subject: [PATCH 12/23] util/admin: fix data race --- util/admin/admin.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/util/admin/admin.go b/util/admin/admin.go index 3de63e478f146..a8654c39c1c44 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -263,8 +263,7 @@ const ( // otherwise it returns an error with a different information. func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices []string) (byte, int, error) { // Add `` for some names like `table name`. - sql := fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s`", dbName, tableName) - tblCnt, err := getCount(ctx, sql) + tblCnt, err := getCount(ctx, fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s`", dbName, tableName)) if err != nil { return 0, 0, errors.Trace(err) } @@ -280,7 +279,7 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices wg.Add(1) go func(num int, idx string, ch chan result) { defer wg.Done() - sql = fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s` USE INDEX(`%s`)", dbName, tableName, idx) + sql := fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s` USE INDEX(`%s`)", dbName, tableName, idx) idxCnt, err := getCount(ctx, sql) if err != nil { ch <- result{ From 63c99ff4d915e5d33dc31b7a758534fc218458da Mon Sep 17 00:00:00 2001 From: Lynn Date: Tue, 11 Dec 2018 18:00:44 +0800 Subject: [PATCH 13/23] util/admin: fix DATA RACE --- util/admin/admin.go | 61 +++++++++++++-------------------------------- 1 file changed, 18 insertions(+), 43 deletions(-) diff --git a/util/admin/admin.go b/util/admin/admin.go index a8654c39c1c44..3b89c1d793948 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -17,7 +17,6 @@ import ( "fmt" "io" "sort" - "sync" "time" "github.com/pingcap/errors" @@ -263,54 +262,30 @@ const ( // otherwise it returns an error with a different information. func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices []string) (byte, int, error) { // Add `` for some names like `table name`. - tblCnt, err := getCount(ctx, fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s`", dbName, tableName)) + sql := fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s`", dbName, tableName) + tblCnt, err := getCount(ctx, sql) if err != nil { return 0, 0, errors.Trace(err) } - wg := sync.WaitGroup{} - type result struct { - greater byte - offset int - err error - } - retCh := make(chan result, len(indices)) for i, idx := range indices { - wg.Add(1) - go func(num int, idx string, ch chan result) { - defer wg.Done() - sql := fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s` USE INDEX(`%s`)", dbName, tableName, idx) - idxCnt, err := getCount(ctx, sql) - if err != nil { - ch <- result{ - greater: InvalidGreater, - offset: num, - err: errors.Trace(err), - } - } - log.Infof("check indices count, table %s cnt %d, index %s cnt %d", tableName, tblCnt, idx, idxCnt) - if tblCnt == idxCnt { - return - } - - var ret byte - if tblCnt > idxCnt { - ret = TblCntGreater - } else if idxCnt > tblCnt { - ret = IdxCntGreater - } - ch <- result{ - greater: ret, - offset: num, - err: errors.Errorf("table count %d != index(%s) count %d", tblCnt, idx, idxCnt), - } - }(i, idx, retCh) - } - wg.Wait() + sql = fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s` USE INDEX(`%s`)", dbName, tableName, idx) + idxCnt, err := getCount(ctx, sql) + if err != nil { + return InvalidGreater, i, errors.Trace(err) + } + log.Infof("check indices count, table %s cnt %d, index %s cnt %d", tableName, tblCnt, idx, idxCnt) + if tblCnt == idxCnt { + continue + } - if len(retCh) > 0 { - ret := <-retCh - return ret.greater, ret.offset, errors.Trace(ret.err) + var ret byte + if tblCnt > idxCnt { + ret = TblCntGreater + } else if idxCnt > tblCnt { + ret = IdxCntGreater + } + return ret, i, errors.Errorf("table count %d != index(%s) count %d", tblCnt, idx, idxCnt) } return 0, 0, nil } From 39ab60557822cba1d48e0ead6ebc8d619e10fe24 Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 17 Dec 2018 20:58:35 +0800 Subject: [PATCH 14/23] executor, util: address comments --- executor/admin_test.go | 4 ++-- executor/executor.go | 9 ++++----- util/admin/admin.go | 7 +++---- 3 files changed, 9 insertions(+), 11 deletions(-) diff --git a/executor/admin_test.go b/executor/admin_test.go index 37fb8b7333beb..ca931a6f62168 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -458,7 +458,7 @@ func (s *testSuite) TestAdminCheckTableFailed(c *C) { err = txn.Commit(context.Background()) c.Assert(err, IsNil) _, err = tk.Exec("admin check table admin_test") - c.Assert(err.Error(), Equals, "admin check table admin_test, index c2 handle 2 more than one") + c.Assert(err.Error(), Equals, "admin check table admin_test, index c2, at least two indices have the same handle 2 more than one") // Table count = index count. txn, err = s.store.Begin() @@ -470,7 +470,7 @@ func (s *testSuite) TestAdminCheckTableFailed(c *C) { err = txn.Commit(context.Background()) c.Assert(err, IsNil) _, err = tk.Exec("admin check table admin_test") - c.Assert(err.Error(), Equals, "admin check table admin_test, index c2 handle 1 more than one") + c.Assert(err.Error(), Equals, "admin check table admin_test, index c2, at least two indices have the same handle 1 more than one") // Recover records. txn, err = s.store.Begin() diff --git a/executor/executor.go b/executor/executor.go index ae1e4c2d7168d..ddeed05c68d81 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -474,7 +474,8 @@ func (e *CheckTableExec) checkIndexHandle(ctx context.Context, num int, src *Ind datum := chunkRow.GetDatum(0, retFieldTypes[0]) h := datum.GetInt64() if _, ok := handles[h]; ok { - err = errors.Errorf("admin check table %s, index %s handle %d more than one", e.tblInfo.Name, e.indices[num].Meta().Name, h) + err = errors.Errorf("admin check table %s, index %s, at least two indices have the same handle %d more than one", + e.tblInfo.Name, e.indices[num].Meta().Name, h) e.retCh <- errors.Trace(err) return errors.Trace(err) } @@ -507,12 +508,10 @@ func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { log.Warnf("check table %v, greater %v index %s err: %v", e.tblInfo.Name, greater, idxNames[idxOffset], errors.ErrorStack(err)) tbl := e.srcs[idxOffset].table txn := e.ctx.Txn(true) - switch greater { - case admin.IdxCntGreater: + if greater == admin.IdxCntGreater { err = e.checkIndexHandle(ctx, idxOffset, e.srcs[idxOffset]) - case admin.TblCntGreater: + } else if greater == admin.TblCntGreater { err = e.checkTableRecord(txn, tbl, e.indices[idxOffset]) - default: } if err != nil && admin.ErrDataInConsistent.Equal(err) { return ErrAdminCheckTable.GenWithStack("%v err:%v", tbl.Meta().Name, err) diff --git a/util/admin/admin.go b/util/admin/admin.go index 3b89c1d793948..f0b276dfca1eb 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -250,7 +250,6 @@ func getCount(ctx sessionctx.Context, sql string) (int64, error) { // Count greater Types const ( - InvalidGreater byte = 0 // TblCntGreater means that the number of table rows is more than the number of index rows. TblCntGreater byte = 1 // IdxCntGreater means that the number of index rows is more than the number of table rows. @@ -258,8 +257,9 @@ const ( ) // CheckIndicesCount compares indices count with table count. +// It returns the count greater type, the index offset and an error. // It returns nil if the count from the index is equal to the count from the table columns, -// otherwise it returns an error with a different information. +// otherwise it returns an error and the corresponding index's offset. func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices []string) (byte, int, error) { // Add `` for some names like `table name`. sql := fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s`", dbName, tableName) @@ -267,12 +267,11 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices if err != nil { return 0, 0, errors.Trace(err) } - for i, idx := range indices { sql = fmt.Sprintf("SELECT COUNT(*) FROM `%s`.`%s` USE INDEX(`%s`)", dbName, tableName, idx) idxCnt, err := getCount(ctx, sql) if err != nil { - return InvalidGreater, i, errors.Trace(err) + return 0, i, errors.Trace(err) } log.Infof("check indices count, table %s cnt %d, index %s cnt %d", tableName, tblCnt, idx, idxCnt) if tblCnt == idxCnt { From 89fb60aff9a89153ce6e26d46ed9aab672de82f3 Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 8 Jul 2019 10:39:42 +0800 Subject: [PATCH 15/23] *: check the index values --- executor/admin_test.go | 13 +-- executor/builder.go | 7 +- executor/distsql.go | 110 ++++++++++---------- executor/executor.go | 6 +- executor/executor_test.go | 2 +- expression/column.go | 2 + expression/scalar_function.go | 1 + planner/core/physical_plans.go | 2 + planner/core/planbuilder.go | 56 ++++++++-- store/mockstore/mocktikv/cop_handler_dag.go | 2 +- store/mockstore/mocktikv/executor.go | 1 + types/datum.go | 3 + util/admin/admin_integration_test.go | 4 +- 13 files changed, 126 insertions(+), 83 deletions(-) diff --git a/executor/admin_test.go b/executor/admin_test.go index 636dcb58a613c..d441eee71f07c 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -420,7 +420,8 @@ func (s *testSuite2) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) tblInfo := tbl.Meta() // idxInfo := findIndexByName("c3", tblInfo.Indices) - idxInfo := findIndexByName("c2", tblInfo.Indices) + // idxInfo := findIndexByName("c2", tblInfo.Indices) + idxInfo := tblInfo.Indices[1] indexOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) sc := s.ctx.GetSessionVars().StmtCtx tk.Se.GetSessionVars().IndexLookupSize = 3 @@ -431,7 +432,7 @@ func (s *testSuite2) TestAdminCheckTableFailed(c *C) { // Index c2 is missing 11. txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(11), 1) + err = indexOpr.Delete(sc, txn, types.MakeDatums(11), 1, nil) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -465,9 +466,9 @@ func (s *testSuite2) TestAdminCheckTableFailed(c *C) { // Two indices have the same handle. txn, err = s.store.Begin() c.Assert(err, IsNil) - // err = indexOpr.Delete(sc, txn, types.MakeDatums(13), 2) + // err = indexOpr.Delete(sc, txn, types.MakeDatums(13), 2, nil) // c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(12), 2) + err = indexOpr.Delete(sc, txn, types.MakeDatums(12), 2, nil) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -481,7 +482,7 @@ func (s *testSuite2) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(12), 2) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(11), 1) + err = indexOpr.Delete(sc, txn, types.MakeDatums(11), 1, nil) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -492,7 +493,7 @@ func (s *testSuite2) TestAdminCheckTableFailed(c *C) { // Recover records. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(21), 1) + err = indexOpr.Delete(sc, txn, types.MakeDatums(21), 1, nil) c.Assert(err, IsNil) _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(11), 1) c.Assert(err, IsNil) diff --git a/executor/builder.go b/executor/builder.go index 26a547e2f17cf..01d45f3920be7 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -354,7 +354,7 @@ func buildIndexLookUpChecker(b *executorBuilder, readerPlan *plannercore.Physica b.err = errors.Trace(err) return } - readerExec.genExprs = genExprs + // readerExec.genExprs = genExprs readerExec.isCheckOp = true readerExec.ranges = ranger.FullRange() } @@ -1962,8 +1962,8 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn } is := v.IndexPlans[0].(*plannercore.PhysicalIndexScan) indexReq.OutputOffsets = []uint32{uint32(len(is.Index.Columns))} - log.Warnf(".......................... dag req:%v, output offsets %v, cols %v", - indexReq.Executors[0].IdxScan, indexReq.OutputOffsets, is.Index.Columns) + log.Warnf(".......................... dag req:%v, output offsets %v, cols %v, exprs %d", + indexReq.Executors[0].IdxScan, indexReq.OutputOffsets, is.Index.Columns, len(is.GenExprs)) tbl, _ := b.is.TableByID(is.Table.ID) for i := 0; i < v.Schema().Len(); i++ { @@ -1998,6 +1998,7 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn tblPlans: v.TablePlans, } + e.genExprs = is.GenExprs if containsLimit(indexReq.Executors) { e.feedback = statistics.NewQueryFeedback(0, nil, 0, is.Desc) } else { diff --git a/executor/distsql.go b/executor/distsql.go index e085a5e4421ad..eb13b03106e70 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -715,7 +715,8 @@ func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, for len(handles) < w.batchSize { chk.SetRequiredRows(w.batchSize-len(handles), w.maxChunkSize) err = errors.Trace(idxResult.Next(ctx, chk)) - log.Warnf("===================================================== extract task handles, batch size: %v, rows: %v", w.batchSize, chk.NumRows()) + log.Warnf("===================================================== extract task handles, batch size: %v, rows: %v, keep order %v, ret chk %v", + w.batchSize, chk.NumRows(), w.keepOrder, retChk) if err != nil { return handles, nil, err } @@ -726,17 +727,25 @@ func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, h := chk.GetRow(i).GetInt64(chk.NumCols() - 1) handles = append(handles, h) - if w.isCheckOp { - if retChk == nil { - retChk = chunk.NewChunkWithCapacity(w.tps, w.maxChunkSize) - } - retChk.AppendRow(chunk.MutRowFromDatums(chk.GetRow(i).GetDatumRow(w.tps)).ToRow()) - } + // if w.isCheckOp { + // if retChk == nil { + // retChk = chunk.NewChunkWithCapacity(w.tps, w.maxChunkSize) + // } + // // retChk.Append(chk, 0, chk.NumRows())// (chunk.MutRowFromDatums(chk.GetRow(i).GetDatumRow(w.tps)).ToRow()) + // } if len(w.tps) > 1 { - log.Infof("xxx ..... cols len %v, tps len %v, datum1: %v, datum2: %v, handle: %v", - chk.NumCols(), len(w.tps), chk.GetRow(i).GetDatum(0, w.tps[0]), chk.GetRow(i).GetDatum(1, w.tps[1]), h) + colVal0 := chk.GetRow(i).GetDatum(0, w.tps[0]) + colVal1 := chk.GetRow(i).GetDatum(1, w.tps[1]) + log.Infof("xxx ..... cols len %v, tps len %v, datum1: %v, datum2: %v, handle: %v, col %v, col %#v", + chk.NumCols(), len(w.tps), colVal0, colVal1, h, colVal0, colVal0) } } + if w.isCheckOp { + if retChk == nil { + retChk = chunk.NewChunkWithCapacity(w.tps, w.maxChunkSize) + } + retChk.Append(chk, 0, chk.NumRows()) + } } w.batchSize *= 2 if w.batchSize > w.maxBatchSize { @@ -750,15 +759,19 @@ func (w *indexWorker) buildTableTask(handles []int64, retChk *chunk.Chunk) *look if w.keepOrder { // Save the index order. indexOrder = make(map[int64]int, len(handles)) + var str string for i, h := range handles { indexOrder[h] = i + str += fmt.Sprintf(" h %d, offset %d;", h, i) } + log.Infof("......... %s, ret chk %v", str, retChk) } task := &lookupTableTask{ handles: handles, indexOrder: indexOrder, idxRows: retChk, } + task.doneCh = make(chan error, 1) return task } @@ -783,16 +796,16 @@ type tableWorker struct { func (w *tableWorker) pickAndExecTask(ctx context.Context) { var task *lookupTableTask var ok bool - // defer func() { - // if r := recover(); r != nil { - // buf := make([]byte, 4096) - // stackSize := runtime.Stack(buf, false) - // buf = buf[:stackSize] - // log.Errorf("tableWorker panic stack is:\n%s", buf) - // logutil.Logger(ctx).Error("tableWorker in IndexLookUpExecutor panicked", zap.String("stack", string(buf))) - // task.doneCh <- errors.Errorf("%v", r) - // } - // }() + defer func() { + if r := recover(); r != nil { + buf := make([]byte, 4096) + stackSize := runtime.Stack(buf, false) + buf = buf[:stackSize] + log.Errorf("tableWorker panic stack is:\n%s", buf) + logutil.Logger(ctx).Error("tableWorker in IndexLookUpExecutor panicked", zap.String("stack", string(buf))) + task.doneCh <- errors.Errorf("%v", r) + } + }() for { // Don't check ctx.Done() on purpose. If background worker get the signal and all // exit immediately, session's goroutine doesn't know this and still calling Next(), @@ -845,11 +858,6 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta } tblReaderExec := tableReader.(*TableReaderExecutor) - // str := "" - // for i := 0; i < chk.NumRows(); i++ { - // str += fmt.Sprintf("no.%d, row %v", i, chk.GetRow(i).GetDatumRow(w.tps)) - // } - // log.Warnf("rows %v, cnt %v", str, chk.NumRows()) i := 0 iter := chunk.NewIterator4Chunk(chk) for row := iter.Begin(); row != iter.End(); row = iter.Next() { @@ -861,6 +869,8 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta for j, col := range w.cols { if col.IsGenerated() && !col.GeneratedStored { expr := w.genExprs[model.TableColumnID{TableID: w.tbl.Meta().ID, ColumnID: col.ID}] + log.Infof("xxx ------------------------------------------- idx %v, j %v, row len %v, cols len %v, col %v, rows %v, handles %v, expr %v", + w.checkIndexValue.idxInfo.Name, j, row.Len(), len(w.cols), col, chk.NumRows(), len(task.handles), expr) // Eval the column value val, err := expr.Eval(row) if err != nil { @@ -901,9 +911,7 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta i, offset, j, idxRow.GetInt64(idxRow.Len()-1), handle, tableReader.Schema().Columns[j].ColName, val, idxRow.GetDatum(j, tp)) if ret != 0 { - log.Errorf("handle %#v, index:%#v != record:%#v", handle, idxRow.GetDatum(j, tp), vals[j]) - // panic("xxx") - return errors.Errorf("handle %#v, index:%#v != record:%#v", handle, idxRow.GetDatum(j, tp), vals[j]) + return errors.Errorf("handle %#v, index:%#v != record:%#v", handle, idxRow.GetDatum(j, tp), val) } } i++ @@ -974,35 +982,25 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er sort.Sort(task) } - // if handleCnt != len(task.rows) { - // if w.isCheckOp { - // obtainedHandlesMap := make(map[int64]struct{}, len(task.rows)) - // for _, row := range task.rows { - // handle := row.GetInt64(w.handleIdx) - // obtainedHandlesMap[handle] = struct{}{} - // } - // return errors.Errorf("inconsistent index %s handle count %d isn't equal to value count %d, missing handles %v in a batch", - // w.idxLookup.index.Name.O, handleCnt, len(task.rows), GetLackHandles(task.handles, obtainedHandlesMap)) - // } - // - // if len(w.idxLookup.tblPlans) == 1 { - // obtainedHandlesMap := make(map[int64]struct{}, len(task.rows)) - // for _, row := range task.rows { - // handle := row.GetInt64(w.handleIdx) - // obtainedHandlesMap[handle] = struct{}{} - // } - // - // logutil.Logger(ctx).Error("inconsistent index handles", zap.String("index", w.idxLookup.index.Name.O), - // zap.Int("index_cnt", handleCnt), zap.Int("table_cnt", len(task.rows)), - // zap.Int64s("missing_handles", GetLackHandles(task.handles, obtainedHandlesMap)), - // zap.Int64s("total_handles", task.handles)) - // - // // table scan in double read can never has conditions according to convertToIndexScan. - // // if this table scan has no condition, the number of rows it returns must equal to the length of handles. - // return errors.Errorf("inconsistent index %s handle count %d isn't equal to value count %d", - // w.idxLookup.index.Name.O, handleCnt, len(task.rows)) - // } - // } + if handleCnt != len(task.rows) { + if len(w.idxLookup.tblPlans) == 1 { + obtainedHandlesMap := make(map[int64]struct{}, len(task.rows)) + for _, row := range task.rows { + handle := row.GetInt64(w.handleIdx) + obtainedHandlesMap[handle] = struct{}{} + } + + logutil.Logger(ctx).Error("inconsistent index handles", zap.String("index", w.idxLookup.index.Name.O), + zap.Int("index_cnt", handleCnt), zap.Int("table_cnt", len(task.rows)), + zap.Int64s("missing_handles", GetLackHandles(task.handles, obtainedHandlesMap)), + zap.Int64s("total_handles", task.handles)) + + // table scan in double read can never has conditions according to convertToIndexScan. + // if this table scan has no condition, the number of rows it returns must equal to the length of handles. + return errors.Errorf("inconsistent index %s handle count %d isn't equal to value count %d", + w.idxLookup.index.Name.O, handleCnt, len(task.rows)) + } + } return nil } diff --git a/executor/executor.go b/executor/executor.go index 8051f4a400f70..71809afd65a06 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -157,7 +157,7 @@ func newBaseExecutor(ctx sessionctx.Context, schema *expression.Schema, id fmt.S cols := schema.Columns e.retFieldTypes = make([]*types.FieldType, len(cols)) for i := range cols { - log.Warnf("------- new executor, id %s, col %v", id, cols[i]) + log.Warnf("------- new executor, id %s, col %v, no. %d", id, cols[i], cols[i].Index) e.retFieldTypes[i] = cols[i].RetType } } @@ -495,9 +495,9 @@ func (e *CheckTableExec) checkIndexHandle(ctx context.Context, num int, src *Ind var err error handles := make(map[int64]struct{}, 1024) for { - log.Infof("====================== check idnex handle. 0") + log.Infof("====================== check no.%d idx handle. 0", num) err = src.Next(ctx, chk) - log.Infof("====================== check idnex handle. 1") + log.Infof("====================== check no.%d idx handle. 1", num) if err != nil { break } diff --git a/executor/executor_test.go b/executor/executor_test.go index dfc5d246f8546..ccb8038746458 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3065,7 +3065,7 @@ func (s *testSuite) TestContainDotColumn(c *C) { c.Assert(terr.Code(), Equals, terror.ErrCode(mysql.ErrWrongTableName)) } -func (s *testSuite) TestCheckIndexx(c *C) { +func (s *testSuite) TestCheckIndex(c *C) { s.ctx = mock.NewContext() s.ctx.Store = s.store se, err := session.CreateSession4Test(s.store) diff --git a/expression/column.go b/expression/column.go index 99b94e0ea4414..6a46fdf62fc66 100644 --- a/expression/column.go +++ b/expression/column.go @@ -17,6 +17,7 @@ import ( "fmt" "strings" + "github.com/ngaut/log" "github.com/pingcap/errors" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -285,6 +286,7 @@ func (col *Column) EvalDuration(ctx sessionctx.Context, row chunk.Row) (types.Du // EvalJSON returns JSON representation of Column. func (col *Column) EvalJSON(ctx sessionctx.Context, row chunk.Row) (json.BinaryJSON, bool, error) { + log.Infof("........... idx %d, col %v", col.Index, col.ColName) if row.IsNull(col.Index) { return json.BinaryJSON{}, true, nil } diff --git a/expression/scalar_function.go b/expression/scalar_function.go index 77ce863a39406..4457967eab7c3 100644 --- a/expression/scalar_function.go +++ b/expression/scalar_function.go @@ -207,6 +207,7 @@ func (sf *ScalarFunction) Eval(row chunk.Row) (d types.Datum, err error) { case types.ETDecimal: res, isNull, err = sf.EvalDecimal(sf.GetCtx(), row) case types.ETDatetime, types.ETTimestamp: + // log.Infof("............. expr %v, row idx %d, row len %v", sf, row.Idx(), row.Len()) res, isNull, err = sf.EvalTime(sf.GetCtx(), row) case types.ETDuration: res, isNull, err = sf.EvalDuration(sf.GetCtx(), row) diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 32d257090270d..047c3027c21d6 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -119,6 +119,8 @@ type PhysicalIndexScan struct { // The index scan may be on a partition. isPartition bool physicalTableID int64 + + GenExprs map[model.TableColumnID]expression.Expression } // PhysicalMemTable reads memory table. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 1f577667faeb2..a0961b3252a4a 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -570,8 +570,7 @@ func (b *PlanBuilder) buildCheckIndex(dbName model.CIStr, as *ast.AdminStmt) (Pl // TODO: Handle generated column. genExprs := make(map[model.TableColumnID]expression.Expression) - reader := b.buildPhysicalIndexLookUpReader(dbName, tbl, idx, 1, genExprs) - return reader, nil + return b.buildPhysicalIndexLookUpReader(dbName, tbl, idx, 1, genExprs) // id := 1 // columns := make([]*model.ColumnInfo, 0, len(idx.Columns)) // schema := expression.NewSchema(make([]*expression.Column, 0, len(idx.Columns))...) @@ -690,6 +689,8 @@ func (b *PlanBuilder) buildAdmin(as *ast.AdminStmt) (Plan, error) { func getGenColumns(expr expression.Expression) []*expression.Column { col, ok := expr.(*expression.Column) if ok { + // return []*expression.Column{col.Clone().(*expression.Column)} + log.Infof("get gen col %#v", col) return []*expression.Column{col} } @@ -701,7 +702,6 @@ func getGenColumns(expr expression.Expression) []*expression.Column { for _, arg := range scalaFunc.GetArgs() { retCols := getGenColumns(arg) if retCols != nil { - log.Infof("get gen col %#v", col) cols = append(cols, retCols...) } } @@ -709,15 +709,42 @@ func getGenColumns(expr expression.Expression) []*expression.Column { } func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl table.Table, idx *model.IndexInfo, id int, - genExprs map[model.TableColumnID]expression.Expression) Plan { + genExprs map[model.TableColumnID]expression.Expression) (Plan, error) { tblInfo := tbl.Meta() columns := make([]*model.ColumnInfo, 0, len(idx.Columns)) tblColumns := make([]*model.ColumnInfo, 0, len(tbl.Cols())) - schema := expression.NewSchema(make([]*expression.Column, 0, len(idx.Columns))...) + + exprs := make(map[model.TableColumnID]expression.Expression) + mockTablePlan := LogicalTableDual{}.Init(b.ctx) + mockTablePlan.SetSchema(expression.TableInfo2SchemaWithDBName(b.ctx, dbName, tblInfo)) + for _, column := range idx.Columns { + col := table.FindCol(tbl.Cols(), column.Name.L) + if !col.IsGenerated() { + continue + } + log.Infof("222 gen col %v", col) + columnName := &ast.ColumnName{Name: column.Name} + columnName.SetText(column.Name.O) + + colExpr, _, err := mockTablePlan.findColumn(columnName) + if err != nil { + return nil, errors.Trace(err) + } + + expr, _, err := b.rewrite(col.GeneratedExpr, mockTablePlan, nil, true) + if err != nil { + return nil, errors.Trace(err) + } + expr = expression.BuildCastFunction(b.ctx, expr, colExpr.GetType()) + genColumnID := model.TableColumnID{TableID: tblInfo.ID, ColumnID: col.ColumnInfo.ID} + exprs[genColumnID] = expr + } + str := "" str2 := "" var genCols []*expression.Column colsMap := make(map[int64]struct{}) + schema := expression.NewSchema(make([]*expression.Column, 0, len(idx.Columns))...) for _, idxCol := range idx.Columns { for _, col := range tblInfo.Columns { if idxCol.Name.L == col.Name.L { @@ -733,7 +760,8 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab str2 += fmt.Sprintf("col %v, field tp %v ", col.Name, col.FieldType) } genColumnID := model.TableColumnID{TableID: tblInfo.ID, ColumnID: col.ID} - if expr, ok := genExprs[genColumnID]; ok { + // if expr, ok := genExprs[genColumnID]; ok { + if expr, ok := exprs[genColumnID]; ok { cols := getGenColumns(expr) if cols != nil { genCols = append(genCols, cols...) @@ -747,7 +775,6 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab if _, ok := colsMap[col.ID]; !ok { c := table.FindCol(tbl.Cols(), col.ColName.O) if c != nil { - log.Infof("222 gen col %v, field tp %v; ", col.ColName, col.RetType) col.Index = len(tblColumns) tblColumns = append(tblColumns, c.ColumnInfo) tblSchema.Append(&expression.Column{ @@ -755,7 +782,7 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(), RetType: &c.FieldType, }) - str2 += fmt.Sprintf("col %v, field tp %v; ", c.Name, c.FieldType) + str2 += fmt.Sprintf("no.%d, col %v, field tp %v; ", col.Index, c.Name, c.FieldType) colsMap[c.ID] = struct{}{} } } @@ -771,7 +798,7 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab } str2 += fmt.Sprintf("col %v", tblColumns[len(tblColumns)-1].Name) tblSchema.Append(handleCol) - log.Warnf("table %v, idx:%v, columns %#v, tbl columns %#v, len %v", tblInfo.Name, idx.Name, str, str2, len(tblColumns)) + log.Warnf("********************************* table %v, idx %v, columns %#v, tbl columns %#v, len %v, exprs %d", tblInfo.Name, idx.Name, str, str2, len(tblColumns), len(exprs)) is := PhysicalIndexScan{ Table: tblInfo, TableAsName: &tblInfo.Name, @@ -781,6 +808,7 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab dataSourceSchema: schema, Ranges: ranger.FullRange(), KeepOrder: false, + GenExprs: exprs, }.Init(b.ctx) is.stats = property.NewSimpleStats(0) cop := &copTask{indexPlan: is} @@ -793,7 +821,7 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab is.initSchema(id, idx, true) t := finishCopTask(b.ctx, cop) rootT := t.(*rootTask) - return rootT.p + return rootT.p, nil } func (b *PlanBuilder) buildPhysicalIndexLookUpReaders(dbName model.CIStr, tbl table.Table, @@ -808,8 +836,14 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReaders(dbName model.CIStr, tbl ta log.Warnf("index %s state %s isn't public in table %s", idxInfo.Name, idxInfo.State, tblInfo.Name) } else { indices = append(indices, idx) - reader := b.buildPhysicalIndexLookUpReader(dbName, tbl, idxInfo, i, genExprs) + reader, err := b.buildPhysicalIndexLookUpReader(dbName, tbl, idxInfo, i, genExprs) + if err != nil { + return nil, nil, err + } indexLookUpReaders = append(indexLookUpReaders, reader) + for i, expr := range genExprs { + log.Infof("******** no.%d, expr %v", i, expr.ExplainInfo()) + } } } if len(indexLookUpReaders) == 0 { diff --git a/store/mockstore/mocktikv/cop_handler_dag.go b/store/mockstore/mocktikv/cop_handler_dag.go index 3972f7b1983ef..3f46c7d1c116b 100644 --- a/store/mockstore/mocktikv/cop_handler_dag.go +++ b/store/mockstore/mocktikv/cop_handler_dag.go @@ -192,7 +192,7 @@ func (h *rpcHandler) buildDAG(ctx *dagContext, executors []*tipb.Executor) (exec func (h *rpcHandler) buildTableScan(ctx *dagContext, executor *tipb.Executor) (*tableScanExec, error) { columns := executor.TblScan.Columns - log.Infof("table scan, columns %v, col %v", len(columns), columns[0]) + log.Infof("table scan, columns %v, the first col %v", len(columns), columns[0]) ctx.evalCtx.setColumnInfo(columns) ranges, err := h.extractKVRanges(ctx.keyRanges, executor.TblScan.Desc) if err != nil { diff --git a/store/mockstore/mocktikv/executor.go b/store/mockstore/mocktikv/executor.go index d12206a48a7fc..f67cf820856e4 100644 --- a/store/mockstore/mocktikv/executor.go +++ b/store/mockstore/mocktikv/executor.go @@ -358,6 +358,7 @@ func (e *indexScanExec) Next(ctx context.Context) (value [][]byte, err error) { e.counts[e.cursor]++ } } + // log.Errorf("xxxxxxxxxxxx %v", value) return value, nil } diff --git a/types/datum.go b/types/datum.go index 92dd3dc40ba86..daca7016731a8 100644 --- a/types/datum.go +++ b/types/datum.go @@ -249,6 +249,9 @@ func (d *Datum) SetMysqlBit(b BinaryLiteral) { // GetMysqlDecimal gets Decimal value func (d *Datum) GetMysqlDecimal() *MyDecimal { + if d.x == nil { + return nil + } return d.x.(*MyDecimal) } diff --git a/util/admin/admin_integration_test.go b/util/admin/admin_integration_test.go index c6a226e01f4da..0123175850b21 100644 --- a/util/admin/admin_integration_test.go +++ b/util/admin/admin_integration_test.go @@ -58,8 +58,8 @@ func (s *testAdminSuite) TearDownSuite(c *C) { func (s *testAdminSuite) TestAdminCheckTable(c *C) { // test NULL value. tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - // test index column has pk-handle column + tk.MustExec("use test") // + // test index column has pk-handle column// tk.MustExec("drop table if exists t") tk.MustExec("create table t(a bigint unsigned primary key, b int, c int, index idx(a, b));") tk.MustExec("insert into t values(1, 1, 1)") From 10cb5b04c0ade8ccf71d80173623e8cf7eecefbb Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 15 Jul 2019 17:15:26 +0800 Subject: [PATCH 16/23] executor, planner: remove logs --- executor/builder.go | 20 +------ executor/distsql.go | 53 ------------------ executor/executor.go | 12 ++-- planner/core/common_plans.go | 2 - planner/core/planbuilder.go | 106 ++++++++--------------------------- 5 files changed, 32 insertions(+), 161 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 01d45f3920be7..dc4c75a44abcc 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -25,7 +25,6 @@ import ( "github.com/cznic/mathutil" "github.com/cznic/sortutil" - "github.com/ngaut/log" "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" @@ -306,8 +305,7 @@ func (b *executorBuilder) buildCheckIndex(v *plannercore.CheckIndex) Executor { return nil } - genExprs := make(map[model.TableColumnID]expression.Expression) - buildIndexLookUpChecker(b, v.IndexLookUpReader, readerExec, genExprs) + buildIndexLookUpChecker(b, v.IndexLookUpReader, readerExec) e := &CheckIndexExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), @@ -321,7 +319,7 @@ func (b *executorBuilder) buildCheckIndex(v *plannercore.CheckIndex) Executor { } func buildIndexLookUpChecker(b *executorBuilder, readerPlan *plannercore.PhysicalIndexLookUpReader, - readerExec *IndexLookUpExecutor, genExprs map[model.TableColumnID]expression.Expression) { + readerExec *IndexLookUpExecutor) { readerExec.keepOrder = true is := readerPlan.IndexPlans[0].(*plannercore.PhysicalIndexScan) readerExec.dagPB.OutputOffsets = make([]uint32, 0, len(is.Index.Columns)) @@ -333,13 +331,8 @@ func buildIndexLookUpChecker(b *executorBuilder, readerPlan *plannercore.Physica for _, col := range is.Columns { tps = append(tps, &col.FieldType) } - // tblTps := make([]*types.FieldType, 0, len(readerExec.columns)+1) - // for _, col := range readerExec.columns { - // tblTps = append(tblTps, &col.FieldType) - // } tps = append(tps, types.NewFieldType(mysql.TypeLonglong)) readerExec.tps = tps - // readerExec.tblTps = tblTps readerExec.tbl = readerExec.table readerExec.idxInfo = readerExec.index @@ -354,7 +347,6 @@ func buildIndexLookUpChecker(b *executorBuilder, readerPlan *plannercore.Physica b.err = errors.Trace(err) return } - // readerExec.genExprs = genExprs readerExec.isCheckOp = true readerExec.ranges = ranger.FullRange() } @@ -367,7 +359,7 @@ func (b *executorBuilder) buildCheckTable(v *plannercore.CheckTable) Executor { b.err = errors.Trace(err) return nil } - buildIndexLookUpChecker(b, readerPlan, readerExec, v.GenExprs) + buildIndexLookUpChecker(b, readerPlan, readerExec) readerExecs = append(readerExecs, readerExec) } @@ -381,7 +373,6 @@ func (b *executorBuilder) buildCheckTable(v *plannercore.CheckTable) Executor { srcs: readerExecs, exitCh: make(chan struct{}), retCh: make(chan error, len(v.Indices)), - genExprs: v.GenExprs, } return e } @@ -1962,15 +1953,11 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn } is := v.IndexPlans[0].(*plannercore.PhysicalIndexScan) indexReq.OutputOffsets = []uint32{uint32(len(is.Index.Columns))} - log.Warnf(".......................... dag req:%v, output offsets %v, cols %v, exprs %d", - indexReq.Executors[0].IdxScan, indexReq.OutputOffsets, is.Index.Columns, len(is.GenExprs)) tbl, _ := b.is.TableByID(is.Table.ID) for i := 0; i < v.Schema().Len(); i++ { tableReq.OutputOffsets = append(tableReq.OutputOffsets, uint32(i)) } - log.Warnf(".......................... dag req:%v, output offsets %v, cols %v, len %v", - tableReq.Executors[0].TblScan, tableReq.OutputOffsets, v.Schema(), len(tableReq.Executors[0].TblScan.Columns)) ts := v.TablePlans[0].(*plannercore.PhysicalTableScan) if isPartition, physicalTableID := ts.IsPartition(); isPartition { @@ -1997,7 +1984,6 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn idxPlans: v.IndexPlans, tblPlans: v.TablePlans, } - e.genExprs = is.GenExprs if containsLimit(indexReq.Executors) { e.feedback = statistics.NewQueryFeedback(0, nil, 0, is.Desc) diff --git a/executor/distsql.go b/executor/distsql.go index eb13b03106e70..e80648ac6ac83 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -24,7 +24,6 @@ import ( "time" "unsafe" - "github.com/ngaut/log" "github.com/pingcap/errors" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -475,8 +474,6 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, kvRanges []k tps = e.tps } // Since the first read only need handle information. So its returned col is only 1. - log.Warnf(".......................... dag req:%v, len %v", e.dagPB.Executors[0].IdxScan, len(tps)) - // result, err := distsql.SelectWithRuntimeStats(ctx, e.ctx, kvReq, []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)}, e.feedback, getPhysicalPlanIDs(e.idxPlans)) result, err := distsql.SelectWithRuntimeStats(ctx, e.ctx, kvReq, tps, e.feedback, getPhysicalPlanIDs(e.idxPlans)) if err != nil { return err @@ -715,8 +712,6 @@ func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, for len(handles) < w.batchSize { chk.SetRequiredRows(w.batchSize-len(handles), w.maxChunkSize) err = errors.Trace(idxResult.Next(ctx, chk)) - log.Warnf("===================================================== extract task handles, batch size: %v, rows: %v, keep order %v, ret chk %v", - w.batchSize, chk.NumRows(), w.keepOrder, retChk) if err != nil { return handles, nil, err } @@ -726,19 +721,6 @@ func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, for i := 0; i < chk.NumRows(); i++ { h := chk.GetRow(i).GetInt64(chk.NumCols() - 1) handles = append(handles, h) - - // if w.isCheckOp { - // if retChk == nil { - // retChk = chunk.NewChunkWithCapacity(w.tps, w.maxChunkSize) - // } - // // retChk.Append(chk, 0, chk.NumRows())// (chunk.MutRowFromDatums(chk.GetRow(i).GetDatumRow(w.tps)).ToRow()) - // } - if len(w.tps) > 1 { - colVal0 := chk.GetRow(i).GetDatum(0, w.tps[0]) - colVal1 := chk.GetRow(i).GetDatum(1, w.tps[1]) - log.Infof("xxx ..... cols len %v, tps len %v, datum1: %v, datum2: %v, handle: %v, col %v, col %#v", - chk.NumCols(), len(w.tps), colVal0, colVal1, h, colVal0, colVal0) - } } if w.isCheckOp { if retChk == nil { @@ -759,12 +741,9 @@ func (w *indexWorker) buildTableTask(handles []int64, retChk *chunk.Chunk) *look if w.keepOrder { // Save the index order. indexOrder = make(map[int64]int, len(handles)) - var str string for i, h := range handles { indexOrder[h] = i - str += fmt.Sprintf(" h %d, offset %d;", h, i) } - log.Infof("......... %s, ret chk %v", str, retChk) } task := &lookupTableTask{ handles: handles, @@ -801,7 +780,6 @@ func (w *tableWorker) pickAndExecTask(ctx context.Context) { buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] - log.Errorf("tableWorker panic stack is:\n%s", buf) logutil.Logger(ctx).Error("tableWorker in IndexLookUpExecutor panicked", zap.String("stack", string(buf))) task.doneCh <- errors.Errorf("%v", r) } @@ -843,8 +821,6 @@ func adjustDatumKind(vals1, vals2 []types.Datum) { func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, tableReader Executor) error { for { chk := newFirstChunk(tableReader) - // err := tableReader.Next(ctx, chk) - //err := tableReader.Next(ctx,chunk.NewRecordBatch(chk)) err := tableReader.Next(ctx, chk) if err != nil { return errors.Trace(err) @@ -869,8 +845,6 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta for j, col := range w.cols { if col.IsGenerated() && !col.GeneratedStored { expr := w.genExprs[model.TableColumnID{TableID: w.tbl.Meta().ID, ColumnID: col.ID}] - log.Infof("xxx ------------------------------------------- idx %v, j %v, row len %v, cols len %v, col %v, rows %v, handles %v, expr %v", - w.checkIndexValue.idxInfo.Name, j, row.Len(), len(w.cols), col, chk.NumRows(), len(task.handles), expr) // Eval the column value val, err := expr.Eval(row) if err != nil { @@ -880,36 +854,16 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta if err != nil { return errors.Trace(err) } - - // if val.Kind() == types.KindMysqlTime && sysLoc != time.UTC { - // t := val.GetMysqlTime() - // if t.Type == mysql.TypeTimestamp { - // err := t.ConvertTimeZone(sysLoc, time.UTC) - // if err != nil { - // return errors.Trace(err) - // } - // val.SetMysqlTime(t) - // } - // } vals = append(vals, val) - log.Infof("xxx ------------------------------------------- col %v, val %v", col, val) } else { vals = append(vals, row.GetDatum(j, &col.FieldType)) - log.Infof("xxx ------------------------------------------- j %v, row len %v, cols len %v, col %v, rows %v, handles %v, val %v", - j, row.Len(), len(w.cols), col, chk.NumRows(), len(task.handles), vals[j]) } } vals = tables.TruncateIndexValuesIfNeeded(w.tbl.Meta(), w.idxInfo, vals) for j, val := range vals { col := w.cols[j] tp := &col.FieldType - log.Infof("xxx ------------------------------------------- i:%v, j:%v, idx cols %v, tbl cols %d, col %v, idx %v, tbl %v, tp %v", - i, j, idxRow.Len(), len(tableReader.Schema().Columns), col, idxRow.GetDatum(j, tp), val, tp) ret := chunk.Compare(idxRow, j, &val) - - log.Warnf("xxx no.%v, order %v, j %v, row handle: %v, idx handle: %v, name:%v, tbl d:%v, idx d:%v", - i, offset, j, idxRow.GetInt64(idxRow.Len()-1), handle, - tableReader.Schema().Columns[j].ColName, val, idxRow.GetDatum(j, tp)) if ret != 0 { return errors.Errorf("handle %#v, index:%#v != record:%#v", handle, idxRow.GetDatum(j, tp), val) } @@ -932,13 +886,6 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er defer terror.Call(tableReader.Close) if w.isCheckOp { - // str := "" - // for i, row := range task.idxRows { - // str += fmt.Sprintf("no.%d row %v ", i, row.GetDatumRow(w.tps)) - // } - // log.Errorf("======= execute task. rows %v\n handles: %v", str, task.handles) - tblReaderExec := tableReader.(*TableReaderExecutor) - log.Errorf("======= execute task. handles: %v, order %v, tps %v", task.handles, tblReaderExec.keepOrder, w.tps) return w.compareData(ctx, task, tableReader) } diff --git a/executor/executor.go b/executor/executor.go index 71809afd65a06..26e127c52e962 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -465,8 +465,6 @@ type CheckTableExec struct { is infoschema.InfoSchema exitCh chan struct{} retCh chan error - - genExprs map[model.TableColumnID]expression.Expression } // Open implements the Executor Open interface. @@ -544,7 +542,7 @@ func (e *CheckTableExec) Next(ctx context.Context, req *chunk.Chunk) error { if greater == admin.IdxCntGreater { err = e.checkIndexHandle(ctx, idxOffset, e.srcs[idxOffset]) } else if greater == admin.TblCntGreater { - err = e.checkTableRecord(tbl, e.indices[idxOffset]) + err = e.checkTableRecord(tbl, idxOffset) } if err != nil && admin.ErrDataInConsistent.Equal(err) { return ErrAdminCheckTable.GenWithStack("%v err:%v", tbl.Meta().Name, err) @@ -568,20 +566,22 @@ func (e *CheckTableExec) Next(ctx context.Context, req *chunk.Chunk) error { return nil } -func (e *CheckTableExec) checkTableRecord(tbl table.Table, idx table.Index) error { +func (e *CheckTableExec) checkTableRecord(tbl table.Table, idxOffset int) error { + idx := e.indices[idxOffset] + genExprs := e.srcs[idxOffset].genExprs txn, err := e.ctx.Txn(true) if err != nil { return err } if tbl.Meta().GetPartitionInfo() == nil { - return admin.CheckRecordAndIndex(e.ctx, txn, tbl, idx, e.genExprs) + return admin.CheckRecordAndIndex(e.ctx, txn, tbl, idx, genExprs) } info := tbl.Meta().GetPartitionInfo() for _, def := range info.Definitions { pid := def.ID partition := tbl.(table.PartitionedTable).GetPartition(pid) - if err := admin.CheckRecordAndIndex(e.ctx, txn, partition, idx, e.genExprs); err != nil { + if err := admin.CheckRecordAndIndex(e.ctx, txn, partition, idx, genExprs); err != nil { return errors.Trace(err) } } diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index fe68691f72fc4..d1f2cea3b16ba 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -78,8 +78,6 @@ type CheckTable struct { TblInfo *model.TableInfo Indices []table.Index IndexLookUpReaders []*PhysicalIndexLookUpReader - - GenExprs map[model.TableColumnID]expression.Expression } // RecoverIndex is used for backfilling corrupted index data. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index a0961b3252a4a..b502f424c1614 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -15,11 +15,12 @@ package core import ( "bytes" + "context" "fmt" + "go.uber.org/zap" "strings" "github.com/cznic/mathutil" - "github.com/ngaut/log" "github.com/pingcap/errors" "github.com/pingcap/parser" "github.com/pingcap/parser/ast" @@ -38,6 +39,7 @@ import ( "github.com/pingcap/tidb/types" driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" ) @@ -569,8 +571,8 @@ func (b *PlanBuilder) buildCheckIndex(dbName model.CIStr, as *ast.AdminStmt) (Pl } // TODO: Handle generated column. - genExprs := make(map[model.TableColumnID]expression.Expression) - return b.buildPhysicalIndexLookUpReader(dbName, tbl, idx, 1, genExprs) + // genExprs := make(map[model.TableColumnID]expression.Expression) + return b.buildPhysicalIndexLookUpReader(dbName, tbl, idx, 1) // id := 1 // columns := make([]*model.ColumnInfo, 0, len(idx.Columns)) // schema := expression.NewSchema(make([]*expression.Column, 0, len(idx.Columns))...) @@ -686,35 +688,13 @@ func (b *PlanBuilder) buildAdmin(as *ast.AdminStmt) (Plan, error) { return ret, nil } -func getGenColumns(expr expression.Expression) []*expression.Column { - col, ok := expr.(*expression.Column) - if ok { - // return []*expression.Column{col.Clone().(*expression.Column)} - log.Infof("get gen col %#v", col) - return []*expression.Column{col} - } - - scalaFunc, isScalaFunc := expr.(*expression.ScalarFunction) - if !isScalaFunc { - return nil - } - cols := make([]*expression.Column, 0, len(scalaFunc.GetArgs())) - for _, arg := range scalaFunc.GetArgs() { - retCols := getGenColumns(arg) - if retCols != nil { - cols = append(cols, retCols...) - } - } - return cols -} - -func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl table.Table, idx *model.IndexInfo, id int, - genExprs map[model.TableColumnID]expression.Expression) (Plan, error) { +func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl table.Table, idx *model.IndexInfo, id int) (Plan, error) { tblInfo := tbl.Meta() columns := make([]*model.ColumnInfo, 0, len(idx.Columns)) tblColumns := make([]*model.ColumnInfo, 0, len(tbl.Cols())) - exprs := make(map[model.TableColumnID]expression.Expression) + // Get generated expressions. + genExprs := make(map[model.TableColumnID]expression.Expression) mockTablePlan := LogicalTableDual{}.Init(b.ctx) mockTablePlan.SetSchema(expression.TableInfo2SchemaWithDBName(b.ctx, dbName, tblInfo)) for _, column := range idx.Columns { @@ -722,7 +702,6 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab if !col.IsGenerated() { continue } - log.Infof("222 gen col %v", col) columnName := &ast.ColumnName{Name: column.Name} columnName.SetText(column.Name.O) @@ -737,11 +716,10 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab } expr = expression.BuildCastFunction(b.ctx, expr, colExpr.GetType()) genColumnID := model.TableColumnID{TableID: tblInfo.ID, ColumnID: col.ColumnInfo.ID} - exprs[genColumnID] = expr + genExprs[genColumnID] = expr } - str := "" - str2 := "" + // Get generated columns. var genCols []*expression.Column colsMap := make(map[int64]struct{}) schema := expression.NewSchema(make([]*expression.Column, 0, len(idx.Columns))...) @@ -756,20 +734,14 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab RetType: &col.FieldType, }) colsMap[col.ID] = struct{}{} - str += fmt.Sprintf("col %v, field tp %v ", col.Name, col.FieldType) - str2 += fmt.Sprintf("col %v, field tp %v ", col.Name, col.FieldType) } genColumnID := model.TableColumnID{TableID: tblInfo.ID, ColumnID: col.ID} - // if expr, ok := genExprs[genColumnID]; ok { - if expr, ok := exprs[genColumnID]; ok { - cols := getGenColumns(expr) - if cols != nil { - genCols = append(genCols, cols...) - } + if expr, ok := genExprs[genColumnID]; ok { + cols := expression.ExtractColumns(expr) + genCols = append(genCols, cols...) } } } - tblSchema := schema.Clone() for _, col := range genCols { if _, ok := colsMap[col.ID]; !ok { @@ -782,7 +754,6 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(), RetType: &c.FieldType, }) - str2 += fmt.Sprintf("no.%d, col %v, field tp %v; ", col.Index, c.Name, c.FieldType) colsMap[c.ID] = struct{}{} } } @@ -796,9 +767,8 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(), ID: model.ExtraHandleID, } - str2 += fmt.Sprintf("col %v", tblColumns[len(tblColumns)-1].Name) tblSchema.Append(handleCol) - log.Warnf("********************************* table %v, idx %v, columns %#v, tbl columns %#v, len %v, exprs %d", tblInfo.Name, idx.Name, str, str2, len(tblColumns), len(exprs)) + is := PhysicalIndexScan{ Table: tblInfo, TableAsName: &tblInfo.Name, @@ -808,13 +778,11 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab dataSourceSchema: schema, Ranges: ranger.FullRange(), KeepOrder: false, - GenExprs: exprs, + GenExprs: genExprs, }.Init(b.ctx) is.stats = property.NewSimpleStats(0) cop := &copTask{indexPlan: is} // It's double read case. - // ts := PhysicalTableScan{Columns: columns, Table: is.Table}.Init(b.ctx) - // ts.SetSchema(is.dataSourceSchema) ts := PhysicalTableScan{Columns: tblColumns, Table: is.Table}.Init(b.ctx) ts.SetSchema(tblSchema) cop.tablePlan = ts @@ -824,8 +792,7 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab return rootT.p, nil } -func (b *PlanBuilder) buildPhysicalIndexLookUpReaders(dbName model.CIStr, tbl table.Table, - genExprs map[model.TableColumnID]expression.Expression) ([]Plan, []table.Index, error) { +func (b *PlanBuilder) buildPhysicalIndexLookUpReaders(dbName model.CIStr, tbl table.Table) ([]Plan, []table.Index, error) { tblInfo := tbl.Meta() // get index information indices := make([]table.Index, 0, len(tblInfo.Indices)) @@ -833,17 +800,15 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReaders(dbName model.CIStr, tbl ta for i, idx := range tbl.Indices() { idxInfo := idx.Meta() if idxInfo.State != model.StatePublic { - log.Warnf("index %s state %s isn't public in table %s", idxInfo.Name, idxInfo.State, tblInfo.Name) + logutil.Logger(context.Background()).Info("build physical index lookup reader, the index isn't public", + zap.String("index", idxInfo.Name.O), zap.Stringer("state", idxInfo.State), zap.String("table", tblInfo.Name.O)) } else { indices = append(indices, idx) - reader, err := b.buildPhysicalIndexLookUpReader(dbName, tbl, idxInfo, i, genExprs) + reader, err := b.buildPhysicalIndexLookUpReader(dbName, tbl, idxInfo, i) if err != nil { return nil, nil, err } indexLookUpReaders = append(indexLookUpReaders, reader) - for i, expr := range genExprs { - log.Infof("******** no.%d, expr %v", i, expr.ExplainInfo()) - } } } if len(indexLookUpReaders) == 0 { @@ -855,45 +820,20 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReaders(dbName model.CIStr, tbl ta func (b *PlanBuilder) buildAdminCheckTable(as *ast.AdminStmt) (*CheckTable, error) { tbl := as.Tables[0] p := &CheckTable{ - DBName: tbl.Schema.O, - TblInfo: tbl.TableInfo, - GenExprs: make(map[model.TableColumnID]expression.Expression), + DBName: tbl.Schema.O, + TblInfo: tbl.TableInfo, } mockTablePlan := LogicalTableDual{}.Init(b.ctx) tableInfo := as.Tables[0].TableInfo schema := expression.TableInfo2SchemaWithDBName(b.ctx, tbl.Schema, tableInfo) + mockTablePlan.SetSchema(schema) table, ok := b.is.TableByID(tableInfo.ID) if !ok { return nil, infoschema.ErrTableNotExists.GenWithStackByArgs(tbl.DBInfo.Name.O, tableInfo.Name.O) } - mockTablePlan.SetSchema(schema) - - // Calculate generated columns. - columns := table.Cols() - for _, column := range columns { - if !column.IsGenerated() { - continue - } - columnName := &ast.ColumnName{Name: column.Name} - columnName.SetText(column.Name.O) - - colExpr, _, err := mockTablePlan.findColumn(columnName) - if err != nil { - return nil, errors.Trace(err) - } - - expr, _, err := b.rewrite(column.GeneratedExpr, mockTablePlan, nil, true) - if err != nil { - return nil, errors.Trace(err) - } - expr = expression.BuildCastFunction(b.ctx, expr, colExpr.GetType()) - genColumnID := model.TableColumnID{TableID: tableInfo.ID, ColumnID: column.ColumnInfo.ID} - p.GenExprs[genColumnID] = expr - } - - readerPlans, indices, err := b.buildPhysicalIndexLookUpReaders(tbl.Schema, table, p.GenExprs) + readerPlans, indices, err := b.buildPhysicalIndexLookUpReaders(tbl.Schema, table) if err != nil { return nil, errors.Trace(err) } From dd72622baf9e6466a4968bee03d442517fe2d658 Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 15 Jul 2019 18:30:01 +0800 Subject: [PATCH 17/23] *: remove logs --- executor/admin_test.go | 24 ++++--------- executor/ddl.go | 1 - executor/distsql.go | 13 ++++--- executor/executor.go | 10 +++--- expression/column.go | 2 -- expression/scalar_function.go | 1 - planner/core/plan_to_pb.go | 7 ---- planner/core/planbuilder.go | 38 --------------------- server/http_handler.go | 2 -- store/mockstore/mocktikv/cop_handler_dag.go | 5 --- store/mockstore/mocktikv/executor.go | 1 - types/datum.go | 3 -- util/admin/admin_integration_test.go | 4 +-- util/chunk/compare.go | 2 -- util/codec/codec.go | 2 -- 15 files changed, 18 insertions(+), 97 deletions(-) diff --git a/executor/admin_test.go b/executor/admin_test.go index d441eee71f07c..7c61da3e2d64a 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -18,7 +18,6 @@ import ( "fmt" "time" - "github.com/ngaut/log" . "github.com/pingcap/check" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/executor" @@ -405,10 +404,7 @@ func (s *testSuite2) TestAdminCheckTableFailed(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") tk.MustExec("create table admin_test (c1 int, c2 int, c3 varchar(255) default '1', primary key(c1), key(c3), unique key(c2), key(c2, c3))") - // tk.MustExec("create table admin_test (c1 int, c2 int, c3 varchar(255) default '1', primary key(c1), key(c3))") - // tk.MustExec("create table admin_test (c1 int, c2 int, c3 varchar(255) default '1', primary key(c1), unique key(c2))") - // tk.MustExec("insert admin_test (c1, c2, c3) values (1, 11, 'a'), (2, 12, 'b'), (5, 15, 'c'), (10, 20, 'd'), (20, 30, 'e')") - tk.MustExec("insert admin_test (c1, c2, c3) values (1, 11, '1'), (2, 12, '2'), (5, 15, '3'), (10, 20, '4'), (20, 30, '5')") + tk.MustExec("insert admin_test (c1, c2, c3) values (1, 11, 'a'), (2, 12, 'b'), (5, 15, 'c'), (10, 20, 'd'), (20, 30, 'e')") // Make some corrupted index. Build the index information. s.ctx = mock.NewContext() @@ -419,8 +415,6 @@ func (s *testSuite2) TestAdminCheckTableFailed(c *C) { tbl, err := is.TableByName(dbName, tblName) c.Assert(err, IsNil) tblInfo := tbl.Meta() - // idxInfo := findIndexByName("c3", tblInfo.Indices) - // idxInfo := findIndexByName("c2", tblInfo.Indices) idxInfo := tblInfo.Indices[1] indexOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) sc := s.ctx.GetSessionVars().StmtCtx @@ -436,16 +430,13 @@ func (s *testSuite2) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) - log.Errorf("...................................................................................... 00") _, err = tk.Exec("admin check table admin_test") c.Assert(err.Error(), Equals, "[executor:8003]admin_test err:[admin:1]index: != record:&admin.RecordData{Handle:1, Values:[]types.Datum{types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:11, b:[]uint8(nil), x:interface {}(nil)}}}") c.Assert(executor.ErrAdminCheckTable.Equal(err), IsTrue) r := tk.MustQuery("admin recover index admin_test c2") r.Check(testkit.Rows("1 5")) - log.Errorf("...................................................................................... 01") tk.MustExec("admin check table admin_test") - log.Errorf("...................................................................................... 1") // Add one row of index. // Table count < index count. @@ -454,26 +445,24 @@ func (s *testSuite2) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(21), 1) c.Assert(err, IsNil) - // _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(13), 2) - // c.Assert(err, IsNil) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(13), 2) + c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) _, err = tk.Exec("admin check table admin_test") - c.Assert(err.Error(), Equals, "handle 1, index:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:21, b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:11, b:[]uint8(nil), x:interface {}(nil)}") - log.Errorf("...................................................................................... 2") + c.Assert(err.Error(), Equals, "handle 2, index:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:13, b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:12, b:[]uint8(nil), x:interface {}(nil)}") // Table count = index count. // Two indices have the same handle. txn, err = s.store.Begin() c.Assert(err, IsNil) - // err = indexOpr.Delete(sc, txn, types.MakeDatums(13), 2, nil) - // c.Assert(err, IsNil) + err = indexOpr.Delete(sc, txn, types.MakeDatums(13), 2, nil) + c.Assert(err, IsNil) err = indexOpr.Delete(sc, txn, types.MakeDatums(12), 2, nil) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) _, err = tk.Exec("admin check table admin_test") - // c.Assert(err.Error(), Equals, "admin check table admin_test, index c2, at least two indices have the same handle 1 more than one") c.Assert(err.Error(), Equals, "handle 1, index:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:21, b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:11, b:[]uint8(nil), x:interface {}(nil)}") // Table count = index count. @@ -487,7 +476,6 @@ func (s *testSuite2) TestAdminCheckTableFailed(c *C) { err = txn.Commit(context.Background()) c.Assert(err, IsNil) _, err = tk.Exec("admin check table admin_test") - // c.Assert(err.Error(), Equals, "admin check table admin_test, index c2, at least two indices have the same handle 1 more than one") c.Assert(err.Error(), Equals, "handle 1, index:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:21, b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:11, b:[]uint8(nil), x:interface {}(nil)}") // Recover records. diff --git a/executor/ddl.go b/executor/ddl.go index 0d8a421a6149d..39fddecb6aa84 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -267,7 +267,6 @@ func (e *DDLExec) executeDropTableOrView(s *ast.DropTableStmt) error { } if config.CheckTableBeforeDrop { - logutil.BgLogger().Info("xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx") logutil.BgLogger().Warn("admin check table before drop", zap.String("database", fullti.Schema.O), zap.String("table", fullti.Name.O), diff --git a/executor/distsql.go b/executor/distsql.go index e80648ac6ac83..e7c6e54063a59 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -480,13 +480,12 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, kvRanges []k } result.Fetch(ctx) worker := &indexWorker{ - idxLookup: e, - workCh: workCh, - finished: e.finished, - resultCh: e.resultCh, - keepOrder: e.keepOrder, - batchSize: initBatchSize, - // batchSize: e.maxChunkSize, + idxLookup: e, + workCh: workCh, + finished: e.finished, + resultCh: e.resultCh, + keepOrder: e.keepOrder, + batchSize: initBatchSize, checkIndexValue: e.checkIndexValue, maxBatchSize: e.ctx.GetSessionVars().IndexLookupSize, maxChunkSize: e.maxChunkSize, diff --git a/executor/executor.go b/executor/executor.go index 26e127c52e962..df8c1526fd791 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -23,7 +23,6 @@ import ( "time" "github.com/cznic/mathutil" - "github.com/ngaut/log" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/parser/ast" @@ -157,7 +156,6 @@ func newBaseExecutor(ctx sessionctx.Context, schema *expression.Schema, id fmt.S cols := schema.Columns e.retFieldTypes = make([]*types.FieldType, len(cols)) for i := range cols { - log.Warnf("------- new executor, id %s, col %v, no. %d", id, cols[i], cols[i].Index) e.retFieldTypes[i] = cols[i].RetType } } @@ -493,9 +491,7 @@ func (e *CheckTableExec) checkIndexHandle(ctx context.Context, num int, src *Ind var err error handles := make(map[int64]struct{}, 1024) for { - log.Infof("====================== check no.%d idx handle. 0", num) err = src.Next(ctx, chk) - log.Infof("====================== check no.%d idx handle. 1", num) if err != nil { break } @@ -537,7 +533,6 @@ func (e *CheckTableExec) Next(ctx context.Context, req *chunk.Chunk) error { } greater, idxOffset, err := admin.CheckIndicesCount(e.ctx, e.dbName, e.tblInfo.Name.O, idxNames) if err != nil { - log.Warnf("check table %v, greater %v index %s err: %v", e.tblInfo.Name, greater, idxNames[idxOffset], err) tbl := e.srcs[idxOffset].table if greater == admin.IdxCntGreater { err = e.checkIndexHandle(ctx, idxOffset, e.srcs[idxOffset]) @@ -553,7 +548,10 @@ func (e *CheckTableExec) Next(ctx context.Context, req *chunk.Chunk) error { // The number of table rows is equal to the number of index rows. for i := range e.srcs { go func(num int) { - e.checkIndexHandle(ctx, num, e.srcs[num]) + err1 := e.checkIndexHandle(ctx, num, e.srcs[num]) + if err1 != nil { + logutil.Logger(ctx).Info("check index handle failed", zap.Error(err)) + } }(i) } diff --git a/expression/column.go b/expression/column.go index 6a46fdf62fc66..99b94e0ea4414 100644 --- a/expression/column.go +++ b/expression/column.go @@ -17,7 +17,6 @@ import ( "fmt" "strings" - "github.com/ngaut/log" "github.com/pingcap/errors" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -286,7 +285,6 @@ func (col *Column) EvalDuration(ctx sessionctx.Context, row chunk.Row) (types.Du // EvalJSON returns JSON representation of Column. func (col *Column) EvalJSON(ctx sessionctx.Context, row chunk.Row) (json.BinaryJSON, bool, error) { - log.Infof("........... idx %d, col %v", col.Index, col.ColName) if row.IsNull(col.Index) { return json.BinaryJSON{}, true, nil } diff --git a/expression/scalar_function.go b/expression/scalar_function.go index 4457967eab7c3..77ce863a39406 100644 --- a/expression/scalar_function.go +++ b/expression/scalar_function.go @@ -207,7 +207,6 @@ func (sf *ScalarFunction) Eval(row chunk.Row) (d types.Datum, err error) { case types.ETDecimal: res, isNull, err = sf.EvalDecimal(sf.GetCtx(), row) case types.ETDatetime, types.ETTimestamp: - // log.Infof("............. expr %v, row idx %d, row len %v", sf, row.Idx(), row.Len()) res, isNull, err = sf.EvalTime(sf.GetCtx(), row) case types.ETDuration: res, isNull, err = sf.EvalDuration(sf.GetCtx(), row) diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 2fe89f6b07130..4694f7bcc5c14 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -14,9 +14,6 @@ package core import ( - "fmt" - - "github.com/ngaut/log" "github.com/pingcap/errors" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" @@ -98,7 +95,6 @@ func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) Columns: model.ColumnsToProto(columns, p.Table.PKIsHandle), Desc: p.Desc, } - log.Warnf("tbl scan ------- to pb, len %v", len(columns)) err := SetPBColumnsDefaultValue(ctx, tsExec.Columns, p.Columns) return &tipb.Executor{Tp: tipb.ExecType_TypeTableScan, TblScan: tsExec}, err } @@ -122,16 +118,13 @@ func checkCoverIndex(idx *model.IndexInfo, ranges []*ranger.Range) bool { func (p *PhysicalIndexScan) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { columns := make([]*model.ColumnInfo, 0, p.schema.Len()) tableColumns := p.Table.Cols() - str := "" for _, col := range p.schema.Columns { if col.ID == model.ExtraHandleID { columns = append(columns, model.NewExtraHandleColInfo()) } else { columns = append(columns, model.FindColumnInfo(tableColumns, col.ColName.L)) } - str += fmt.Sprintf("id %d, col %v; ", col.ID, col) } - log.Warnf("idx scan ------- to pb, %v", str) idxExec := &tipb.IndexScan{ TableId: p.Table.ID, IndexId: p.Index.ID, diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index b502f424c1614..02d5123cd7985 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -570,45 +570,7 @@ func (b *PlanBuilder) buildCheckIndex(dbName model.CIStr, as *ast.AdminStmt) (Pl return nil, errors.Errorf("index %s state %s isn't public", as.Index, idx.State) } - // TODO: Handle generated column. - // genExprs := make(map[model.TableColumnID]expression.Expression) return b.buildPhysicalIndexLookUpReader(dbName, tbl, idx, 1) - // id := 1 - // columns := make([]*model.ColumnInfo, 0, len(idx.Columns)) - // schema := expression.NewSchema(make([]*expression.Column, 0, len(idx.Columns))...) - // for _, idxCol := range idx.Columns { - // for _, col := range tblInfo.Columns { - // if idxCol.Name.L == col.Name.L { - // columns = append(columns, col) - // schema.Append(&expression.Column{ - // ColName: col.Name, - // UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(), - // RetType: &col.FieldType, - // }) - // } - // } - // } - // is := PhysicalIndexScan{ - // Table: tblInfo, - // TableAsName: &tblName.Name, - // DBName: dbName, - // Columns: columns, - // Index: idx, - // dataSourceSchema: schema, - // Ranges: ranger.FullRange(), - // KeepOrder: false, - // }.Init(b.ctx) - // is.stats = property.NewSimpleStats(0) - // cop := &copTask{indexPlan: is} - // // It's double read case. - // ts := PhysicalTableScan{Columns: columns, Table: is.Table}.Init(b.ctx) - // ts.SetSchema(is.dataSourceSchema) - // cop.tablePlan = ts - // is.initSchema(id, idx, true) - // t := finishCopTask(b.ctx, cop) - // - // rootT := t.(*rootTask) - // return rootT.p, nil } func (b *PlanBuilder) buildAdmin(as *ast.AdminStmt) (Plan, error) { diff --git a/server/http_handler.go b/server/http_handler.go index 00b1090415126..a925c5a653c1d 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -582,11 +582,9 @@ func (h settingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { writeError(w, err1) return } - log.Warnf("-------------- time %v", atomic.LoadUint32(&variable.DDLSlowOprThreshold)) if threshold > 0 { atomic.StoreUint32(&variable.DDLSlowOprThreshold, uint32(threshold)) } - log.Warnf("-------------- time %v", atomic.LoadUint32(&variable.DDLSlowOprThreshold)) } if checkMb4ValueInUtf8 := req.Form.Get("check_mb4_value_in_utf8"); checkMb4ValueInUtf8 != "" { switch checkMb4ValueInUtf8 { diff --git a/store/mockstore/mocktikv/cop_handler_dag.go b/store/mockstore/mocktikv/cop_handler_dag.go index 3f46c7d1c116b..d02e87800d9a8 100644 --- a/store/mockstore/mocktikv/cop_handler_dag.go +++ b/store/mockstore/mocktikv/cop_handler_dag.go @@ -20,7 +20,6 @@ import ( "time" "github.com/golang/protobuf/proto" - "github.com/ngaut/log" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/errorpb" @@ -68,7 +67,6 @@ func (h *rpcHandler) handleCopDAGRequest(req *coprocessor.Request) *coprocessor. rowCnt int ) ctx := context.TODO() - log.Errorf("tp: %v, offsets:%v", dagReq.Executors[0].Tp, dagReq.OutputOffsets) for { var row [][]byte row, err = e.Next(ctx) @@ -192,7 +190,6 @@ func (h *rpcHandler) buildDAG(ctx *dagContext, executors []*tipb.Executor) (exec func (h *rpcHandler) buildTableScan(ctx *dagContext, executor *tipb.Executor) (*tableScanExec, error) { columns := executor.TblScan.Columns - log.Infof("table scan, columns %v, the first col %v", len(columns), columns[0]) ctx.evalCtx.setColumnInfo(columns) ranges, err := h.extractKVRanges(ctx.keyRanges, executor.TblScan.Desc) if err != nil { @@ -217,7 +214,6 @@ func (h *rpcHandler) buildTableScan(ctx *dagContext, executor *tipb.Executor) (* func (h *rpcHandler) buildIndexScan(ctx *dagContext, executor *tipb.Executor) (*indexScanExec, error) { var err error columns := executor.IdxScan.Columns - log.Warnf("index scan 111, columns %v", len(executor.IdxScan.Columns)) ctx.evalCtx.setColumnInfo(columns) length := len(columns) pkStatus := pkColNotExists @@ -233,7 +229,6 @@ func (h *rpcHandler) buildIndexScan(ctx *dagContext, executor *tipb.Executor) (* pkStatus = pkColIsSigned columns = columns[:length-1] } - log.Warnf("index scan 222, columns %v, col %#v, pk status:%v", len(columns), columns[0], pkStatus) ranges, err := h.extractKVRanges(ctx.keyRanges, executor.IdxScan.Desc) if err != nil { return nil, errors.Trace(err) diff --git a/store/mockstore/mocktikv/executor.go b/store/mockstore/mocktikv/executor.go index f67cf820856e4..d12206a48a7fc 100644 --- a/store/mockstore/mocktikv/executor.go +++ b/store/mockstore/mocktikv/executor.go @@ -358,7 +358,6 @@ func (e *indexScanExec) Next(ctx context.Context) (value [][]byte, err error) { e.counts[e.cursor]++ } } - // log.Errorf("xxxxxxxxxxxx %v", value) return value, nil } diff --git a/types/datum.go b/types/datum.go index daca7016731a8..92dd3dc40ba86 100644 --- a/types/datum.go +++ b/types/datum.go @@ -249,9 +249,6 @@ func (d *Datum) SetMysqlBit(b BinaryLiteral) { // GetMysqlDecimal gets Decimal value func (d *Datum) GetMysqlDecimal() *MyDecimal { - if d.x == nil { - return nil - } return d.x.(*MyDecimal) } diff --git a/util/admin/admin_integration_test.go b/util/admin/admin_integration_test.go index 0123175850b21..c6a226e01f4da 100644 --- a/util/admin/admin_integration_test.go +++ b/util/admin/admin_integration_test.go @@ -58,8 +58,8 @@ func (s *testAdminSuite) TearDownSuite(c *C) { func (s *testAdminSuite) TestAdminCheckTable(c *C) { // test NULL value. tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") // - // test index column has pk-handle column// + tk.MustExec("use test") + // test index column has pk-handle column tk.MustExec("drop table if exists t") tk.MustExec("create table t(a bigint unsigned primary key, b int, c int, index idx(a, b));") tk.MustExec("insert into t values(1, 1, 1)") diff --git a/util/chunk/compare.go b/util/chunk/compare.go index 9923e5d05fc39..a8adbfc4174a9 100644 --- a/util/chunk/compare.go +++ b/util/chunk/compare.go @@ -16,7 +16,6 @@ package chunk import ( "sort" - "github.com/ngaut/log" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" @@ -186,7 +185,6 @@ func Compare(row Row, colIdx int, ad *types.Datum) int { case types.KindFloat64: return types.CompareFloat64(row.GetFloat64(colIdx), ad.GetFloat64()) case types.KindString, types.KindBytes, types.KindBinaryLiteral, types.KindMysqlBit: - log.Errorf("compare idx %v, len %v, ad kind %d", colIdx, row.Len(), ad.Kind()) return types.CompareString(row.GetString(colIdx), ad.GetString()) case types.KindMysqlDecimal: l, r := row.GetMyDecimal(colIdx), ad.GetMysqlDecimal() diff --git a/util/codec/codec.go b/util/codec/codec.go index 3ed3c4b201c74..19cbfff6c7977 100644 --- a/util/codec/codec.go +++ b/util/codec/codec.go @@ -17,7 +17,6 @@ import ( "encoding/binary" "time" - "github.com/ngaut/log" "github.com/pingcap/errors" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" @@ -562,7 +561,6 @@ func NewDecoder(chk *chunk.Chunk, timezone *time.Location) *Decoder { // DecodeOne decodes one value to chunk and returns the remained bytes. func (decoder *Decoder) DecodeOne(b []byte, colIdx int, ft *types.FieldType) (remain []byte, err error) { if len(b) < 1 { - log.Errorf("-------- idx %d, fp %v", colIdx, ft) return nil, errors.New("invalid encoded key") } chk := decoder.chk From 697c31a3ab4bdaa2ccf0a40c7a675f219c4eb614 Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 22 Jul 2019 18:43:23 +0800 Subject: [PATCH 18/23] *: update comments and address comments --- executor/admin_test.go | 8 ++++---- executor/distsql.go | 9 +++++---- planner/core/planbuilder.go | 2 +- util/admin/admin.go | 3 ++- 4 files changed, 12 insertions(+), 10 deletions(-) diff --git a/executor/admin_test.go b/executor/admin_test.go index 7c61da3e2d64a..ed063becdb6ef 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -404,7 +404,7 @@ func (s *testSuite2) TestAdminCheckTableFailed(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") tk.MustExec("create table admin_test (c1 int, c2 int, c3 varchar(255) default '1', primary key(c1), key(c3), unique key(c2), key(c2, c3))") - tk.MustExec("insert admin_test (c1, c2, c3) values (1, 11, 'a'), (2, 12, 'b'), (5, 15, 'c'), (10, 20, 'd'), (20, 30, 'e')") + tk.MustExec("insert admin_test (c1, c2, c3) values (-1, -10, 'z'), (1, 11, 'a'), (2, 12, 'b'), (5, 15, 'c'), (10, 20, 'd'), (20, 30, 'e')") // Make some corrupted index. Build the index information. s.ctx = mock.NewContext() @@ -426,16 +426,16 @@ func (s *testSuite2) TestAdminCheckTableFailed(c *C) { // Index c2 is missing 11. txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(11), 1, nil) + err = indexOpr.Delete(sc, txn, types.MakeDatums(-10), -1, nil) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) _, err = tk.Exec("admin check table admin_test") c.Assert(err.Error(), Equals, - "[executor:8003]admin_test err:[admin:1]index: != record:&admin.RecordData{Handle:1, Values:[]types.Datum{types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:11, b:[]uint8(nil), x:interface {}(nil)}}}") + "[executor:8003]admin_test err:[admin:1]index: != record:&admin.RecordData{Handle:-1, Values:[]types.Datum{types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:-10, b:[]uint8(nil), x:interface {}(nil)}}}") c.Assert(executor.ErrAdminCheckTable.Equal(err), IsTrue) r := tk.MustQuery("admin recover index admin_test c2") - r.Check(testkit.Rows("1 5")) + r.Check(testkit.Rows("1 6")) tk.MustExec("admin check table admin_test") // Add one row of index. diff --git a/executor/distsql.go b/executor/distsql.go index e7c6e54063a59..facf89a27ef29 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -366,7 +366,7 @@ type IndexLookUpExecutor struct { // memTracker is used to track the memory usage of this executor. memTracker *memory.Tracker - // isCheckOp is used to determine whether we need to check the consistency of the index data. + // checkIndexValue is used to check the consistency of the index data. checkIndexValue corColInIdxSide bool @@ -650,7 +650,7 @@ type indexWorker struct { maxBatchSize int maxChunkSize int - // isCheckOp is used to determine whether we need to check the consistency of the index data. + // checkIndexValue is used to check the consistency of the index data. checkIndexValue } @@ -707,6 +707,7 @@ func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, if w.isCheckOp { chk = chunk.NewChunkWithCapacity(w.tps, w.maxChunkSize) } + handleOffset := chk.NumCols() - 1 handles = make([]int64, 0, w.batchSize) for len(handles) < w.batchSize { chk.SetRequiredRows(w.batchSize-len(handles), w.maxChunkSize) @@ -718,7 +719,7 @@ func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, return handles, retChk, nil } for i := 0; i < chk.NumRows(); i++ { - h := chk.GetRow(i).GetInt64(chk.NumCols() - 1) + h := chk.GetRow(i).GetInt64(handleOffset) handles = append(handles, h) } if w.isCheckOp { @@ -766,7 +767,7 @@ type tableWorker struct { // memTracker is used to track the memory usage of this executor. memTracker *memory.Tracker - // isCheckOp is used to determine whether we need to check the consistency of the index data. + // checkIndexValue is used to check the consistency of the index data. checkIndexValue } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 02d5123cd7985..72e08951e4fe3 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -17,7 +17,6 @@ import ( "bytes" "context" "fmt" - "go.uber.org/zap" "strings" "github.com/cznic/mathutil" @@ -41,6 +40,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" + "go.uber.org/zap" ) type visitInfo struct { diff --git a/util/admin/admin.go b/util/admin/admin.go index 7d4b326376ed6..70592a2a2b60e 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -16,6 +16,7 @@ package admin import ( "fmt" "io" + "math" "sort" "time" @@ -461,7 +462,7 @@ func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table cols[i] = t.Cols()[col.Offset] } - startKey := t.RecordKey(0) + startKey := t.RecordKey(math.MinInt64) filterFunc := func(h1 int64, vals1 []types.Datum, cols []*table.Column) (bool, error) { for i, val := range vals1 { col := cols[i] From 992527aee11ce6942b1da29f62d85c7f805460cb Mon Sep 17 00:00:00 2001 From: Lynn Date: Wed, 24 Jul 2019 16:26:05 +0800 Subject: [PATCH 19/23] executor: clean up and address comments --- executor/admin_test.go | 39 ++++++++++++++------- executor/builder.go | 1 - executor/distsql.go | 78 +++++++++++++++++++++++------------------- executor/executor.go | 13 ------- 4 files changed, 69 insertions(+), 62 deletions(-) diff --git a/executor/admin_test.go b/executor/admin_test.go index ed063becdb6ef..b5109ebae6bc2 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -404,7 +404,7 @@ func (s *testSuite2) TestAdminCheckTableFailed(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") tk.MustExec("create table admin_test (c1 int, c2 int, c3 varchar(255) default '1', primary key(c1), key(c3), unique key(c2), key(c2, c3))") - tk.MustExec("insert admin_test (c1, c2, c3) values (-1, -10, 'z'), (1, 11, 'a'), (2, 12, 'b'), (5, 15, 'c'), (10, 20, 'd'), (20, 30, 'e')") + tk.MustExec("insert admin_test (c1, c2, c3) values (-10, -20, 'y'), (-1, -10, 'z'), (1, 11, 'a'), (2, 12, 'b'), (5, 15, 'c'), (10, 20, 'd'), (20, 30, 'e')") // Make some corrupted index. Build the index information. s.ctx = mock.NewContext() @@ -435,22 +435,37 @@ func (s *testSuite2) TestAdminCheckTableFailed(c *C) { "[executor:8003]admin_test err:[admin:1]index: != record:&admin.RecordData{Handle:-1, Values:[]types.Datum{types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:-10, b:[]uint8(nil), x:interface {}(nil)}}}") c.Assert(executor.ErrAdminCheckTable.Equal(err), IsTrue) r := tk.MustQuery("admin recover index admin_test c2") - r.Check(testkit.Rows("1 6")) + r.Check(testkit.Rows("1 7")) tk.MustExec("admin check table admin_test") // Add one row of index. // Table count < index count. - // Index c2 has two more values ​​than table data: 21, 13. + // Index c2 has one more values ​​than table data: 0, and the handle 0 hasn't correlative record. txn, err = s.store.Begin() c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(21), 1) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(0), 0) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + _, err = tk.Exec("admin check table admin_test") + c.Assert(err.Error(), Equals, "handle 0, index:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:0, b:[]uint8(nil), x:interface {}(nil)} != record:") + + // Add one row of index. + // Table count < index count. + // Index c2 has two more values ​​than table data: 10, 13, and these handles have correlative record. + txn, err = s.store.Begin() + c.Assert(err, IsNil) + err = indexOpr.Delete(sc, txn, types.MakeDatums(0), 0, nil) + c.Assert(err, IsNil) + // Make sure the index value "19" is smaller "21". Then we scan to "19" before "21". + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(19), 10) c.Assert(err, IsNil) _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(13), 2) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) _, err = tk.Exec("admin check table admin_test") - c.Assert(err.Error(), Equals, "handle 2, index:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:13, b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:12, b:[]uint8(nil), x:interface {}(nil)}") + c.Assert(err.Error(), Equals, "col c2, handle 2, index:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:13, b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:12, b:[]uint8(nil), x:interface {}(nil)}") // Table count = index count. // Two indices have the same handle. @@ -463,34 +478,34 @@ func (s *testSuite2) TestAdminCheckTableFailed(c *C) { err = txn.Commit(context.Background()) c.Assert(err, IsNil) _, err = tk.Exec("admin check table admin_test") - c.Assert(err.Error(), Equals, "handle 1, index:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:21, b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:11, b:[]uint8(nil), x:interface {}(nil)}") + c.Assert(err.Error(), Equals, "col c2, handle 10, index:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:19, b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:20, b:[]uint8(nil), x:interface {}(nil)}") // Table count = index count. - // Index c2 has one line of data is 21, the corresponding table data is 11. + // Index c2 has one line of data is 19, the corresponding table data is 20. txn, err = s.store.Begin() c.Assert(err, IsNil) _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(12), 2) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(11), 1, nil) + err = indexOpr.Delete(sc, txn, types.MakeDatums(20), 10, nil) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) _, err = tk.Exec("admin check table admin_test") - c.Assert(err.Error(), Equals, "handle 1, index:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:21, b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:11, b:[]uint8(nil), x:interface {}(nil)}") + c.Assert(err.Error(), Equals, "col c2, handle 10, index:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:19, b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, collation:0x0, decimal:0x0, length:0x0, i:20, b:[]uint8(nil), x:interface {}(nil)}") // Recover records. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(21), 1, nil) + err = indexOpr.Delete(sc, txn, types.MakeDatums(19), 10, nil) c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(11), 1) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(20), 10) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) tk.MustExec("admin check table admin_test") } -func (s *testSuite1) TestAdminCheckTablex(c *C) { +func (s *testSuite1) TestAdminCheckTable(c *C) { // test NULL value. tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/builder.go b/executor/builder.go index 2bbd28485d24a..18668ba74d71e 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -322,7 +322,6 @@ func (b *executorBuilder) buildCheckIndex(v *plannercore.CheckIndex) Executor { func buildIndexLookUpChecker(b *executorBuilder, readerPlan *plannercore.PhysicalIndexLookUpReader, readerExec *IndexLookUpExecutor) { - readerExec.keepOrder = true is := readerPlan.IndexPlans[0].(*plannercore.PhysicalIndexScan) readerExec.dagPB.OutputOffsets = make([]uint32, 0, len(is.Index.Columns)) for i := 0; i <= len(is.Index.Columns); i++ { diff --git a/executor/distsql.go b/executor/distsql.go index f1c3776eff0eb..83567e15a7012 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -71,6 +71,9 @@ type lookupTableTask struct { // The handles fetched from index is originally ordered by index, but we need handles to be ordered by itself // to do table request. indexOrder map[int64]int + // duplicatedIndexOrder map likes indexOrder. But it's used when isCheckOp is true and + // the same handle of index has multiple values. + duplicatedIndexOrder map[int64]int // memUsage records the memory usage of this task calculated by table worker. // memTracker is used to release memUsage after task is done and unused. @@ -644,7 +647,12 @@ func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectRes } } }() - chk := chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeLonglong)}, w.idxLookup.maxChunkSize) + var chk *chunk.Chunk + if w.isCheckOp { + chk = chunk.NewChunkWithCapacity(w.tps, w.maxChunkSize) + } else { + chk = chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeLonglong)}, w.idxLookup.maxChunkSize) + } for { handles, retChunk, err := w.extractTaskHandles(ctx, chk, result) if err != nil { @@ -673,9 +681,6 @@ func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectRes func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, idxResult distsql.SelectResult) ( handles []int64, retChk *chunk.Chunk, err error) { - if w.isCheckOp { - chk = chunk.NewChunkWithCapacity(w.tps, w.maxChunkSize) - } handleOffset := chk.NumCols() - 1 handles = make([]int64, 0, w.batchSize) for len(handles) < w.batchSize { @@ -693,7 +698,7 @@ func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, } if w.isCheckOp { if retChk == nil { - retChk = chunk.NewChunkWithCapacity(w.tps, w.maxChunkSize) + retChk = chunk.NewChunkWithCapacity(w.tps, w.batchSize) } retChk.Append(chk, 0, chk.NumRows()) } @@ -707,6 +712,7 @@ func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, func (w *indexWorker) buildTableTask(handles []int64, retChk *chunk.Chunk) *lookupTableTask { var indexOrder map[int64]int + var duplicatedIndexOrder map[int64]int if w.keepOrder { // Save the index order. indexOrder = make(map[int64]int, len(handles)) @@ -714,10 +720,25 @@ func (w *indexWorker) buildTableTask(handles []int64, retChk *chunk.Chunk) *look indexOrder[h] = i } } + + if w.isCheckOp { + // Save the index order. + indexOrder = make(map[int64]int, len(handles)) + duplicatedIndexOrder = make(map[int64]int) + for i, h := range handles { + if _, ok := indexOrder[h]; ok { + duplicatedIndexOrder[h] = i + } else { + indexOrder[h] = i + } + } + } + task := &lookupTableTask{ - handles: handles, - indexOrder: indexOrder, - idxRows: retChk, + handles: handles, + indexOrder: indexOrder, + duplicatedIndexOrder: duplicatedIndexOrder, + idxRows: retChk, } task.doneCh = make(chan error, 1) @@ -770,26 +791,10 @@ func (w *tableWorker) pickAndExecTask(ctx context.Context) { } } -func adjustDatumKind(vals1, vals2 []types.Datum) { - if len(vals1) != len(vals2) { - return - } - - for i, val1 := range vals1 { - val2 := vals2[i] - if val1.Kind() != val2.Kind() { - if (val1.Kind() == types.KindBytes || val1.Kind() == types.KindString) && - (val2.Kind() == types.KindBytes || val2.Kind() == types.KindString) { - vals1[i].SetBytes(val1.GetBytes()) - vals2[i].SetBytes(val2.GetBytes()) - } - } - } -} - func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, tableReader Executor) error { + chk := newFirstChunk(tableReader) + vals := make([]types.Datum, 0, len(w.cols)) for { - chk := newFirstChunk(tableReader) err := tableReader.Next(ctx, chk) if err != nil { return errors.Trace(err) @@ -803,15 +808,17 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta } tblReaderExec := tableReader.(*TableReaderExecutor) - i := 0 iter := chunk.NewIterator4Chunk(chk) for row := iter.Begin(); row != iter.End(); row = iter.Next() { handle := row.GetInt64(row.Len() - 1) - offset := task.indexOrder[handle] + offset, ok := task.indexOrder[handle] + if !ok { + offset = task.duplicatedIndexOrder[handle] + } delete(task.indexOrder, handle) idxRow := task.idxRows.GetRow(offset) - vals := make([]types.Datum, 0, len(w.cols)) - for j, col := range w.cols { + vals = vals[:0] + for i, col := range w.cols { if col.IsGenerated() && !col.GeneratedStored { expr := w.genExprs[model.TableColumnID{TableID: w.tbl.Meta().ID, ColumnID: col.ID}] // Eval the column value @@ -825,19 +832,18 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta } vals = append(vals, val) } else { - vals = append(vals, row.GetDatum(j, &col.FieldType)) + vals = append(vals, row.GetDatum(i, &col.FieldType)) } } vals = tables.TruncateIndexValuesIfNeeded(w.tbl.Meta(), w.idxInfo, vals) - for j, val := range vals { - col := w.cols[j] + for i, val := range vals { + col := w.cols[i] tp := &col.FieldType - ret := chunk.Compare(idxRow, j, &val) + ret := chunk.Compare(idxRow, i, &val) if ret != 0 { - return errors.Errorf("handle %#v, index:%#v != record:%#v", handle, idxRow.GetDatum(j, tp), val) + return errors.Errorf("col %s, handle %#v, index:%#v != record:%#v", col.Name, handle, idxRow.GetDatum(i, tp), val) } } - i++ } } diff --git a/executor/executor.go b/executor/executor.go index df8c1526fd791..ed1c9a7b9b665 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -486,10 +486,8 @@ func (e *CheckTableExec) checkIndexHandle(ctx context.Context, num int, src *Ind retFieldTypes[i] = cols[i].RetType } chk := chunk.New(retFieldTypes, e.initCap, e.maxChunkSize) - iter := chunk.NewIterator4Chunk(chk) var err error - handles := make(map[int64]struct{}, 1024) for { err = src.Next(ctx, chk) if err != nil { @@ -498,17 +496,6 @@ func (e *CheckTableExec) checkIndexHandle(ctx context.Context, num int, src *Ind if chk.NumRows() == 0 { break } - for chunkRow := iter.Begin(); chunkRow != iter.End(); chunkRow = iter.Next() { - datum := chunkRow.GetDatum(0, retFieldTypes[0]) - h := datum.GetInt64() - if _, ok := handles[h]; ok { - err = errors.Errorf("admin check table %s, index %s, at least two indices have the same handle %d more than one", - e.tblInfo.Name, e.indices[num].Meta().Name, h) - e.retCh <- errors.Trace(err) - return errors.Trace(err) - } - handles[h] = struct{}{} - } select { case <-e.exitCh: From 9f1d27d14fe548671842b5a071780e787c57470d Mon Sep 17 00:00:00 2001 From: Lynn Date: Fri, 26 Jul 2019 19:26:42 +0800 Subject: [PATCH 20/23] planner, executor: remove possible the PK column --- executor/builder.go | 3 +++ executor/distsql.go | 2 +- planner/core/physical_plans.go | 3 +++ planner/core/planbuilder.go | 35 +++++++++++++++++++++------------- 4 files changed, 29 insertions(+), 14 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 18668ba74d71e..bdd43be106594 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1994,7 +1994,10 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn idxPlans: v.IndexPlans, tblPlans: v.TablePlans, } + // These are used to check the consistency of the index data. e.genExprs = is.GenExprs + e.handleIdx = ts.HandleIdx + if containsLimit(indexReq.Executors) { e.feedback = statistics.NewQueryFeedback(0, nil, 0, is.Desc) } else { diff --git a/executor/distsql.go b/executor/distsql.go index 83567e15a7012..943c551146d60 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -810,7 +810,7 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta tblReaderExec := tableReader.(*TableReaderExecutor) iter := chunk.NewIterator4Chunk(chk) for row := iter.Begin(); row != iter.End(); row = iter.Next() { - handle := row.GetInt64(row.Len() - 1) + handle := row.GetInt64(w.handleIdx) offset, ok := task.indexOrder[handle] if !ok { offset = task.duplicatedIndexOrder[handle] diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 88733c7c33580..a43f3fa03f340 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -162,6 +162,9 @@ type PhysicalTableScan struct { physicalTableID int64 rangeDecidedBy []*expression.Column + + // HandleIdx is the index of handle, which is only used for admin check table. + HandleIdx int } // IsPartition returns true and partition ID if it's actually a partition. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 505fb3721c503..051b6e9135a76 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -687,6 +687,7 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab // Get generated columns. var genCols []*expression.Column + pkOffset := -1 colsMap := make(map[int64]struct{}) schema := expression.NewSchema(make([]*expression.Column, 0, len(idx.Columns))...) for _, idxCol := range idx.Columns { @@ -697,9 +698,11 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab schema.Append(&expression.Column{ ColName: col.Name, UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(), - RetType: &col.FieldType, - }) + RetType: &col.FieldType}) colsMap[col.ID] = struct{}{} + if mysql.HasPriKeyFlag(col.Flag) { + pkOffset = len(tblColumns) - 1 + } } genColumnID := model.TableColumnID{TableID: tblInfo.ID, ColumnID: col.ID} if expr, ok := genExprs[genColumnID]; ok { @@ -718,22 +721,27 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab tblSchema.Append(&expression.Column{ ColName: c.Name, UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(), - RetType: &c.FieldType, - }) + RetType: &c.FieldType}) colsMap[c.ID] = struct{}{} + if mysql.HasPriKeyFlag(c.Flag) { + pkOffset = len(tblColumns) - 1 + } } } } - tblColumns = append(tblColumns, model.NewExtraHandleColInfo()) - handleCol := &expression.Column{ - DBName: dbName, - TblName: tblInfo.Name, - ColName: model.ExtraHandleName, - RetType: types.NewFieldType(mysql.TypeLonglong), - UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(), - ID: model.ExtraHandleID, + if !tbl.Meta().PKIsHandle || pkOffset == -1 { + tblColumns = append(tblColumns, model.NewExtraHandleColInfo()) + handleCol := &expression.Column{ + DBName: dbName, + TblName: tblInfo.Name, + ColName: model.ExtraHandleName, + RetType: types.NewFieldType(mysql.TypeLonglong), + UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(), + ID: model.ExtraHandleID, + } + tblSchema.Append(handleCol) + pkOffset = len(tblColumns) - 1 } - tblSchema.Append(handleCol) is := PhysicalIndexScan{ Table: tblInfo, @@ -752,6 +760,7 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(dbName model.CIStr, tbl tab ts := PhysicalTableScan{Columns: tblColumns, Table: is.Table}.Init(b.ctx) ts.SetSchema(tblSchema) cop.tablePlan = ts + ts.HandleIdx = pkOffset is.initSchema(id, idx, true) t := finishCopTask(b.ctx, cop) rootT := t.(*rootTask) From 2c7d2d25d269e31f9f103f26fce87b90fd436c28 Mon Sep 17 00:00:00 2001 From: Lynn Date: Tue, 30 Jul 2019 19:22:55 +0800 Subject: [PATCH 21/23] *: add concurrency limit and address comments --- executor/builder.go | 7 +-- executor/distsql.go | 30 ++++----- executor/executor.go | 26 ++++++-- go.mod | 1 - go.sum | 2 - planner/core/planbuilder.go | 118 +++++++++++++++++++++--------------- util/admin/admin.go | 5 +- 7 files changed, 111 insertions(+), 78 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 058d4b905c918..4295eaa80e334 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -322,6 +322,7 @@ func (b *executorBuilder) buildCheckIndex(v *plannercore.CheckIndex) Executor { return e } +// buildIndexLookUpChecker builds check information to IndexLookUpReader. func buildIndexLookUpChecker(b *executorBuilder, readerPlan *plannercore.PhysicalIndexLookUpReader, readerExec *IndexLookUpExecutor) { is := readerPlan.IndexPlans[0].(*plannercore.PhysicalIndexScan) @@ -329,14 +330,12 @@ func buildIndexLookUpChecker(b *executorBuilder, readerPlan *plannercore.Physica for i := 0; i <= len(is.Index.Columns); i++ { readerExec.dagPB.OutputOffsets = append(readerExec.dagPB.OutputOffsets, uint32(i)) } - // set tps tps := make([]*types.FieldType, 0, len(is.Columns)+1) for _, col := range is.Columns { tps = append(tps, &col.FieldType) } tps = append(tps, types.NewFieldType(mysql.TypeLonglong)) - readerExec.tps = tps - readerExec.tbl = readerExec.table + readerExec.idxColTps = tps readerExec.idxInfo = readerExec.index colNames := make([]string, 0, len(is.Columns)) @@ -345,7 +344,7 @@ func buildIndexLookUpChecker(b *executorBuilder, readerPlan *plannercore.Physica } var err error - readerExec.cols, err = table.FindCols(readerExec.table.Cols(), colNames, true) + readerExec.idxTblCols, err = table.FindCols(readerExec.table.Cols(), colNames, true) if err != nil { b.err = errors.Trace(err) return diff --git a/executor/distsql.go b/executor/distsql.go index 2e79e12deff38..198ca216423d5 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -354,12 +354,11 @@ type IndexLookUpExecutor struct { } type checkIndexValue struct { - isCheckOp bool - tps []*types.FieldType - tbl table.Table - idxInfo *model.IndexInfo - cols []*table.Column - genExprs map[model.TableColumnID]expression.Expression + isCheckOp bool + idxColTps []*types.FieldType + idxInfo *model.IndexInfo + idxTblCols []*table.Column + genExprs map[model.TableColumnID]expression.Expression } // Open implements the Executor Open interface. @@ -448,7 +447,7 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, kvRanges []k } tps := []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)} if e.isCheckOp { - tps = e.tps + tps = e.idxColTps } // Since the first read only need handle information. So its returned col is only 1. result, err := distsql.SelectWithRuntimeStats(ctx, e.ctx, kvReq, tps, e.feedback, getPhysicalPlanIDs(e.idxPlans)) @@ -652,7 +651,7 @@ func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectRes }() var chk *chunk.Chunk if w.isCheckOp { - chk = chunk.NewChunkWithCapacity(w.tps, w.maxChunkSize) + chk = chunk.NewChunkWithCapacity(w.idxColTps, w.maxChunkSize) } else { chk = chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeLonglong)}, w.idxLookup.maxChunkSize) } @@ -701,7 +700,7 @@ func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, } if w.isCheckOp { if retChk == nil { - retChk = chunk.NewChunkWithCapacity(w.tps, w.batchSize) + retChk = chunk.NewChunkWithCapacity(w.idxColTps, w.batchSize) } retChk.Append(chk, 0, chk.NumRows()) } @@ -796,7 +795,8 @@ func (w *tableWorker) pickAndExecTask(ctx context.Context) { func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, tableReader Executor) error { chk := newFirstChunk(tableReader) - vals := make([]types.Datum, 0, len(w.cols)) + tblInfo := w.idxLookup.table.Meta() + vals := make([]types.Datum, 0, len(w.idxTblCols)) for { err := tableReader.Next(ctx, chk) if err != nil { @@ -805,7 +805,7 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta if chk.NumRows() == 0 { for h := range task.indexOrder { idxRow := task.idxRows.GetRow(task.indexOrder[h]) - return errors.Errorf("handle %#v, index:%#v != record:%#v", h, idxRow.GetDatum(0, w.tps[0]), nil) + return errors.Errorf("handle %#v, index:%#v != record:%#v", h, idxRow.GetDatum(0, w.idxColTps[0]), nil) } break } @@ -821,9 +821,9 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta delete(task.indexOrder, handle) idxRow := task.idxRows.GetRow(offset) vals = vals[:0] - for i, col := range w.cols { + for i, col := range w.idxTblCols { if col.IsGenerated() && !col.GeneratedStored { - expr := w.genExprs[model.TableColumnID{TableID: w.tbl.Meta().ID, ColumnID: col.ID}] + expr := w.genExprs[model.TableColumnID{TableID: tblInfo.ID, ColumnID: col.ID}] // Eval the column value val, err := expr.Eval(row) if err != nil { @@ -838,9 +838,9 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta vals = append(vals, row.GetDatum(i, &col.FieldType)) } } - vals = tables.TruncateIndexValuesIfNeeded(w.tbl.Meta(), w.idxInfo, vals) + vals = tables.TruncateIndexValuesIfNeeded(tblInfo, w.idxInfo, vals) for i, val := range vals { - col := w.cols[i] + col := w.idxTblCols[i] tp := &col.FieldType ret := chunk.Compare(idxRow, i, &val) if ret != 0 { diff --git a/executor/executor.go b/executor/executor.go index 810f82422504e..81361ca25581b 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" @@ -507,6 +508,12 @@ func (e *CheckTableExec) checkIndexHandle(ctx context.Context, num int, src *Ind return errors.Trace(err) } +func (e *CheckTableExec) handlePanic(r interface{}) { + if r != nil { + e.retCh <- errors.Errorf("%v", r) + } +} + // Next implements the Executor Next interface. func (e *CheckTableExec) Next(ctx context.Context, req *chunk.Chunk) error { if e.done || len(e.srcs) == 0 { @@ -533,13 +540,24 @@ func (e *CheckTableExec) Next(ctx context.Context, req *chunk.Chunk) error { } // The number of table rows is equal to the number of index rows. + // TODO: Make the value of concurrency adjustable. And we can consider the number of records. + concurrency := 3 + wg := sync.WaitGroup{} for i := range e.srcs { + wg.Add(1) go func(num int) { - err1 := e.checkIndexHandle(ctx, num, e.srcs[num]) - if err1 != nil { - logutil.Logger(ctx).Info("check index handle failed", zap.Error(err)) - } + defer wg.Done() + util.WithRecovery(func() { + err1 := e.checkIndexHandle(ctx, num, e.srcs[num]) + if err1 != nil { + logutil.Logger(ctx).Info("check index handle failed", zap.Error(err)) + } + }, e.handlePanic) }(i) + + if (i+1)%concurrency == 0 { + wg.Wait() + } } for i := 0; i < len(e.srcs); i++ { diff --git a/go.mod b/go.mod index 7664320524d5b..6f5060725a50f 100644 --- a/go.mod +++ b/go.mod @@ -31,7 +31,6 @@ require ( github.com/modern-go/reflect2 v1.0.1 // indirect github.com/montanaflynn/stats v0.0.0-20180911141734-db72e6cae808 // indirect github.com/myesui/uuid v1.0.0 // indirect - github.com/ngaut/log v0.0.0-20180314031856-b8e36e7ba5ac github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef github.com/opentracing/basictracer-go v1.0.0 diff --git a/go.sum b/go.sum index 2e7a072af9c73..368f4dd6e0d9a 100644 --- a/go.sum +++ b/go.sum @@ -132,8 +132,6 @@ github.com/montanaflynn/stats v0.0.0-20180911141734-db72e6cae808 h1:pmpDGKLw4n82 github.com/montanaflynn/stats v0.0.0-20180911141734-db72e6cae808/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/myesui/uuid v1.0.0 h1:xCBmH4l5KuvLYc5L7AS7SZg9/jKdIFubM7OVoLqaQUI= github.com/myesui/uuid v1.0.0/go.mod h1:2CDfNgU0LR8mIdO8vdWd8i9gWWxLlcoIGGpSNgafq84= -github.com/ngaut/log v0.0.0-20180314031856-b8e36e7ba5ac h1:wyheT2lPXRQqYPWY2IVW5BTLrbqCsnhL61zK2R5goLA= -github.com/ngaut/log v0.0.0-20180314031856-b8e36e7ba5ac/go.mod h1:ueVCjKQllPmX7uEvCYnZD5b8qjidGf1TCH61arVe4SU= github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 h1:7KAv7KMGTTqSmYZtNdcNTgsos+vFzULLwyElndwn+5c= github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7/go.mod h1:iWMfgwqYW+e8n5lC/jjNEhwcjbRDpl5NT7n2h+4UNcI= github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef h1:K0Fn+DoFqNqktdZtdV3bPQ/0cuYh2H4rkg0tytX/07k= diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index ff3740ed0f0a9..80af88e989aa7 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -657,83 +657,107 @@ func (b *PlanBuilder) buildAdmin(ctx context.Context, as *ast.AdminStmt) (Plan, return ret, nil } -func (b *PlanBuilder) buildPhysicalIndexLookUpReader(ctx context.Context, dbName model.CIStr, tbl table.Table, idx *model.IndexInfo, id int) (Plan, error) { +// getGenExprs gets generated expressions map. +func (b *PlanBuilder) getGenExprs(ctx context.Context, dbName model.CIStr, tbl table.Table, idx *model.IndexInfo) ( + map[model.TableColumnID]expression.Expression, error) { tblInfo := tbl.Meta() - columns := make([]*model.ColumnInfo, 0, len(idx.Columns)) - tblColumns := make([]*model.ColumnInfo, 0, len(tbl.Cols())) - - // Get generated expressions. - genExprs := make(map[model.TableColumnID]expression.Expression) + genExprsMap := make(map[model.TableColumnID]expression.Expression) + exprs := make([]expression.Expression, 0, len(tbl.Cols())) + genExprIdxs := make([]model.TableColumnID, len(tbl.Cols())) mockTablePlan := LogicalTableDual{}.Init(b.ctx) mockTablePlan.SetSchema(expression.TableInfo2SchemaWithDBName(b.ctx, dbName, tblInfo)) - for _, column := range idx.Columns { - col := table.FindCol(tbl.Cols(), column.Name.L) - if !col.IsGenerated() { - continue + for i, colExpr := range mockTablePlan.Schema().Columns { + col := tbl.Cols()[i] + var expr expression.Expression + expr = colExpr + if col.IsGenerated() && !col.GeneratedStored { + var err error + expr, _, err = b.rewrite(ctx, col.GeneratedExpr, mockTablePlan, nil, true) + if err != nil { + return nil, errors.Trace(err) + } + expr = expression.BuildCastFunction(b.ctx, expr, colExpr.GetType()) + found := false + for _, column := range idx.Columns { + if strings.EqualFold(col.Name.L, column.Name.L) { + found = true + break + } + } + if found { + genColumnID := model.TableColumnID{TableID: tblInfo.ID, ColumnID: col.ColumnInfo.ID} + genExprsMap[genColumnID] = expr + genExprIdxs[i] = genColumnID + } } - columnName := &ast.ColumnName{Name: column.Name} - columnName.SetText(column.Name.O) - - colExpr, _, err := mockTablePlan.findColumn(columnName) - if err != nil { - return nil, errors.Trace(err) + exprs = append(exprs, expr) + } + // Re-iterate expressions to handle those virtual generated columns that refers to the other generated columns. + for i, expr := range exprs { + exprs[i] = expression.ColumnSubstitute(expr, mockTablePlan.Schema(), exprs) + if _, ok := genExprsMap[genExprIdxs[i]]; ok { + genExprsMap[genExprIdxs[i]] = exprs[i] } + } + return genExprsMap, nil +} - expr, _, err := b.rewrite(ctx, col.GeneratedExpr, mockTablePlan, nil, true) - if err != nil { - return nil, errors.Trace(err) - } - expr = expression.BuildCastFunction(b.ctx, expr, colExpr.GetType()) - genColumnID := model.TableColumnID{TableID: tblInfo.ID, ColumnID: col.ColumnInfo.ID} - genExprs[genColumnID] = expr +func (b *PlanBuilder) buildPhysicalIndexLookUpReader(ctx context.Context, dbName model.CIStr, tbl table.Table, idx *model.IndexInfo, id int) (Plan, error) { + genExprsMap, err := b.getGenExprs(ctx, dbName, tbl, idx) + if err != nil { + return nil, errors.Trace(err) } // Get generated columns. var genCols []*expression.Column pkOffset := -1 + tblInfo := tbl.Meta() colsMap := make(map[int64]struct{}) schema := expression.NewSchema(make([]*expression.Column, 0, len(idx.Columns))...) + idxReaderCols := make([]*model.ColumnInfo, 0, len(idx.Columns)) + tblReaderCols := make([]*model.ColumnInfo, 0, len(tbl.Cols())) for _, idxCol := range idx.Columns { for _, col := range tblInfo.Columns { if idxCol.Name.L == col.Name.L { - columns = append(columns, col) - tblColumns = append(tblColumns, col) + idxReaderCols = append(idxReaderCols, col) + tblReaderCols = append(tblReaderCols, col) schema.Append(&expression.Column{ ColName: col.Name, UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(), RetType: &col.FieldType}) colsMap[col.ID] = struct{}{} if mysql.HasPriKeyFlag(col.Flag) { - pkOffset = len(tblColumns) - 1 + pkOffset = len(tblReaderCols) - 1 } } genColumnID := model.TableColumnID{TableID: tblInfo.ID, ColumnID: col.ID} - if expr, ok := genExprs[genColumnID]; ok { + if expr, ok := genExprsMap[genColumnID]; ok { cols := expression.ExtractColumns(expr) genCols = append(genCols, cols...) } } } + // Add generated columns to tblSchema and tblReaderCols. tblSchema := schema.Clone() for _, col := range genCols { if _, ok := colsMap[col.ID]; !ok { c := table.FindCol(tbl.Cols(), col.ColName.O) if c != nil { - col.Index = len(tblColumns) - tblColumns = append(tblColumns, c.ColumnInfo) + col.Index = len(tblReaderCols) + tblReaderCols = append(tblReaderCols, c.ColumnInfo) tblSchema.Append(&expression.Column{ ColName: c.Name, UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(), RetType: &c.FieldType}) colsMap[c.ID] = struct{}{} if mysql.HasPriKeyFlag(c.Flag) { - pkOffset = len(tblColumns) - 1 + pkOffset = len(tblReaderCols) - 1 } } } } if !tbl.Meta().PKIsHandle || pkOffset == -1 { - tblColumns = append(tblColumns, model.NewExtraHandleColInfo()) + tblReaderCols = append(tblReaderCols, model.NewExtraHandleColInfo()) handleCol := &expression.Column{ DBName: dbName, TblName: tblInfo.Name, @@ -743,30 +767,27 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(ctx context.Context, dbName ID: model.ExtraHandleID, } tblSchema.Append(handleCol) - pkOffset = len(tblColumns) - 1 + pkOffset = len(tblReaderCols) - 1 } is := PhysicalIndexScan{ Table: tblInfo, TableAsName: &tblInfo.Name, DBName: dbName, - Columns: columns, + Columns: idxReaderCols, Index: idx, dataSourceSchema: schema, Ranges: ranger.FullRange(), - KeepOrder: false, - GenExprs: genExprs, + GenExprs: genExprsMap, }.Init(b.ctx) is.stats = property.NewSimpleStats(0) - cop := &copTask{indexPlan: is} // It's double read case. - ts := PhysicalTableScan{Columns: tblColumns, Table: is.Table}.Init(b.ctx) + ts := PhysicalTableScan{Columns: tblReaderCols, Table: is.Table}.Init(b.ctx) ts.SetSchema(tblSchema) - cop.tablePlan = ts + cop := &copTask{indexPlan: is, tablePlan: ts} ts.HandleIdx = pkOffset is.initSchema(id, idx, true) - t := finishCopTask(b.ctx, cop) - rootT := t.(*rootTask) + rootT := finishCopTask(b.ctx, cop).(*rootTask) return rootT.p, nil } @@ -780,14 +801,14 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReaders(ctx context.Context, dbNam if idxInfo.State != model.StatePublic { logutil.Logger(context.Background()).Info("build physical index lookup reader, the index isn't public", zap.String("index", idxInfo.Name.O), zap.Stringer("state", idxInfo.State), zap.String("table", tblInfo.Name.O)) - } else { - indices = append(indices, idx) - reader, err := b.buildPhysicalIndexLookUpReader(ctx, dbName, tbl, idxInfo, i) - if err != nil { - return nil, nil, err - } - indexLookUpReaders = append(indexLookUpReaders, reader) + continue + } + indices = append(indices, idx) + reader, err := b.buildPhysicalIndexLookUpReader(ctx, dbName, tbl, idxInfo, i) + if err != nil { + return nil, nil, err } + indexLookUpReaders = append(indexLookUpReaders, reader) } if len(indexLookUpReaders) == 0 { return nil, nil, nil @@ -802,10 +823,7 @@ func (b *PlanBuilder) buildAdminCheckTable(ctx context.Context, as *ast.AdminStm TblInfo: tbl.TableInfo, } - mockTablePlan := LogicalTableDual{}.Init(b.ctx) tableInfo := as.Tables[0].TableInfo - schema := expression.TableInfo2SchemaWithDBName(b.ctx, tbl.Schema, tableInfo) - mockTablePlan.SetSchema(schema) table, ok := b.is.TableByID(tableInfo.ID) if !ok { return nil, infoschema.ErrTableNotExists.GenWithStackByArgs(tbl.DBInfo.Name.O, tableInfo.Name.O) diff --git a/util/admin/admin.go b/util/admin/admin.go index 1b102e46d2e7f..fce94e497bc93 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -14,13 +14,13 @@ package admin import ( + "context" "fmt" "io" "math" "sort" "time" - "github.com/ngaut/log" "github.com/pingcap/errors" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -293,7 +293,8 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices if err != nil { return 0, i, errors.Trace(err) } - log.Infof("check indices count, table %s cnt %d, index %s cnt %d", tableName, tblCnt, idx, idxCnt) + logutil.Logger(context.Background()).Info("check indices count, table %s cnt %d, index %s cnt %d", + zap.String("table", tableName), zap.Int64("cnt", tblCnt), zap.Reflect("index", idx), zap.Int64("cnt", idxCnt)) if tblCnt == idxCnt { continue } From d0fb55cfe46f324ce931b55991b8708614a3bab5 Mon Sep 17 00:00:00 2001 From: Lynn Date: Wed, 31 Jul 2019 12:50:59 +0800 Subject: [PATCH 22/23] executor: address comments --- executor/builder.go | 1 - executor/distsql.go | 3 +-- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 58a85cce47589..0d16788f1b197 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -336,7 +336,6 @@ func buildIndexLookUpChecker(b *executorBuilder, readerPlan *plannercore.Physica } tps = append(tps, types.NewFieldType(mysql.TypeLonglong)) readerExec.idxColTps = tps - readerExec.idxInfo = readerExec.index colNames := make([]string, 0, len(is.Columns)) for _, col := range is.Columns { diff --git a/executor/distsql.go b/executor/distsql.go index 8cf6062298a8c..455aaf0d8c462 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -361,7 +361,6 @@ type IndexLookUpExecutor struct { type checkIndexValue struct { isCheckOp bool idxColTps []*types.FieldType - idxInfo *model.IndexInfo idxTblCols []*table.Column genExprs map[model.TableColumnID]expression.Expression } @@ -843,7 +842,7 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta vals = append(vals, row.GetDatum(i, &col.FieldType)) } } - vals = tables.TruncateIndexValuesIfNeeded(tblInfo, w.idxInfo, vals) + vals = tables.TruncateIndexValuesIfNeeded(tblInfo, w.idxLookup.index, vals) for i, val := range vals { col := w.idxTblCols[i] tp := &col.FieldType From 61ab6a7f182575f36a63791227f6d2695434505d Mon Sep 17 00:00:00 2001 From: Lynn Date: Wed, 31 Jul 2019 13:34:14 +0800 Subject: [PATCH 23/23] executor: address comments --- executor/builder.go | 12 +++++------- executor/distsql.go | 19 +++++++++---------- 2 files changed, 14 insertions(+), 17 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 0d16788f1b197..d2192c66512a7 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -330,26 +330,27 @@ func buildIndexLookUpChecker(b *executorBuilder, readerPlan *plannercore.Physica for i := 0; i <= len(is.Index.Columns); i++ { readerExec.dagPB.OutputOffsets = append(readerExec.dagPB.OutputOffsets, uint32(i)) } + readerExec.ranges = ranger.FullRange() + ts := readerPlan.TablePlans[0].(*plannercore.PhysicalTableScan) + readerExec.handleIdx = ts.HandleIdx + tps := make([]*types.FieldType, 0, len(is.Columns)+1) for _, col := range is.Columns { tps = append(tps, &col.FieldType) } tps = append(tps, types.NewFieldType(mysql.TypeLonglong)) - readerExec.idxColTps = tps + readerExec.checkIndexValue = &checkIndexValue{genExprs: is.GenExprs, idxColTps: tps} colNames := make([]string, 0, len(is.Columns)) for _, col := range is.Columns { colNames = append(colNames, col.Name.O) } - var err error readerExec.idxTblCols, err = table.FindCols(readerExec.table.Cols(), colNames, true) if err != nil { b.err = errors.Trace(err) return } - readerExec.isCheckOp = true - readerExec.ranges = ranger.FullRange() } func (b *executorBuilder) buildCheckTable(v *plannercore.CheckTable) Executor { @@ -1933,9 +1934,6 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn idxPlans: v.IndexPlans, tblPlans: v.TablePlans, } - // These are used to check the consistency of the index data. - e.genExprs = is.GenExprs - e.handleIdx = ts.HandleIdx if containsLimit(indexReq.Executors) { e.feedback = statistics.NewQueryFeedback(0, nil, 0, is.Desc) diff --git a/executor/distsql.go b/executor/distsql.go index 455aaf0d8c462..ffb28e46fb784 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -72,7 +72,7 @@ type lookupTableTask struct { // The handles fetched from index is originally ordered by index, but we need handles to be ordered by itself // to do table request. indexOrder map[int64]int - // duplicatedIndexOrder map likes indexOrder. But it's used when isCheckOp is true and + // duplicatedIndexOrder map likes indexOrder. But it's used when checkIndexValue isn't nil and // the same handle of index has multiple values. duplicatedIndexOrder map[int64]int @@ -347,7 +347,7 @@ type IndexLookUpExecutor struct { memTracker *memory.Tracker // checkIndexValue is used to check the consistency of the index data. - checkIndexValue + *checkIndexValue corColInIdxSide bool idxPlans []plannercore.PhysicalPlan @@ -359,7 +359,6 @@ type IndexLookUpExecutor struct { } type checkIndexValue struct { - isCheckOp bool idxColTps []*types.FieldType idxTblCols []*table.Column genExprs map[model.TableColumnID]expression.Expression @@ -450,7 +449,7 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, kvRanges []k return err } tps := []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)} - if e.isCheckOp { + if e.checkIndexValue != nil { tps = e.idxColTps } // Since the first read only need handle information. So its returned col is only 1. @@ -629,7 +628,7 @@ type indexWorker struct { maxChunkSize int // checkIndexValue is used to check the consistency of the index data. - checkIndexValue + *checkIndexValue } // fetchHandles fetches a batch of handles from index data and builds the index lookup tasks. @@ -654,7 +653,7 @@ func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectRes } }() var chk *chunk.Chunk - if w.isCheckOp { + if w.checkIndexValue != nil { chk = chunk.NewChunkWithCapacity(w.idxColTps, w.maxChunkSize) } else { chk = chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeLonglong)}, w.idxLookup.maxChunkSize) @@ -702,7 +701,7 @@ func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, h := chk.GetRow(i).GetInt64(handleOffset) handles = append(handles, h) } - if w.isCheckOp { + if w.checkIndexValue != nil { if retChk == nil { retChk = chunk.NewChunkWithCapacity(w.idxColTps, w.batchSize) } @@ -727,7 +726,7 @@ func (w *indexWorker) buildTableTask(handles []int64, retChk *chunk.Chunk) *look } } - if w.isCheckOp { + if w.checkIndexValue != nil { // Save the index order. indexOrder = make(map[int64]int, len(handles)) duplicatedIndexOrder = make(map[int64]int) @@ -764,7 +763,7 @@ type tableWorker struct { memTracker *memory.Tracker // checkIndexValue is used to check the consistency of the index data. - checkIndexValue + *checkIndexValue } // pickAndExecTask picks tasks from workCh, and execute them. @@ -867,7 +866,7 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er } defer terror.Call(tableReader.Close) - if w.isCheckOp { + if w.checkIndexValue != nil { return w.compareData(ctx, task, tableReader) }