Skip to content

Commit

Permalink
statistics: right deal with error for reading stats from storage (#58048
Browse files Browse the repository at this point in the history
)

ref #57901
  • Loading branch information
hawkingrei authored Dec 6, 2024
1 parent 5de1105 commit 1521bf7
Show file tree
Hide file tree
Showing 5 changed files with 38 additions and 2 deletions.
2 changes: 1 addition & 1 deletion pkg/statistics/handle/handletest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ go_test(
],
flaky = True,
race = "on",
shard_count = 33,
shard_count = 34,
deps = [
"//pkg/config",
"//pkg/domain",
Expand Down
28 changes: 28 additions & 0 deletions pkg/statistics/handle/handletest/handle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1490,3 +1490,31 @@ func TestSkipMissingPartitionStats(t *testing.T) {
return false
})
}

func TestStatsCacheUpdateTimeout(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'")
tk.MustExec("set @@tidb_skip_missing_partition_stats = 1")
tk.MustExec("create table t (a int, b int, c int, index idx_b(b)) partition by range (a) (partition p0 values less than (100), partition p1 values less than (200), partition p2 values less than (300))")
tk.MustExec("insert into t values (1,1,1), (2,2,2), (101,101,101), (102,102,102), (201,201,201), (202,202,202)")
analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a", "b", "c")
h := dom.StatsHandle()
require.NoError(t, h.DumpStatsDeltaToKV(true))
tk.MustExec("analyze table t partition p0, p1")
tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
tblInfo := tbl.Meta()
globalStats := h.GetTableStats(tblInfo)
require.Equal(t, 6, int(globalStats.RealtimeCount))
require.Equal(t, 2, int(globalStats.ModifyCount))
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/statistics/handle/util/ExecRowsTimeout", "return()"))
defer func() {
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/statistics/handle/util/ExecRowsTimeout"))
}()
require.Error(t, h.Update(context.Background(), dom.InfoSchema()))
globalStats2 := h.GetTableStats(tblInfo)
require.Equal(t, 6, int(globalStats2.RealtimeCount))
require.Equal(t, 2, int(globalStats2.ModifyCount))
}
5 changes: 4 additions & 1 deletion pkg/statistics/handle/storage/read.go
Original file line number Diff line number Diff line change
Expand Up @@ -539,8 +539,11 @@ func TableStatsFromStorage(sctx sessionctx.Context, snapshot uint64, tableInfo *
table.RealtimeCount = realtimeCount

rows, _, err := util.ExecRows(sctx, "select table_id, is_index, hist_id, distinct_count, version, null_count, tot_col_size, stats_ver, flag, correlation, last_analyze_pos from mysql.stats_histograms where table_id = %?", tableID)
if err != nil {
return nil, err
}
// Check deleted table.
if err != nil || len(rows) == 0 {
if len(rows) == 0 {
return nil, nil
}
for _, row := range rows {
Expand Down
1 change: 1 addition & 0 deletions pkg/statistics/handle/util/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ go_library(
"//pkg/util/sqlexec",
"//pkg/util/sqlexec/mock",
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_tiancaiamao_gp//:gp",
"@com_github_tikv_client_go_v2//oracle",
"@org_golang_x_exp//maps",
Expand Down
4 changes: 4 additions & 0 deletions pkg/statistics/handle/util/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"time"

"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/meta/model"
"github.com/pingcap/tidb/pkg/parser/terror"
Expand Down Expand Up @@ -229,6 +230,9 @@ func ExecWithCtx(

// ExecRows is a helper function to execute sql and return rows and fields.
func ExecRows(sctx sessionctx.Context, sql string, args ...any) (rows []chunk.Row, fields []*resolve.ResultField, err error) {
failpoint.Inject("ExecRowsTimeout", func() {
failpoint.Return(nil, nil, errors.New("inject timeout error"))
})
return ExecRowsWithCtx(StatsCtx, sctx, sql, args...)
}

Expand Down

0 comments on commit 1521bf7

Please sign in to comment.