From c1903e96e1d6d31574bf34b0bdf5756b411a1e5e Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Thu, 10 Aug 2023 17:37:56 +0800 Subject: [PATCH 1/9] This is an automated cherry-pick of #45938 Signed-off-by: ti-chi-bot --- executor/analyze.go | 3 +- executor/analyze_test.go | 4 +- executor/analyzetest/analyze_test.go | 66 ++-- executor/builder.go | 38 ++- executor/infoschema_reader_test.go | 2 +- planner/core/indexmerge_path_test.go | 4 +- planner/core/integration_test.go | 2 +- statistics/analyze_jobs.go | 6 + .../handle/handletest/analyze/analyze_test.go | 298 ++++++++++++++++++ statistics/integration_test.go | 4 +- 10 files changed, 384 insertions(+), 43 deletions(-) create mode 100644 statistics/handle/handletest/analyze/analyze_test.go diff --git a/executor/analyze.go b/executor/analyze.go index f3adc7f2bb33b..dd5c20440638e 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -592,7 +592,8 @@ func finishJobWithLog(sctx sessionctx.Context, job *statistics.AnalyzeJob, analy zap.String("job info", job.JobInfo), zap.Time("start time", job.StartTime), zap.Time("end time", job.EndTime), - zap.String("cost", job.EndTime.Sub(job.StartTime).String())) + zap.String("cost", job.EndTime.Sub(job.StartTime).String()), + zap.String("sample rate reason", job.SampleRateReason)) } } diff --git a/executor/analyze_test.go b/executor/analyze_test.go index a6cdea833df50..ce5e9b4a9130a 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -433,8 +433,8 @@ func TestMergeGlobalStatsWithUnAnalyzedPartition(t *testing.T) { tk.MustExec("analyze table t partition p2 index idxc;") tk.MustQuery("show warnings").Check(testkit.Rows( "Warning 1105 The version 2 would collect all statistics not only the selected indexes", - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p2")) + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p2, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"")) tk.MustExec("analyze table t partition p0;") tk.MustQuery("show warnings").Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0")) + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0, reason to use this rate is \"use min(1, 110000/2) as the sample-rate=1\"")) } diff --git a/executor/analyzetest/analyze_test.go b/executor/analyzetest/analyze_test.go index d052c8b950873..c00b2fe5812ca 100644 --- a/executor/analyzetest/analyze_test.go +++ b/executor/analyzetest/analyze_test.go @@ -668,14 +668,14 @@ func TestAdjustSampleRateNote(t *testing.T) { result := tk.MustQuery("show stats_meta where table_name = 't'") require.Equal(t, "220000", result.Rows()[0][5]) tk.MustExec("analyze table t") - tk.MustQuery("show warnings").Check(testkit.Rows("Note 1105 Analyze use auto adjusted sample rate 0.500000 for table test.t")) + tk.MustQuery("show warnings").Check(testkit.Rows("Note 1105 Analyze use auto adjusted sample rate 0.500000 for table test.t, reason to use this rate is \"use min(1, 110000/220000) as the sample-rate=0.5\"")) tk.MustExec("insert into t values(1),(1),(1)") require.NoError(t, statsHandle.DumpStatsDeltaToKV(handle.DumpAll)) require.NoError(t, statsHandle.Update(is)) result = tk.MustQuery("show stats_meta where table_name = 't'") require.Equal(t, "3", result.Rows()[0][5]) tk.MustExec("analyze table t") - tk.MustQuery("show warnings").Check(testkit.Rows("Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t")) + tk.MustQuery("show warnings").Check(testkit.Rows("Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/3) as the sample-rate=1\"")) } func TestFastAnalyze4GlobalStats(t *testing.T) { @@ -924,7 +924,7 @@ func TestSmallTableAnalyzeV2(t *testing.T) { tk.MustExec("create table small_table_inject_pd(a int)") tk.MustExec("insert into small_table_inject_pd values(1), (2), (3), (4), (5)") tk.MustExec("analyze table small_table_inject_pd") - tk.MustQuery("show warnings").Check(testkit.Rows("Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.small_table_inject_pd")) + tk.MustQuery("show warnings").Check(testkit.Rows("Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.small_table_inject_pd, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"")) tk.MustExec(` create table small_table_inject_pd_with_partition( a int @@ -936,9 +936,9 @@ create table small_table_inject_pd_with_partition( tk.MustExec("insert into small_table_inject_pd_with_partition values(1), (6), (11)") tk.MustExec("analyze table small_table_inject_pd_with_partition") tk.MustQuery("show warnings").Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.small_table_inject_pd_with_partition's partition p0", - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.small_table_inject_pd_with_partition's partition p1", - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.small_table_inject_pd_with_partition's partition p2", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.small_table_inject_pd_with_partition's partition p0, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.small_table_inject_pd_with_partition's partition p1, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.small_table_inject_pd_with_partition's partition p2, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", )) rows := [][]interface{}{ {"global", "a"}, @@ -1435,7 +1435,7 @@ func TestAnalyzeColumnsWithPrimaryKey(t *testing.T) { case model.ColumnList: tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", )) case model.PredicateColumns: @@ -1503,7 +1503,7 @@ func TestAnalyzeColumnsWithIndex(t *testing.T) { case model.ColumnList: tk.MustExec("analyze table t columns c with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Columns b,d are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", )) case model.PredicateColumns: @@ -1580,7 +1580,7 @@ func TestAnalyzeColumnsWithClusteredIndex(t *testing.T) { case model.ColumnList: tk.MustExec("analyze table t columns c with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Columns b,d are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", )) case model.PredicateColumns: @@ -1661,8 +1661,8 @@ func TestAnalyzeColumnsWithDynamicPartitionTable(t *testing.T) { case model.ColumnList: tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0", - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", )) case model.PredicateColumns: @@ -1813,8 +1813,8 @@ func TestAnalyzeColumnsWithStaticPartitionTable(t *testing.T) { case model.ColumnList: tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0", - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", )) case model.PredicateColumns: @@ -1918,7 +1918,7 @@ func TestAnalyzeColumnsWithExtendedStats(t *testing.T) { case model.ColumnList: tk.MustExec("analyze table t columns b with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", )) case model.PredicateColumns: @@ -1988,7 +1988,7 @@ func TestAnalyzeColumnsWithVirtualColumnIndex(t *testing.T) { case model.ColumnList: tk.MustExec("analyze table t columns b with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", )) case model.PredicateColumns: @@ -2110,6 +2110,26 @@ func TestAnalyzeColumnsAfterAnalyzeAll(t *testing.T) { } } +func TestAnalyzeSampleRateReason(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b int)") + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) + + tk.MustExec(`analyze table t`) + tk.MustQuery(`show warnings`).Sort().Check(testkit.Rows( + `Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is "use min(1, 110000/10000) as the sample-rate=1"`)) + + tk.MustExec(`insert into t values (1, 1), (2, 2), (3, 3)`) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) + tk.MustExec(`analyze table t`) + tk.MustQuery(`show warnings`).Sort().Check(testkit.Rows( + `Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is "TiDB assumes that the table is empty, use sample-rate=1"`)) +} + func TestAnalyzeColumnsErrorAndWarning(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) @@ -2134,7 +2154,7 @@ func TestAnalyzeColumnsErrorAndWarning(t *testing.T) { // If no predicate column is collected, analyze predicate columns gives a warning and falls back to analyze all columns. tk.MustExec("analyze table t predicate columns") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t", + `Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is "use min(1, 110000/10000) as the sample-rate=1"`, "Warning 1105 No predicate column has been collected yet for table test.t so all columns are analyzed", )) rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_analyzed_at is not null").Rows() @@ -2159,7 +2179,7 @@ func TestAnalyzeColumnsErrorAndWarning(t *testing.T) { tk.MustExec("analyze table t predicate columns") } tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t", + `Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is "TiDB assumes that the table is empty, use sample-rate=1"`, "Warning 1105 Table test.t has version 1 statistics so all the columns must be analyzed to overwrite the current statistics", )) }(val) @@ -2741,7 +2761,7 @@ PARTITION BY RANGE ( a ) ( // analyze partition with options under dynamic mode tk.MustExec("analyze table t partition p0 columns a,b,c with 1 topn, 3 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Ignore columns and options when analyze partition in dynamic mode", "Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`", "Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`", @@ -2755,7 +2775,7 @@ PARTITION BY RANGE ( a ) ( tk.MustExec("analyze table t partition p0") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0, reason to use this rate is \"use min(1, 110000/9) as the sample-rate=1\"", "Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`", "Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`", )) @@ -2810,7 +2830,7 @@ PARTITION BY RANGE ( a ) ( tk.MustExec("set @@session.tidb_partition_prune_mode = 'dynamic'") tk.MustExec("analyze table t partition p1 columns a,b,d with 1 topn, 3 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 8244 Build global-level stats failed due to missing partition-level column stats: table `t` partition `p0` column `d`, please run analyze table to refresh columns of all partitions", )) @@ -2819,7 +2839,7 @@ PARTITION BY RANGE ( a ) ( tk.MustExec("set global tidb_persist_analyze_options = true") tk.MustExec("analyze table t partition p1 columns a,b,d with 1 topn, 3 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1, reason to use this rate is \"use min(1, 110000/5) as the sample-rate=1\"", "Warning 1105 Ignore columns and options when analyze partition in dynamic mode", "Warning 8244 Build global-level stats failed due to missing partition-level column stats: table `t` partition `p0` column `d`, please run analyze table to refresh columns of all partitions", )) @@ -2828,7 +2848,7 @@ PARTITION BY RANGE ( a ) ( tk.MustExec("insert into mysql.analyze_options values (?,?,?,?,?,?,?)", pi.Definitions[1].ID, 0, 0, 1, 1, "DEFAULT", "") tk.MustExec("analyze table t partition p1 columns a,b,d with 1 topn, 3 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1, reason to use this rate is \"use min(1, 110000/5) as the sample-rate=1\"", "Warning 1105 Ignore columns and options when analyze partition in dynamic mode", "Warning 8244 Build global-level stats failed due to missing partition-level column stats: table `t` partition `p0` column `d`, please run analyze table to refresh columns of all partitions", )) @@ -3233,7 +3253,7 @@ func TestAnalyzeColumnsSkipMVIndexJsonCol(t *testing.T) { tk.MustExec("analyze table t columns a") tk.MustQuery("show warnings").Sort().Check(testkit.Rows(""+ - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Columns b are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", "Warning 1105 analyzing multi-valued indexes is not supported, skip idx_c")) tk.MustQuery("select job_info from mysql.analyze_jobs where table_schema = 'test' and table_name = 't'").Check(testkit.Rows( diff --git a/executor/builder.go b/executor/builder.go index 96173435f1364..5d74586712ff0 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -17,6 +17,7 @@ package executor import ( "bytes" "context" + "fmt" "math" "strconv" "strings" @@ -2599,32 +2600,40 @@ func (b *executorBuilder) buildAnalyzeSamplingPushdown(task plannercore.AnalyzeC modifyCount = int64(val.(int)) }) sampleRate := new(float64) + var sampleRateReason string if opts[ast.AnalyzeOptNumSamples] == 0 { *sampleRate = math.Float64frombits(opts[ast.AnalyzeOptSampleRate]) if *sampleRate < 0 { +<<<<<<< HEAD *sampleRate = b.getAdjustedSampleRate(b.ctx, task) +======= + *sampleRate, sampleRateReason = b.getAdjustedSampleRate(task) +>>>>>>> 6fb20c9d306 (planner: log the reason why the sample-rate is chosen when analyzing table (#45938)) if task.PartitionName != "" { sc.AppendNote(errors.Errorf( - "Analyze use auto adjusted sample rate %f for table %s.%s's partition %s", + `Analyze use auto adjusted sample rate %f for table %s.%s's partition %s, reason to use this rate is "%s"`, *sampleRate, task.DBName, task.TableName, task.PartitionName, + sampleRateReason, )) } else { sc.AppendNote(errors.Errorf( - "Analyze use auto adjusted sample rate %f for table %s.%s", + `Analyze use auto adjusted sample rate %f for table %s.%s, reason to use this rate is "%s"`, *sampleRate, task.DBName, task.TableName, + sampleRateReason, )) } } } job := &statistics.AnalyzeJob{ - DBName: task.DBName, - TableName: task.TableName, - PartitionName: task.PartitionName, + DBName: task.DBName, + TableName: task.TableName, + PartitionName: task.PartitionName, + SampleRateReason: sampleRateReason, } base := baseAnalyzeExec{ @@ -2681,11 +2690,16 @@ func (b *executorBuilder) buildAnalyzeSamplingPushdown(task plannercore.AnalyzeC // If we take n = 1e12, a 300*k sample still gives <= 0.66 bin size error with probability 0.99. // So if we don't consider the top-n values, we can keep the sample size at 300*256. // But we may take some top-n before building the histogram, so we increase the sample a little. +<<<<<<< HEAD func (b *executorBuilder) getAdjustedSampleRate(sctx sessionctx.Context, task plannercore.AnalyzeColumnsTask) float64 { statsHandle := domain.GetDomain(sctx).StatsHandle() +======= +func (b *executorBuilder) getAdjustedSampleRate(task plannercore.AnalyzeColumnsTask) (sampleRate float64, reason string) { + statsHandle := domain.GetDomain(b.ctx).StatsHandle() +>>>>>>> 6fb20c9d306 (planner: log the reason why the sample-rate is chosen when analyzing table (#45938)) defaultRate := 0.001 if statsHandle == nil { - return defaultRate + return defaultRate, fmt.Sprintf("statsHandler is nil, use the default-rate=%v", defaultRate) } var statsTbl *statistics.Table tid := task.TableID.GetStatisticsID() @@ -2697,11 +2711,11 @@ func (b *executorBuilder) getAdjustedSampleRate(sctx sessionctx.Context, task pl approxiCount, hasPD := b.getApproximateTableCountFromStorage(sctx, tid, task) // If there's no stats meta and no pd, return the default rate. if statsTbl == nil && !hasPD { - return defaultRate + return defaultRate, fmt.Sprintf("TiDB cannot get the row count of the table, use the default-rate=%v", defaultRate) } // If the count in stats_meta is still 0 and there's no information from pd side, we scan all rows. if statsTbl.RealtimeCount == 0 && !hasPD { - return 1 + return 1, "TiDB assumes that the table is empty and cannot get row count from PD, use sample-rate=1" } // we have issue https://github.com/pingcap/tidb/issues/29216. // To do a workaround for this issue, we check the approxiCount from the pd side to do a comparison. @@ -2710,15 +2724,17 @@ func (b *executorBuilder) getAdjustedSampleRate(sctx sessionctx.Context, task pl if float64(statsTbl.RealtimeCount*5) < approxiCount { // Confirmed by TiKV side, the experience error rate of the approximate count is about 20%. // So we increase the number to 150000 to reduce this error rate. - return math.Min(1, 150000/approxiCount) + sampleRate = math.Min(1, 150000/approxiCount) + return sampleRate, fmt.Sprintf("Row count in stats_meta is much smaller compared with the row count got by PD, use min(1, 15000/%v) as the sample-rate=%v", approxiCount, sampleRate) } // If we don't go into the above if branch and we still detect the count is zero. Return 1 to prevent the dividing zero. if statsTbl.RealtimeCount == 0 { - return 1 + return 1, "TiDB assumes that the table is empty, use sample-rate=1" } // We are expected to scan about 100000 rows or so. // Since there's tiny error rate around the count from the stats meta, we use 110000 to get a little big result - return math.Min(1, config.DefRowsForSampleRate/float64(statsTbl.RealtimeCount)) + sampleRate = math.Min(1, config.DefRowsForSampleRate/float64(statsTbl.RealtimeCount)) + return sampleRate, fmt.Sprintf("use min(1, %v/%v) as the sample-rate=%v", config.DefRowsForSampleRate, statsTbl.RealtimeCount, sampleRate) } func (b *executorBuilder) getApproximateTableCountFromStorage(sctx sessionctx.Context, tid int64, task plannercore.AnalyzeColumnsTask) (float64, bool) { diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 67c3c87eac512..64b423f3dbe24 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -594,7 +594,7 @@ func TestForAnalyzeStatus(t *testing.T) { tk.MustExec("create table t1 (a int, b int, index idx(a))") tk.MustExec("insert into t1 values (1,2),(3,4)") tk.MustExec("analyze table t1") - tk.MustQuery("show warnings").Check(testkit.Rows("Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t1")) // 1 note. + tk.MustQuery("show warnings").Check(testkit.Rows("Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t1, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"")) // 1 note. require.NoError(t, dom.StatsHandle().LoadNeededHistograms()) tk.MustExec("CREATE ROLE r_t1 ;") tk.MustExec("GRANT ALL PRIVILEGES ON test.t1 TO r_t1;") diff --git a/planner/core/indexmerge_path_test.go b/planner/core/indexmerge_path_test.go index e676b82d6f1a1..66d1a0bd9baec 100644 --- a/planner/core/indexmerge_path_test.go +++ b/planner/core/indexmerge_path_test.go @@ -39,12 +39,12 @@ index idx2(a, b, (cast(j->'$.str' as char(10) array)), c))`) tk.MustExec("set tidb_analyze_version=2") tk.MustExec("analyze table t") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 analyzing multi-valued indexes is not supported, skip idx", "Warning 1105 analyzing multi-valued indexes is not supported, skip idx2")) tk.MustExec("analyze table t index idx") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"TiDB assumes that the table is empty, use sample-rate=1\"", "Warning 1105 The version 2 would collect all statistics not only the selected indexes", "Warning 1105 analyzing multi-valued indexes is not supported, skip idx", "Warning 1105 analyzing multi-valued indexes is not supported, skip idx2")) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index b667281a4a0ca..dbaf58752ddfa 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -2952,7 +2952,7 @@ func TestIncrementalAnalyzeStatsVer2(t *testing.T) { require.Len(t, warns, 3) require.EqualError(t, warns[0].Err, "The version 2 would collect all statistics not only the selected indexes") require.EqualError(t, warns[1].Err, "The version 2 stats would ignore the INCREMENTAL keyword and do full sampling") - require.EqualError(t, warns[2].Err, "Analyze use auto adjusted sample rate 1.000000 for table test.t") + require.EqualError(t, warns[2].Err, "Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/3) as the sample-rate=1\"") rows = tk.MustQuery(fmt.Sprintf("select distinct_count from mysql.stats_histograms where table_id = %d and is_index = 1", tblID)).Rows() require.Len(t, rows, 1) require.Equal(t, "6", rows[0][0]) diff --git a/statistics/analyze_jobs.go b/statistics/analyze_jobs.go index 34ae95be58a3c..930cfabdfe0a3 100644 --- a/statistics/analyze_jobs.go +++ b/statistics/analyze_jobs.go @@ -26,9 +26,15 @@ type AnalyzeJob struct { TableName string PartitionName string JobInfo string +<<<<<<< HEAD StartTime time.Time EndTime time.Time Progress AnalyzeProgress +======= + + SampleRateReason string // why this sample-rate is chosen + Progress AnalyzeProgress +>>>>>>> 6fb20c9d306 (planner: log the reason why the sample-rate is chosen when analyzing table (#45938)) } // AnalyzeProgress represents the process of one analyze job. diff --git a/statistics/handle/handletest/analyze/analyze_test.go b/statistics/handle/handletest/analyze/analyze_test.go new file mode 100644 index 0000000000000..6c42871bc8a62 --- /dev/null +++ b/statistics/handle/handletest/analyze/analyze_test.go @@ -0,0 +1,298 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package analyze + +import ( + "bytes" + "fmt" + "testing" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/statistics/handle" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" +) + +// nolint:unused +func checkForGlobalStatsWithOpts(t *testing.T, dom *domain.Domain, db, tt, pp string, topn, buckets int) { + tbl, err := dom.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(tt)) + require.NoError(t, err) + + tblInfo := tbl.Meta() + physicalID := tblInfo.ID + if pp != "global" { + for _, def := range tbl.Meta().GetPartitionInfo().Definitions { + if def.Name.L == pp { + physicalID = def.ID + } + } + } + tblStats, err := dom.StatsHandle().TableStatsFromStorage(tblInfo, physicalID, true, 0) + require.NoError(t, err) + + delta := buckets/2 + 10 + for _, idxStats := range tblStats.Indices { + if len(idxStats.Buckets) == 0 { + continue // it's not loaded + } + numTopN := idxStats.TopN.Num() + numBuckets := len(idxStats.Buckets) + // since the hist-building algorithm doesn't stipulate the final bucket number to be equal to the expected number exactly, + // we have to check the results by a range here. + require.Equal(t, topn, numTopN) + require.GreaterOrEqual(t, numBuckets, buckets-delta) + require.LessOrEqual(t, numBuckets, buckets+delta) + } + for _, colStats := range tblStats.Columns { + if len(colStats.Buckets) == 0 { + continue // it's not loaded + } + numTopN := colStats.TopN.Num() + numBuckets := len(colStats.Buckets) + require.Equal(t, topn, numTopN) + require.GreaterOrEqual(t, numBuckets, buckets-delta) + require.LessOrEqual(t, numBuckets, buckets+delta) + } +} + +// nolint:unused +func prepareForGlobalStatsWithOptsV2(t *testing.T, dom *domain.Domain, tk *testkit.TestKit, tblName, dbName string) { + tk.MustExec("create database if not exists " + dbName) + tk.MustExec("use " + dbName) + tk.MustExec("drop table if exists " + tblName) + tk.MustExec(` create table ` + tblName + ` (a int, key(a)) partition by range (a) ` + + `(partition p0 values less than (100000), partition p1 values less than (200000))`) + buf1 := bytes.NewBufferString("insert into " + tblName + " values (0)") + buf2 := bytes.NewBufferString("insert into " + tblName + " values (100000)") + for i := 0; i < 1000; i++ { + buf1.WriteString(fmt.Sprintf(", (%v)", 2)) + buf2.WriteString(fmt.Sprintf(", (%v)", 100002)) + buf1.WriteString(fmt.Sprintf(", (%v)", 1)) + buf2.WriteString(fmt.Sprintf(", (%v)", 100001)) + buf1.WriteString(fmt.Sprintf(", (%v)", 0)) + buf2.WriteString(fmt.Sprintf(", (%v)", 100000)) + } + for i := 0; i < 5000; i += 3 { + buf1.WriteString(fmt.Sprintf(", (%v)", i)) + buf2.WriteString(fmt.Sprintf(", (%v)", 100000+i)) + } + tk.MustExec(buf1.String()) + tk.MustExec(buf2.String()) + tk.MustExec("set @@tidb_analyze_version=2") + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) +} + +// nolint:unused +func prepareForGlobalStatsWithOpts(t *testing.T, dom *domain.Domain, tk *testkit.TestKit, tblName, dbName string) { + tk.MustExec("create database if not exists " + dbName) + tk.MustExec("use " + dbName) + tk.MustExec("drop table if exists " + tblName) + tk.MustExec(` create table ` + tblName + ` (a int, key(a)) partition by range (a) ` + + `(partition p0 values less than (100000), partition p1 values less than (200000))`) + buf1 := bytes.NewBufferString("insert into " + tblName + " values (0)") + buf2 := bytes.NewBufferString("insert into " + tblName + " values (100000)") + for i := 0; i < 5000; i += 3 { + buf1.WriteString(fmt.Sprintf(", (%v)", i)) + buf2.WriteString(fmt.Sprintf(", (%v)", 100000+i)) + } + for i := 0; i < 1000; i++ { + buf1.WriteString(fmt.Sprintf(", (%v)", 0)) + buf2.WriteString(fmt.Sprintf(", (%v)", 100000)) + } + tk.MustExec(buf1.String()) + tk.MustExec(buf2.String()) + tk.MustExec("set @@tidb_analyze_version=2") + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) +} + +func TestAnalyzeVirtualCol(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int generated always as (-a) virtual, c int generated always as (-a) stored, index (c))") + tk.MustExec("insert into t(a) values(2),(1),(1),(3),(NULL)") + tk.MustExec("set @@tidb_analyze_version = 2") + tk.MustExec("analyze table t") + require.Len(t, tk.MustQuery("show stats_histograms where table_name ='t'").Rows(), 3) +} + +func TestAnalyzeGlobalStatsWithOpts1(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + prepareForGlobalStatsWithOpts(t, dom, tk, "test_gstats_opt", "test_gstats_opt") + + // nolint:unused + type opt struct { + topn int + buckets int + err bool + } + + cases := []opt{ + {1, 37, false}, + {2, 47, false}, + {10, 77, false}, + {77, 219, false}, + {-31, 222, true}, + {10, -77, true}, + {10000, 47, true}, + {77, 47000, true}, + } + for _, ca := range cases { + sql := fmt.Sprintf("analyze table test_gstats_opt with %v topn, %v buckets", ca.topn, ca.buckets) + if !ca.err { + tk.MustExec(sql) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt", "test_gstats_opt", "global", ca.topn, ca.buckets) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt", "test_gstats_opt", "p0", ca.topn, ca.buckets) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt", "test_gstats_opt", "p1", ca.topn, ca.buckets) + } else { + err := tk.ExecToErr(sql) + require.Error(t, err) + } + } +} + +func TestAnalyzeGlobalStatsWithOpts2(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + originalVal1 := tk.MustQuery("select @@tidb_persist_analyze_options").Rows()[0][0].(string) + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_persist_analyze_options = %v", originalVal1)) + }() + tk.MustExec("set global tidb_persist_analyze_options=false") + prepareForGlobalStatsWithOptsV2(t, dom, tk, "test_gstats_opt2", "test_gstats_opt2") + + tk.MustExec("analyze table test_gstats_opt2 with 2 topn, 10 buckets, 1000 samples") + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "global", 2, 10) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p0", 2, 10) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p1", 2, 10) + + // analyze a partition to let its options be different with others' + tk.MustExec("analyze table test_gstats_opt2 partition p0 with 3 topn, 20 buckets") + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "global", 3, 20) // use new options + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p0", 3, 20) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p1", 2, 10) + + tk.MustExec("analyze table test_gstats_opt2 partition p1 with 1 topn, 15 buckets") + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "global", 1, 15) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p0", 3, 20) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p1", 1, 15) + + tk.MustExec("analyze table test_gstats_opt2 partition p0 with 2 topn, 10 buckets") // change back to 2 topn + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "global", 2, 10) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p0", 2, 10) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p1", 1, 15) +} + +func TestAnalyzeWithDynamicPartitionPruneMode(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_partition_prune_mode = '" + string(variable.Dynamic) + "'") + tk.MustExec("set @@tidb_analyze_version = 2") + tk.MustExec(`create table t (a int, key(a)) partition by range(a) + (partition p0 values less than (10), + partition p1 values less than (22))`) + tk.MustExec(`insert into t values (1), (2), (3), (10), (11)`) + tk.MustExec(`analyze table t with 1 topn, 2 buckets`) + rows := tk.MustQuery("show stats_buckets where partition_name = 'global' and is_index=1").Rows() + require.Len(t, rows, 2) + require.Equal(t, "4", rows[1][6]) + tk.MustExec("insert into t values (1), (2), (2)") + tk.MustExec("analyze table t partition p0 with 1 topn, 2 buckets") + rows = tk.MustQuery("show stats_buckets where partition_name = 'global' and is_index=1").Rows() + require.Len(t, rows, 2) + require.Equal(t, "5", rows[1][6]) + tk.MustExec("insert into t values (3)") + tk.MustExec("analyze table t partition p0 index a with 1 topn, 2 buckets") + rows = tk.MustQuery("show stats_buckets where partition_name = 'global' and is_index=1").Rows() + require.Len(t, rows, 1) + require.Equal(t, "6", rows[0][6]) +} + +func TestFMSWithAnalyzePartition(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_partition_prune_mode = '" + string(variable.Dynamic) + "'") + tk.MustExec("set @@tidb_analyze_version = 2") + tk.MustExec(`create table t (a int, key(a)) partition by range(a) + (partition p0 values less than (10), + partition p1 values less than (22))`) + tk.MustExec(`insert into t values (1), (2), (3), (10), (11)`) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("0")) + tk.MustExec("analyze table t partition p0 with 1 topn, 2 buckets") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows( + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", + "Warning 1105 Ignore columns and options when analyze partition in dynamic mode", + "Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`", + "Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`", + )) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("2")) +} + +func TestFastAnalyzeColumnHistWithNullValue(t *testing.T) { + store := testkit.CreateMockStore(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t (a int)") + testKit.MustExec("insert into t values (1), (2), (3), (4), (NULL)") + testKit.MustExec("set @@session.tidb_analyze_version = 1") + testKit.MustExec("set @@tidb_enable_fast_analyze=1") + defer testKit.MustExec("set @@tidb_enable_fast_analyze=0") + testKit.MustExec("analyze table t with 0 topn, 2 buckets") + // If NULL is in hist, the min(lower_bound) will be "". + testKit.MustQuery("select min(lower_bound) from mysql.stats_buckets").Check(testkit.Rows("1")) +} + +func TestAnalyzeIncrementalEvictedIndex(t *testing.T) { + t.Skip("now we don't support to evict index") + restore := config.RestoreFunc() + defer restore() + config.UpdateGlobal(func(conf *config.Config) { + conf.Performance.EnableStatsCacheMemQuota = true + }) + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_analyze_version = 1") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b varchar(10), index idx_b (b))") + tk.MustExec("analyze table test.t") + tbl, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.Nil(t, err) + tblStats := domain.GetDomain(tk.Session()).StatsHandle().GetTableStats(tbl.Meta()) + for _, index := range tblStats.Indices { + require.False(t, index.IsEvicted()) + } + + domain.GetDomain(tk.Session()).StatsHandle().SetStatsCacheCapacity(1) + tblStats = domain.GetDomain(tk.Session()).StatsHandle().GetTableStats(tbl.Meta()) + for _, index := range tblStats.Indices { + require.True(t, index.IsEvicted()) + } + + require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertEvictIndex", `return(true)`)) + tk.MustExec("analyze incremental table test.t index idx_b") + require.Nil(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertEvictIndex")) +} diff --git a/statistics/integration_test.go b/statistics/integration_test.go index ec6a9f0aaffb2..e998753560109 100644 --- a/statistics/integration_test.go +++ b/statistics/integration_test.go @@ -169,7 +169,7 @@ func TestChangeVerTo2BehaviorWithPersistedOptions(t *testing.T) { tk.MustExec("analyze table t index idx") tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead", "Warning 1105 The version 2 would collect all statistics not only the selected indexes", - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t")) // since fallback to ver2 path, should do samplerate adjustment + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/3) as the sample-rate=1\"")) // since fallback to ver2 path, should do samplerate adjustment require.NoError(t, h.Update(is)) statsTblT = h.GetTableStats(tblT.Meta()) for _, idx := range statsTblT.Indices { @@ -178,7 +178,7 @@ func TestChangeVerTo2BehaviorWithPersistedOptions(t *testing.T) { tk.MustExec("analyze table t index") tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead", "Warning 1105 The version 2 would collect all statistics not only the selected indexes", - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t")) + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/3) as the sample-rate=1\"")) require.NoError(t, h.Update(is)) statsTblT = h.GetTableStats(tblT.Meta()) for _, idx := range statsTblT.Indices { From f2a009c8dff2b7b1a46efc7f7ccdf96f90bc5e64 Mon Sep 17 00:00:00 2001 From: qw4990 Date: Thu, 12 Oct 2023 17:17:20 +0800 Subject: [PATCH 2/9] fixup --- executor/builder.go | 11 +---------- statistics/analyze_jobs.go | 6 +----- 2 files changed, 2 insertions(+), 15 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 5d74586712ff0..aa9a87d2d4a05 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2604,11 +2604,7 @@ func (b *executorBuilder) buildAnalyzeSamplingPushdown(task plannercore.AnalyzeC if opts[ast.AnalyzeOptNumSamples] == 0 { *sampleRate = math.Float64frombits(opts[ast.AnalyzeOptSampleRate]) if *sampleRate < 0 { -<<<<<<< HEAD - *sampleRate = b.getAdjustedSampleRate(b.ctx, task) -======= *sampleRate, sampleRateReason = b.getAdjustedSampleRate(task) ->>>>>>> 6fb20c9d306 (planner: log the reason why the sample-rate is chosen when analyzing table (#45938)) if task.PartitionName != "" { sc.AppendNote(errors.Errorf( `Analyze use auto adjusted sample rate %f for table %s.%s's partition %s, reason to use this rate is "%s"`, @@ -2690,13 +2686,8 @@ func (b *executorBuilder) buildAnalyzeSamplingPushdown(task plannercore.AnalyzeC // If we take n = 1e12, a 300*k sample still gives <= 0.66 bin size error with probability 0.99. // So if we don't consider the top-n values, we can keep the sample size at 300*256. // But we may take some top-n before building the histogram, so we increase the sample a little. -<<<<<<< HEAD -func (b *executorBuilder) getAdjustedSampleRate(sctx sessionctx.Context, task plannercore.AnalyzeColumnsTask) float64 { - statsHandle := domain.GetDomain(sctx).StatsHandle() -======= -func (b *executorBuilder) getAdjustedSampleRate(task plannercore.AnalyzeColumnsTask) (sampleRate float64, reason string) { +func (b *executorBuilder) getAdjustedSampleRate(sctx sessionctx.Context, task plannercore.AnalyzeColumnsTask) (sampleRate float64, reason string) { statsHandle := domain.GetDomain(b.ctx).StatsHandle() ->>>>>>> 6fb20c9d306 (planner: log the reason why the sample-rate is chosen when analyzing table (#45938)) defaultRate := 0.001 if statsHandle == nil { return defaultRate, fmt.Sprintf("statsHandler is nil, use the default-rate=%v", defaultRate) diff --git a/statistics/analyze_jobs.go b/statistics/analyze_jobs.go index 930cfabdfe0a3..4f8cb2af2e401 100644 --- a/statistics/analyze_jobs.go +++ b/statistics/analyze_jobs.go @@ -26,15 +26,11 @@ type AnalyzeJob struct { TableName string PartitionName string JobInfo string -<<<<<<< HEAD StartTime time.Time EndTime time.Time - Progress AnalyzeProgress -======= - SampleRateReason string // why this sample-rate is chosen Progress AnalyzeProgress ->>>>>>> 6fb20c9d306 (planner: log the reason why the sample-rate is chosen when analyzing table (#45938)) + SampleRateReason string // why this sample-rate is chosen } // AnalyzeProgress represents the process of one analyze job. From c16144eac0d8b00a7672b34223e29bc981a93e2b Mon Sep 17 00:00:00 2001 From: qw4990 Date: Thu, 12 Oct 2023 17:18:53 +0800 Subject: [PATCH 3/9] fixup --- .../handle/handletest/analyze/analyze_test.go | 298 ------------------ 1 file changed, 298 deletions(-) delete mode 100644 statistics/handle/handletest/analyze/analyze_test.go diff --git a/statistics/handle/handletest/analyze/analyze_test.go b/statistics/handle/handletest/analyze/analyze_test.go deleted file mode 100644 index 6c42871bc8a62..0000000000000 --- a/statistics/handle/handletest/analyze/analyze_test.go +++ /dev/null @@ -1,298 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package analyze - -import ( - "bytes" - "fmt" - "testing" - - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics/handle" - "github.com/pingcap/tidb/testkit" - "github.com/stretchr/testify/require" -) - -// nolint:unused -func checkForGlobalStatsWithOpts(t *testing.T, dom *domain.Domain, db, tt, pp string, topn, buckets int) { - tbl, err := dom.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(tt)) - require.NoError(t, err) - - tblInfo := tbl.Meta() - physicalID := tblInfo.ID - if pp != "global" { - for _, def := range tbl.Meta().GetPartitionInfo().Definitions { - if def.Name.L == pp { - physicalID = def.ID - } - } - } - tblStats, err := dom.StatsHandle().TableStatsFromStorage(tblInfo, physicalID, true, 0) - require.NoError(t, err) - - delta := buckets/2 + 10 - for _, idxStats := range tblStats.Indices { - if len(idxStats.Buckets) == 0 { - continue // it's not loaded - } - numTopN := idxStats.TopN.Num() - numBuckets := len(idxStats.Buckets) - // since the hist-building algorithm doesn't stipulate the final bucket number to be equal to the expected number exactly, - // we have to check the results by a range here. - require.Equal(t, topn, numTopN) - require.GreaterOrEqual(t, numBuckets, buckets-delta) - require.LessOrEqual(t, numBuckets, buckets+delta) - } - for _, colStats := range tblStats.Columns { - if len(colStats.Buckets) == 0 { - continue // it's not loaded - } - numTopN := colStats.TopN.Num() - numBuckets := len(colStats.Buckets) - require.Equal(t, topn, numTopN) - require.GreaterOrEqual(t, numBuckets, buckets-delta) - require.LessOrEqual(t, numBuckets, buckets+delta) - } -} - -// nolint:unused -func prepareForGlobalStatsWithOptsV2(t *testing.T, dom *domain.Domain, tk *testkit.TestKit, tblName, dbName string) { - tk.MustExec("create database if not exists " + dbName) - tk.MustExec("use " + dbName) - tk.MustExec("drop table if exists " + tblName) - tk.MustExec(` create table ` + tblName + ` (a int, key(a)) partition by range (a) ` + - `(partition p0 values less than (100000), partition p1 values less than (200000))`) - buf1 := bytes.NewBufferString("insert into " + tblName + " values (0)") - buf2 := bytes.NewBufferString("insert into " + tblName + " values (100000)") - for i := 0; i < 1000; i++ { - buf1.WriteString(fmt.Sprintf(", (%v)", 2)) - buf2.WriteString(fmt.Sprintf(", (%v)", 100002)) - buf1.WriteString(fmt.Sprintf(", (%v)", 1)) - buf2.WriteString(fmt.Sprintf(", (%v)", 100001)) - buf1.WriteString(fmt.Sprintf(", (%v)", 0)) - buf2.WriteString(fmt.Sprintf(", (%v)", 100000)) - } - for i := 0; i < 5000; i += 3 { - buf1.WriteString(fmt.Sprintf(", (%v)", i)) - buf2.WriteString(fmt.Sprintf(", (%v)", 100000+i)) - } - tk.MustExec(buf1.String()) - tk.MustExec(buf2.String()) - tk.MustExec("set @@tidb_analyze_version=2") - tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") - require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) -} - -// nolint:unused -func prepareForGlobalStatsWithOpts(t *testing.T, dom *domain.Domain, tk *testkit.TestKit, tblName, dbName string) { - tk.MustExec("create database if not exists " + dbName) - tk.MustExec("use " + dbName) - tk.MustExec("drop table if exists " + tblName) - tk.MustExec(` create table ` + tblName + ` (a int, key(a)) partition by range (a) ` + - `(partition p0 values less than (100000), partition p1 values less than (200000))`) - buf1 := bytes.NewBufferString("insert into " + tblName + " values (0)") - buf2 := bytes.NewBufferString("insert into " + tblName + " values (100000)") - for i := 0; i < 5000; i += 3 { - buf1.WriteString(fmt.Sprintf(", (%v)", i)) - buf2.WriteString(fmt.Sprintf(", (%v)", 100000+i)) - } - for i := 0; i < 1000; i++ { - buf1.WriteString(fmt.Sprintf(", (%v)", 0)) - buf2.WriteString(fmt.Sprintf(", (%v)", 100000)) - } - tk.MustExec(buf1.String()) - tk.MustExec(buf2.String()) - tk.MustExec("set @@tidb_analyze_version=2") - tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") - require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) -} - -func TestAnalyzeVirtualCol(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int generated always as (-a) virtual, c int generated always as (-a) stored, index (c))") - tk.MustExec("insert into t(a) values(2),(1),(1),(3),(NULL)") - tk.MustExec("set @@tidb_analyze_version = 2") - tk.MustExec("analyze table t") - require.Len(t, tk.MustQuery("show stats_histograms where table_name ='t'").Rows(), 3) -} - -func TestAnalyzeGlobalStatsWithOpts1(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - tk := testkit.NewTestKit(t, store) - prepareForGlobalStatsWithOpts(t, dom, tk, "test_gstats_opt", "test_gstats_opt") - - // nolint:unused - type opt struct { - topn int - buckets int - err bool - } - - cases := []opt{ - {1, 37, false}, - {2, 47, false}, - {10, 77, false}, - {77, 219, false}, - {-31, 222, true}, - {10, -77, true}, - {10000, 47, true}, - {77, 47000, true}, - } - for _, ca := range cases { - sql := fmt.Sprintf("analyze table test_gstats_opt with %v topn, %v buckets", ca.topn, ca.buckets) - if !ca.err { - tk.MustExec(sql) - checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt", "test_gstats_opt", "global", ca.topn, ca.buckets) - checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt", "test_gstats_opt", "p0", ca.topn, ca.buckets) - checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt", "test_gstats_opt", "p1", ca.topn, ca.buckets) - } else { - err := tk.ExecToErr(sql) - require.Error(t, err) - } - } -} - -func TestAnalyzeGlobalStatsWithOpts2(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - tk := testkit.NewTestKit(t, store) - originalVal1 := tk.MustQuery("select @@tidb_persist_analyze_options").Rows()[0][0].(string) - defer func() { - tk.MustExec(fmt.Sprintf("set global tidb_persist_analyze_options = %v", originalVal1)) - }() - tk.MustExec("set global tidb_persist_analyze_options=false") - prepareForGlobalStatsWithOptsV2(t, dom, tk, "test_gstats_opt2", "test_gstats_opt2") - - tk.MustExec("analyze table test_gstats_opt2 with 2 topn, 10 buckets, 1000 samples") - checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "global", 2, 10) - checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p0", 2, 10) - checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p1", 2, 10) - - // analyze a partition to let its options be different with others' - tk.MustExec("analyze table test_gstats_opt2 partition p0 with 3 topn, 20 buckets") - checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "global", 3, 20) // use new options - checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p0", 3, 20) - checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p1", 2, 10) - - tk.MustExec("analyze table test_gstats_opt2 partition p1 with 1 topn, 15 buckets") - checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "global", 1, 15) - checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p0", 3, 20) - checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p1", 1, 15) - - tk.MustExec("analyze table test_gstats_opt2 partition p0 with 2 topn, 10 buckets") // change back to 2 topn - checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "global", 2, 10) - checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p0", 2, 10) - checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p1", 1, 15) -} - -func TestAnalyzeWithDynamicPartitionPruneMode(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set @@tidb_partition_prune_mode = '" + string(variable.Dynamic) + "'") - tk.MustExec("set @@tidb_analyze_version = 2") - tk.MustExec(`create table t (a int, key(a)) partition by range(a) - (partition p0 values less than (10), - partition p1 values less than (22))`) - tk.MustExec(`insert into t values (1), (2), (3), (10), (11)`) - tk.MustExec(`analyze table t with 1 topn, 2 buckets`) - rows := tk.MustQuery("show stats_buckets where partition_name = 'global' and is_index=1").Rows() - require.Len(t, rows, 2) - require.Equal(t, "4", rows[1][6]) - tk.MustExec("insert into t values (1), (2), (2)") - tk.MustExec("analyze table t partition p0 with 1 topn, 2 buckets") - rows = tk.MustQuery("show stats_buckets where partition_name = 'global' and is_index=1").Rows() - require.Len(t, rows, 2) - require.Equal(t, "5", rows[1][6]) - tk.MustExec("insert into t values (3)") - tk.MustExec("analyze table t partition p0 index a with 1 topn, 2 buckets") - rows = tk.MustQuery("show stats_buckets where partition_name = 'global' and is_index=1").Rows() - require.Len(t, rows, 1) - require.Equal(t, "6", rows[0][6]) -} - -func TestFMSWithAnalyzePartition(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set @@tidb_partition_prune_mode = '" + string(variable.Dynamic) + "'") - tk.MustExec("set @@tidb_analyze_version = 2") - tk.MustExec(`create table t (a int, key(a)) partition by range(a) - (partition p0 values less than (10), - partition p1 values less than (22))`) - tk.MustExec(`insert into t values (1), (2), (3), (10), (11)`) - tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("0")) - tk.MustExec("analyze table t partition p0 with 1 topn, 2 buckets") - tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", - "Warning 1105 Ignore columns and options when analyze partition in dynamic mode", - "Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`", - "Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`", - )) - tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("2")) -} - -func TestFastAnalyzeColumnHistWithNullValue(t *testing.T) { - store := testkit.CreateMockStore(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("drop table if exists t") - testKit.MustExec("create table t (a int)") - testKit.MustExec("insert into t values (1), (2), (3), (4), (NULL)") - testKit.MustExec("set @@session.tidb_analyze_version = 1") - testKit.MustExec("set @@tidb_enable_fast_analyze=1") - defer testKit.MustExec("set @@tidb_enable_fast_analyze=0") - testKit.MustExec("analyze table t with 0 topn, 2 buckets") - // If NULL is in hist, the min(lower_bound) will be "". - testKit.MustQuery("select min(lower_bound) from mysql.stats_buckets").Check(testkit.Rows("1")) -} - -func TestAnalyzeIncrementalEvictedIndex(t *testing.T) { - t.Skip("now we don't support to evict index") - restore := config.RestoreFunc() - defer restore() - config.UpdateGlobal(func(conf *config.Config) { - conf.Performance.EnableStatsCacheMemQuota = true - }) - store, dom := testkit.CreateMockStoreAndDomain(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("set @@tidb_analyze_version = 1") - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b varchar(10), index idx_b (b))") - tk.MustExec("analyze table test.t") - tbl, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.Nil(t, err) - tblStats := domain.GetDomain(tk.Session()).StatsHandle().GetTableStats(tbl.Meta()) - for _, index := range tblStats.Indices { - require.False(t, index.IsEvicted()) - } - - domain.GetDomain(tk.Session()).StatsHandle().SetStatsCacheCapacity(1) - tblStats = domain.GetDomain(tk.Session()).StatsHandle().GetTableStats(tbl.Meta()) - for _, index := range tblStats.Indices { - require.True(t, index.IsEvicted()) - } - - require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertEvictIndex", `return(true)`)) - tk.MustExec("analyze incremental table test.t index idx_b") - require.Nil(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertEvictIndex")) -} From 6b4108b39c51723f3be39b226783dbeae8f6c1d0 Mon Sep 17 00:00:00 2001 From: qw4990 Date: Thu, 12 Oct 2023 17:20:23 +0800 Subject: [PATCH 4/9] fixup --- statistics/handle/handletest/handle_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/statistics/handle/handletest/handle_test.go b/statistics/handle/handletest/handle_test.go index df2562b1d2f4b..dd18db4b46ef7 100644 --- a/statistics/handle/handletest/handle_test.go +++ b/statistics/handle/handletest/handle_test.go @@ -2186,7 +2186,7 @@ func TestFMSWithAnalyzePartition(t *testing.T) { tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("0")) tk.MustExec("analyze table t partition p0 with 1 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Ignore columns and options when analyze partition in dynamic mode", "Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`", "Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`", From 05d7a09b7acd4f25c72ebaf084be9bc16b9453a8 Mon Sep 17 00:00:00 2001 From: qw4990 Date: Mon, 16 Oct 2023 10:35:15 +0800 Subject: [PATCH 5/9] fixup --- executor/builder.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executor/builder.go b/executor/builder.go index aa9a87d2d4a05..75afaa3f96ace 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2604,7 +2604,7 @@ func (b *executorBuilder) buildAnalyzeSamplingPushdown(task plannercore.AnalyzeC if opts[ast.AnalyzeOptNumSamples] == 0 { *sampleRate = math.Float64frombits(opts[ast.AnalyzeOptSampleRate]) if *sampleRate < 0 { - *sampleRate, sampleRateReason = b.getAdjustedSampleRate(task) + *sampleRate, sampleRateReason = b.getAdjustedSampleRate(b.ctx, task) if task.PartitionName != "" { sc.AppendNote(errors.Errorf( `Analyze use auto adjusted sample rate %f for table %s.%s's partition %s, reason to use this rate is "%s"`, From 93c572d5e04d49aca869df25a089e42cf55a082b Mon Sep 17 00:00:00 2001 From: qw4990 Date: Mon, 16 Oct 2023 11:04:13 +0800 Subject: [PATCH 6/9] fixup --- planner/core/indexmerge_path_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/indexmerge_path_test.go b/planner/core/indexmerge_path_test.go index 66d1a0bd9baec..5efdce0b2cf1a 100644 --- a/planner/core/indexmerge_path_test.go +++ b/planner/core/indexmerge_path_test.go @@ -44,7 +44,7 @@ index idx2(a, b, (cast(j->'$.str' as char(10) array)), c))`) "Warning 1105 analyzing multi-valued indexes is not supported, skip idx2")) tk.MustExec("analyze table t index idx") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"TiDB assumes that the table is empty, use sample-rate=1\"", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"TiDB assumes that the table is empty and cannot get row count from PD, use sample-rate=1\"", "Warning 1105 The version 2 would collect all statistics not only the selected indexes", "Warning 1105 analyzing multi-valued indexes is not supported, skip idx", "Warning 1105 analyzing multi-valued indexes is not supported, skip idx2")) From 64506e95586f10cd4e7e7b1262fb008c386de8ac Mon Sep 17 00:00:00 2001 From: qw4990 Date: Mon, 16 Oct 2023 11:39:37 +0800 Subject: [PATCH 7/9] fixup --- executor/analyzetest/analyze_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executor/analyzetest/analyze_test.go b/executor/analyzetest/analyze_test.go index c00b2fe5812ca..3513f2378ca38 100644 --- a/executor/analyzetest/analyze_test.go +++ b/executor/analyzetest/analyze_test.go @@ -2179,7 +2179,7 @@ func TestAnalyzeColumnsErrorAndWarning(t *testing.T) { tk.MustExec("analyze table t predicate columns") } tk.MustQuery("show warnings").Sort().Check(testkit.Rows( - `Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is "TiDB assumes that the table is empty, use sample-rate=1"`, + `Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is "TiDB assumes that the table is empty and cannot get row count from PD, use sample-rate=1"`, "Warning 1105 Table test.t has version 1 statistics so all the columns must be analyzed to overwrite the current statistics", )) }(val) From fc0e024fc869d328610615db595752d9353b21b8 Mon Sep 17 00:00:00 2001 From: qw4990 Date: Mon, 16 Oct 2023 12:08:22 +0800 Subject: [PATCH 8/9] fixup --- executor/analyzetest/analyze_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executor/analyzetest/analyze_test.go b/executor/analyzetest/analyze_test.go index 3513f2378ca38..aff035fd4135a 100644 --- a/executor/analyzetest/analyze_test.go +++ b/executor/analyzetest/analyze_test.go @@ -2127,7 +2127,7 @@ func TestAnalyzeSampleRateReason(t *testing.T) { require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) tk.MustExec(`analyze table t`) tk.MustQuery(`show warnings`).Sort().Check(testkit.Rows( - `Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is "TiDB assumes that the table is empty, use sample-rate=1"`)) + `Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"TiDB assumes that the table is empty and cannot get row count from PD, use sample-rate=1\"`)) } func TestAnalyzeColumnsErrorAndWarning(t *testing.T) { From c5ebfa8dd314da8369d65014ad94e66f059d771b Mon Sep 17 00:00:00 2001 From: qw4990 Date: Mon, 16 Oct 2023 12:18:47 +0800 Subject: [PATCH 9/9] fixup --- executor/analyzetest/analyze_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executor/analyzetest/analyze_test.go b/executor/analyzetest/analyze_test.go index aff035fd4135a..cae564758b6be 100644 --- a/executor/analyzetest/analyze_test.go +++ b/executor/analyzetest/analyze_test.go @@ -2127,7 +2127,7 @@ func TestAnalyzeSampleRateReason(t *testing.T) { require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) tk.MustExec(`analyze table t`) tk.MustQuery(`show warnings`).Sort().Check(testkit.Rows( - `Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"TiDB assumes that the table is empty and cannot get row count from PD, use sample-rate=1\"`)) + `Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is "TiDB assumes that the table is empty and cannot get row count from PD, use sample-rate=1"`)) } func TestAnalyzeColumnsErrorAndWarning(t *testing.T) {