forked from pingcap/tidb
-
Notifications
You must be signed in to change notification settings - Fork 2
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
test: Add auto-analyze priority calculator test suite (pingcap#55683)
close pingcap#55682
- Loading branch information
1 parent
9171ae7
commit 86ba31d
Showing
6 changed files
with
1,079 additions
and
1 deletion.
There are no files selected for viewing
21 changes: 21 additions & 0 deletions
21
pkg/statistics/handle/autoanalyze/priorityqueue/calculatoranalysis/BUILD.bazel
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,21 @@ | ||
load("@io_bazel_rules_go//go:def.bzl", "go_test") | ||
|
||
go_test( | ||
name = "calculatoranalysis_test", | ||
timeout = "short", | ||
srcs = [ | ||
"calculator_analysis_test.go", | ||
"main_test.go", | ||
], | ||
data = glob(["testdata/**"]), | ||
flaky = True, | ||
deps = [ | ||
"//pkg/sessionctx", | ||
"//pkg/sessionctx/sysproctrack", | ||
"//pkg/statistics/handle/autoanalyze/priorityqueue", | ||
"//pkg/statistics/handle/types", | ||
"//pkg/testkit/testsetup", | ||
"@com_github_stretchr_testify//require", | ||
"@org_uber_go_goleak//:goleak", | ||
], | ||
) |
59 changes: 59 additions & 0 deletions
59
pkg/statistics/handle/autoanalyze/priorityqueue/calculatoranalysis/README.md
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,59 @@ | ||
# Auto-Analyze Priority Calculator Test | ||
|
||
This test suite evaluates the effectiveness of the auto-analyze priority calculator for database statistics. The calculator prioritizes which tables should be analyzed based on factors such as table size, data changes, and time since last analysis. | ||
|
||
## Core Principles | ||
|
||
1. **Data Generation**: The test generates diverse scenarios using combinations of: | ||
- Table sizes (ranging from 1K to 100M rows) | ||
- Changes in data | ||
- Time since last analysis (ranging from 10 seconds to 3 days) | ||
|
||
2. **Priority Calculation**: For each scenario, the priority calculator computes a weight based on: | ||
- Change percentage (changes / table size) | ||
- Table size | ||
- Time since last analysis | ||
|
||
3. **Realistic Constraints**: The test data is generated with realistic constraints: | ||
- Change rate calculation based on table size | ||
- Changes capped at 300% of table size | ||
- Minimum meaningful change of 1000 rows | ||
|
||
## How It Works | ||
|
||
1. `generateTestData()` creates test scenarios directly in memory. | ||
2. `TestPriorityCalculatorWithGeneratedData()` is the main test function that: | ||
- Generates test data using `generateTestData()` | ||
- Calculates new priorities using `PriorityCalculator` | ||
- Sorts the priorities in descending order | ||
- Compares the results with `calculated_priorities.golden.csv` | ||
|
||
## Key Files | ||
|
||
- `calculator_analysis_test.go`: Contains the main test logic and data generation | ||
- `calculated_priorities.golden.csv`: Expected output with calculated priorities for all scenarios | ||
|
||
## Running the Test | ||
|
||
To run the test: | ||
```shell | ||
go test -v ./pkg/statistics/handle/autoanalyze/priorityqueue/calculatoranalysis | ||
``` | ||
If you want to update the golden file, run the test with `-update` flag: | ||
```shell | ||
go test -v ./pkg/statistics/handle/autoanalyze/priorityqueue/calculatoranalysis -update | ||
``` | ||
|
||
## Output | ||
|
||
The test compares the generated priorities against the golden file `calculated_priorities.golden.csv` with | ||
- ID | ||
- CalculatedPriority | ||
- TableSize | ||
- Changes | ||
- TimeSinceLastAnalyze | ||
- ChangeRatio | ||
|
||
This output allows for analysis of how the priority calculator behaves across different scenarios. | ||
|
||
This test helps ensure that the priority calculator provides consistent and predictable behavior in the auto-analyze feature across a wide range of realistic scenarios. |
277 changes: 277 additions & 0 deletions
277
...tatistics/handle/autoanalyze/priorityqueue/calculatoranalysis/calculator_analysis_test.go
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,277 @@ | ||
// Copyright 2024 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 priorityqueue_test | ||
|
||
import ( | ||
"bytes" | ||
"cmp" | ||
"encoding/csv" | ||
"flag" | ||
"fmt" | ||
"math" | ||
"os" | ||
"slices" | ||
"testing" | ||
"time" | ||
|
||
"github.com/pingcap/tidb/pkg/sessionctx" | ||
"github.com/pingcap/tidb/pkg/sessionctx/sysproctrack" | ||
"github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/priorityqueue" | ||
"github.com/pingcap/tidb/pkg/statistics/handle/types" | ||
"github.com/stretchr/testify/require" | ||
) | ||
|
||
const ( | ||
baseChangeRate = 0.001 // Base change rate: 0.1% per second | ||
changeRateDecayLog = 3 // Controls how quickly the change rate decays for larger tables | ||
smallTableThreshold = 100000 // Tables smaller than this use the base change rate | ||
maxChangePercentage = 3.0 // Maximum change capped at 300% of table size | ||
) | ||
|
||
var update = flag.Bool("update", false, "update .golden files") | ||
|
||
// Please read README.md for more details. | ||
// To update golden file, run the test with -update flag. | ||
func TestPriorityCalculatorWithGeneratedData(t *testing.T) { | ||
jobs := generateTestData(t) | ||
|
||
calculator := priorityqueue.NewPriorityCalculator() | ||
|
||
newPriorities := calculateNewPriorities(jobs, calculator) | ||
sortPrioritiesByWeight(newPriorities) | ||
|
||
// Convert newPriorities to CSV string | ||
var buf bytes.Buffer | ||
writer := csv.NewWriter(&buf) | ||
writer.Write([]string{"ID", "CalculatedPriority", "TableSize", "Changes", "TimeSinceLastAnalyze", "ChangeRatio"}) | ||
for _, p := range newPriorities { | ||
writer.Write([]string{ | ||
fmt.Sprintf("%d", p.job.ID), | ||
fmt.Sprintf("%.4f", p.priority), | ||
fmt.Sprintf("%.0f", p.job.TableSize), | ||
fmt.Sprintf("%.0f", p.job.Changes), | ||
fmt.Sprintf("%.0f", p.job.TimeSinceLastAnalyze), | ||
fmt.Sprintf("%.4f", p.changeRatio), | ||
}) | ||
} | ||
writer.Flush() | ||
got := buf.Bytes() | ||
|
||
// Compare with golden file | ||
golden := "testdata/calculated_priorities.golden.csv" | ||
if *update { | ||
t.Log("updating golden file") | ||
err := os.WriteFile(golden, got, 0644) | ||
require.NoError(t, err, "Failed to update golden file") | ||
} | ||
|
||
want, err := os.ReadFile(golden) | ||
require.NoError(t, err, "Failed to read golden file") | ||
|
||
require.Equal(t, want, got) | ||
} | ||
|
||
// calculateNewPriorities calculates priorities for all jobs | ||
func calculateNewPriorities(jobs []TestJob, calculator *priorityqueue.PriorityCalculator) []jobWithPriority { | ||
newPriorities := make([]jobWithPriority, 0, len(jobs)) | ||
for _, job := range jobs { | ||
priority := calculator.CalculateWeight(&job) | ||
changeRatio := job.Changes / job.TableSize // Calculate change ratio | ||
newPriorities = append(newPriorities, jobWithPriority{job, priority, changeRatio}) | ||
} | ||
return newPriorities | ||
} | ||
|
||
// sortPrioritiesByWeight sorts jobs by priority in descending order | ||
func sortPrioritiesByWeight(priorities []jobWithPriority) { | ||
slices.SortStableFunc(priorities, func(i, j jobWithPriority) int { | ||
return cmp.Compare(j.priority, i.priority) | ||
}) | ||
} | ||
|
||
// generateCombinations generates test data combinations for analyzing table statistics. | ||
// | ||
// The function creates combinations based on different table sizes and time intervals | ||
// since the last analysis. It calculates reasonable change amounts for each combination | ||
// using the following principles: | ||
// | ||
// 1. Maximum Change Rate: Assumes a maximum change rate of 0.1% of the table size per second. | ||
// This rate can be adjusted based on real-world observations. | ||
// | ||
// 2. Change Calculation: For each table size and time interval, it calculates a maximum | ||
// reasonable change as: tableSize * maxChangeRate * timeSinceLastAnalyze. | ||
// | ||
// 3. Change Limit: The maximum change is capped at 300% of the table size to prevent | ||
// unrealistic scenarios for very long time intervals. | ||
// | ||
// 4. Change Variety: Generates six different change amounts for each combination: | ||
// 1%, 10%, 50%, 100%, 200%, and 300% of the calculated maximum change. | ||
// This variety allows testing the priority calculator under different degrees of modification. | ||
// | ||
// This approach creates a dataset that simulates real-world scenarios by considering | ||
// the relationship between table size, time since last analysis, and the magnitude of changes. | ||
func generateCombinations(tableSizes, analyzeTimes []int64) [][]int64 { | ||
var combinations [][]int64 | ||
id := 1 | ||
for _, size := range tableSizes { | ||
for _, time := range analyzeTimes { | ||
// Calculate the maximum reasonable change based on table size and time since last analysis | ||
maxChange := calculateMaxChange(size, time) | ||
|
||
// Generate a range of changes from 10% to 300% of the maximum change | ||
// This allows us to test the priority calculator with various degrees of data modification | ||
changes := []int64{ | ||
maxChange / 10, // 10% of max change | ||
maxChange / 5, // 20% of max change | ||
maxChange / 2, // 50% of max change | ||
maxChange, // 100% of max change | ||
maxChange * 2, // 200% of max change | ||
maxChange * 3, // 300% of max change | ||
} | ||
|
||
for _, change := range changes { | ||
// Only add meaningful changes (> 0) and limit to 300% of table size | ||
// This ensures we don't generate unrealistic scenarios | ||
if change > 0 && change <= size*3 { | ||
combinations = append(combinations, []int64{int64(id), size, change, time}) | ||
id++ | ||
} | ||
} | ||
} | ||
} | ||
return combinations | ||
} | ||
|
||
// calculateMaxChange computes the maximum reasonable change for a given table size and time interval | ||
func calculateMaxChange(tableSize, timeSinceLastAnalyze int64) int64 { | ||
// Determine the change rate based on table size | ||
var changeRate float64 | ||
if tableSize < smallTableThreshold { | ||
// For small tables, use the base change rate | ||
changeRate = baseChangeRate | ||
} else { | ||
// For larger tables, apply a logarithmic decay to the change rate | ||
// This assumes that larger tables change proportionally slower | ||
changeRate = baseChangeRate * math.Pow(0.5, math.Log10(float64(tableSize))/changeRateDecayLog) | ||
} | ||
|
||
// Calculate the maximum change | ||
maxChange := float64(tableSize) * changeRate * float64(timeSinceLastAnalyze) | ||
|
||
// Cap the maximum change at a percentage of the table size to prevent unrealistic scenarios | ||
return int64(math.Min(maxChange, float64(tableSize)*maxChangePercentage)) | ||
} | ||
|
||
// generateTestData generates test data for the priority calculator | ||
func generateTestData(t *testing.T) []TestJob { | ||
tableSizes := []int64{ | ||
1000, // 1K | ||
5000, // 5K | ||
10000, // 10K | ||
50000, // 50K | ||
100000, // 100K | ||
500000, // 500K | ||
1000000, // 1M | ||
5000000, // 5M | ||
10000000, // 10M | ||
50000000, // 50M | ||
100000000, // 100M | ||
} | ||
analyzeTimes := []int64{ | ||
10, // 10 seconds | ||
60, // 1 minute | ||
300, // 5 minutes | ||
900, // 15 minutes | ||
1800, // 30 minutes | ||
3600, // 1 hour | ||
7200, // 2 hours | ||
14400, // 4 hours | ||
28800, // 8 hours | ||
43200, // 12 hours | ||
86400, // 1 day | ||
172800, // 2 days | ||
259200, // 3 days | ||
} | ||
|
||
combinations := generateCombinations(tableSizes, analyzeTimes) | ||
|
||
jobs := make([]TestJob, len(combinations)) | ||
for i, combo := range combinations { | ||
jobs[i] = TestJob{ | ||
ID: int(combo[0]), | ||
TableSize: float64(combo[1]), | ||
Changes: float64(combo[2]), | ||
TimeSinceLastAnalyze: float64(combo[3]), | ||
} | ||
} | ||
|
||
t.Logf("Generated %d test jobs", len(jobs)) | ||
return jobs | ||
} | ||
|
||
type jobWithPriority struct { | ||
job TestJob | ||
priority float64 | ||
changeRatio float64 | ||
} | ||
|
||
type TestJob struct { | ||
ID int | ||
TableSize float64 | ||
Changes float64 | ||
TimeSinceLastAnalyze float64 | ||
} | ||
|
||
// Analyze implements AnalysisJob. | ||
func (j *TestJob) Analyze(statsHandle types.StatsHandle, sysProcTracker sysproctrack.Tracker) error { | ||
panic("unimplemented") | ||
} | ||
|
||
// GetWeight implements AnalysisJob. | ||
func (j *TestJob) GetWeight() float64 { | ||
panic("unimplemented") | ||
} | ||
|
||
// IsValidToAnalyze implements AnalysisJob. | ||
func (j *TestJob) IsValidToAnalyze(sctx sessionctx.Context) (bool, string) { | ||
panic("unimplemented") | ||
} | ||
|
||
// SetWeight implements AnalysisJob. | ||
func (j *TestJob) SetWeight(weight float64) { | ||
panic("unimplemented") | ||
} | ||
|
||
// String implements AnalysisJob. | ||
func (j *TestJob) String() string { | ||
panic("unimplemented") | ||
} | ||
|
||
// GetTableID implements AnalysisJob. | ||
func (j *TestJob) GetTableID() int64 { | ||
return int64(j.ID) | ||
} | ||
|
||
func (j *TestJob) GetIndicators() priorityqueue.Indicators { | ||
return priorityqueue.Indicators{ | ||
ChangePercentage: j.Changes / j.TableSize, | ||
TableSize: j.TableSize, | ||
LastAnalysisDuration: time.Duration(j.TimeSinceLastAnalyze) * time.Second, | ||
} | ||
} | ||
|
||
func (j *TestJob) HasNewlyAddedIndex() bool { | ||
return false | ||
} |
31 changes: 31 additions & 0 deletions
31
pkg/statistics/handle/autoanalyze/priorityqueue/calculatoranalysis/main_test.go
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,31 @@ | ||
// Copyright 2024 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 priorityqueue_test | ||
|
||
import ( | ||
"testing" | ||
|
||
"github.com/pingcap/tidb/pkg/testkit/testsetup" | ||
"go.uber.org/goleak" | ||
) | ||
|
||
func TestMain(m *testing.M) { | ||
opts := []goleak.Option{ | ||
goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).flushDaemon"), | ||
goleak.IgnoreTopFunction("github.com/bazelbuild/rules_go/go/tools/bzltestutil.RegisterTimeoutHandler.func1"), | ||
} | ||
testsetup.SetupForCommonTest() | ||
goleak.VerifyTestMain(m, opts...) | ||
} |
Oops, something went wrong.