Skip to content

Commit 202043e

Browse files
committed
migrate test for testify in analyze_test.go, some tests failed
1 parent ed384b2 commit 202043e

File tree

2 files changed

+565
-440
lines changed

2 files changed

+565
-440
lines changed

executor/analyze_serial_test.go

Lines changed: 302 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,302 @@
1+
// Copyright 2021 PingCAP, Inc.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
// See the License for the specific language governing permissions and
13+
// limitations under the License.
14+
15+
package executor_test
16+
17+
import (
18+
"testing"
19+
20+
"github.com/pingcap/failpoint"
21+
"github.com/pingcap/tidb/domain"
22+
"github.com/pingcap/tidb/parser/model"
23+
"github.com/pingcap/tidb/session"
24+
"github.com/pingcap/tidb/statistics"
25+
"github.com/pingcap/tidb/statistics/handle"
26+
"github.com/pingcap/tidb/testkit"
27+
"github.com/pingcap/tidb/types"
28+
"github.com/pingcap/tidb/util/codec"
29+
"github.com/pingcap/tidb/util/collate"
30+
"github.com/stretchr/testify/require"
31+
)
32+
33+
func TestFastAnalyze4GlobalStats(t *testing.T) {
34+
store, clean := testkit.CreateMockStore(t)
35+
defer clean()
36+
37+
tk := testkit.NewTestKit(t, store)
38+
tk.MustExec(`create database if not exists test_fast_gstats`)
39+
tk.MustExec("use test_fast_gstats")
40+
tk.MustExec("set @@session.tidb_enable_fast_analyze=1")
41+
tk.MustExec("set @@session.tidb_build_stats_concurrency=1")
42+
// test fast analyze in dynamic mode
43+
tk.MustExec("set @@session.tidb_analyze_version = 2;")
44+
tk.MustExec("set @@session.tidb_partition_prune_mode = 'dynamic';")
45+
tk.MustExec("drop table if exists test_fast_gstats;")
46+
tk.MustExec("create table test_fast_gstats(a int, b int) PARTITION BY HASH(a) PARTITIONS 2;")
47+
tk.MustExec("insert into test_fast_gstats values(1,1),(3,3),(4,4),(2,2),(5,5);")
48+
err := tk.ExecToErr("analyze table test_fast_gstats;")
49+
require.EqualError(t, err, "Fast analyze hasn't reached General Availability and only support analyze version 1 currently.")
50+
}
51+
52+
func TestAnalyzeIndex(t *testing.T) {
53+
store, clean := testkit.CreateMockStore(t)
54+
defer clean()
55+
56+
tk := testkit.NewTestKit(t, store)
57+
tk.MustExec("use test")
58+
tk.MustExec("drop table if exists t1")
59+
tk.MustExec("create table t1 (id int, v int, primary key(id), index k(v))")
60+
tk.MustExec("insert into t1(id, v) values(1, 2), (2, 2), (3, 2), (4, 2), (5, 1), (6, 3), (7, 4)")
61+
tk.MustExec("set @@tidb_analyze_version=1")
62+
tk.MustExec("analyze table t1 index k")
63+
require.Greater(t, len(tk.MustQuery("show stats_buckets where table_name = 't1' and column_name = 'k' and is_index = 1").Rows()), 0)
64+
tk.MustExec("set @@tidb_analyze_version=default")
65+
tk.MustExec("analyze table t1")
66+
require.Greater(t, len(tk.MustQuery("show stats_topn where table_name = 't1' and column_name = 'k' and is_index = 1").Rows()), 0)
67+
68+
func() {
69+
defer tk.MustExec("set @@session.tidb_enable_fast_analyze=0")
70+
tk.MustExec("drop stats t1")
71+
tk.MustExec("set @@session.tidb_enable_fast_analyze=1")
72+
tk.MustExec("set @@tidb_analyze_version=1")
73+
tk.MustExec("analyze table t1 index k")
74+
require.Greater(t, len(tk.MustQuery("show stats_buckets where table_name = 't1' and column_name = 'k' and is_index = 1").Rows()), 1)
75+
}()
76+
}
77+
78+
func TestAnalyzeIncremental(t *testing.T) {
79+
store, clean := testkit.CreateMockStore(t)
80+
defer clean()
81+
dom, err := session.BootstrapSession(store)
82+
require.NoError(t, err)
83+
84+
tk := testkit.NewTestKit(t, store)
85+
tk.MustExec("use test")
86+
tk.MustExec("set @@tidb_analyze_version = 1")
87+
tk.Session().GetSessionVars().EnableStreaming = false
88+
testAnalyzeIncremental(tk, t, dom)
89+
}
90+
91+
func TestAnalyzeIncrementalStreaming(t *testing.T) {
92+
t.Skip("unistore hasn't support streaming yet.")
93+
store, clean := testkit.CreateMockStore(t)
94+
dom, err := session.BootstrapSession(store)
95+
require.NoError(t, err)
96+
97+
defer clean()
98+
tk := testkit.NewTestKit(t, store)
99+
tk.MustExec("use test")
100+
tk.Session().GetSessionVars().EnableStreaming = true
101+
testAnalyzeIncremental(tk, t, dom)
102+
}
103+
104+
func testAnalyzeIncremental(tk *testkit.TestKit, t *testing.T, dom *domain.Domain) {
105+
tk.MustExec("use test")
106+
tk.MustExec("drop table if exists t")
107+
tk.MustExec("create table t(a int, b int, primary key(a), index idx(b))")
108+
tk.MustExec("analyze incremental table t index")
109+
tk.MustQuery("show stats_buckets").Check(testkit.Rows())
110+
tk.MustExec("insert into t values (1,1)")
111+
tk.MustExec("analyze incremental table t index")
112+
tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t idx 1 0 1 1 1 1 0"))
113+
tk.MustExec("insert into t values (2,2)")
114+
tk.MustExec("analyze incremental table t index")
115+
tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t a 0 1 2 1 2 2 0", "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0"))
116+
tk.MustExec("analyze incremental table t index")
117+
// Result should not change.
118+
tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t a 0 1 2 1 2 2 0", "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0"))
119+
120+
// Test analyze incremental with feedback.
121+
tk.MustExec("insert into t values (3,3)")
122+
oriProbability := statistics.FeedbackProbability.Load()
123+
oriMinLogCount := handle.MinLogScanCount.Load()
124+
defer func() {
125+
statistics.FeedbackProbability.Store(oriProbability)
126+
handle.MinLogScanCount.Store(oriMinLogCount)
127+
}()
128+
statistics.FeedbackProbability.Store(1)
129+
handle.MinLogScanCount.Store(0)
130+
is := dom.InfoSchema()
131+
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
132+
require.NoError(t, err)
133+
tblInfo := table.Meta()
134+
tk.MustQuery("select * from t use index(idx) where b = 3")
135+
tk.MustQuery("select * from t where a > 1")
136+
h := dom.StatsHandle()
137+
require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll))
138+
require.NoError(t, h.DumpStatsFeedbackToKV())
139+
require.NoError(t, h.HandleUpdateStats(is))
140+
require.NoError(t, h.Update(is))
141+
require.NoError(t, h.LoadNeededHistograms())
142+
tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t a 0 1 3 0 2 2147483647 0", "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0"))
143+
tblStats := h.GetTableStats(tblInfo)
144+
val, err := codec.EncodeKey(tk.Session().GetSessionVars().StmtCtx, nil, types.NewIntDatum(3))
145+
require.NoError(t, err)
146+
require.Equal(t, uint64(1), tblStats.Indices[tblInfo.Indices[0].ID].QueryBytes(val))
147+
require.False(t, statistics.IsAnalyzed(tblStats.Indices[tblInfo.Indices[0].ID].Flag))
148+
require.False(t, statistics.IsAnalyzed(tblStats.Columns[tblInfo.Columns[0].ID].Flag))
149+
150+
tk.MustExec("analyze incremental table t index")
151+
require.NoError(t, h.LoadNeededHistograms())
152+
tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t a 0 1 2 1 2 2 0", "test t a 0 2 3 1 3 3 0",
153+
"test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0", "test t idx 1 2 3 1 3 3 0"))
154+
tblStats = h.GetTableStats(tblInfo)
155+
require.Equal(t, uint64(1), tblStats.Indices[tblInfo.Indices[0].ID].QueryBytes(val))
156+
157+
// test analyzeIndexIncremental for global-level stats;
158+
tk.MustExec("set @@session.tidb_analyze_version = 1;")
159+
tk.MustQuery("select @@tidb_analyze_version").Check(testkit.Rows("1"))
160+
tk.MustExec("set @@tidb_partition_prune_mode = 'static';")
161+
tk.MustExec("drop table if exists t;")
162+
tk.MustExec(`create table t (a int, b int, primary key(a), index idx(b)) partition by range (a) (
163+
partition p0 values less than (10),
164+
partition p1 values less than (20),
165+
partition p2 values less than (30)
166+
);`)
167+
tk.MustExec("analyze incremental table t index")
168+
require.NoError(t, h.LoadNeededHistograms())
169+
tk.MustQuery("show stats_buckets").Check(testkit.Rows())
170+
tk.MustExec("insert into t values (1,1)")
171+
tk.MustExec("analyze incremental table t index")
172+
tk.MustQuery("show warnings").Check(testkit.Rows()) // no warning
173+
require.NoError(t, h.LoadNeededHistograms())
174+
tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t p0 a 0 0 1 1 1 1 0", "test t p0 idx 1 0 1 1 1 1 0"))
175+
tk.MustExec("insert into t values (2,2)")
176+
tk.MustExec("analyze incremental table t index")
177+
require.NoError(t, h.LoadNeededHistograms())
178+
tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t p0 a 0 0 1 1 1 1 0", "test t p0 a 0 1 2 1 2 2 0", "test t p0 idx 1 0 1 1 1 1 0", "test t p0 idx 1 1 2 1 2 2 0"))
179+
tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic';")
180+
tk.MustExec("insert into t values (11,11)")
181+
err = tk.ExecToErr("analyze incremental table t index")
182+
require.Equal(t, "[stats]: global statistics for partitioned tables unavailable in ANALYZE INCREMENTAL", err.Error())
183+
}
184+
185+
func TestIssue27429(t *testing.T) {
186+
collate.SetNewCollationEnabledForTest(true)
187+
defer collate.SetNewCollationEnabledForTest(false)
188+
store, clean := testkit.CreateMockStore(t)
189+
defer clean()
190+
191+
tk := testkit.NewTestKit(t, store)
192+
tk.MustExec("use test")
193+
tk.MustExec("drop table if exists t")
194+
tk.MustExec("create table test.t(id int, value varchar(20) charset utf8mb4 collate utf8mb4_general_ci, value1 varchar(20) charset utf8mb4 collate utf8mb4_bin)")
195+
tk.MustExec("insert into test.t values (1, 'abc', 'abc '),(4, 'Abc', 'abc'),(3,'def', 'def ');")
196+
197+
tk.MustQuery("select upper(group_concat(distinct value order by 1)) from test.t;").Check(testkit.Rows("ABC,DEF"))
198+
tk.MustQuery("select upper(group_concat(distinct value)) from test.t;").Check(testkit.Rows("ABC,DEF"))
199+
}
200+
201+
func TestIssue20874(t *testing.T) {
202+
collate.SetNewCollationEnabledForTest(true)
203+
defer collate.SetNewCollationEnabledForTest(false)
204+
store, clean := testkit.CreateMockStore(t)
205+
defer clean()
206+
207+
tk := testkit.NewTestKit(t, store)
208+
tk.MustExec("use test")
209+
tk.MustExec("drop table if exists t")
210+
tk.MustExec("delete from mysql.stats_histograms")
211+
tk.MustExec("create table t (a char(10) collate utf8mb4_unicode_ci not null, b char(20) collate utf8mb4_general_ci not null, key idxa(a), key idxb(b))")
212+
tk.MustExec("insert into t values ('#', 'C'), ('$', 'c'), ('a', 'a')")
213+
tk.MustExec("set @@tidb_analyze_version=1")
214+
tk.MustExec("analyze table t")
215+
tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check(testkit.Rows(
216+
"test t a 0 0 1 1 \x02\xd2 \x02\xd2 0",
217+
"test t a 0 1 2 1 \x0e\x0f \x0e\x0f 0",
218+
"test t a 0 2 3 1 \x0e3 \x0e3 0",
219+
"test t b 0 0 1 1 \x00A \x00A 0",
220+
"test t b 0 1 3 2 \x00C \x00C 0",
221+
"test t idxa 1 0 1 1 \x02\xd2 \x02\xd2 0",
222+
"test t idxa 1 1 2 1 \x0e\x0f \x0e\x0f 0",
223+
"test t idxa 1 2 3 1 \x0e3 \x0e3 0",
224+
"test t idxb 1 0 1 1 \x00A \x00A 0",
225+
"test t idxb 1 1 3 2 \x00C \x00C 0",
226+
))
227+
tk.MustQuery("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, correlation from mysql.stats_histograms").Sort().Check(testkit.Rows(
228+
"0 1 3 0 9 1 1",
229+
"0 2 2 0 9 1 -0.5",
230+
"1 1 3 0 0 1 0",
231+
"1 2 2 0 0 1 0",
232+
))
233+
tk.MustExec("set @@tidb_analyze_version=2")
234+
tk.MustExec("analyze table t")
235+
tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check(testkit.Rows(
236+
"test t a 0 \x02\xd2 1",
237+
"test t a 0 \x0e\x0f 1",
238+
"test t a 0 \x0e3 1",
239+
"test t b 0 \x00A 1",
240+
"test t b 0 \x00C 2",
241+
"test t idxa 1 \x02\xd2 1",
242+
"test t idxa 1 \x0e\x0f 1",
243+
"test t idxa 1 \x0e3 1",
244+
"test t idxb 1 \x00A 1",
245+
"test t idxb 1 \x00C 2",
246+
))
247+
tk.MustQuery("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, correlation from mysql.stats_histograms").Sort().Check(testkit.Rows(
248+
"0 1 3 0 6 2 1",
249+
"0 2 2 0 6 2 -0.5",
250+
"1 1 3 0 6 2 0",
251+
"1 2 2 0 6 2 0",
252+
))
253+
}
254+
255+
func TestAnalyzeClusteredIndexPrimary(t *testing.T) {
256+
store, clean := testkit.CreateMockStore(t)
257+
defer clean()
258+
259+
tk := testkit.NewTestKit(t, store)
260+
tk.MustExec("use test")
261+
tk.MustExec("drop table if exists t0")
262+
tk.MustExec("drop table if exists t1")
263+
tk.MustExec("create table t0(a varchar(20), primary key(a) clustered)")
264+
tk.MustExec("create table t1(a varchar(20), primary key(a))")
265+
tk.MustExec("insert into t0 values('1111')")
266+
tk.MustExec("insert into t1 values('1111')")
267+
tk.MustExec("set @@session.tidb_analyze_version = 1")
268+
tk.MustExec("analyze table t0 index primary")
269+
tk.MustExec("analyze table t1 index primary")
270+
tk.MustQuery("show stats_buckets").Check(testkit.Rows(
271+
"test t0 PRIMARY 1 0 1 1 1111 1111 0",
272+
"test t1 PRIMARY 1 0 1 1 1111 1111 0"))
273+
tk.MustExec("set @@session.tidb_analyze_version = 2")
274+
tk.MustExec("analyze table t0")
275+
tk.MustExec("analyze table t1")
276+
tk.MustQuery("show stats_topn").Sort().Check(testkit.Rows(""+
277+
"test t0 PRIMARY 1 1111 1",
278+
"test t0 a 0 1111 1",
279+
"test t1 PRIMARY 1 1111 1",
280+
"test t1 a 0 1111 1"))
281+
}
282+
283+
func TestAnalyzeSamplingWorkPanic(t *testing.T) {
284+
store, clean := testkit.CreateMockStore(t)
285+
defer clean()
286+
287+
tk := testkit.NewTestKit(t, store)
288+
tk.MustExec("use test")
289+
tk.MustExec("set @@session.tidb_analyze_version = 2")
290+
tk.MustExec("create table t(a int)")
291+
tk.MustExec("insert into t values(1), (2), (3), (4), (5), (6), (7), (8), (9), (10), (11), (12)")
292+
tk.MustExec("split table t between (-9223372036854775808) and (9223372036854775807) regions 12")
293+
294+
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingBuildWorkerPanic", "return(1)"))
295+
err := tk.ExecToErr("analyze table t")
296+
require.NotNil(t, err)
297+
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingBuildWorkerPanic"))
298+
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingMergeWorkerPanic", "return(1)"))
299+
err = tk.ExecToErr("analyze table t")
300+
require.NotNil(t, err)
301+
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingMergeWorkerPanic"))
302+
}

0 commit comments

Comments
 (0)