Skip to content

Commit

Permalink
*: kill auto analyze out of the auto analyze time windows (#55062)
Browse files Browse the repository at this point in the history
close #55283
  • Loading branch information
hawkingrei authored Aug 9, 2024
1 parent 2a731e6 commit b066365
Show file tree
Hide file tree
Showing 6 changed files with 90 additions and 15 deletions.
1 change: 1 addition & 0 deletions pkg/domain/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ go_library(
"//pkg/sessionctx/variable",
"//pkg/statistics",
"//pkg/statistics/handle",
"//pkg/statistics/handle/autoanalyze",
"//pkg/statistics/handle/logutil",
"//pkg/statistics/handle/util",
"//pkg/store/helper",
Expand Down
21 changes: 21 additions & 0 deletions pkg/domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,9 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx/sysproctrack"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/statistics/handle"
"github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze"
statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil"
handleutil "github.com/pingcap/tidb/pkg/statistics/handle/util"
"github.com/pingcap/tidb/pkg/store/helper"
"github.com/pingcap/tidb/pkg/ttl/ttlworker"
"github.com/pingcap/tidb/pkg/types"
Expand Down Expand Up @@ -915,6 +917,24 @@ func (do *Domain) topologySyncerKeeper() {
}
}

// CheckAutoAnalyzeWindows checks the auto analyze windows and kill the auto analyze process if it is not in the window.
func (do *Domain) CheckAutoAnalyzeWindows() {
se, err := do.sysSessionPool.Get()

if err != nil {
logutil.BgLogger().Warn("get system session failed", zap.Error(err))
return
}
// Make sure the session is new.
sctx := se.(sessionctx.Context)
defer do.sysSessionPool.Put(se)
if !autoanalyze.CheckAutoAnalyzeWindow(sctx) {
for _, id := range handleutil.GlobalAutoAnalyzeProcessList.All() {
do.SysProcTracker().KillSysProcess(id)
}
}
}

func (do *Domain) refreshMDLCheckTableInfo() {
se, err := do.sysSessionPool.Get()

Expand Down Expand Up @@ -2575,6 +2595,7 @@ func (do *Domain) updateStatsWorker(_ sessionctx.Context, owner owner.Manager) {
if err != nil {
logutil.BgLogger().Debug("GC stats failed", zap.Error(err))
}
do.CheckAutoAnalyzeWindows()
case <-dumpColStatsUsageTicker.C:
err := statsHandle.DumpColStatsUsageToKV()
if err != nil {
Expand Down
42 changes: 29 additions & 13 deletions pkg/statistics/handle/autoanalyze/autoanalyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -318,21 +318,11 @@ func HandleAutoAnalyze(

parameters := exec.GetAutoAnalyzeParameters(sctx)
autoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[variable.TiDBAutoAnalyzeRatio])
// Determine the time window for auto-analysis and verify if the current time falls within this range.
start, end, err := exec.ParseAutoAnalysisWindow(
parameters[variable.TiDBAutoAnalyzeStartTime],
parameters[variable.TiDBAutoAnalyzeEndTime],
)
if err != nil {
statslogutil.StatsLogger().Error(
"parse auto analyze period failed",
zap.Error(err),
)
return false
}
if !timeutil.WithinDayTimePeriod(start, end, time.Now()) {
start, end, ok := checkAutoAnalyzeWindow(parameters)
if !ok {
return false
}

pruneMode := variable.PartitionPruneMode(sctx.GetSessionVars().PartitionPruneMode.Load())

return RandomPickOneTableAndTryAutoAnalyze(
Expand All @@ -346,6 +336,32 @@ func HandleAutoAnalyze(
)
}

// CheckAutoAnalyzeWindow determine the time window for auto-analysis and verify if the current time falls within this range.
// parameters is a map of auto analyze parameters. it is from GetAutoAnalyzeParameters.
func CheckAutoAnalyzeWindow(sctx sessionctx.Context) bool {
parameters := exec.GetAutoAnalyzeParameters(sctx)
_, _, ok := checkAutoAnalyzeWindow(parameters)
return ok
}

func checkAutoAnalyzeWindow(parameters map[string]string) (time.Time, time.Time, bool) {
start, end, err := exec.ParseAutoAnalysisWindow(
parameters[variable.TiDBAutoAnalyzeStartTime],
parameters[variable.TiDBAutoAnalyzeEndTime],
)
if err != nil {
statslogutil.StatsLogger().Error(
"parse auto analyze period failed",
zap.Error(err),
)
return start, end, false
}
if !timeutil.WithinDayTimePeriod(start, end, time.Now()) {
return start, end, false
}
return start, end, true
}

// RandomPickOneTableAndTryAutoAnalyze randomly picks one table and tries to analyze it.
// 1. If the table is not analyzed, analyze it.
// 2. If the table is analyzed, analyze it when "tbl.ModifyCount/tbl.Count > autoAnalyzeRatio".
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 @@ -34,6 +34,7 @@ go_test(
"//pkg/parser/model",
"//pkg/sessionctx",
"//pkg/testkit",
"//pkg/util",
"@com_github_stretchr_testify//require",
],
)
5 changes: 3 additions & 2 deletions pkg/statistics/handle/autoanalyze/exec/exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ func AutoAnalyze(
params ...any,
) {
startTime := time.Now()
_, _, err := execAnalyzeStmt(sctx, statsHandle, sysProcTracker, statsVer, sql, params...)
_, _, err := RunAnalyzeStmt(sctx, statsHandle, sysProcTracker, statsVer, sql, params...)
dur := time.Since(startTime)
metrics.AutoAnalyzeHistogram.Observe(dur.Seconds())
if err != nil {
Expand All @@ -72,7 +72,8 @@ func AutoAnalyze(
}
}

func execAnalyzeStmt(
// RunAnalyzeStmt executes the analyze statement.
func RunAnalyzeStmt(
sctx sessionctx.Context,
statsHandle statstypes.StatsHandle,
sysProcTracker sysproctrack.Tracker,
Expand Down
35 changes: 35 additions & 0 deletions pkg/statistics/handle/autoanalyze/exec/exec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,15 @@ package exec_test

import (
"context"
"fmt"
"testing"
"time"

"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/sessionctx"
"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 @@ -51,3 +54,35 @@ func TestExecAutoAnalyzes(t *testing.T) {
tblStats := handle.GetTableStats(tbl.Meta())
require.Equal(t, int64(3), tblStats.RealtimeCount)
}

func TestKillInWindows(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("create table t1 (a int, b int, index idx(a)) partition by range (a) (partition p0 values less than (2), partition p1 values less than (14))")
tk.MustExec("insert into t1 values (4, 4), (5, 5), (6, 6), (7, 7), (8, 8), (9, 9), (10, 10), (11, 11), (12, 12), (13, 13)")
handle := dom.StatsHandle()
sysProcTracker := dom.SysProcTracker()
now := time.Now()
startTime := now.Add(1 * time.Hour).Format("15:04 -0700")
endTime := now.Add(2 * time.Hour).Format("15:04 -0700")
tk.MustExec(fmt.Sprintf("SET GLOBAL tidb_auto_analyze_start_time='%s'", startTime))
tk.MustExec(fmt.Sprintf("SET GLOBAL tidb_auto_analyze_end_time='%s'", endTime))
var wg util.WaitGroupWrapper
exitCh := make(chan struct{})
wg.Run(func() {
for {
select {
case <-exitCh:
return
default:
dom.CheckAutoAnalyzeWindows()
}
}
})
sctx := tk.Session()
_, _, err := exec.RunAnalyzeStmt(sctx, handle, sysProcTracker, 2, "analyze table %n", "t1")
require.ErrorContains(t, err, "[executor:1317]Query execution was interrupted")
close(exitCh)
wg.Wait()
}

0 comments on commit b066365

Please sign in to comment.