executor: replace logger with zap logger (#9521)
This commit is contained in:
@ -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)
|
||||
logutil.Logger(ctx).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)
|
||||
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 {
|
||||
@ -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
|
||||
@ -388,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())
|
||||
|
||||
@ -31,10 +31,11 @@ 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"
|
||||
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)
|
||||
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
|
||||
@ -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)
|
||||
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("index", e.index.Meta().Name.O), zap.ByteString("indexKey", key),
|
||||
zap.Int64("handleInTable", rows[i].handle), zap.Int64("handleInIndex", 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)
|
||||
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))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -26,12 +26,12 @@ 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"
|
||||
"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
|
||||
@ -317,9 +317,9 @@ func (w *HashAggPartialWorker) getChildInput() bool {
|
||||
}
|
||||
|
||||
func recoveryHashAgg(output chan *AfFinalResult, r interface{}) {
|
||||
err := errors.Errorf("%v", r)
|
||||
output <- &AfFinalResult{err: errors.Errorf("%v", r)}
|
||||
buf := util.GetStack()
|
||||
log.Errorf("panic in the recoverable goroutine: %v, stack trace:\n%s", r, buf)
|
||||
logutil.Logger(context.Background()).Error("parallel hash aggregation panicked", zap.Error(err))
|
||||
}
|
||||
|
||||
func (w *HashAggPartialWorker) run(ctx sessionctx.Context, waitGroup *sync.WaitGroup, finalConcurrency int) {
|
||||
@ -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 {
|
||||
log.Error(errors.ErrorStack(err))
|
||||
logutil.Logger(context.Background()).Error("HashAggFinalWorker failed to append final result to Chunk", zap.Error(err))
|
||||
}
|
||||
}
|
||||
if len(w.aggFuncs) == 0 {
|
||||
|
||||
@ -29,9 +29,10 @@ 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"
|
||||
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 {
|
||||
logutil.Logger(ctx).Error("analyze failed", zap.Error(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))
|
||||
logutil.Logger(ctx).Error("save stats to storage failed", zap.Error(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)
|
||||
logutil.Logger(context.Background()).Error("analyze worker panicked", zap.String("stack", string(buf)))
|
||||
metrics.PanicCounter.WithLabelValues(metrics.LabelAnalyze).Inc()
|
||||
resultCh <- statistics.AnalyzeResult{
|
||||
Err: errAnalyzeWorkerPanic,
|
||||
|
||||
@ -24,9 +24,10 @@ 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/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))
|
||||
logutil.Logger(ctx).Error("checksum failed", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
e.handleResult(result)
|
||||
|
||||
@ -26,7 +26,8 @@ import (
|
||||
"github.com/pingcap/tidb/planner"
|
||||
plannercore "github.com/pingcap/tidb/planner/core"
|
||||
"github.com/pingcap/tidb/sessionctx"
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/pingcap/tidb/util/logutil"
|
||||
"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)
|
||||
logutil.Logger(context.Background()).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())
|
||||
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)
|
||||
}
|
||||
|
||||
@ -32,8 +32,9 @@ 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"
|
||||
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)
|
||||
logutil.Logger(context.Background()).Error("active txn failed", zap.Error(err))
|
||||
return errors.Trace(err1)
|
||||
}
|
||||
schemaInfoErr := checker.Check(txn.StartTS())
|
||||
@ -252,7 +253,10 @@ 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)
|
||||
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 {
|
||||
|
||||
@ -38,10 +38,11 @@ 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"
|
||||
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))
|
||||
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())
|
||||
@ -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)
|
||||
logutil.Logger(ctx).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)
|
||||
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
|
||||
@ -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)
|
||||
logutil.Logger(ctx).Error("tableWorker in IndexLookUpExecutor panicked", zap.String("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)
|
||||
logutil.Logger(ctx).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)
|
||||
logutil.Logger(ctx).Error("table reader fetch next chunk failed", zap.Error(err))
|
||||
return errors.Trace(err)
|
||||
}
|
||||
if chk.NumRows() == 0 {
|
||||
|
||||
@ -44,8 +44,9 @@ 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"
|
||||
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))
|
||||
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 +1217,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)
|
||||
logutil.Logger(ctx).Error("resultPuller panicked", zap.String("stack", string(buf)))
|
||||
result.err = errors.Errorf("%v", r)
|
||||
e.resultPool <- result
|
||||
e.stopFetchData.Store(true)
|
||||
|
||||
@ -31,10 +31,11 @@ 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"
|
||||
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)
|
||||
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)
|
||||
@ -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)
|
||||
logutil.Logger(ctx).Error("innerWorker panicked", zap.String("stack", string(buf)))
|
||||
// "task != nil" is guaranteed when panic happened.
|
||||
task.doneCh <- errors.Errorf("%v", r)
|
||||
}
|
||||
|
||||
@ -26,7 +26,8 @@ import (
|
||||
"github.com/pingcap/tidb/tablecodec"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/chunk"
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/pingcap/tidb/util/logutil"
|
||||
"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))
|
||||
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.
|
||||
|
||||
@ -26,7 +26,8 @@ import (
|
||||
"github.com/pingcap/tidb/table"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/chunk"
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/pingcap/tidb/util/logutil"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// InsertValues is the data to insert.
|
||||
@ -219,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(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)
|
||||
}
|
||||
@ -524,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.
|
||||
|
||||
@ -25,7 +25,8 @@ import (
|
||||
"github.com/pingcap/tidb/table"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/chunk"
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/pingcap/tidb/util/logutil"
|
||||
"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)
|
||||
logutil.Logger(context.Background()).Info("batch limit hit when inserting rows", zap.Int("maxBatchRows", e.maxChunkSize),
|
||||
zap.Uint64("totalRows", e.rowCount))
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
@ -15,6 +15,7 @@ package executor
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
@ -24,7 +25,8 @@ import (
|
||||
"github.com/pingcap/tidb/sessionctx"
|
||||
"github.com/pingcap/tidb/util"
|
||||
"github.com/pingcap/tidb/util/chunk"
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/pingcap/tidb/util/logutil"
|
||||
"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)
|
||||
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 {
|
||||
|
||||
@ -23,9 +23,10 @@ import (
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/tidb/util"
|
||||
"github.com/pingcap/tidb/util/chunk"
|
||||
"github.com/pingcap/tidb/util/logutil"
|
||||
"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))
|
||||
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
|
||||
}
|
||||
|
||||
@ -24,7 +24,8 @@ import (
|
||||
"github.com/pingcap/tidb/tablecodec"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/chunk"
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/pingcap/tidb/util/logutil"
|
||||
"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))
|
||||
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)
|
||||
|
||||
@ -28,7 +28,8 @@ import (
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/chunk"
|
||||
"github.com/pingcap/tidb/util/gcutil"
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/pingcap/tidb/util/logutil"
|
||||
"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)
|
||||
logutil.Logger(context.Background()).Info("set session var", zap.Uint64("conn", 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)
|
||||
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 {
|
||||
|
||||
@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -32,8 +32,9 @@ 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"
|
||||
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)
|
||||
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 {
|
||||
|
||||
@ -14,6 +14,7 @@
|
||||
package executor
|
||||
|
||||
import (
|
||||
"context"
|
||||
"strings"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
@ -24,7 +25,8 @@ import (
|
||||
"github.com/pingcap/tidb/table"
|
||||
"github.com/pingcap/tidb/table/tables"
|
||||
"github.com/pingcap/tidb/types"
|
||||
log "github.com/sirupsen/logrus"
|
||||
"github.com/pingcap/tidb/util/logutil"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
var (
|
||||
@ -171,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(err)
|
||||
logutil.Logger(context.Background()).Error("data too long for column", zap.String("colName", colName), zap.Int("rowIndex", rowIdx))
|
||||
return errors.Trace(newErr)
|
||||
}
|
||||
|
||||
|
||||
@ -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 {
|
||||
|
||||
Reference in New Issue
Block a user