From a3b37d1b5a747e5948d52f83cc84aae7ad002781 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Fri, 1 Mar 2019 15:55:32 +0800 Subject: [PATCH 1/6] executor: replace logger with zap logger --- executor/adapter.go | 12 +++++++----- executor/admin.go | 17 ++++++++++------- executor/aggregate.go | 10 ++++++---- executor/analyze.go | 10 ++++++---- executor/checksum.go | 5 +++-- executor/compiler.go | 7 ++++--- executor/ddl.go | 7 ++++--- executor/distsql.go | 15 ++++++++------- executor/executor.go | 7 ++++--- executor/index_lookup_join.go | 7 ++++--- executor/insert.go | 5 +++-- executor/insert_common.go | 5 +++-- executor/load_data.go | 7 ++++--- executor/projection.go | 6 ++++-- executor/radix_hash_join.go | 9 +++++---- executor/replace.go | 5 +++-- executor/set.go | 7 ++++--- executor/show_stats.go | 5 ++--- executor/simple.go | 5 +++-- executor/write.go | 5 +++-- 20 files changed, 90 insertions(+), 66 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index f37ad05c5a1ab..55e278164af36 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -23,6 +23,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -40,7 +41,8 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) // processinfoSetter is the interface use to set current running process info. @@ -286,7 +288,7 @@ func (a *ExecStmt) handleNoDelayExecutor(ctx context.Context, sctx sessionctx.Co txnTS := uint64(0) // Don't active pending txn here. if txn, err1 := sctx.Txn(false); err1 != nil { - log.Error(err1) + log.Error("Get current transaction failed", zap.Error(err)) } else { if txn.Valid() { txnTS = txn.StartTS() @@ -313,7 +315,7 @@ func (a *ExecStmt) buildExecutor(ctx sessionctx.Context) (Executor, error) { return nil, errors.Trace(err) } if isPointGet { - log.Debugf("con:%d InitTxnWithStartTS %s", ctx.GetSessionVars().ConnectionID, a.Text) + log.Debug("InitTxnWithStartTS", zap.Uint64("con", ctx.GetSessionVars().ConnectionID), zap.String("text", a.Text)) err = ctx.InitTxnWithStartTS(math.MaxUint64) } else if ctx.GetSessionVars().SnapshotTS != 0 { if _, ok := a.Plan.(*plannercore.CheckTable); ok { @@ -363,13 +365,13 @@ var QueryReplacer = strings.NewReplacer("\r", " ", "\n", " ", "\t", " ") // LogSlowQuery is used to print the slow query in the log files. func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { level := log.GetLevel() - if level < log.WarnLevel { + if level > zapcore.WarnLevel { return } cfg := config.GetGlobalConfig() costTime := time.Since(a.StartTime) threshold := time.Duration(atomic.LoadUint64(&cfg.Log.SlowThreshold)) * time.Millisecond - if costTime < threshold && level < log.DebugLevel { + if costTime < threshold && level > zapcore.DebugLevel { return } sql := a.Text diff --git a/executor/admin.go b/executor/admin.go index bc62de1084fda..bbb99b36b5a36 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -18,6 +18,7 @@ import ( "math" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/terror" @@ -34,7 +35,7 @@ import ( "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-tipb" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) var ( @@ -308,8 +309,9 @@ func (e *RecoverIndexExec) backfillIndex(ctx context.Context) (int64, int64, err totalScanCnt += result.scanRowCount if totalScanCnt-lastLogCnt >= 50000 { lastLogCnt = totalScanCnt - log.Infof("[recover-index] recover table:%v, index:%v, totalAddedCnt:%v, totalScanCnt:%v, nextHandle: %v", - e.table.Meta().Name.O, e.index.Meta().Name.O, totalAddedCnt, totalScanCnt, result.nextHandle) + log.Info("Recover index", zap.String("table", e.table.Meta().Name.O), + zap.String("index", e.index.Meta().Name.O), zap.Int64("total added cnt", totalAddedCnt), + zap.Int64("total scan cnt", totalScanCnt), zap.Int64("next handle", result.nextHandle)) } // no more rows @@ -393,8 +395,9 @@ func (e *RecoverIndexExec) batchMarkDup(txn kv.Transaction, rows []recoverRows) } if handle != rows[i].handle { - log.Warnf("[recover-index] The constraint of unique index:%v is broken, handle:%v is not equal handle:%v with idxKey:%v.", - e.index.Meta().Name.O, handle, rows[i].handle, key) + log.Warn("The constraint of unique index is broken when recovering index", + zap.String("index name", e.index.Meta().Name.O), zap.ByteString("key value", key), + zap.Int64("handle", rows[i].handle), zap.Int64("another handle", handle)) } } rows[i].skip = true @@ -520,8 +523,8 @@ func (e *CleanupIndexExec) deleteDanglingIdx(txn kv.Transaction, values map[stri } e.removeCnt++ if e.removeCnt%e.batchSize == 0 { - log.Infof("[cleaning up dangling index] table: %v, index: %v, count: %v.", - e.table.Meta().Name.String(), e.index.Meta().Name.String(), e.removeCnt) + log.Info("Cleaning up dangling index", zap.String("table", e.table.Meta().Name.String()), + zap.String("index", e.index.Meta().Name.String()), zap.Uint64("count", e.removeCnt)) } } } diff --git a/executor/aggregate.go b/executor/aggregate.go index 6444aa58f2707..e188760c21359 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -20,6 +20,7 @@ import ( opentracing "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/executor/aggfuncs" "github.com/pingcap/tidb/expression" @@ -30,8 +31,8 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/set" - log "github.com/sirupsen/logrus" "github.com/spaolacci/murmur3" + "go.uber.org/zap" ) type aggPartialResultMapper map[string][]aggfuncs.PartialResult @@ -318,9 +319,10 @@ func (w *HashAggPartialWorker) getChildInput() bool { } func recoveryHashAgg(output chan *AfFinalResult, r interface{}) { - output <- &AfFinalResult{err: errors.Errorf("%v", r)} + err := errors.Errorf("%v", r) + output <- &AfFinalResult{err: err} buf := util.GetStack() - log.Errorf("panic in the recoverable goroutine: %v, stack trace:\n%s", r, buf) + log.Error("Panic in the recoverable goroutine", zap.Error(err), zap.String("stack trace", string(buf))) } func (w *HashAggPartialWorker) run(ctx sessionctx.Context, waitGroup *sync.WaitGroup, finalConcurrency int) { @@ -475,7 +477,7 @@ func (w *HashAggFinalWorker) getFinalResult(sctx sessionctx.Context) { partialResults := w.getPartialResult(sctx.GetSessionVars().StmtCtx, []byte(groupKey), w.partialResultMap) for i, af := range w.aggFuncs { if err := af.AppendFinalResult2Chunk(sctx, partialResults[i], result); err != nil { - log.Error(errors.ErrorStack(err)) + log.Error("Append final result failed", zap.String("error stack", errors.ErrorStack(err))) } } if len(w.aggFuncs) == 0 { diff --git a/executor/analyze.go b/executor/analyze.go index 67497220fa959..ed8d79cd3cc30 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -19,6 +19,7 @@ import ( "strconv" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/distsql" @@ -31,7 +32,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tipb/go-tipb" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) var _ Executor = &AnalyzeExec{} @@ -72,15 +73,16 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.RecordBatch) error { err = result.Err if errors.Trace(err) == errAnalyzeWorkerPanic { panicCnt++ + } else { + log.Error("Analyze failed", zap.String("error stack", errors.ErrorStack(err))) } - log.Error(errors.ErrorStack(err)) continue } for i, hg := range result.Hist { err1 := statsHandle.SaveStatsToStorage(result.PhysicalTableID, result.Count, result.IsIndex, hg, result.Cms[i], 1) if err1 != nil { err = err1 - log.Error(errors.ErrorStack(err)) + log.Error("Save stats to storage failed", zap.String("error stack", errors.ErrorStack(err))) continue } } @@ -122,7 +124,7 @@ func (e *AnalyzeExec) analyzeWorker(taskCh <-chan *analyzeTask, resultCh chan<- buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] - log.Errorf("analyzeWorker panic stack is:\n%s", buf) + log.Error("Analyze worker panicked", zap.String("panic stack", string(buf))) metrics.PanicCounter.WithLabelValues(metrics.LabelAnalyze).Inc() resultCh <- statistics.AnalyzeResult{ Err: errAnalyzeWorkerPanic, diff --git a/executor/checksum.go b/executor/checksum.go index 3eff498d4f2b6..25b7aecda4e50 100644 --- a/executor/checksum.go +++ b/executor/checksum.go @@ -18,6 +18,7 @@ import ( "strconv" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/kv" @@ -26,7 +27,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tipb/go-tipb" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) var _ Executor = &ChecksumTableExec{} @@ -70,7 +71,7 @@ func (e *ChecksumTableExec) Open(ctx context.Context) error { result := <-resultCh if result.Error != nil { err = result.Error - log.Error(errors.ErrorStack(err)) + log.Error("Checksum failed", zap.String("error stack", errors.ErrorStack(err))) continue } e.handleResult(result) diff --git a/executor/compiler.go b/executor/compiler.go index e263fbffbfcd7..ddf28777fc5da 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -19,6 +19,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/infoschema" @@ -26,7 +27,7 @@ import ( "github.com/pingcap/tidb/planner" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) // Compiler compiles an ast.StmtNode to a physical plan. @@ -76,7 +77,7 @@ func logExpensiveQuery(stmtNode ast.StmtNode, finalPlan plannercore.Plan) (expen if len(sql) > logSQLLen { sql = fmt.Sprintf("%s len(%d)", sql[:logSQLLen], len(sql)) } - log.Warnf("[EXPENSIVE_QUERY] %s", sql) + log.Warn("[EXPENSIVE_QUERY]", zap.String("sql", sql)) return } @@ -305,7 +306,7 @@ func GetInfoSchema(ctx sessionctx.Context) infoschema.InfoSchema { var is infoschema.InfoSchema if snap := sessVar.SnapshotInfoschema; snap != nil { is = snap.(infoschema.InfoSchema) - log.Infof("con:%d use snapshot schema %d", sessVar.ConnectionID, is.SchemaMetaVersion()) + log.Info("Use snapshot schema", zap.Uint64("con", sessVar.ConnectionID), zap.Int64("schema meta version", is.SchemaMetaVersion())) } else { is = sessVar.TxnCtx.InfoSchema.(infoschema.InfoSchema) } diff --git a/executor/ddl.go b/executor/ddl.go index 5f545a9bfe619..c0e5b7398498f 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -19,6 +19,7 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -33,7 +34,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/sqlexec" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) // DDLExec represents a DDL executor. @@ -60,7 +61,7 @@ func (e *DDLExec) toErr(err error) error { checker := domain.NewSchemaChecker(dom, e.is.SchemaMetaVersion(), nil) txn, err1 := e.ctx.Txn(true) if err1 != nil { - log.Error(err) + log.Error("Active txn failed", zap.Error(err)) return errors.Trace(err1) } schemaInfoErr := checker.Check(txn.StartTS()) @@ -252,7 +253,7 @@ func (e *DDLExec) executeDropTableOrView(s *ast.DropTableStmt) error { } if config.CheckTableBeforeDrop { - log.Warnf("admin check table `%s`.`%s` before drop.", fullti.Schema.O, fullti.Name.O) + log.Warn("Admin check table before drop", zap.String("table name", fmt.Sprintf("`%s`.`%s`", fullti.Schema.O, fullti.Name.O))) sql := fmt.Sprintf("admin check table `%s`.`%s`", fullti.Schema.O, fullti.Name.O) _, _, err = e.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(e.ctx, sql) if err != nil { diff --git a/executor/distsql.go b/executor/distsql.go index d0a8807aaf5a7..3d902a98f9011 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -25,6 +25,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" @@ -41,7 +42,7 @@ import ( "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tipb/go-tipb" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) var ( @@ -464,7 +465,7 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, kvRanges []k } cancel() if err := result.Close(); err != nil { - log.Error("close Select result failed:", errors.ErrorStack(err)) + log.Error("Close Select result failed", zap.String("error stack", errors.ErrorStack(err))) } if e.runtimeStats != nil { copStats := e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.idxPlans[len(e.idxPlans)-1].ExplainID()) @@ -516,7 +517,7 @@ func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, handles []in } tableReader, err := e.dataReaderBuilder.buildTableReaderFromHandles(ctx, tableReaderExec, handles) if err != nil { - log.Error(err) + log.Error("Build table reader from handles failed", zap.Error(err)) return nil, errors.Trace(err) } return tableReader, nil @@ -612,7 +613,7 @@ func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectRes buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] - log.Errorf("indexWorker panic stack is:\n%s", buf) + log.Error("Index worker panicked", zap.String("panic stack", string(buf))) err4Panic := errors.Errorf("%v", r) doneCh := make(chan error, 1) doneCh <- err4Panic @@ -713,7 +714,7 @@ 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) + log.Error("Table worker panicked", zap.String("panic stack", string(buf))) task.doneCh <- errors.Errorf("%v", r) } }() @@ -739,7 +740,7 @@ func (w *tableWorker) pickAndExecTask(ctx context.Context) { func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) error { tableReader, err := w.buildTblReader(ctx, task.handles) if err != nil { - log.Error(err) + log.Error("Build table reader failed", zap.Error(err)) return errors.Trace(err) } defer terror.Call(tableReader.Close) @@ -754,7 +755,7 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er chk := tableReader.newFirstChunk() err = tableReader.Next(ctx, chunk.NewRecordBatch(chk)) if err != nil { - log.Error(err) + log.Error("Table reader fetch next chunk failed", zap.Error(err)) return errors.Trace(err) } if chk.NumRows() == 0 { diff --git a/executor/executor.go b/executor/executor.go index 0b35afb995a2a..9eee4da1e8b24 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -25,6 +25,7 @@ import ( "github.com/cznic/mathutil" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -45,7 +46,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/memory" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) var ( @@ -466,7 +467,7 @@ func (e *CheckTableExec) Next(ctx context.Context, req *chunk.RecordBatch) error err = e.doCheckTable(tb) } if err != nil { - log.Warnf("%v error:%v", t.Name, errors.ErrorStack(err)) + log.Warn("Check table failed", zap.String("table name", t.Name.O), zap.String("error stack", errors.ErrorStack(err))) if admin.ErrDataInConsistent.Equal(err) { return ErrAdminCheckTable.GenWithStack("%v err:%v", t.Name, err) } @@ -1215,7 +1216,7 @@ func (e *UnionExec) resultPuller(ctx context.Context, childID int) { buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] - log.Errorf("resultPuller panic stack is:\n%s", buf) + log.Error("resultPuller panicked", zap.String("panic stack", string(buf))) result.err = errors.Errorf("%v", r) e.resultPool <- result e.stopFetchData.Store(true) diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 786fc17d17a44..b366d1ef25965 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -23,6 +23,7 @@ import ( "unsafe" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/expression" @@ -34,7 +35,7 @@ import ( "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/mvmap" "github.com/pingcap/tidb/util/ranger" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) var _ Executor = &IndexLookUpJoin{} @@ -305,7 +306,7 @@ func (ow *outerWorker) run(ctx context.Context, wg *sync.WaitGroup) { buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] - log.Errorf("outerWorker panic stack is:\n%s", buf) + log.Error("outerWorker panicked", zap.String("panic stack", string(buf))) task := &lookUpJoinTask{doneCh: make(chan error, 1)} task.doneCh <- errors.Errorf("%v", r) ow.pushToChan(ctx, task, ow.resultCh) @@ -407,7 +408,7 @@ func (iw *innerWorker) run(ctx context.Context, wg *sync.WaitGroup) { buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] - log.Errorf("innerWorker panic stack is:\n%s", buf) + log.Error("innerWorker panicked", zap.String("panic stack", string(buf))) // "task != nil" is guaranteed when panic happened. task.doneCh <- errors.Errorf("%v", r) } diff --git a/executor/insert.go b/executor/insert.go index fac4723471f91..ec6e5a94f5ab0 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -19,6 +19,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" @@ -26,7 +27,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) // InsertExec represents an insert executor. @@ -171,7 +172,7 @@ func (e *InsertExec) Open(ctx context.Context) error { func (e *InsertExec) updateDupRow(row toBeCheckedRow, handle int64, onDuplicate []*expression.Assignment) error { oldRow, err := e.getOldRow(e.ctx, e.Table, handle) if err != nil { - log.Errorf("[insert on dup] handle is %d for the to-be-inserted row %s", handle, types.DatumsToStrNoErr(row.row)) + log.Error("Get old row failed when insert on dup", zap.Int64("handle", handle), zap.String("to be inserted row", types.DatumsToStrNoErr(row.row))) return errors.Trace(err) } // Do update row. diff --git a/executor/insert_common.go b/executor/insert_common.go index a20268c7e4422..843b65cf071c1 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -18,6 +18,7 @@ import ( "fmt" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -26,7 +27,7 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) // InsertValues is the data to insert. @@ -221,7 +222,7 @@ func (e *InsertValues) handleErr(col *table.Column, val *types.Datum, rowIdx int if types.ErrTruncated.Equal(err) { valStr, err1 := val.ToString() if err1 != nil { - log.Warn(err1) + log.Warn("Truncate error", zap.Error(err1)) } return table.ErrTruncatedWrongValueForField.GenWithStackByArgs(types.TypeStr(col.Tp), valStr, col.Name.O, rowIdx+1) } diff --git a/executor/load_data.go b/executor/load_data.go index 4ad67808e0dfc..06c2746317599 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -19,13 +19,14 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) // LoadDataExec represents a load data executor. @@ -247,8 +248,8 @@ func (e *LoadDataInfo) InsertData(prevData, curData []byte) ([]byte, bool, error e.rowCount++ if e.maxRowsInBatch != 0 && e.rowCount%e.maxRowsInBatch == 0 { reachLimit = true - log.Infof("This insert rows has reached the batch %d, current total rows %d", - e.maxRowsInBatch, e.rowCount) + log.Info("This insert rows has reached the batch limit", zap.Int("max rows in batch", e.maxChunkSize), + zap.Uint64("current total rows", e.rowCount)) break } } diff --git a/executor/projection.go b/executor/projection.go index d7a5f314f1076..ac3147f29d451 100644 --- a/executor/projection.go +++ b/executor/projection.go @@ -15,16 +15,18 @@ package executor import ( "context" + "fmt" "sync/atomic" "time" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) // This file contains the implementation of the physical Projection Operator: @@ -378,7 +380,7 @@ func recoveryProjection(output *projectionOutput, r interface{}) { output.done <- errors.Errorf("%v", r) } buf := util.GetStack() - log.Errorf("panic in the recoverable goroutine: %v, stack trace:\n%s", r, buf) + log.Error("Projection executor panicked", zap.String("error", fmt.Sprintf("%v", r)), zap.String("stack trace", string(buf))) } func readProjectionInput(inputCh <-chan *projectionInput, finishCh <-chan struct{}) *projectionInput { diff --git a/executor/radix_hash_join.go b/executor/radix_hash_join.go index 7acbaf934ac18..22511b416c71f 100644 --- a/executor/radix_hash_join.go +++ b/executor/radix_hash_join.go @@ -21,11 +21,12 @@ import ( "unsafe" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mvmap" - log "github.com/sirupsen/logrus" "github.com/spaolacci/murmur3" + "go.uber.org/zap" ) var ( @@ -175,9 +176,9 @@ func (e *RadixHashJoinExec) preAlloc4InnerParts() (err error) { if e.numNonEmptyPart < len(e.innerParts) { numTotalPart := len(e.innerParts) numEmptyPart := numTotalPart - e.numNonEmptyPart - log.Debugf("[EMPTY_PART_IN_RADIX_HASH_JOIN] txn_start_ts:%v, num_empty_parts:%v, "+ - "num_total_parts:%v, empty_ratio:%v", e.ctx.GetSessionVars().TxnCtx.StartTS, - numEmptyPart, numTotalPart, float64(numEmptyPart)/float64(numTotalPart)) + log.Debug("Empty partition in radix hash join", zap.Uint64("txn_start_ts", e.ctx.GetSessionVars().TxnCtx.StartTS), + zap.Int("num_empty_parts", numEmptyPart), zap.Int("num_total_parts", numTotalPart), + zap.Float64("empty_ratio", float64(numEmptyPart)/float64(numTotalPart))) } return } diff --git a/executor/replace.go b/executor/replace.go index c83b6402ab9a6..7830a7c6d937e 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -19,12 +19,13 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) // ReplaceExec represents a replace executor. @@ -57,7 +58,7 @@ func (e *ReplaceExec) removeRow(handle int64, r toBeCheckedRow) (bool, error) { newRow := r.row oldRow, err := e.batchChecker.getOldRow(e.ctx, r.t, handle) if err != nil { - log.Errorf("[replace] handle is %d for the to-be-inserted row %v", handle, types.DatumsToStrNoErr(r.row)) + log.Error("Get old row failed when replace", zap.Int64("handle", handle), zap.String("to be inserted row", types.DatumsToStrNoErr(r.row))) return false, errors.Trace(err) } rowUnchanged, err := types.EqualDatums(e.ctx.GetSessionVars().StmtCtx, oldRow, newRow) diff --git a/executor/set.go b/executor/set.go index fe6711dbfbf67..41ed6ab01979b 100644 --- a/executor/set.go +++ b/executor/set.go @@ -19,6 +19,7 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/terror" @@ -28,7 +29,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/gcutil" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) // SetExecutor executes set statement. @@ -175,7 +176,7 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e valStr, err = value.ToString() terror.Log(errors.Trace(err)) } - log.Infof("con:%d %s=%s", sessionVars.ConnectionID, name, valStr) + log.Info("Set session var", zap.Uint64("con", sessionVars.ConnectionID), zap.String("name", name), zap.String("val", valStr)) } return nil @@ -226,7 +227,7 @@ func (e *SetExecutor) loadSnapshotInfoSchemaIfNeeded(name string) error { vars.SnapshotInfoschema = nil return nil } - log.Infof("con:%d loadSnapshotInfoSchema, SnapshotTS:%d", vars.ConnectionID, vars.SnapshotTS) + log.Info("Load snapshot info schema", zap.Uint64("con", vars.ConnectionID), zap.Uint64("SnapshotTS", vars.SnapshotTS)) dom := domain.GetDomain(e.ctx) snapInfo, err := dom.GetSnapshotInfoSchema(vars.SnapshotTS) if err != nil { diff --git a/executor/show_stats.go b/executor/show_stats.go index b5410c360fdcd..e40d9dde1a2a5 100644 --- a/executor/show_stats.go +++ b/executor/show_stats.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/types" - log "github.com/sirupsen/logrus" ) func (e *ShowExec) fetchShowStatsMeta() error { @@ -122,12 +121,12 @@ func (e *ShowExec) fetchShowStatsBuckets() error { pi := tbl.GetPartitionInfo() if pi == nil { if err := e.appendTableForStatsBuckets(db.Name.O, tbl.Name.O, "", h.GetTableStats(tbl)); err != nil { - log.Error(errors.ErrorStack(err)) + return err } } else { for _, def := range pi.Definitions { if err := e.appendTableForStatsBuckets(db.Name.O, tbl.Name.O, def.Name.O, h.GetPartitionStats(tbl, def.ID)); err != nil { - log.Error(errors.ErrorStack(err)) + return err } } } diff --git a/executor/simple.go b/executor/simple.go index 6a72297b6d705..88f371045c56d 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -19,6 +19,7 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/model" @@ -33,7 +34,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sqlexec" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) // SimpleExec represents simple statement executor. @@ -145,7 +146,7 @@ func (e *SimpleExec) executeCommit(s *ast.CommitStmt) { func (e *SimpleExec) executeRollback(s *ast.RollbackStmt) error { sessVars := e.ctx.GetSessionVars() - log.Debugf("con:%d execute rollback statement", sessVars.ConnectionID) + log.Debug("Execute rollback statement", zap.Uint64("con", sessVars.ConnectionID)) sessVars.SetStatusFlag(mysql.ServerStatusInTrans, false) txn, err := e.ctx.Txn(true) if err != nil { diff --git a/executor/write.go b/executor/write.go index 58e591a0bb424..f6e69fc30857d 100644 --- a/executor/write.go +++ b/executor/write.go @@ -17,6 +17,7 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" @@ -24,7 +25,7 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) var ( @@ -171,7 +172,7 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu // so we reset the error msg here, and wrap old err with errors.Wrap. func resetErrDataTooLong(colName string, rowIdx int, err error) error { newErr := types.ErrDataTooLong.GenWithStack("Data too long for column '%v' at row %v", colName, rowIdx) - log.Error(err) + log.Error("Data too long for column", zap.String("col name", colName), zap.Int("row index", rowIdx)) return errors.Trace(newErr) } From e022b85e936cb9d8062b88a3c2bb267e3b7ace41 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Mon, 11 Mar 2019 16:14:24 +0800 Subject: [PATCH 2/6] address comments --- executor/adapter.go | 4 ++-- executor/admin.go | 16 ++++++++-------- executor/aggregate.go | 6 +++--- executor/analyze.go | 8 ++++---- executor/checksum.go | 4 ++-- executor/compiler.go | 6 +++--- executor/ddl.go | 6 +++--- executor/distsql.go | 14 +++++++------- executor/executor.go | 6 +++--- executor/index_lookup_join.go | 6 +++--- executor/insert.go | 4 ++-- executor/insert_common.go | 6 +++--- executor/load_data.go | 6 +++--- executor/projection.go | 4 ++-- executor/radix_hash_join.go | 8 ++++---- executor/replace.go | 4 ++-- executor/set.go | 6 +++--- executor/simple.go | 4 ++-- executor/write.go | 5 +++-- 19 files changed, 62 insertions(+), 61 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 4ff4a1e9f66da..842a014960446 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -288,7 +288,7 @@ func (a *ExecStmt) handleNoDelayExecutor(ctx context.Context, sctx sessionctx.Co txnTS := uint64(0) // Don't active pending txn here. if txn, err1 := sctx.Txn(false); err1 != nil { - log.Error("Get current transaction failed", zap.Error(err)) + logutil.Logger(ctx).Error("get current transaction failed", zap.Error(err)) } else { if txn.Valid() { txnTS = txn.StartTS() @@ -315,7 +315,7 @@ func (a *ExecStmt) buildExecutor(ctx sessionctx.Context) (Executor, error) { return nil, errors.Trace(err) } if isPointGet { - log.Debug("InitTxnWithStartTS", zap.Uint64("con", ctx.GetSessionVars().ConnectionID), zap.String("text", a.Text)) + logutil.Logger(context.Background()).Debug("init txnStartTS with MaxUint64", zap.Uint64("conn", ctx.GetSessionVars().ConnectionID), zap.String("text", a.Text)) err = ctx.InitTxnWithStartTS(math.MaxUint64) } else if ctx.GetSessionVars().SnapshotTS != 0 { if _, ok := a.Plan.(*plannercore.CheckTable); ok { diff --git a/executor/admin.go b/executor/admin.go index 6d3deb9ad4f60..d746fbb3cf6b1 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -18,7 +18,6 @@ import ( "math" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/terror" @@ -32,6 +31,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-tipb" @@ -309,9 +309,9 @@ func (e *RecoverIndexExec) backfillIndex(ctx context.Context) (int64, int64, err totalScanCnt += result.scanRowCount if totalScanCnt-lastLogCnt >= 50000 { lastLogCnt = totalScanCnt - log.Info("Recover index", zap.String("table", e.table.Meta().Name.O), - zap.String("index", e.index.Meta().Name.O), zap.Int64("total added cnt", totalAddedCnt), - zap.Int64("total scan cnt", totalScanCnt), zap.Int64("next handle", result.nextHandle)) + logutil.Logger(ctx).Info("recover index", zap.String("table", e.table.Meta().Name.O), + zap.String("index", e.index.Meta().Name.O), zap.Int64("totalAddedCnt", totalAddedCnt), + zap.Int64("totalScanCnt", totalScanCnt), zap.Int64("nextHandle", result.nextHandle)) } // no more rows @@ -395,9 +395,9 @@ func (e *RecoverIndexExec) batchMarkDup(txn kv.Transaction, rows []recoverRows) } if handle != rows[i].handle { - log.Warn("The constraint of unique index is broken when recovering index", - zap.String("index name", e.index.Meta().Name.O), zap.ByteString("key value", key), - zap.Int64("handle", rows[i].handle), zap.Int64("another handle", handle)) + logutil.Logger(context.Background()).Warn("the constraint of unique index is broken when recovering index", + zap.String("indexName", e.index.Meta().Name.O), zap.ByteString("key", key), + zap.Int64("handle", rows[i].handle), zap.Int64("anotherHandle", handle)) } } rows[i].skip = true @@ -523,7 +523,7 @@ func (e *CleanupIndexExec) deleteDanglingIdx(txn kv.Transaction, values map[stri } e.removeCnt++ if e.removeCnt%e.batchSize == 0 { - log.Info("Cleaning up dangling index", zap.String("table", e.table.Meta().Name.String()), + logutil.Logger(context.Background()).Info("cleaning up dangling index", zap.String("table", e.table.Meta().Name.String()), zap.String("index", e.index.Meta().Name.String()), zap.Uint64("count", e.removeCnt)) } } diff --git a/executor/aggregate.go b/executor/aggregate.go index bbd18e63bca31..f8ce64142b2cf 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -20,7 +20,6 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/executor/aggfuncs" "github.com/pingcap/tidb/expression" @@ -30,6 +29,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/set" "github.com/spaolacci/murmur3" "go.uber.org/zap" @@ -321,7 +321,7 @@ func recoveryHashAgg(output chan *AfFinalResult, r interface{}) { err := errors.Errorf("%v", r) output <- &AfFinalResult{err: err} buf := util.GetStack() - log.Error("Panic in the recoverable goroutine", zap.Error(err), zap.String("stack trace", string(buf))) + logutil.Logger(context.Background()).Error("panic in the recoverable goroutine", zap.Error(err), zap.String("stack", string(buf))) } func (w *HashAggPartialWorker) run(ctx sessionctx.Context, waitGroup *sync.WaitGroup, finalConcurrency int) { @@ -475,7 +475,7 @@ func (w *HashAggFinalWorker) getFinalResult(sctx sessionctx.Context) { partialResults := w.getPartialResult(sctx.GetSessionVars().StmtCtx, []byte(groupKey), w.partialResultMap) for i, af := range w.aggFuncs { if err := af.AppendFinalResult2Chunk(sctx, partialResults[i], result); err != nil { - log.Error("Append final result failed", zap.String("error stack", errors.ErrorStack(err))) + logutil.Logger(context.Background()).Error("append final result failed", zap.Error(err)) } } if len(w.aggFuncs) == 0 { diff --git a/executor/analyze.go b/executor/analyze.go index ed8d79cd3cc30..1c934b2e368d7 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -19,7 +19,6 @@ import ( "strconv" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/distsql" @@ -30,6 +29,7 @@ import ( "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" @@ -74,7 +74,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.RecordBatch) error { if errors.Trace(err) == errAnalyzeWorkerPanic { panicCnt++ } else { - log.Error("Analyze failed", zap.String("error stack", errors.ErrorStack(err))) + logutil.Logger(ctx).Error("analyze failed", zap.Error(err)) } continue } @@ -82,7 +82,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.RecordBatch) error { err1 := statsHandle.SaveStatsToStorage(result.PhysicalTableID, result.Count, result.IsIndex, hg, result.Cms[i], 1) if err1 != nil { err = err1 - log.Error("Save stats to storage failed", zap.String("error stack", errors.ErrorStack(err))) + logutil.Logger(ctx).Error("save stats to storage failed", zap.Error(err)) continue } } @@ -124,7 +124,7 @@ func (e *AnalyzeExec) analyzeWorker(taskCh <-chan *analyzeTask, resultCh chan<- buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] - log.Error("Analyze worker panicked", zap.String("panic stack", string(buf))) + logutil.Logger(context.Background()).Error("analyze worker panicked", zap.String("stack", string(buf))) metrics.PanicCounter.WithLabelValues(metrics.LabelAnalyze).Inc() resultCh <- statistics.AnalyzeResult{ Err: errAnalyzeWorkerPanic, diff --git a/executor/checksum.go b/executor/checksum.go index 25b7aecda4e50..889b40c40f924 100644 --- a/executor/checksum.go +++ b/executor/checksum.go @@ -18,13 +18,13 @@ import ( "strconv" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" @@ -71,7 +71,7 @@ func (e *ChecksumTableExec) Open(ctx context.Context) error { result := <-resultCh if result.Error != nil { err = result.Error - log.Error("Checksum failed", zap.String("error stack", errors.ErrorStack(err))) + logutil.Logger(ctx).Error("checksum failed", zap.Error(err)) continue } e.handleResult(result) diff --git a/executor/compiler.go b/executor/compiler.go index 0a6f21a726e5a..07472386b7815 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -19,7 +19,6 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/infoschema" @@ -27,6 +26,7 @@ import ( "github.com/pingcap/tidb/planner" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) @@ -77,7 +77,7 @@ func logExpensiveQuery(stmtNode ast.StmtNode, finalPlan plannercore.Plan) (expen if len(sql) > logSQLLen { sql = fmt.Sprintf("%s len(%d)", sql[:logSQLLen], len(sql)) } - log.Warn("[EXPENSIVE_QUERY]", zap.String("sql", sql)) + logutil.Logger(context.Background()).Warn("[EXPENSIVE_QUERY]", zap.String("sql", sql)) return } @@ -306,7 +306,7 @@ func GetInfoSchema(ctx sessionctx.Context) infoschema.InfoSchema { var is infoschema.InfoSchema if snap := sessVar.SnapshotInfoschema; snap != nil { is = snap.(infoschema.InfoSchema) - log.Info("Use snapshot schema", zap.Uint64("con", sessVar.ConnectionID), zap.Int64("schema meta version", is.SchemaMetaVersion())) + logutil.Logger(context.Background()).Info("use snapshot schema", zap.Uint64("conn", sessVar.ConnectionID), zap.Int64("schemaVersion", is.SchemaMetaVersion())) } else { is = sessVar.TxnCtx.InfoSchema.(infoschema.InfoSchema) } diff --git a/executor/ddl.go b/executor/ddl.go index c0e5b7398498f..e920c09c4f27a 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -19,7 +19,6 @@ import ( "strings" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -33,6 +32,7 @@ import ( "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/gcutil" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" "go.uber.org/zap" ) @@ -61,7 +61,7 @@ func (e *DDLExec) toErr(err error) error { checker := domain.NewSchemaChecker(dom, e.is.SchemaMetaVersion(), nil) txn, err1 := e.ctx.Txn(true) if err1 != nil { - log.Error("Active txn failed", zap.Error(err)) + logutil.Logger(context.Background()).Error("active txn failed", zap.Error(err)) return errors.Trace(err1) } schemaInfoErr := checker.Check(txn.StartTS()) @@ -253,7 +253,7 @@ func (e *DDLExec) executeDropTableOrView(s *ast.DropTableStmt) error { } if config.CheckTableBeforeDrop { - log.Warn("Admin check table before drop", zap.String("table name", fmt.Sprintf("`%s`.`%s`", fullti.Schema.O, fullti.Name.O))) + logutil.Logger(context.Background()).Warn("admin check table before drop", zap.String("tableName", fmt.Sprintf("`%s`.`%s`", fullti.Schema.O, fullti.Name.O))) sql := fmt.Sprintf("admin check table `%s`.`%s`", fullti.Schema.O, fullti.Name.O) _, _, err = e.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(e.ctx, sql) if err != nil { diff --git a/executor/distsql.go b/executor/distsql.go index 3d902a98f9011..76222a310e1d4 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -25,7 +25,6 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" @@ -39,6 +38,7 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tipb/go-tipb" @@ -465,7 +465,7 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, kvRanges []k } cancel() if err := result.Close(); err != nil { - log.Error("Close Select result failed", zap.String("error stack", errors.ErrorStack(err))) + logutil.Logger(ctx).Error("close Select result failed", zap.Error(err)) } if e.runtimeStats != nil { copStats := e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.idxPlans[len(e.idxPlans)-1].ExplainID()) @@ -517,7 +517,7 @@ func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, handles []in } tableReader, err := e.dataReaderBuilder.buildTableReaderFromHandles(ctx, tableReaderExec, handles) if err != nil { - log.Error("Build table reader from handles failed", zap.Error(err)) + logutil.Logger(ctx).Error("build table reader from handles failed", zap.Error(err)) return nil, errors.Trace(err) } return tableReader, nil @@ -613,7 +613,7 @@ func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectRes buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] - log.Error("Index worker panicked", zap.String("panic stack", string(buf))) + logutil.Logger(ctx).Error("index worker panicked", zap.String("stack", string(buf))) err4Panic := errors.Errorf("%v", r) doneCh := make(chan error, 1) doneCh <- err4Panic @@ -714,7 +714,7 @@ func (w *tableWorker) pickAndExecTask(ctx context.Context) { buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] - log.Error("Table worker panicked", zap.String("panic stack", string(buf))) + logutil.Logger(ctx).Error("table worker panicked", zap.String("stack", string(buf))) task.doneCh <- errors.Errorf("%v", r) } }() @@ -740,7 +740,7 @@ func (w *tableWorker) pickAndExecTask(ctx context.Context) { func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) error { tableReader, err := w.buildTblReader(ctx, task.handles) if err != nil { - log.Error("Build table reader failed", zap.Error(err)) + logutil.Logger(ctx).Error("build table reader failed", zap.Error(err)) return errors.Trace(err) } defer terror.Call(tableReader.Close) @@ -755,7 +755,7 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er chk := tableReader.newFirstChunk() err = tableReader.Next(ctx, chunk.NewRecordBatch(chk)) if err != nil { - log.Error("Table reader fetch next chunk failed", zap.Error(err)) + logutil.Logger(ctx).Error("table reader fetch next chunk failed", zap.Error(err)) return errors.Trace(err) } if chk.NumRows() == 0 { diff --git a/executor/executor.go b/executor/executor.go index 9eee4da1e8b24..3ada6743ebfee 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -25,7 +25,6 @@ import ( "github.com/cznic/mathutil" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -45,6 +44,7 @@ import ( "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "go.uber.org/zap" ) @@ -467,7 +467,7 @@ func (e *CheckTableExec) Next(ctx context.Context, req *chunk.RecordBatch) error err = e.doCheckTable(tb) } if err != nil { - log.Warn("Check table failed", zap.String("table name", t.Name.O), zap.String("error stack", errors.ErrorStack(err))) + logutil.Logger(ctx).Warn("check table failed", zap.String("tableName", t.Name.O), zap.Error(err)) if admin.ErrDataInConsistent.Equal(err) { return ErrAdminCheckTable.GenWithStack("%v err:%v", t.Name, err) } @@ -1216,7 +1216,7 @@ func (e *UnionExec) resultPuller(ctx context.Context, childID int) { buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] - log.Error("resultPuller panicked", zap.String("panic stack", string(buf))) + logutil.Logger(ctx).Error("resultPuller panicked", zap.String("stack", string(buf))) result.err = errors.Errorf("%v", r) e.resultPool <- result e.stopFetchData.Store(true) diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index b366d1ef25965..a9a2b4e4f25a1 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -23,7 +23,6 @@ import ( "unsafe" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/expression" @@ -32,6 +31,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/mvmap" "github.com/pingcap/tidb/util/ranger" @@ -306,7 +306,7 @@ func (ow *outerWorker) run(ctx context.Context, wg *sync.WaitGroup) { buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] - log.Error("outerWorker panicked", zap.String("panic stack", string(buf))) + logutil.Logger(ctx).Error("outerWorker panicked", zap.String("stack", string(buf))) task := &lookUpJoinTask{doneCh: make(chan error, 1)} task.doneCh <- errors.Errorf("%v", r) ow.pushToChan(ctx, task, ow.resultCh) @@ -408,7 +408,7 @@ func (iw *innerWorker) run(ctx context.Context, wg *sync.WaitGroup) { buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] - log.Error("innerWorker panicked", zap.String("panic stack", string(buf))) + logutil.Logger(ctx).Error("innerWorker panicked", zap.String("stack", string(buf))) // "task != nil" is guaranteed when panic happened. task.doneCh <- errors.Errorf("%v", r) } diff --git a/executor/insert.go b/executor/insert.go index ec6e5a94f5ab0..8af65ba73f6be 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -19,7 +19,6 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" @@ -27,6 +26,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) @@ -172,7 +172,7 @@ func (e *InsertExec) Open(ctx context.Context) error { func (e *InsertExec) updateDupRow(row toBeCheckedRow, handle int64, onDuplicate []*expression.Assignment) error { oldRow, err := e.getOldRow(e.ctx, e.Table, handle) if err != nil { - log.Error("Get old row failed when insert on dup", zap.Int64("handle", handle), zap.String("to be inserted row", types.DatumsToStrNoErr(row.row))) + logutil.Logger(context.Background()).Error("get old row failed when insert on dup", zap.Int64("handle", handle), zap.String("toBeInsertedRow", types.DatumsToStrNoErr(row.row))) return errors.Trace(err) } // Do update row. diff --git a/executor/insert_common.go b/executor/insert_common.go index 54dd345c0f76f..1326724c4db85 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -18,7 +18,6 @@ import ( "fmt" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -27,6 +26,7 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) @@ -220,7 +220,7 @@ func (e *InsertValues) handleErr(col *table.Column, val *types.Datum, rowIdx int if types.ErrTruncated.Equal(err) { valStr, err1 := val.ToString() if err1 != nil { - log.Warn("Truncate error", zap.Error(err1)) + logutil.Logger(context.Background()).Warn("truncate error", zap.Error(err1)) } return table.ErrTruncatedWrongValueForField.GenWithStackByArgs(types.TypeStr(col.Tp), valStr, col.Name.O, rowIdx+1) } @@ -525,7 +525,7 @@ func (e *InsertValues) adjustAutoIncrementDatum(d types.Datum, hasValue bool, c func (e *InsertValues) handleWarning(err error, logInfo string) { sc := e.ctx.GetSessionVars().StmtCtx sc.AppendWarning(err) - log.Warn(logInfo) + logutil.Logger(context.Background()).Warn(logInfo) } // batchCheckAndInsert checks rows with duplicate errors. diff --git a/executor/load_data.go b/executor/load_data.go index 06c2746317599..2fddacc893c5b 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -19,13 +19,13 @@ import ( "strings" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) @@ -248,8 +248,8 @@ func (e *LoadDataInfo) InsertData(prevData, curData []byte) ([]byte, bool, error e.rowCount++ if e.maxRowsInBatch != 0 && e.rowCount%e.maxRowsInBatch == 0 { reachLimit = true - log.Info("This insert rows has reached the batch limit", zap.Int("max rows in batch", e.maxChunkSize), - zap.Uint64("current total rows", e.rowCount)) + logutil.Logger(context.Background()).Info("this insert rows has reached the batch limit", zap.Int("maxBatchRows", e.maxChunkSize), + zap.Uint64("totalRows", e.rowCount)) break } } diff --git a/executor/projection.go b/executor/projection.go index ac3147f29d451..5458165e0e6f9 100644 --- a/executor/projection.go +++ b/executor/projection.go @@ -21,11 +21,11 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) @@ -380,7 +380,7 @@ func recoveryProjection(output *projectionOutput, r interface{}) { output.done <- errors.Errorf("%v", r) } buf := util.GetStack() - log.Error("Projection executor panicked", zap.String("error", fmt.Sprintf("%v", r)), zap.String("stack trace", string(buf))) + logutil.Logger(context.Background()).Error("projection executor panicked", zap.String("error", fmt.Sprintf("%v", r)), zap.String("stack", string(buf))) } func readProjectionInput(inputCh <-chan *projectionInput, finishCh <-chan struct{}) *projectionInput { diff --git a/executor/radix_hash_join.go b/executor/radix_hash_join.go index 22511b416c71f..cc0633f391bb8 100644 --- a/executor/radix_hash_join.go +++ b/executor/radix_hash_join.go @@ -21,9 +21,9 @@ import ( "unsafe" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mvmap" "github.com/spaolacci/murmur3" "go.uber.org/zap" @@ -176,9 +176,9 @@ func (e *RadixHashJoinExec) preAlloc4InnerParts() (err error) { if e.numNonEmptyPart < len(e.innerParts) { numTotalPart := len(e.innerParts) numEmptyPart := numTotalPart - e.numNonEmptyPart - log.Debug("Empty partition in radix hash join", zap.Uint64("txn_start_ts", e.ctx.GetSessionVars().TxnCtx.StartTS), - zap.Int("num_empty_parts", numEmptyPart), zap.Int("num_total_parts", numTotalPart), - zap.Float64("empty_ratio", float64(numEmptyPart)/float64(numTotalPart))) + logutil.Logger(context.Background()).Debug("empty partition in radix hash join", zap.Uint64("txnStartTS", e.ctx.GetSessionVars().TxnCtx.StartTS), + zap.Int("numEmptyParts", numEmptyPart), zap.Int("numTotalParts", numTotalPart), + zap.Float64("emptyRatio", float64(numEmptyPart)/float64(numTotalPart))) } return } diff --git a/executor/replace.go b/executor/replace.go index 7830a7c6d937e..48ac8883dea2a 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -19,12 +19,12 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) @@ -58,7 +58,7 @@ func (e *ReplaceExec) removeRow(handle int64, r toBeCheckedRow) (bool, error) { newRow := r.row oldRow, err := e.batchChecker.getOldRow(e.ctx, r.t, handle) if err != nil { - log.Error("Get old row failed when replace", zap.Int64("handle", handle), zap.String("to be inserted row", types.DatumsToStrNoErr(r.row))) + logutil.Logger(context.Background()).Error("get old row failed when replace", zap.Int64("handle", handle), zap.String("toBeInsertedRow", types.DatumsToStrNoErr(r.row))) return false, errors.Trace(err) } rowUnchanged, err := types.EqualDatums(e.ctx.GetSessionVars().StmtCtx, oldRow, newRow) diff --git a/executor/set.go b/executor/set.go index 41ed6ab01979b..a5b2a7f6a7d80 100644 --- a/executor/set.go +++ b/executor/set.go @@ -19,7 +19,6 @@ import ( "strings" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/terror" @@ -29,6 +28,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/gcutil" + "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) @@ -176,7 +176,7 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e valStr, err = value.ToString() terror.Log(errors.Trace(err)) } - log.Info("Set session var", zap.Uint64("con", sessionVars.ConnectionID), zap.String("name", name), zap.String("val", valStr)) + logutil.Logger(context.Background()).Info("set session var", zap.Uint64("conn", sessionVars.ConnectionID), zap.String("name", name), zap.String("val", valStr)) } return nil @@ -227,7 +227,7 @@ func (e *SetExecutor) loadSnapshotInfoSchemaIfNeeded(name string) error { vars.SnapshotInfoschema = nil return nil } - log.Info("Load snapshot info schema", zap.Uint64("con", vars.ConnectionID), zap.Uint64("SnapshotTS", vars.SnapshotTS)) + logutil.Logger(context.Background()).Info("load snapshot info schema", zap.Uint64("conn", vars.ConnectionID), zap.Uint64("SnapshotTS", vars.SnapshotTS)) dom := domain.GetDomain(e.ctx) snapInfo, err := dom.GetSnapshotInfoSchema(vars.SnapshotTS) if err != nil { diff --git a/executor/simple.go b/executor/simple.go index 88f371045c56d..43d3815f75683 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -19,7 +19,6 @@ import ( "strings" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/model" @@ -33,6 +32,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" "go.uber.org/zap" ) @@ -146,7 +146,7 @@ func (e *SimpleExec) executeCommit(s *ast.CommitStmt) { func (e *SimpleExec) executeRollback(s *ast.RollbackStmt) error { sessVars := e.ctx.GetSessionVars() - log.Debug("Execute rollback statement", zap.Uint64("con", sessVars.ConnectionID)) + logutil.Logger(context.Background()).Debug("execute rollback statement", zap.Uint64("conn", sessVars.ConnectionID)) sessVars.SetStatusFlag(mysql.ServerStatusInTrans, false) txn, err := e.ctx.Txn(true) if err != nil { diff --git a/executor/write.go b/executor/write.go index f6e69fc30857d..97c67bbca0710 100644 --- a/executor/write.go +++ b/executor/write.go @@ -14,10 +14,10 @@ package executor import ( + "context" "strings" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) @@ -172,7 +173,7 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu // so we reset the error msg here, and wrap old err with errors.Wrap. func resetErrDataTooLong(colName string, rowIdx int, err error) error { newErr := types.ErrDataTooLong.GenWithStack("Data too long for column '%v' at row %v", colName, rowIdx) - log.Error("Data too long for column", zap.String("col name", colName), zap.Int("row index", rowIdx)) + logutil.Logger(context.Background()).Error("data too long for column", zap.String("colName", colName), zap.Int("rowIndex", rowIdx)) return errors.Trace(newErr) } From e8250b8eca472c7025605e8cde0358674371a829 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Tue, 12 Mar 2019 17:14:27 +0800 Subject: [PATCH 3/6] address comments --- executor/admin.go | 6 +++--- executor/aggregate.go | 8 +++----- executor/ddl.go | 5 ++++- 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/executor/admin.go b/executor/admin.go index d746fbb3cf6b1..386b515455b85 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -395,9 +395,9 @@ func (e *RecoverIndexExec) batchMarkDup(txn kv.Transaction, rows []recoverRows) } if handle != rows[i].handle { - logutil.Logger(context.Background()).Warn("the constraint of unique index is broken when recovering index", - zap.String("indexName", e.index.Meta().Name.O), zap.ByteString("key", key), - zap.Int64("handle", rows[i].handle), zap.Int64("anotherHandle", handle)) + logutil.Logger(context.Background()).Warn("recover index: the constraint of unique index is broken, handle in index is not equal to handle in table", + zap.String("indexName", e.index.Meta().Name.O), zap.ByteString("indexKey", key), + zap.Int64("handleInTable", rows[i].handle), zap.Int64("handleInIndex", handle)) } } rows[i].skip = true diff --git a/executor/aggregate.go b/executor/aggregate.go index f8ce64142b2cf..2c163704adc59 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/logutil" @@ -319,9 +318,8 @@ func (w *HashAggPartialWorker) getChildInput() bool { func recoveryHashAgg(output chan *AfFinalResult, r interface{}) { err := errors.Errorf("%v", r) - output <- &AfFinalResult{err: err} - buf := util.GetStack() - logutil.Logger(context.Background()).Error("panic in the recoverable goroutine", zap.Error(err), zap.String("stack", string(buf))) + output <- &AfFinalResult{err: errors.Errorf("%v", r)} + logutil.Logger(context.Background()).Error("panic in the recoverable goroutine", zap.Error(err)) } func (w *HashAggPartialWorker) run(ctx sessionctx.Context, waitGroup *sync.WaitGroup, finalConcurrency int) { @@ -475,7 +473,7 @@ func (w *HashAggFinalWorker) getFinalResult(sctx sessionctx.Context) { partialResults := w.getPartialResult(sctx.GetSessionVars().StmtCtx, []byte(groupKey), w.partialResultMap) for i, af := range w.aggFuncs { if err := af.AppendFinalResult2Chunk(sctx, partialResults[i], result); err != nil { - logutil.Logger(context.Background()).Error("append final result failed", zap.Error(err)) + logutil.Logger(context.Background()).Error("HashAggFinalWorker append final result to Chunk failed", zap.Error(err)) } } if len(w.aggFuncs) == 0 { diff --git a/executor/ddl.go b/executor/ddl.go index e920c09c4f27a..68606d001c492 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -253,7 +253,10 @@ func (e *DDLExec) executeDropTableOrView(s *ast.DropTableStmt) error { } if config.CheckTableBeforeDrop { - logutil.Logger(context.Background()).Warn("admin check table before drop", zap.String("tableName", fmt.Sprintf("`%s`.`%s`", fullti.Schema.O, fullti.Name.O))) + logutil.Logger(context.Background()).Warn("admin check table before drop", + zap.String("database", fullti.Schema.O), + zap.String("table", fullti.Name.O), + ) sql := fmt.Sprintf("admin check table `%s`.`%s`", fullti.Schema.O, fullti.Name.O) _, _, err = e.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(e.ctx, sql) if err != nil { From 04eecfaf90fdc03662167323b17ae472868f5bda Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Wed, 13 Mar 2019 12:44:16 +0800 Subject: [PATCH 4/6] address comments --- executor/admin.go | 2 +- executor/aggregate.go | 2 +- executor/compiler.go | 2 +- executor/distsql.go | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/executor/admin.go b/executor/admin.go index 386b515455b85..bdf81cf0be9c0 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -396,7 +396,7 @@ func (e *RecoverIndexExec) batchMarkDup(txn kv.Transaction, rows []recoverRows) if handle != rows[i].handle { logutil.Logger(context.Background()).Warn("recover index: the constraint of unique index is broken, handle in index is not equal to handle in table", - zap.String("indexName", e.index.Meta().Name.O), zap.ByteString("indexKey", key), + zap.String("index", e.index.Meta().Name.O), zap.ByteString("indexKey", key), zap.Int64("handleInTable", rows[i].handle), zap.Int64("handleInIndex", handle)) } } diff --git a/executor/aggregate.go b/executor/aggregate.go index 2c163704adc59..909a77bca1fd6 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -319,7 +319,7 @@ func (w *HashAggPartialWorker) getChildInput() bool { func recoveryHashAgg(output chan *AfFinalResult, r interface{}) { err := errors.Errorf("%v", r) output <- &AfFinalResult{err: errors.Errorf("%v", r)} - logutil.Logger(context.Background()).Error("panic in the recoverable goroutine", zap.Error(err)) + logutil.Logger(context.Background()).Error("parallel hash aggregation panicked", zap.Error(err)) } func (w *HashAggPartialWorker) run(ctx sessionctx.Context, waitGroup *sync.WaitGroup, finalConcurrency int) { diff --git a/executor/compiler.go b/executor/compiler.go index 07472386b7815..b9bf046ae2d4d 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -77,7 +77,7 @@ func logExpensiveQuery(stmtNode ast.StmtNode, finalPlan plannercore.Plan) (expen if len(sql) > logSQLLen { sql = fmt.Sprintf("%s len(%d)", sql[:logSQLLen], len(sql)) } - logutil.Logger(context.Background()).Warn("[EXPENSIVE_QUERY]", zap.String("sql", sql)) + logutil.Logger(context.Background()).Warn("EXPENSIVE_QUERY", zap.String("SQL", sql)) return } diff --git a/executor/distsql.go b/executor/distsql.go index 76222a310e1d4..884c7275bdad0 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -613,7 +613,7 @@ func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectRes buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] - logutil.Logger(ctx).Error("index worker panicked", zap.String("stack", string(buf))) + logutil.Logger(ctx).Error("indexWorker in IndexLookupExecutor panicked", zap.String("stack", string(buf))) err4Panic := errors.Errorf("%v", r) doneCh := make(chan error, 1) doneCh <- err4Panic @@ -714,7 +714,7 @@ func (w *tableWorker) pickAndExecTask(ctx context.Context) { buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] - logutil.Logger(ctx).Error("table worker panicked", zap.String("stack", string(buf))) + logutil.Logger(ctx).Error("tableWorker in IndexLookUpExecutor panicked", zap.String("stack", string(buf))) task.doneCh <- errors.Errorf("%v", r) } }() From 74fc48744d9722e5df302559771e3bf49aff71f1 Mon Sep 17 00:00:00 2001 From: Kenan Yao Date: Thu, 14 Mar 2019 11:15:42 +0800 Subject: [PATCH 5/6] Apply suggestions from code review Co-Authored-By: lamxTyler --- executor/admin.go | 2 +- executor/aggregate.go | 2 +- executor/load_data.go | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/executor/admin.go b/executor/admin.go index bdf81cf0be9c0..36762feb23aa7 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -523,7 +523,7 @@ func (e *CleanupIndexExec) deleteDanglingIdx(txn kv.Transaction, values map[stri } e.removeCnt++ if e.removeCnt%e.batchSize == 0 { - logutil.Logger(context.Background()).Info("cleaning up dangling index", zap.String("table", e.table.Meta().Name.String()), + logutil.Logger(context.Background()).Info("clean up dangling index", zap.String("table", e.table.Meta().Name.String()), zap.String("index", e.index.Meta().Name.String()), zap.Uint64("count", e.removeCnt)) } } diff --git a/executor/aggregate.go b/executor/aggregate.go index 909a77bca1fd6..96e7e6c1dab16 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -473,7 +473,7 @@ func (w *HashAggFinalWorker) getFinalResult(sctx sessionctx.Context) { partialResults := w.getPartialResult(sctx.GetSessionVars().StmtCtx, []byte(groupKey), w.partialResultMap) for i, af := range w.aggFuncs { if err := af.AppendFinalResult2Chunk(sctx, partialResults[i], result); err != nil { - logutil.Logger(context.Background()).Error("HashAggFinalWorker append final result to Chunk failed", zap.Error(err)) + logutil.Logger(context.Background()).Error("HashAggFinalWorker failed to append final result to Chunk", zap.Error(err)) } } if len(w.aggFuncs) == 0 { diff --git a/executor/load_data.go b/executor/load_data.go index 2fddacc893c5b..cb09f6f76f2a6 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -248,7 +248,7 @@ func (e *LoadDataInfo) InsertData(prevData, curData []byte) ([]byte, bool, error e.rowCount++ if e.maxRowsInBatch != 0 && e.rowCount%e.maxRowsInBatch == 0 { reachLimit = true - logutil.Logger(context.Background()).Info("this insert rows has reached the batch limit", zap.Int("maxBatchRows", e.maxChunkSize), + logutil.Logger(context.Background()).Info("batch limit hit when inserting rows", zap.Int("maxBatchRows", e.maxChunkSize), zap.Uint64("totalRows", e.rowCount)) break } From 9cfde5d466e22b7e56ab9d7a4b4f954c6b977594 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Thu, 14 Mar 2019 14:28:21 +0800 Subject: [PATCH 6/6] fix merge error --- executor/adapter.go | 8 +++----- util/logutil/log.go | 4 ++-- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 7527798beac35..384bb2ecfd101 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -390,13 +390,11 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { } execDetail := sessVars.StmtCtx.GetExecDetails() if costTime < threshold { - if logutil.SlowQueryLogger.IsLevelEnabled(log.DebugLevel) { - _, digest := sessVars.StmtCtx.SQLDigest() - logutil.SlowQueryLogger.Debugf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, sql)) - } + _, digest := sessVars.StmtCtx.SQLDigest() + logutil.SlowQueryZapLogger.Debug(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, sql)) } else { _, digest := sessVars.StmtCtx.SQLDigest() - logutil.SlowQueryLogger.Warnf(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, sql)) + logutil.SlowQueryZapLogger.Warn(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, sql)) metrics.TotalQueryProcHistogram.Observe(costTime.Seconds()) metrics.TotalCopProcHistogram.Observe(execDetail.ProcessTime.Seconds()) metrics.TotalCopWaitHistogram.Observe(execDetail.WaitTime.Seconds()) diff --git a/util/logutil/log.go b/util/logutil/log.go index edb626b3d20a6..4e6a7fc7ce7a3 100644 --- a/util/logutil/log.go +++ b/util/logutil/log.go @@ -284,8 +284,8 @@ func initFileLog(cfg *zaplog.FileLogConfig, logger *log.Logger) error { // SlowQueryLogger is used to log slow query, InitLogger will modify it according to config file. var SlowQueryLogger = log.StandardLogger() -// SlowQueryZapLogger is used to log slow query, InitZapLogger will set it according to config file. -var SlowQueryZapLogger *zap.Logger +// SlowQueryZapLogger is used to log slow query, InitZapLogger will modify it according to config file. +var SlowQueryZapLogger = zaplog.L() // InitLogger initializes PD's logger. func InitLogger(cfg *LogConfig) error {