Skip to content

Commit

Permalink
statistics: reduce the log output (#57184)
Browse files Browse the repository at this point in the history
ref #55906
  • Loading branch information
Rustin170506 authored Nov 7, 2024
1 parent b522e98 commit 87669fb
Show file tree
Hide file tree
Showing 3 changed files with 24 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ go_library(
"//pkg/statistics/handle/util",
"//pkg/util",
"//pkg/util/intest",
"//pkg/util/logutil",
"//pkg/util/timeutil",
"@com_github_pingcap_errors//:errors",
"@com_github_tikv_client_go_v2//oracle",
Expand Down
28 changes: 22 additions & 6 deletions pkg/statistics/handle/autoanalyze/priorityqueue/queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/intest"
"github.com/pingcap/tidb/pkg/util/logutil"
"go.uber.org/zap"
)

Expand All @@ -43,6 +44,12 @@ const (
mustRetryJobRequeueInterval = time.Minute * 5
)

// If the process takes longer than this threshold, we will log it as a slow log.
const slowLogThreshold = 150 * time.Millisecond

// Every 15 minutes, at most 1 log will be output.
var queueSamplerLogger = logutil.SampleLoggerFactory(15*time.Minute, 1, zap.String(logutil.LogFieldCategory, "stats"))

// pqHeap is an interface that wraps the methods of a priority queue heap.
type pqHeap interface {
// getByKey returns the job by the given table ID.
Expand Down Expand Up @@ -312,13 +319,13 @@ func (pq *AnalysisPriorityQueue) run() {
statslogutil.StatsLogger().Info("Priority queue stopped")
return
case <-dmlChangesFetchInterval.C:
statslogutil.StatsLogger().Info("Start to fetch DML changes of jobs")
queueSamplerLogger().Info("Start to fetch DML changes of tables")
pq.ProcessDMLChanges()
case <-timeRefreshInterval.C:
statslogutil.StatsLogger().Info("Start to refresh last analysis durations of jobs")
queueSamplerLogger().Info("Start to refresh last analysis durations of jobs")
pq.RefreshLastAnalysisDuration()
case <-mustRetryJobRequeueInterval.C:
statslogutil.StatsLogger().Info("Start to request must retry jobs")
queueSamplerLogger().Info("Start to requeue must retry jobs")
pq.RequeueMustRetryJobs()
}
}
Expand All @@ -334,7 +341,10 @@ func (pq *AnalysisPriorityQueue) ProcessDMLChanges() {
if err := statsutil.CallWithSCtx(pq.statsHandle.SPool(), func(sctx sessionctx.Context) error {
start := time.Now()
defer func() {
statslogutil.StatsLogger().Info("DML changes processed", zap.Duration("duration", time.Since(start)))
duration := time.Since(start)
if duration > slowLogThreshold {
queueSamplerLogger().Info("DML changes processed", zap.Duration("duration", duration))
}
}()

parameters := exec.GetAutoAnalyzeParameters(sctx)
Expand Down Expand Up @@ -609,7 +619,10 @@ func (pq *AnalysisPriorityQueue) RequeueMustRetryJobs() {
if err := statsutil.CallWithSCtx(pq.statsHandle.SPool(), func(sctx sessionctx.Context) error {
start := time.Now()
defer func() {
statslogutil.StatsLogger().Info("Must retry jobs requeued", zap.Duration("duration", time.Since(start)))
duration := time.Since(start)
if duration > slowLogThreshold {
queueSamplerLogger().Info("Must retry jobs requeued", zap.Duration("duration", duration))
}
}()

is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema)
Expand Down Expand Up @@ -642,7 +655,10 @@ func (pq *AnalysisPriorityQueue) RefreshLastAnalysisDuration() {
if err := statsutil.CallWithSCtx(pq.statsHandle.SPool(), func(sctx sessionctx.Context) error {
start := time.Now()
defer func() {
statslogutil.StatsLogger().Info("Last analysis duration refreshed", zap.Duration("duration", time.Since(start)))
duration := time.Since(start)
if duration > slowLogThreshold {
queueSamplerLogger().Info("Last analysis duration refreshed", zap.Duration("duration", duration))
}
}()
jobs := pq.syncFields.inner.list()
currentTs, err := statsutil.GetStartTS(sctx)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -518,7 +518,7 @@ func TestExchangeTablePartition(t *testing.T) {
testKit.MustExec("analyze table t2")
require.NoError(t, h.Update(context.Background(), do.InfoSchema()))
// Insert some data.
testKit.MustExec("insert into t1 values (1,2),(2,2)")
testKit.MustExec("insert into t1 values (1,2),(2,2),(3,3),(4,4)")
testKit.MustExec("insert into t2 values (1,2)")
require.NoError(t, h.DumpStatsDeltaToKV(true))
require.NoError(t, h.Update(context.Background(), do.InfoSchema()))
Expand Down

0 comments on commit 87669fb

Please sign in to comment.