Skip to content

Commit

Permalink
feat: build the table analysis job queue
Browse files Browse the repository at this point in the history
Signed-off-by: hi-rustin <[email protected]>

refactor: rename

Signed-off-by: hi-rustin <[email protected]>

refactor: remove partition part

Signed-off-by: hi-rustin <[email protected]>

refactor: use common functions from exec

Signed-off-by: hi-rustin <[email protected]>

fix: use exec

Signed-off-by: hi-rustin <[email protected]>

feat: calculate time and size

Signed-off-by: hi-rustin <[email protected]>

test: add test case

Signed-off-by: hi-rustin <[email protected]>

test: add more cases

Signed-off-by: hi-rustin <[email protected]>

fix: make clippy happy

Signed-off-by: hi-rustin <[email protected]>

fix: set txn correctly

Signed-off-by: hi-rustin <[email protected]>

fix: use physical time

Signed-off-by: hi-rustin <[email protected]>

chroe: add TODO

Signed-off-by: hi-rustin <[email protected]>

fix: make lint happy

Signed-off-by: hi-rustin <[email protected]>

fix: compile issue

Signed-off-by: hi-rustin <[email protected]>

fix: build issue

Signed-off-by: hi-rustin <[email protected]>

fix: compile issue

Signed-off-by: hi-rustin <[email protected]>

fix: build issue

Signed-off-by: hi-rustin <[email protected]>

refactor: better code

Signed-off-by: hi-rustin <[email protected]>
  • Loading branch information
Rustin170506 committed Feb 14, 2024
1 parent fe31308 commit 6defd3c
Show file tree
Hide file tree
Showing 22 changed files with 419 additions and 114 deletions.
2 changes: 1 addition & 1 deletion pkg/executor/test/analyzetest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ go_test(
"//pkg/sessionctx",
"//pkg/sessionctx/variable",
"//pkg/statistics",
"//pkg/statistics/handle/autoanalyze",
"//pkg/statistics/handle/autoanalyze/exec",
"//pkg/testkit",
"//pkg/util/dbterror/exeerrors",
"@com_github_pingcap_errors//:errors",
Expand Down
28 changes: 14 additions & 14 deletions pkg/executor/test/analyzetest/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/statistics"
"github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze"
"github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/util/dbterror/exeerrors"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -689,11 +689,11 @@ func TestSavedAnalyzeOptions(t *testing.T) {
tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_ratio = %v", originalVal2))
}()
tk.MustExec("set global tidb_auto_analyze_ratio = 0.01")
originalVal3 := autoanalyze.AutoAnalyzeMinCnt
originalVal3 := exec.AutoAnalyzeMinCnt
defer func() {
autoanalyze.AutoAnalyzeMinCnt = originalVal3
exec.AutoAnalyzeMinCnt = originalVal3
}()
autoanalyze.AutoAnalyzeMinCnt = 0
exec.AutoAnalyzeMinCnt = 0

tk.MustExec("use test")
tk.MustExec("set @@session.tidb_analyze_version = 2")
Expand Down Expand Up @@ -1031,11 +1031,11 @@ func TestSavedAnalyzeColumnOptions(t *testing.T) {
tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_ratio = %v", originalVal2))
}()
tk.MustExec("set global tidb_auto_analyze_ratio = 0.01")
originalVal3 := autoanalyze.AutoAnalyzeMinCnt
originalVal3 := exec.AutoAnalyzeMinCnt
defer func() {
autoanalyze.AutoAnalyzeMinCnt = originalVal3
exec.AutoAnalyzeMinCnt = originalVal3
}()
autoanalyze.AutoAnalyzeMinCnt = 0
exec.AutoAnalyzeMinCnt = 0
originalVal4 := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string)
defer func() {
tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal4))
Expand Down Expand Up @@ -1884,9 +1884,9 @@ func testKillAutoAnalyze(t *testing.T, ver int) {
tk := testkit.NewTestKit(t, store)
oriStart := tk.MustQuery("select @@tidb_auto_analyze_start_time").Rows()[0][0].(string)
oriEnd := tk.MustQuery("select @@tidb_auto_analyze_end_time").Rows()[0][0].(string)
autoanalyze.AutoAnalyzeMinCnt = 0
exec.AutoAnalyzeMinCnt = 0
defer func() {
autoanalyze.AutoAnalyzeMinCnt = 1000
exec.AutoAnalyzeMinCnt = 1000
tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='%v'", oriStart))
tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='%v'", oriEnd))
}()
Expand Down Expand Up @@ -1967,9 +1967,9 @@ func TestKillAutoAnalyzeIndex(t *testing.T) {
tk := testkit.NewTestKit(t, store)
oriStart := tk.MustQuery("select @@tidb_auto_analyze_start_time").Rows()[0][0].(string)
oriEnd := tk.MustQuery("select @@tidb_auto_analyze_end_time").Rows()[0][0].(string)
autoanalyze.AutoAnalyzeMinCnt = 0
exec.AutoAnalyzeMinCnt = 0
defer func() {
autoanalyze.AutoAnalyzeMinCnt = 1000
exec.AutoAnalyzeMinCnt = 1000
tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='%v'", oriStart))
tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='%v'", oriEnd))
}()
Expand Down Expand Up @@ -2722,12 +2722,12 @@ func TestAutoAnalyzeAwareGlobalVariableChange(t *testing.T) {
"3 0",
))

originalVal1 := autoanalyze.AutoAnalyzeMinCnt
originalVal1 := exec.AutoAnalyzeMinCnt
originalVal2 := tk.MustQuery("select @@global.tidb_auto_analyze_ratio").Rows()[0][0].(string)
autoanalyze.AutoAnalyzeMinCnt = 0
exec.AutoAnalyzeMinCnt = 0
tk.MustExec("set global tidb_auto_analyze_ratio = 0.001")
defer func() {
autoanalyze.AutoAnalyzeMinCnt = originalVal1
exec.AutoAnalyzeMinCnt = originalVal1
tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_ratio = %v", originalVal2))
}()

Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/test/analyzetest/memorycontrol/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@ go_test(
"//pkg/config",
"//pkg/executor",
"//pkg/sessionctx/variable",
"//pkg/statistics/handle/autoanalyze",
"//pkg/statistics/handle/autoanalyze/exec",
"//pkg/testkit",
"//pkg/util",
"@com_github_pingcap_failpoint//:failpoint",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import (

"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/executor"
"github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze"
"github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/util"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -143,12 +143,12 @@ func TestGlobalMemoryControlForAutoAnalyze(t *testing.T) {
require.Len(t, rs0.Rows(), 0)

h := dom.StatsHandle()
originalVal4 := autoanalyze.AutoAnalyzeMinCnt
originalVal4 := exec.AutoAnalyzeMinCnt
originalVal5 := tk.MustQuery("select @@global.tidb_auto_analyze_ratio").Rows()[0][0].(string)
autoanalyze.AutoAnalyzeMinCnt = 0
exec.AutoAnalyzeMinCnt = 0
tk.MustExec("set global tidb_auto_analyze_ratio = 0.001")
defer func() {
autoanalyze.AutoAnalyzeMinCnt = originalVal4
exec.AutoAnalyzeMinCnt = originalVal4
tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_ratio = %v", originalVal5))
}()

Expand Down
2 changes: 1 addition & 1 deletion pkg/statistics/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ go_test(
"//pkg/parser/mysql",
"//pkg/sessionctx",
"//pkg/sessionctx/stmtctx",
"//pkg/statistics/handle/autoanalyze",
"//pkg/statistics/handle/autoanalyze/exec",
"//pkg/testkit",
"//pkg/testkit/testdata",
"//pkg/testkit/testmain",
Expand Down
1 change: 1 addition & 0 deletions pkg/statistics/handle/autoanalyze/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ go_test(
"//pkg/sessionctx",
"//pkg/sessionctx/variable",
"//pkg/statistics",
"//pkg/statistics/handle/autoanalyze/exec",
"//pkg/statistics/handle/util",
"//pkg/statistics/handle/util/test",
"//pkg/testkit",
Expand Down
51 changes: 5 additions & 46 deletions pkg/statistics/handle/autoanalyze/autoanalyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ package autoanalyze
import (
"context"
"fmt"
"math"
"math/rand"
"net"
"strconv"
Expand Down Expand Up @@ -259,14 +258,6 @@ func (sa *statsAnalyze) CheckAnalyzeVersion(tblInfo *model.TableInfo, physicalID
return statistics.CheckAnalyzeVerOnTable(tbl, version)
}

func parseAutoAnalyzeRatio(ratio string) float64 {
autoAnalyzeRatio, err := strconv.ParseFloat(ratio, 64)
if err != nil {
return variable.DefAutoAnalyzeRatio
}
return math.Max(autoAnalyzeRatio, 0)
}

// parseAnalyzePeriod parses the start and end time for auto analyze.
// It parses the times in UTC location.
func parseAnalyzePeriod(start, end string) (time.Time, time.Time, error) {
Expand All @@ -284,19 +275,6 @@ func parseAnalyzePeriod(start, end string) (time.Time, time.Time, error) {
return s, e, err
}

func getAutoAnalyzeParameters(sctx sessionctx.Context) map[string]string {
sql := "select variable_name, variable_value from mysql.global_variables where variable_name in (%?, %?, %?)"
rows, _, err := statsutil.ExecWithOpts(sctx, nil, sql, variable.TiDBAutoAnalyzeRatio, variable.TiDBAutoAnalyzeStartTime, variable.TiDBAutoAnalyzeEndTime)
if err != nil {
return map[string]string{}
}
parameters := make(map[string]string, len(rows))
for _, row := range rows {
parameters[row.GetString(0)] = row.GetString(1)
}
return parameters
}

// HandleAutoAnalyze analyzes the newly created table or index.
func HandleAutoAnalyze(
sctx sessionctx.Context,
Expand All @@ -313,8 +291,8 @@ func HandleAutoAnalyze(
}
}()

parameters := getAutoAnalyzeParameters(sctx)
autoAnalyzeRatio := parseAutoAnalyzeRatio(parameters[variable.TiDBAutoAnalyzeRatio])
parameters := exec.GetAutoAnalyzeParameters(sctx)
autoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[variable.TiDBAutoAnalyzeRatio])
// Get the available time period for auto analyze and check if the current time is in the period.
start, end, err := parseAnalyzePeriod(
parameters[variable.TiDBAutoAnalyzeStartTime],
Expand Down Expand Up @@ -473,10 +451,6 @@ func getPartitionStats(
return partitionStats
}

// AutoAnalyzeMinCnt means if the count of table is less than this value, we don't need to do auto analyze.
// Exported for testing.
var AutoAnalyzeMinCnt int64 = 1000

// Determine whether the table and index require analysis.
func tryAutoAnalyzeTable(
sctx sessionctx.Context,
Expand All @@ -491,7 +465,7 @@ func tryAutoAnalyzeTable(
// 1. If the statistics are either not loaded or are classified as pseudo, there is no need for analyze
// 2. If the table is too small, we don't want to waste time to analyze it.
// Leave the opportunity to other bigger tables.
if statsTbl == nil || statsTbl.RealtimeCount < AutoAnalyzeMinCnt || statsTbl.Pseudo {
if statsTbl == nil || statsTbl.RealtimeCount < exec.AutoAnalyzeMinCnt || statsTbl.Pseudo {
return false
}

Expand Down Expand Up @@ -548,7 +522,7 @@ func tryAutoAnalyzeTable(
//
// Exposed for test.
func NeedAnalyzeTable(tbl *statistics.Table, autoAnalyzeRatio float64) (bool, string) {
analyzed := TableAnalyzed(tbl)
analyzed := exec.TableAnalyzed(tbl)
if !analyzed {
return true, "table unanalyzed"
}
Expand All @@ -567,21 +541,6 @@ func NeedAnalyzeTable(tbl *statistics.Table, autoAnalyzeRatio float64) (bool, st
return true, fmt.Sprintf("too many modifications(%v/%v>%v)", tbl.ModifyCount, tblCnt, autoAnalyzeRatio)
}

// TableAnalyzed checks if any column or index of the table has been analyzed.
func TableAnalyzed(tbl *statistics.Table) bool {
for _, col := range tbl.Columns {
if col.IsAnalyzed() {
return true
}
}
for _, idx := range tbl.Indices {
if idx.IsAnalyzed() {
return true
}
}
return false
}

// It is very similar to tryAutoAnalyzeTable, but it commits the analyze job in batch for partitions.
func tryAutoAnalyzePartitionTableInDynamicMode(
sctx sessionctx.Context,
Expand All @@ -602,7 +561,7 @@ func tryAutoAnalyzePartitionTableInDynamicMode(
// 1. If the stats are not loaded, we don't need to analyze it.
// 2. If the table is too small, we don't want to waste time to analyze it.
// Leave the opportunity to other bigger tables.
if partitionStatsTbl.Pseudo || partitionStatsTbl.RealtimeCount < AutoAnalyzeMinCnt {
if partitionStatsTbl.Pseudo || partitionStatsTbl.RealtimeCount < exec.AutoAnalyzeMinCnt {
continue
}
if needAnalyze, reason := NeedAnalyzeTable(
Expand Down
29 changes: 15 additions & 14 deletions pkg/statistics/handle/autoanalyze/autoanalyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/statistics"
"github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze"
"github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec"
statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util"
"github.com/pingcap/tidb/pkg/statistics/handle/util/test"
"github.com/pingcap/tidb/pkg/testkit"
Expand All @@ -54,9 +55,9 @@ func TestAutoAnalyzeLockedTable(t *testing.T) {
tk.MustExec("lock stats t")
is := dom.InfoSchema()
require.NoError(t, h.Update(is))
autoanalyze.AutoAnalyzeMinCnt = 0
exec.AutoAnalyzeMinCnt = 0
defer func() {
autoanalyze.AutoAnalyzeMinCnt = 1000
exec.AutoAnalyzeMinCnt = 1000
}()
// Try to analyze the locked table, it should not analyze the table.
require.False(t, dom.StatsHandle().HandleAutoAnalyze())
Expand All @@ -82,9 +83,9 @@ func TestDisableAutoAnalyze(t *testing.T) {

// Set auto analyze ratio to 0.
tk.MustExec("set @@global.tidb_auto_analyze_ratio = 0")
autoanalyze.AutoAnalyzeMinCnt = 0
exec.AutoAnalyzeMinCnt = 0
defer func() {
autoanalyze.AutoAnalyzeMinCnt = 1000
exec.AutoAnalyzeMinCnt = 1000
}()
// Even auto analyze ratio is set to 0, we still need to analyze the unanalyzed tables.
require.True(t, dom.StatsHandle().HandleAutoAnalyze())
Expand All @@ -107,9 +108,9 @@ func TestAutoAnalyzeOnChangeAnalyzeVer(t *testing.T) {
tk.MustExec("insert into t values(1)")
tk.MustExec("set @@global.tidb_analyze_version = 1")
do := dom
autoanalyze.AutoAnalyzeMinCnt = 0
exec.AutoAnalyzeMinCnt = 0
defer func() {
autoanalyze.AutoAnalyzeMinCnt = 1000
exec.AutoAnalyzeMinCnt = 1000
}()
h := do.StatsHandle()
err := h.HandleDDLEvent(<-h.DDLEventCh())
Expand Down Expand Up @@ -184,12 +185,12 @@ func TestTableAnalyzed(t *testing.T) {

require.NoError(t, h.Update(is))
statsTbl := h.GetTableStats(tableInfo)
require.False(t, autoanalyze.TableAnalyzed(statsTbl))
require.False(t, exec.TableAnalyzed(statsTbl))

testKit.MustExec("analyze table t")
require.NoError(t, h.Update(is))
statsTbl = h.GetTableStats(tableInfo)
require.True(t, autoanalyze.TableAnalyzed(statsTbl))
require.True(t, exec.TableAnalyzed(statsTbl))

h.Clear()
oriLease := h.Lease()
Expand All @@ -200,7 +201,7 @@ func TestTableAnalyzed(t *testing.T) {
}()
require.NoError(t, h.Update(is))
statsTbl = h.GetTableStats(tableInfo)
require.True(t, autoanalyze.TableAnalyzed(statsTbl))
require.True(t, exec.TableAnalyzed(statsTbl))
}

func TestNeedAnalyzeTable(t *testing.T) {
Expand Down Expand Up @@ -287,10 +288,10 @@ func TestAutoAnalyzeSkipColumnTypes(t *testing.T) {
require.NoError(t, h.Update(dom.InfoSchema()))
tk.MustExec("set @@global.tidb_analyze_skip_column_types = 'json,blob,mediumblob,text,mediumtext'")

originalVal := autoanalyze.AutoAnalyzeMinCnt
autoanalyze.AutoAnalyzeMinCnt = 0
originalVal := exec.AutoAnalyzeMinCnt
exec.AutoAnalyzeMinCnt = 0
defer func() {
autoanalyze.AutoAnalyzeMinCnt = originalVal
exec.AutoAnalyzeMinCnt = originalVal
}()
require.True(t, h.HandleAutoAnalyze())
tk.MustQuery("select job_info from mysql.analyze_jobs where job_info like '%auto analyze table%'").Check(testkit.Rows("auto analyze table columns a, b, d with 256 buckets, 500 topn, 1 samplerate"))
Expand Down Expand Up @@ -319,7 +320,7 @@ func TestAutoAnalyzeOnEmptyTable(t *testing.T) {
// to pass the stats.Pseudo check in autoAnalyzeTable
tk.MustExec("analyze table t")
// to pass the AutoAnalyzeMinCnt check in autoAnalyzeTable
tk.MustExec("insert into t values (1)" + strings.Repeat(", (1)", int(autoanalyze.AutoAnalyzeMinCnt)))
tk.MustExec("insert into t values (1)" + strings.Repeat(", (1)", int(exec.AutoAnalyzeMinCnt)))
require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true))
require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema()))

Expand Down Expand Up @@ -354,7 +355,7 @@ func TestAutoAnalyzeOutOfSpecifiedTime(t *testing.T) {
// to pass the stats.Pseudo check in autoAnalyzeTable
tk.MustExec("analyze table t")
// to pass the AutoAnalyzeMinCnt check in autoAnalyzeTable
tk.MustExec("insert into t values (1)" + strings.Repeat(", (1)", int(autoanalyze.AutoAnalyzeMinCnt)))
tk.MustExec("insert into t values (1)" + strings.Repeat(", (1)", int(exec.AutoAnalyzeMinCnt)))
require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true))
require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema()))

Expand Down
1 change: 1 addition & 0 deletions pkg/statistics/handle/autoanalyze/exec/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ go_library(
"//pkg/metrics",
"//pkg/parser/ast",
"//pkg/sessionctx",
"//pkg/sessionctx/variable",
"//pkg/statistics",
"//pkg/statistics/handle/logutil",
"//pkg/statistics/handle/types",
Expand Down
Loading

0 comments on commit 6defd3c

Please sign in to comment.