Skip to content

Commit

Permalink
statistics: add bucket ndv for index histogram (#20580)
Browse files Browse the repository at this point in the history
Co-authored-by: Yuanjia Zhang <zhangyuanjia@pingcap.com>
Co-authored-by: ti-srebot <66930949+ti-srebot@users.noreply.github.com>
  • Loading branch information
3 people committed Jan 13, 2021
1 parent 68aaa2f commit 3dd842f
Show file tree
Hide file tree
Showing 26 changed files with 439 additions and 267 deletions.
2 changes: 1 addition & 1 deletion cmd/explaintest/run-tests.sh
Original file line number Diff line number Diff line change
Expand Up @@ -208,7 +208,7 @@ if [ "${TIDB_TEST_STORE_NAME}" = "tikv" ]; then
$tidb_server -P "$port" -status "$status" -config config.toml -store tikv -path "${TIKV_PATH}" > $explain_test_log 2>&1 &
SERVER_PID=$!
else
$tidb_server -P "$port" -status "$status" -config config.toml -store mocktikv -path "" > $explain_test_log 2>&1 &
$tidb_server -P "$port" -status "$status" -config config.toml -store unistore -path "" > $explain_test_log 2>&1 &
SERVER_PID=$!
fi
echo "tidb-server(PID: $SERVER_PID) started"
Expand Down
3 changes: 1 addition & 2 deletions distsql/select_result.go
Original file line number Diff line number Diff line change
Expand Up @@ -149,9 +149,8 @@ func (r *selectResult) fetchResp(ctx context.Context) error {
sc.AppendWarning(dbterror.ClassTiKV.Synthesize(terror.ErrCode(warning.Code), warning.Msg))
}
if r.feedback != nil {
r.feedback.Update(resultSubset.GetStartKey(), r.selectResp.OutputCounts)
r.feedback.Update(resultSubset.GetStartKey(), r.selectResp.OutputCounts, r.selectResp.Ndvs)
}

r.partialCount++

hasStats, ok := resultSubset.(CopRuntimeStats)
Expand Down
2 changes: 1 addition & 1 deletion distsql/stream.go
Original file line number Diff line number Diff line change
Expand Up @@ -105,7 +105,7 @@ func (r *streamResult) readDataFromResponse(ctx context.Context, resp kv.Respons
if err != nil {
return false, errors.Trace(err)
}
r.feedback.Update(resultSubset.GetStartKey(), stream.OutputCounts)
r.feedback.Update(resultSubset.GetStartKey(), stream.OutputCounts, stream.Ndvs)
r.partialCount++

hasStats, ok := resultSubset.(CopRuntimeStats)
Expand Down
12 changes: 8 additions & 4 deletions executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -335,6 +335,10 @@ func (e *AnalyzeIndexExec) buildStatsFromResult(result distsql.SelectResult, nee
cms = statistics.NewCMSketch(int32(e.opts[ast.AnalyzeOptCMSketchDepth]), int32(e.opts[ast.AnalyzeOptCMSketchWidth]))
topn = statistics.NewTopN(int(e.opts[ast.AnalyzeOptNumTopN]))
}
statsVer := statistics.Version1
if e.analyzePB.IdxReq.Version != nil {
statsVer = int(*e.analyzePB.IdxReq.Version)
}
for {
data, err := result.NextRaw(context.TODO())
if err != nil {
Expand All @@ -350,7 +354,7 @@ func (e *AnalyzeIndexExec) buildStatsFromResult(result distsql.SelectResult, nee
}
respHist := statistics.HistogramFromProto(resp.Hist)
e.job.Update(int64(respHist.TotalRowCount()))
hist, err = statistics.MergeHistograms(e.ctx.GetSessionVars().StmtCtx, hist, respHist, int(e.opts[ast.AnalyzeOptNumBuckets]))
hist, err = statistics.MergeHistograms(e.ctx.GetSessionVars().StmtCtx, hist, respHist, int(e.opts[ast.AnalyzeOptNumBuckets]), statsVer)
if err != nil {
return nil, nil, nil, err
}
Expand Down Expand Up @@ -556,7 +560,7 @@ func (e *AnalyzeColumnsExec) buildStats(ranges []*ranger.Range, needExtStats boo
if hasPkHist(e.handleCols) {
respHist := statistics.HistogramFromProto(resp.PkHist)
rowCount = int64(respHist.TotalRowCount())
pkHist, err = statistics.MergeHistograms(sc, pkHist, respHist, int(e.opts[ast.AnalyzeOptNumBuckets]))
pkHist, err = statistics.MergeHistograms(sc, pkHist, respHist, int(e.opts[ast.AnalyzeOptNumBuckets]), statistics.Version1)
if err != nil {
return nil, nil, nil, nil, err
}
Expand Down Expand Up @@ -1244,7 +1248,7 @@ func analyzeIndexIncremental(idxExec *analyzeIndexIncrementalExec) analyzeResult
if err != nil {
return analyzeResult{Err: err, job: idxExec.job}
}
hist, err = statistics.MergeHistograms(idxExec.ctx.GetSessionVars().StmtCtx, idxExec.oldHist, hist, int(idxExec.opts[ast.AnalyzeOptNumBuckets]))
hist, err = statistics.MergeHistograms(idxExec.ctx.GetSessionVars().StmtCtx, idxExec.oldHist, hist, int(idxExec.opts[ast.AnalyzeOptNumBuckets]), statistics.Version1)
if err != nil {
return analyzeResult{Err: err, job: idxExec.job}
}
Expand Down Expand Up @@ -1295,7 +1299,7 @@ func analyzePKIncremental(colExec *analyzePKIncrementalExec) analyzeResult {
return analyzeResult{Err: err, job: colExec.job}
}
hist := hists[0]
hist, err = statistics.MergeHistograms(colExec.ctx.GetSessionVars().StmtCtx, colExec.oldHist, hist, int(colExec.opts[ast.AnalyzeOptNumBuckets]))
hist, err = statistics.MergeHistograms(colExec.ctx.GetSessionVars().StmtCtx, colExec.oldHist, hist, int(colExec.opts[ast.AnalyzeOptNumBuckets]), statistics.Version1)
if err != nil {
return analyzeResult{Err: err, job: colExec.job}
}
Expand Down
97 changes: 49 additions & 48 deletions executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -461,8 +461,8 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) {
tk.MustExec("insert into t2 values (0), (18446744073709551615)")
tk.MustExec("analyze table t2")
tk.MustQuery("show stats_buckets where table_name = 't2'").Check(testkit.Rows(
"test t2 a 0 0 1 1 0 0",
"test t2 a 0 1 2 1 18446744073709551615 18446744073709551615"))
"test t2 a 0 0 1 1 0 0 0",
"test t2 a 0 1 2 1 18446744073709551615 18446744073709551615 0"))

tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.StaticOnly) + `'`)
tk.MustExec(`create table t3 (id int, v int, primary key(id), index k(v)) partition by hash (id) partitions 4`)
Expand Down Expand Up @@ -531,6 +531,7 @@ func (s *testSuite1) TestAnalyzeIncremental(c *C) {
}

func (s *testSuite1) TestAnalyzeIncrementalStreaming(c *C) {
c.Skip("unistore hasn't support streaming yet.")
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.Se.GetSessionVars().EnableStreaming = true
Expand All @@ -545,13 +546,13 @@ func (s *testSuite1) testAnalyzeIncremental(tk *testkit.TestKit, c *C) {
tk.MustQuery("show stats_buckets").Check(testkit.Rows())
tk.MustExec("insert into t values (1,1)")
tk.MustExec("analyze incremental table t index")
tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t idx 1 0 1 1 1 1"))
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"))
tk.MustExec("insert into t values (2,2)")
tk.MustExec("analyze incremental table t index")
tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t a 0 1 2 1 2 2", "test t idx 1 0 1 1 1 1", "test t idx 1 1 2 1 2 2"))
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"))
tk.MustExec("analyze incremental table t index")
// Result should not change.
tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t a 0 1 2 1 2 2", "test t idx 1 0 1 1 1 1", "test t idx 1 1 2 1 2 2"))
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"))

// Test analyze incremental with feedback.
tk.MustExec("insert into t values (3,3)")
Expand All @@ -574,7 +575,7 @@ func (s *testSuite1) testAnalyzeIncremental(tk *testkit.TestKit, c *C) {
c.Assert(h.DumpStatsFeedbackToKV(), IsNil)
c.Assert(h.HandleUpdateStats(is), IsNil)
c.Assert(h.Update(is), IsNil)
tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t a 0 1 3 0 2 2147483647", "test t idx 1 0 1 1 1 1", "test t idx 1 1 2 1 2 2"))
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"))
tblStats := h.GetTableStats(tblInfo)
val, err := codec.EncodeKey(tk.Se.GetSessionVars().StmtCtx, nil, types.NewIntDatum(3))
c.Assert(err, IsNil)
Expand All @@ -583,8 +584,8 @@ func (s *testSuite1) testAnalyzeIncremental(tk *testkit.TestKit, c *C) {
c.Assert(statistics.IsAnalyzed(tblStats.Columns[tblInfo.Columns[0].ID].Flag), IsFalse)

tk.MustExec("analyze incremental table t index")
tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t a 0 1 2 1 2 2", "test t a 0 2 3 1 3 3",
"test t idx 1 0 1 1 1 1", "test t idx 1 1 2 1 2 2", "test t idx 1 2 3 1 3 3"))
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",
"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"))
tblStats = h.GetTableStats(tblInfo)
c.Assert(tblStats.Indices[tblInfo.Indices[0].ID].QueryBytes(val), Equals, uint64(1))
}
Expand Down Expand Up @@ -769,36 +770,36 @@ func (s *testSuite1) TestNormalAnalyzeOnCommonHandle(c *C) {
tk.MustExec("analyze table t1, t2, t3")

tk.MustQuery(`show stats_buckets where table_name in ("t1", "t2", "t3")`).Sort().Check(testkit.Rows(
"test t1 a 0 0 1 1 1 1",
"test t1 a 0 1 2 1 2 2",
"test t1 a 0 2 3 1 3 3",
"test t1 b 0 0 1 1 1 1",
"test t1 b 0 1 2 1 2 2",
"test t1 b 0 2 3 1 3 3",
"test t2 PRIMARY 1 0 1 1 111 111",
"test t2 PRIMARY 1 1 2 1 222 222",
"test t2 PRIMARY 1 2 3 1 333 333",
"test t2 a 0 0 1 1 111 111",
"test t2 a 0 1 2 1 222 222",
"test t2 a 0 2 3 1 333 333",
"test t2 b 0 0 1 1 1 1",
"test t2 b 0 1 2 1 2 2",
"test t2 b 0 2 3 1 3 3",
"test t3 PRIMARY 1 0 1 1 (1, 1) (1, 1)",
"test t3 PRIMARY 1 1 2 1 (2, 2) (2, 2)",
"test t3 PRIMARY 1 2 3 1 (3, 3) (3, 3)",
"test t3 a 0 0 1 1 1 1",
"test t3 a 0 1 2 1 2 2",
"test t3 a 0 2 3 1 3 3",
"test t3 b 0 0 1 1 1 1",
"test t3 b 0 1 2 1 2 2",
"test t3 b 0 2 3 1 3 3",
"test t3 c 0 0 1 1 1 1",
"test t3 c 0 1 2 1 2 2",
"test t3 c 0 2 3 1 3 3",
"test t3 c 1 0 1 1 1 1",
"test t3 c 1 1 2 1 2 2",
"test t3 c 1 2 3 1 3 3"))
"test t1 a 0 0 1 1 1 1 0",
"test t1 a 0 1 2 1 2 2 0",
"test t1 a 0 2 3 1 3 3 0",
"test t1 b 0 0 1 1 1 1 0",
"test t1 b 0 1 2 1 2 2 0",
"test t1 b 0 2 3 1 3 3 0",
"test t2 PRIMARY 1 0 1 1 111 111 0",
"test t2 PRIMARY 1 1 2 1 222 222 0",
"test t2 PRIMARY 1 2 3 1 333 333 0",
"test t2 a 0 0 1 1 111 111 0",
"test t2 a 0 1 2 1 222 222 0",
"test t2 a 0 2 3 1 333 333 0",
"test t2 b 0 0 1 1 1 1 0",
"test t2 b 0 1 2 1 2 2 0",
"test t2 b 0 2 3 1 3 3 0",
"test t3 PRIMARY 1 0 1 1 (1, 1) (1, 1) 0",
"test t3 PRIMARY 1 1 2 1 (2, 2) (2, 2) 0",
"test t3 PRIMARY 1 2 3 1 (3, 3) (3, 3) 0",
"test t3 a 0 0 1 1 1 1 0",
"test t3 a 0 1 2 1 2 2 0",
"test t3 a 0 2 3 1 3 3 0",
"test t3 b 0 0 1 1 1 1 0",
"test t3 b 0 1 2 1 2 2 0",
"test t3 b 0 2 3 1 3 3 0",
"test t3 c 0 0 1 1 1 1 0",
"test t3 c 0 1 2 1 2 2 0",
"test t3 c 0 2 3 1 3 3 0",
"test t3 c 1 0 1 1 1 1 0",
"test t3 c 1 1 2 1 2 2 0",
"test t3 c 1 2 3 1 3 3 0"))
}

func (s *testSuite1) TestDefaultValForAnalyze(c *C) {
Expand Down Expand Up @@ -846,15 +847,15 @@ func (s *testSerialSuite2) TestIssue20874(c *C) {
tk.MustExec("insert into t values ('#', 'C'), ('$', 'c'), ('a', 'a')")
tk.MustExec("analyze table t")
tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check(testkit.Rows(
"test t a 0 0 1 1 \x02\xd2 \x02\xd2",
"test t a 0 1 2 1 \x0e\x0f \x0e\x0f",
"test t a 0 2 3 1 \x0e3 \x0e3",
"test t b 0 0 1 1 \x00A \x00A",
"test t b 0 1 3 2 \x00C \x00C",
"test t idxa 1 0 1 1 \x02\xd2 \x02\xd2",
"test t idxa 1 1 2 1 \x0e\x0f \x0e\x0f",
"test t idxa 1 2 3 1 \x0e3 \x0e3",
"test t idxb 1 0 1 1 \x00A \x00A",
"test t idxb 1 1 3 2 \x00C \x00C",
"test t a 0 0 1 1 \x02\xd2 \x02\xd2 0",
"test t a 0 1 2 1 \x0e\x0f \x0e\x0f 0",
"test t a 0 2 3 1 \x0e3 \x0e3 0",
"test t b 0 0 1 1 \x00A \x00A 0",
"test t b 0 1 3 2 \x00C \x00C 0",
"test t idxa 1 0 1 1 \x02\xd2 \x02\xd2 0",
"test t idxa 1 1 2 1 \x0e\x0f \x0e\x0f 0",
"test t idxa 1 2 3 1 \x0e3 \x0e3 0",
"test t idxb 1 0 1 1 \x00A \x00A 0",
"test t idxb 1 1 3 2 \x00C \x00C 0",
))
}
1 change: 1 addition & 0 deletions executor/show_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -259,6 +259,7 @@ func (e *ShowExec) bucketsToRows(dbName, tblName, partitionName, colName string,
hist.Buckets[i].Repeat,
lowerBoundStr,
upperBoundStr,
hist.Buckets[i].NDV,
})
}
return nil
Expand Down
24 changes: 12 additions & 12 deletions executor/show_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,36 +80,36 @@ func (s *testShowStatsSuite) TestShowStatsBuckets(c *C) {
tk.MustExec("insert into t values (1,1)")
tk.MustExec("analyze table t")
result := tk.MustQuery("show stats_buckets").Sort()
result.Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t b 0 0 1 1 1 1", "test t idx 1 0 1 1 (1, 1) (1, 1)"))
result.Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t b 0 0 1 1 1 1 0", "test t idx 1 0 1 1 (1, 1) (1, 1) 0"))
result = tk.MustQuery("show stats_buckets where column_name = 'idx'")
result.Check(testkit.Rows("test t idx 1 0 1 1 (1, 1) (1, 1)"))
result.Check(testkit.Rows("test t idx 1 0 1 1 (1, 1) (1, 1) 0"))

tk.MustExec("drop table t")
tk.MustExec("create table t (`a` datetime, `b` int, key `idx`(`a`, `b`))")
tk.MustExec("insert into t values (\"2020-01-01\", 1)")
tk.MustExec("analyze table t")
result = tk.MustQuery("show stats_buckets").Sort()
result.Check(testkit.Rows("test t a 0 0 1 1 2020-01-01 00:00:00 2020-01-01 00:00:00", "test t b 0 0 1 1 1 1", "test t idx 1 0 1 1 (2020-01-01 00:00:00, 1) (2020-01-01 00:00:00, 1)"))
result.Check(testkit.Rows("test t a 0 0 1 1 2020-01-01 00:00:00 2020-01-01 00:00:00 0", "test t b 0 0 1 1 1 1 0", "test t idx 1 0 1 1 (2020-01-01 00:00:00, 1) (2020-01-01 00:00:00, 1) 0"))
result = tk.MustQuery("show stats_buckets where column_name = 'idx'")
result.Check(testkit.Rows("test t idx 1 0 1 1 (2020-01-01 00:00:00, 1) (2020-01-01 00:00:00, 1)"))
result.Check(testkit.Rows("test t idx 1 0 1 1 (2020-01-01 00:00:00, 1) (2020-01-01 00:00:00, 1) 0"))

tk.MustExec("drop table t")
tk.MustExec("create table t (`a` date, `b` int, key `idx`(`a`, `b`))")
tk.MustExec("insert into t values (\"2020-01-01\", 1)")
tk.MustExec("analyze table t")
result = tk.MustQuery("show stats_buckets").Sort()
result.Check(testkit.Rows("test t a 0 0 1 1 2020-01-01 2020-01-01", "test t b 0 0 1 1 1 1", "test t idx 1 0 1 1 (2020-01-01, 1) (2020-01-01, 1)"))
result.Check(testkit.Rows("test t a 0 0 1 1 2020-01-01 2020-01-01 0", "test t b 0 0 1 1 1 1 0", "test t idx 1 0 1 1 (2020-01-01, 1) (2020-01-01, 1) 0"))
result = tk.MustQuery("show stats_buckets where column_name = 'idx'")
result.Check(testkit.Rows("test t idx 1 0 1 1 (2020-01-01, 1) (2020-01-01, 1)"))
result.Check(testkit.Rows("test t idx 1 0 1 1 (2020-01-01, 1) (2020-01-01, 1) 0"))

tk.MustExec("drop table t")
tk.MustExec("create table t (`a` timestamp, `b` int, key `idx`(`a`, `b`))")
tk.MustExec("insert into t values (\"2020-01-01\", 1)")
tk.MustExec("analyze table t")
result = tk.MustQuery("show stats_buckets").Sort()
result.Check(testkit.Rows("test t a 0 0 1 1 2020-01-01 00:00:00 2020-01-01 00:00:00", "test t b 0 0 1 1 1 1", "test t idx 1 0 1 1 (2020-01-01 00:00:00, 1) (2020-01-01 00:00:00, 1)"))
result.Check(testkit.Rows("test t a 0 0 1 1 2020-01-01 00:00:00 2020-01-01 00:00:00 0", "test t b 0 0 1 1 1 1 0", "test t idx 1 0 1 1 (2020-01-01 00:00:00, 1) (2020-01-01 00:00:00, 1) 0"))
result = tk.MustQuery("show stats_buckets where column_name = 'idx'")
result.Check(testkit.Rows("test t idx 1 0 1 1 (2020-01-01 00:00:00, 1) (2020-01-01 00:00:00, 1)"))
result.Check(testkit.Rows("test t idx 1 0 1 1 (2020-01-01 00:00:00, 1) (2020-01-01 00:00:00, 1) 0"))
}

func (s *testShowStatsSuite) TestShowStatsHasNullValue(c *C) {
Expand All @@ -124,14 +124,14 @@ func (s *testShowStatsSuite) TestShowStatsHasNullValue(c *C) {
tk.MustExec("insert into t values(1)")
tk.MustExec("analyze table t")
tk.MustQuery("show stats_buckets").Sort().Check(testkit.Rows(
"test t a 0 0 1 1 1 1",
"test t idx 1 0 1 1 1 1",
"test t a 0 0 1 1 1 1 0",
"test t idx 1 0 1 1 1 1 0",
))
tk.MustExec("drop table t")
tk.MustExec("create table t (a int, b int, index idx(a, b))")
tk.MustExec("insert into t values(NULL, NULL)")
tk.MustExec("analyze table t")
tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t idx 1 0 1 1 (NULL, NULL) (NULL, NULL)"))
tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t idx 1 0 1 1 (NULL, NULL) (NULL, NULL) 0"))

tk.MustExec("drop table t")
tk.MustExec("create table t(a int, b int, c int, index idx_b(b), index idx_c_a(c, a))")
Expand Down Expand Up @@ -201,7 +201,7 @@ func (s *testShowStatsSuite) TestShowPartitionStats(c *C) {
c.Assert(result.Rows()[2][3], Equals, "idx")

result = tk.MustQuery("show stats_buckets").Sort()
result.Check(testkit.Rows("test t p0 a 0 0 1 1 1 1", "test t p0 b 0 0 1 1 1 1", "test t p0 idx 1 0 1 1 1 1"))
result.Check(testkit.Rows("test t p0 a 0 0 1 1 1 1 0", "test t p0 b 0 0 1 1 1 1 0", "test t p0 idx 1 0 1 1 1 1 0"))

result = tk.MustQuery("show stats_healthy")
result.Check(testkit.Rows("test t p0 100"))
Expand Down
4 changes: 2 additions & 2 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -3671,9 +3671,9 @@ func buildShowSchema(s *ast.ShowStmt, isView bool, isSequence bool) (schema *exp
mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeDouble, mysql.TypeDouble}
case ast.ShowStatsBuckets:
names = []string{"Db_name", "Table_name", "Partition_name", "Column_name", "Is_index", "Bucket_id", "Count",
"Repeats", "Lower_Bound", "Upper_Bound"}
"Repeats", "Lower_Bound", "Upper_Bound", "Ndv"}
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeTiny, mysql.TypeLonglong,
mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeVarchar, mysql.TypeVarchar}
mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLonglong}
case ast.ShowStatsTopN:
names = []string{"Db_name", "Table_name", "Partition_name", "Column_name", "Is_index", "Value", "Count"}
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeTiny, mysql.TypeVarchar, mysql.TypeLonglong}
Expand Down
15 changes: 13 additions & 2 deletions session/bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -207,7 +207,8 @@ const (
count BIGINT(64) NOT NULL,
repeats BIGINT(64) NOT NULL,
upper_bound BLOB NOT NULL,
lower_bound BLOB ,
lower_bound BLOB ,
ndv BIGINT NOT NULL DEFAULT 0,
UNIQUE INDEX tbl(table_id, is_index, hist_id, bucket_id)
);`

Expand Down Expand Up @@ -456,9 +457,11 @@ const (
version60 = 60
// version61 restore all SQL bindings.
version61 = 61
// version62 add column ndv for mysql.stats_buckets.
version62 = 62

// please make sure this is the largest version
currentBootstrapVersion = version61
currentBootstrapVersion = version62
)

var (
Expand Down Expand Up @@ -524,6 +527,7 @@ var (
upgradeToVer59,
upgradeToVer60,
upgradeToVer61,
upgradeToVer62,
}
)

Expand Down Expand Up @@ -1412,6 +1416,13 @@ func writeMemoryQuotaQuery(s Session) {
mustExecute(s, sql)
}

func upgradeToVer62(s Session, ver int64) {
if ver >= version62 {
return
}
doReentrantDDL(s, "ALTER TABLE mysql.stats_buckets ADD COLUMN `ndv` bigint not null default 0", infoschema.ErrColumnExists)
}

func writeOOMAction(s Session) {
comment := "oom-action is `log` by default in v3.0.x, `cancel` by default in v4.0.11+"
sql := fmt.Sprintf(`INSERT HIGH_PRIORITY INTO %s.%s VALUES ("%s", '%s', '%s') ON DUPLICATE KEY UPDATE VARIABLE_VALUE='%s'`,
Expand Down
Loading

0 comments on commit 3dd842f

Please sign in to comment.