diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job.go b/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job.go index e15385b2b424c..a9f1948bdfb5b 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job.go @@ -111,7 +111,9 @@ func (j *DynamicPartitionedTableAnalysisJob) HasNewlyAddedIndex() bool { // IsValidToAnalyze checks whether the table or partition is valid to analyze. // We need to check each partition to determine whether the table is valid to analyze. -func (j *DynamicPartitionedTableAnalysisJob) IsValidToAnalyze(sctx sessionctx.Context) (bool, string) { +func (j *DynamicPartitionedTableAnalysisJob) IsValidToAnalyze( + sctx sessionctx.Context, +) (bool, string) { if valid, failReason := isValidWeight(j.Weight); !valid { return false, failReason } diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/job.go b/pkg/statistics/handle/autoanalyze/priorityqueue/job.go index 8171f720de2ab..f06e5d93e864f 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/job.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/job.go @@ -19,7 +19,7 @@ import ( "time" "github.com/pingcap/tidb/pkg/sessionctx" - statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil" + "github.com/pingcap/tidb/pkg/statistics/handle/logutil" statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/util/intest" "go.uber.org/zap" @@ -50,7 +50,9 @@ type AnalysisJob interface { // It checks the last failed analysis duration and the average analysis duration. // If the last failed analysis duration is less than 2 times the average analysis duration, // we skip this table to avoid too much failed analysis. - IsValidToAnalyze(sctx sessionctx.Context) (bool, string) + IsValidToAnalyze( + sctx sessionctx.Context, + ) (bool, string) // Analyze executes the analyze statement within a transaction. Analyze( @@ -98,7 +100,7 @@ func isValidToAnalyze( lastFailedAnalysisDuration, err := GetLastFailedAnalysisDuration(sctx, schema, table, partitionNames...) if err != nil { - statslogutil.StatsLogger().Warn( + logutil.SingletonStatsSamplerLogger().Warn( "Fail to get last failed analysis duration", zap.String("schema", schema), zap.String("table", table), @@ -111,7 +113,7 @@ func isValidToAnalyze( averageAnalysisDuration, err := GetAverageAnalysisDuration(sctx, schema, table, partitionNames...) if err != nil { - statslogutil.StatsLogger().Warn( + logutil.SingletonStatsSamplerLogger().Warn( "Fail to get average analysis duration", zap.String("schema", schema), zap.String("table", table), @@ -124,7 +126,7 @@ func isValidToAnalyze( // Last analysis just failed, we should not analyze it again. if lastFailedAnalysisDuration == justFailed { // The last analysis failed, we should not analyze it again. - statslogutil.StatsLogger().Info( + logutil.SingletonStatsSamplerLogger().Info( "Skip analysis because the last analysis just failed", zap.String("schema", schema), zap.String("table", table), @@ -137,7 +139,7 @@ func isValidToAnalyze( // Skip this table to avoid too much failed analysis. onlyFailedAnalysis := lastFailedAnalysisDuration != NoRecord && averageAnalysisDuration == NoRecord if onlyFailedAnalysis && lastFailedAnalysisDuration < defaultFailedAnalysisWaitTime { - statslogutil.StatsLogger().Info( + logutil.SingletonStatsSamplerLogger().Info( fmt.Sprintf("Skip analysis because the last failed analysis duration is less than %v", defaultFailedAnalysisWaitTime), zap.String("schema", schema), zap.String("table", table), @@ -151,7 +153,7 @@ func isValidToAnalyze( meetSkipCondition := lastFailedAnalysisDuration != NoRecord && lastFailedAnalysisDuration < 2*averageAnalysisDuration if meetSkipCondition { - statslogutil.StatsLogger().Info( + logutil.SingletonStatsSamplerLogger().Info( "Skip analysis because the last failed analysis duration is less than 2 times the average analysis duration", zap.String("schema", schema), zap.String("table", table), diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job.go b/pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job.go index dfe4ef4508789..4b8a1ca79e469 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job.go @@ -91,7 +91,9 @@ func (j *NonPartitionedTableAnalysisJob) HasNewlyAddedIndex() bool { // IsValidToAnalyze checks whether the table is valid to analyze. // We will check the last failed job and average analyze duration to determine whether the table is valid to analyze. -func (j *NonPartitionedTableAnalysisJob) IsValidToAnalyze(sctx sessionctx.Context) (bool, string) { +func (j *NonPartitionedTableAnalysisJob) IsValidToAnalyze( + sctx sessionctx.Context, +) (bool, string) { if valid, failReason := isValidWeight(j.Weight); !valid { return false, failReason } diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job.go b/pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job.go index f44818a9599a1..2441cae8657e6 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job.go @@ -104,7 +104,9 @@ func (j *StaticPartitionedTableAnalysisJob) HasNewlyAddedIndex() bool { // IsValidToAnalyze checks whether the partition is valid to analyze. // Only the specified static partition is checked. -func (j *StaticPartitionedTableAnalysisJob) IsValidToAnalyze(sctx sessionctx.Context) (bool, string) { +func (j *StaticPartitionedTableAnalysisJob) IsValidToAnalyze( + sctx sessionctx.Context, +) (bool, string) { if valid, failReason := isValidWeight(j.Weight); !valid { return false, failReason } diff --git a/pkg/statistics/handle/autoanalyze/refresher/refresher.go b/pkg/statistics/handle/autoanalyze/refresher/refresher.go index c2b0c920705c3..d6b3694f8218c 100644 --- a/pkg/statistics/handle/autoanalyze/refresher/refresher.go +++ b/pkg/statistics/handle/autoanalyze/refresher/refresher.go @@ -91,7 +91,7 @@ func (r *Refresher) PickOneTableAndAnalyzeByPriority() bool { if valid, failReason := job.IsValidToAnalyze( sctx, ); !valid { - statslogutil.StatsLogger().Info( + statslogutil.SingletonStatsSamplerLogger().Info( "Table is not ready to analyze", zap.String("failReason", failReason), zap.Stringer("job", job), @@ -116,7 +116,7 @@ func (r *Refresher) PickOneTableAndAnalyzeByPriority() bool { // Only analyze one table each time. return true } - statslogutil.StatsLogger().Debug( + statslogutil.SingletonStatsSamplerLogger().Info( "No table to analyze", ) return false diff --git a/pkg/statistics/handle/logutil/BUILD.bazel b/pkg/statistics/handle/logutil/BUILD.bazel index 49bc63b86585e..78b06985a8f13 100644 --- a/pkg/statistics/handle/logutil/BUILD.bazel +++ b/pkg/statistics/handle/logutil/BUILD.bazel @@ -8,5 +8,6 @@ go_library( deps = [ "//pkg/util/logutil", "@org_uber_go_zap//:zap", + "@org_uber_go_zap//zapcore", ], ) diff --git a/pkg/statistics/handle/logutil/logutil.go b/pkg/statistics/handle/logutil/logutil.go index ea791531a7760..a7c4413404028 100644 --- a/pkg/statistics/handle/logutil/logutil.go +++ b/pkg/statistics/handle/logutil/logutil.go @@ -15,11 +15,42 @@ package logutil import ( + "sync" + "time" + "github.com/pingcap/tidb/pkg/util/logutil" "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) // StatsLogger with category "stats" is used to log statistic related messages. +// Do not use it to log the message that is not related to statistics. func StatsLogger() *zap.Logger { return logutil.BgLogger().With(zap.String("category", "stats")) } + +var ( + initSamplerLoggerOnce sync.Once + samplerLogger *zap.Logger +) + +// SingletonStatsSamplerLogger with category "stats" is used to log statistic related messages. +// It is used to sample the log to avoid too many logs. +// NOTE: Do not create a new logger for each log, it will cause the sampler not work. +// Because we need to record the log count with the same level and message in this specific logger. +// Do not use it to log the message that is not related to statistics. +func SingletonStatsSamplerLogger() *zap.Logger { + init := func() { + if samplerLogger == nil { + // Create a new zapcore sampler with options + // This will log the first 2 log entries with the same level and message in a minute and ignore the rest of the logs. + sampler := zap.WrapCore(func(core zapcore.Core) zapcore.Core { + return zapcore.NewSamplerWithOptions(core, time.Minute, 2, 0) + }) + samplerLogger = StatsLogger().WithOptions(sampler) + } + } + + initSamplerLoggerOnce.Do(init) + return samplerLogger +}