diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/BUILD.bazel b/pkg/statistics/handle/autoanalyze/priorityqueue/BUILD.bazel index 37e5f16d91eea..93559021972ab 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/BUILD.bazel @@ -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", diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/queue.go b/pkg/statistics/handle/autoanalyze/priorityqueue/queue.go index d7f5d21cc9482..daeb03496c1d4 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/queue.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/queue.go @@ -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" ) @@ -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. @@ -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() } } @@ -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) @@ -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) @@ -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) diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler_test.go b/pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler_test.go index 7bd86d9121887..1a67f6b340b8c 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler_test.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler_test.go @@ -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()))