Skip to content

Commit

Permalink
Showing 22 changed files with 88 additions and 52 deletions.
1 change: 1 addition & 0 deletions pkg/domain/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -54,6 +54,7 @@ go_library(
"//pkg/sessionctx/sessionstates",
"//pkg/sessionctx/variable",
"//pkg/statistics/handle",
"//pkg/statistics/handle/logutil",
"//pkg/statistics/handle/util",
"//pkg/store/helper",
"//pkg/telemetry",
3 changes: 2 additions & 1 deletion pkg/domain/domain.go
Original file line number Diff line number Diff line change
@@ -64,6 +64,7 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx/sessionstates"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/statistics/handle"
statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil"
"github.com/pingcap/tidb/pkg/store/helper"
"github.com/pingcap/tidb/pkg/telemetry"
"github.com/pingcap/tidb/pkg/ttl/ttlworker"
@@ -2375,7 +2376,7 @@ func (do *Domain) syncIndexUsageWorker(owner owner.Manager) {
continue
}
if err := handle.GCIndexUsage(); err != nil {
logutil.BgLogger().Error("gc index usage failed", zap.String("category", "stats"), zap.Error(err))
statslogutil.StatsLogger.Error("gc index usage failed", zap.Error(err))
}
}
}
1 change: 1 addition & 0 deletions pkg/statistics/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -34,6 +34,7 @@ go_library(
"//pkg/sessionctx",
"//pkg/sessionctx/stmtctx",
"//pkg/sessionctx/variable",
"//pkg/statistics/handle/logutil",
"//pkg/tablecodec",
"//pkg/types",
"//pkg/util/chunk",
10 changes: 3 additions & 7 deletions pkg/statistics/builder.go
Original file line number Diff line number Diff line change
@@ -21,10 +21,10 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/codec"
"github.com/pingcap/tidb/pkg/util/collate"
"github.com/pingcap/tidb/pkg/util/logutil"
"github.com/pingcap/tidb/pkg/util/memory"
"go.uber.org/zap"
)
@@ -387,14 +387,10 @@ func BuildHistAndTopN(
if foundTwice {
datumString, err := firstTimeSample.ToString()
if err != nil {
logutil.BgLogger().With(
zap.String("category", "stats"),
).Error("try to convert datum to string failed", zap.Error(err))
statslogutil.StatsLogger.Error("try to convert datum to string failed", zap.Error(err))
}

logutil.BgLogger().With(
zap.String("category", "stats"),
).Warn(
statslogutil.StatsLogger.Warn(
"invalid sample data",
zap.Bool("isColumn", isColumn),
zap.Int64("columnID", id),
1 change: 1 addition & 0 deletions pkg/statistics/handle/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -26,6 +26,7 @@ go_library(
"//pkg/statistics/handle/globalstats",
"//pkg/statistics/handle/history",
"//pkg/statistics/handle/lockstats",
"//pkg/statistics/handle/logutil",
"//pkg/statistics/handle/storage",
"//pkg/statistics/handle/syncload",
"//pkg/statistics/handle/usage",
1 change: 1 addition & 0 deletions pkg/statistics/handle/autoanalyze/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -13,6 +13,7 @@ go_library(
"//pkg/sessionctx",
"//pkg/sessionctx/variable",
"//pkg/statistics",
"//pkg/statistics/handle/logutil",
"//pkg/statistics/handle/util",
"//pkg/table",
"//pkg/types",
21 changes: 11 additions & 10 deletions pkg/statistics/handle/autoanalyze/autoanalyze.go
Original file line number Diff line number Diff line change
@@ -31,6 +31,7 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/statistics"
statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil"
statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util"
"github.com/pingcap/tidb/pkg/table"
"github.com/pingcap/tidb/pkg/types"
@@ -155,7 +156,7 @@ func HandleAutoAnalyze(
) (analyzed bool) {
defer func() {
if r := recover(); r != nil {
statsutil.StatsLogger.Error(
statslogutil.StatsLogger.Error(
"HandleAutoAnalyze panicked",
zap.Any("recover", r),
zap.Stack("stack"),
@@ -173,7 +174,7 @@ func HandleAutoAnalyze(
parameters[variable.TiDBAutoAnalyzeEndTime],
)
if err != nil {
statsutil.StatsLogger.Error(
statslogutil.StatsLogger.Error(
"parse auto analyze period failed",
zap.Error(err),
)
@@ -207,7 +208,7 @@ func HandleAutoAnalyze(
tidsAndPids := getAllTidsAndPids(tbls)
lockedTables, err := statsHandle.GetLockedTables(tidsAndPids...)
if err != nil {
statsutil.StatsLogger.Error(
statslogutil.StatsLogger.Error(
"check table lock failed",
zap.Error(err),
)
@@ -298,7 +299,7 @@ func tryAutoAnalyzeTable(
if err != nil {
return false
}
statsutil.StatsLogger.Info(
statslogutil.StatsLogger.Info(
"auto analyze triggered",
zap.String("sql", escaped),
zap.String("reason", reason),
@@ -321,7 +322,7 @@ func tryAutoAnalyzeTable(
return false
}

statsutil.StatsLogger.Info(
statslogutil.StatsLogger.Info(
"auto analyze for unanalyzed indexes",
zap.String("sql", escaped),
)
@@ -402,7 +403,7 @@ func tryAutoAnalyzePartitionTableInDynamicMode(
ratio,
); needAnalyze {
needAnalyzePartitionNames = append(needAnalyzePartitionNames, def.Name.O)
statsutil.StatsLogger.Info(
statslogutil.StatsLogger.Info(
"need to auto analyze",
zap.String("database", db),
zap.String("table", tblInfo.Name.String()),
@@ -427,7 +428,7 @@ func tryAutoAnalyzePartitionTableInDynamicMode(
}

if len(needAnalyzePartitionNames) > 0 {
statsutil.StatsLogger.Info("start to auto analyze",
statslogutil.StatsLogger.Info("start to auto analyze",
zap.String("database", db),
zap.String("table", tblInfo.Name.String()),
zap.Any("partitions", needAnalyzePartitionNames),
@@ -447,7 +448,7 @@ func tryAutoAnalyzePartitionTableInDynamicMode(
sql := getSQL("analyze table %n.%n partition", "", end-start)
params := append([]interface{}{db, tblInfo.Name.O}, needAnalyzePartitionNames[start:end]...)

statsutil.StatsLogger.Info(
statslogutil.StatsLogger.Info(
"auto analyze triggered",
zap.String("database", db),
zap.String("table", tblInfo.Name.String()),
@@ -485,7 +486,7 @@ func tryAutoAnalyzePartitionTableInDynamicMode(
sql := getSQL("analyze table %n.%n partition", " index %n", end-start)
params := append([]interface{}{db, tblInfo.Name.O}, needAnalyzePartitionNames[start:end]...)
params = append(params, idx.Name.O)
statsutil.StatsLogger.Info("auto analyze for unanalyzed",
statslogutil.StatsLogger.Info("auto analyze for unanalyzed",
zap.String("database", db),
zap.String("table", tblInfo.Name.String()),
zap.String("index", idx.Name.String()),
@@ -523,7 +524,7 @@ func execAutoAnalyze(
if err1 != nil {
escaped = ""
}
statsutil.StatsLogger.Error(
statslogutil.StatsLogger.Error(
"auto analyze failed",
zap.String("sql", escaped),
zap.Duration("cost_time", dur),
1 change: 1 addition & 0 deletions pkg/statistics/handle/cache/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -19,6 +19,7 @@ go_library(
"//pkg/statistics/handle/cache/internal/lfu",
"//pkg/statistics/handle/cache/internal/mapcache",
"//pkg/statistics/handle/cache/internal/metrics",
"//pkg/statistics/handle/logutil",
"//pkg/statistics/handle/metrics",
"//pkg/statistics/handle/util",
"//pkg/types",
3 changes: 2 additions & 1 deletion pkg/statistics/handle/cache/statscache.go
Original file line number Diff line number Diff line change
@@ -23,6 +23,7 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/statistics"
"github.com/pingcap/tidb/pkg/statistics/handle/cache/internal/metrics"
statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil"
handle_metrics "github.com/pingcap/tidb/pkg/statistics/handle/metrics"
"github.com/pingcap/tidb/pkg/statistics/handle/util"
"github.com/pingcap/tidb/pkg/util/chunk"
@@ -99,7 +100,7 @@ func (s *StatsCacheImpl) Update(is infoschema.InfoSchema) error {
tbl, err := s.statsHandle.TableStatsFromStorage(tableInfo, physicalID, false, 0)
// Error is not nil may mean that there are some ddl changes on this table, we will not update it.
if err != nil {
logutil.BgLogger().Error("error occurred when read table stats", zap.String("category", "stats"), zap.String("table", tableInfo.Name.O), zap.Error(err))
statslogutil.StatsLogger.Error("error occurred when read table stats", zap.String("table", tableInfo.Name.O), zap.Error(err))
continue
}
if tbl == nil {
1 change: 1 addition & 0 deletions pkg/statistics/handle/globalstats/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -18,6 +18,7 @@ go_library(
"//pkg/sessionctx/stmtctx",
"//pkg/sessiontxn",
"//pkg/statistics",
"//pkg/statistics/handle/logutil",
"//pkg/statistics/handle/storage",
"//pkg/statistics/handle/util",
"//pkg/types",
16 changes: 8 additions & 8 deletions pkg/statistics/handle/globalstats/global_stats_async.go
Original file line number Diff line number Diff line change
@@ -28,10 +28,10 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
"github.com/pingcap/tidb/pkg/statistics"
statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil"
"github.com/pingcap/tidb/pkg/statistics/handle/storage"
"github.com/pingcap/tidb/pkg/statistics/handle/util"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/logutil"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
)
@@ -224,7 +224,7 @@ func (a *AsyncMergePartitionStats2GlobalStats) dealErrPartitionColumnStatsMissin
func (a *AsyncMergePartitionStats2GlobalStats) ioWorker(sctx sessionctx.Context, isIndex bool) (err error) {
defer func() {
if r := recover(); r != nil {
logutil.BgLogger().Warn("ioWorker panic", zap.Stack("stack"), zap.Any("error", r), zap.String("category", "stats"))
statslogutil.StatsLogger.Warn("ioWorker panic", zap.Stack("stack"), zap.Any("error", r))
close(a.ioWorkerExitWhenErrChan)
err = errors.New(fmt.Sprint(r))
}
@@ -259,7 +259,7 @@ func (a *AsyncMergePartitionStats2GlobalStats) ioWorker(sctx sessionctx.Context,
func (a *AsyncMergePartitionStats2GlobalStats) cpuWorker(stmtCtx *stmtctx.StatementContext, sctx sessionctx.Context, opts map[ast.AnalyzeOptionType]uint64, isIndex bool, tz *time.Location, analyzeVersion int) (err error) {
defer func() {
if r := recover(); r != nil {
logutil.BgLogger().Warn("cpuWorker panic", zap.Stack("stack"), zap.Any("error", r), zap.String("category", "stats"))
statslogutil.StatsLogger.Warn("cpuWorker panic", zap.Stack("stack"), zap.Any("error", r))
err = errors.New(fmt.Sprint(r))
}
close(a.cpuWorkerExitChan)
@@ -281,7 +281,7 @@ func (a *AsyncMergePartitionStats2GlobalStats) cpuWorker(stmtCtx *stmtctx.Statem
}
err = a.dealCMSketch()
if err != nil {
logutil.BgLogger().Warn("dealCMSketch failed", zap.Error(err), zap.String("category", "stats"))
statslogutil.StatsLogger.Warn("dealCMSketch failed", zap.Error(err))
return err
}
failpoint.Inject("PanicSameTime", func(val failpoint.Value) {
@@ -292,7 +292,7 @@ func (a *AsyncMergePartitionStats2GlobalStats) cpuWorker(stmtCtx *stmtctx.Statem
})
err = a.dealHistogramAndTopN(stmtCtx, sctx, opts, isIndex, tz, analyzeVersion)
if err != nil {
logutil.BgLogger().Warn("dealHistogramAndTopN failed", zap.Error(err), zap.String("category", "stats"))
statslogutil.StatsLogger.Warn("dealHistogramAndTopN failed", zap.Error(err))
return err
}
return nil
@@ -360,7 +360,7 @@ func (a *AsyncMergePartitionStats2GlobalStats) loadFmsketch(sctx sessionctx.Cont
fmsketch, i,
}:
case <-a.cpuWorkerExitChan:
logutil.BgLogger().Warn("ioWorker detects CPUWorker has exited", zap.String("category", "stats"))
statslogutil.StatsLogger.Warn("ioWorker detects CPUWorker has exited")
return nil
}
}
@@ -391,7 +391,7 @@ func (a *AsyncMergePartitionStats2GlobalStats) loadCMsketch(sctx sessionctx.Cont
cmsketch, i,
}:
case <-a.cpuWorkerExitChan:
logutil.BgLogger().Warn("ioWorker detects CPUWorker has exited", zap.String("category", "stats"))
statslogutil.StatsLogger.Warn("ioWorker detects CPUWorker has exited")
return nil
}
}
@@ -433,7 +433,7 @@ func (a *AsyncMergePartitionStats2GlobalStats) loadHistogramAndTopN(sctx session
NewStatsWrapper(hists, topn), i,
}:
case <-a.cpuWorkerExitChan:
logutil.BgLogger().Warn("ioWorker detects CPUWorker has exited", zap.String("category", "stats"))
statslogutil.StatsLogger.Warn("ioWorker detects CPUWorker has exited")
return nil
}
}
6 changes: 3 additions & 3 deletions pkg/statistics/handle/handle.go
Original file line number Diff line number Diff line change
@@ -27,11 +27,11 @@ import (
"github.com/pingcap/tidb/pkg/statistics/handle/globalstats"
"github.com/pingcap/tidb/pkg/statistics/handle/history"
"github.com/pingcap/tidb/pkg/statistics/handle/lockstats"
statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil"
"github.com/pingcap/tidb/pkg/statistics/handle/storage"
"github.com/pingcap/tidb/pkg/statistics/handle/syncload"
"github.com/pingcap/tidb/pkg/statistics/handle/usage"
"github.com/pingcap/tidb/pkg/statistics/handle/util"
"github.com/pingcap/tidb/pkg/util/logutil"
"github.com/tiancaiamao/gp"
atomic2 "go.uber.org/atomic"
"go.uber.org/zap"
@@ -180,11 +180,11 @@ func (h *Handle) FlushStats() {
for len(h.ddlEventCh) > 0 {
e := <-h.ddlEventCh
if err := h.HandleDDLEvent(e); err != nil {
logutil.BgLogger().Error("handle ddl event fail", zap.String("category", "stats"), zap.Error(err))
statslogutil.StatsLogger.Error("handle ddl event fail", zap.Error(err))
}
}
if err := h.DumpStatsDeltaToKV(true); err != nil {
logutil.BgLogger().Error("dump stats delta fail", zap.String("category", "stats"), zap.Error(err))
statslogutil.StatsLogger.Error("dump stats delta fail", zap.Error(err))
}
}

2 changes: 1 addition & 1 deletion pkg/statistics/handle/lockstats/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -12,8 +12,8 @@ go_library(
deps = [
"//pkg/sessionctx",
"//pkg/statistics/handle/cache",
"//pkg/statistics/handle/logutil",
"//pkg/statistics/handle/util",
"//pkg/util/logutil",
"//pkg/util/sqlexec",
"@com_github_pingcap_errors//:errors",
"@org_uber_go_zap//:zap",
8 changes: 4 additions & 4 deletions pkg/statistics/handle/lockstats/lock_stats.go
Original file line number Diff line number Diff line change
@@ -20,8 +20,8 @@ import (
"strings"

"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/statistics/handle/logutil"
"github.com/pingcap/tidb/pkg/statistics/handle/util"
"github.com/pingcap/tidb/pkg/util/logutil"
"github.com/pingcap/tidb/pkg/util/sqlexec"
"go.uber.org/zap"
)
@@ -155,7 +155,7 @@ func AddLockedTables(
ids = append(ids, pid)
}
}
util.StatsLogger.Info("lock table",
logutil.StatsLogger.Info("lock table",
zap.Any("tables", tables),
)

@@ -209,7 +209,7 @@ func AddLockedPartitions(
pNames = append(pNames, pName)
}

util.StatsLogger.Info("lock partitions",
logutil.StatsLogger.Info("lock partitions",
zap.Int64("tableID", tid),
zap.String("tableName", tableName),
zap.Int64s("partitionIDs", pids),
@@ -290,7 +290,7 @@ func generateStableSkippedPartitionsMessage(ids []int64, tableName string, skipp
func insertIntoStatsTableLocked(sctx sessionctx.Context, tid int64) error {
_, _, err := util.ExecRows(sctx, insertSQL, tid, tid)
if err != nil {
logutil.BgLogger().Error("error occurred when insert mysql.stats_table_locked", zap.String("category", "stats"), zap.Error(err))
logutil.StatsLogger.Error("error occurred when insert mysql.stats_table_locked", zap.Error(err))
return err
}
return nil
5 changes: 3 additions & 2 deletions pkg/statistics/handle/lockstats/unlock_stats.go
Original file line number Diff line number Diff line change
@@ -18,6 +18,7 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/statistics/handle/cache"
statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil"
"github.com/pingcap/tidb/pkg/statistics/handle/util"
"go.uber.org/zap"
)
@@ -52,7 +53,7 @@ func RemoveLockedTables(
}
}

util.StatsLogger.Info("unlock table",
statslogutil.StatsLogger.Info("unlock table",
zap.Any("tables", tables),
)

@@ -105,7 +106,7 @@ func RemoveLockedPartitions(
for pid := range pidNames {
pids = append(pids, pid)
}
util.StatsLogger.Info("unlock partitions",
statslogutil.StatsLogger.Info("unlock partitions",
zap.Int64("tableID", tid),
zap.String("tableName", tableName),
zap.Int64s("partitionIDs", pids),
12 changes: 12 additions & 0 deletions pkg/statistics/handle/logutil/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,12 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "logutil",
srcs = ["logutil.go"],
importpath = "github.com/pingcap/tidb/pkg/statistics/handle/logutil",
visibility = ["//visibility:public"],
deps = [
"//pkg/util/logutil",
"@org_uber_go_zap//:zap",
],
)
Loading

0 comments on commit 44f9430

Please sign in to comment.