From f37e5f36c918b9c1b42519ede47a6afdbfeb8283 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 29 Apr 2024 16:58:36 +0800 Subject: [PATCH] statistics: reduce allocation of types.Context (#52615) (#52963) close pingcap/tidb#52342 --- pkg/statistics/BUILD.bazel | 1 + pkg/statistics/handle/BUILD.bazel | 1 - pkg/statistics/handle/bootstrap.go | 10 ++-------- pkg/statistics/handle/storage/BUILD.bazel | 1 - pkg/statistics/handle/storage/json.go | 8 ++------ pkg/statistics/handle/storage/read.go | 9 ++------- pkg/statistics/histogram.go | 4 ++-- pkg/statistics/scalar.go | 11 ++++++++--- 8 files changed, 17 insertions(+), 28 deletions(-) diff --git a/pkg/statistics/BUILD.bazel b/pkg/statistics/BUILD.bazel index 10e37351b43a5..9eb3f27750d75 100644 --- a/pkg/statistics/BUILD.bazel +++ b/pkg/statistics/BUILD.bazel @@ -41,6 +41,7 @@ go_library( "//pkg/util/chunk", "//pkg/util/codec", "//pkg/util/collate", + "//pkg/util/context", "//pkg/util/dbterror", "//pkg/util/fastrand", "//pkg/util/hack", diff --git a/pkg/statistics/handle/BUILD.bazel b/pkg/statistics/handle/BUILD.bazel index bfd4bbe943a3a..f5787bc595d13 100644 --- a/pkg/statistics/handle/BUILD.bazel +++ b/pkg/statistics/handle/BUILD.bazel @@ -16,7 +16,6 @@ go_library( "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/sessionctx", - "//pkg/sessionctx/stmtctx", "//pkg/sessionctx/sysproctrack", "//pkg/statistics", "//pkg/statistics/handle/autoanalyze", diff --git a/pkg/statistics/handle/bootstrap.go b/pkg/statistics/handle/bootstrap.go index 6ba21b3d3d882..33f2d3c279fb8 100644 --- a/pkg/statistics/handle/bootstrap.go +++ b/pkg/statistics/handle/bootstrap.go @@ -18,7 +18,6 @@ import ( "context" "sync" "sync/atomic" - "time" "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/config" @@ -28,7 +27,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/cache" "github.com/pingcap/tidb/pkg/statistics/handle/initstats" @@ -558,19 +556,15 @@ func (*Handle) initStatsBuckets4Chunk(cache statstypes.StatsCache, iter *chunk.I } hist = &column.Histogram d := types.NewBytesDatum(row.GetBytes(5)) - // Setting TimeZone to time.UTC aligns with HistogramFromStorage and can fix #41938. However, #41985 still exist. - // TODO: do the correct time zone conversion for timestamp-type columns' upper/lower bounds. - sc := stmtctx.NewStmtCtxWithTimeZone(time.UTC) - sc.SetTypeFlags(sc.TypeFlags().WithIgnoreInvalidDateErr(true).WithIgnoreZeroInDate(true)) var err error - lower, err = d.ConvertTo(sc.TypeCtx(), &column.Info.FieldType) + lower, err = d.ConvertTo(statistics.UTCWithAllowInvalidDateCtx, &column.Info.FieldType) if err != nil { logutil.BgLogger().Debug("decode bucket lower bound failed", zap.Error(err)) delete(table.Columns, histID) continue } d = types.NewBytesDatum(row.GetBytes(6)) - upper, err = d.ConvertTo(sc.TypeCtx(), &column.Info.FieldType) + upper, err = d.ConvertTo(statistics.UTCWithAllowInvalidDateCtx, &column.Info.FieldType) if err != nil { logutil.BgLogger().Debug("decode bucket upper bound failed", zap.Error(err)) delete(table.Columns, histID) diff --git a/pkg/statistics/handle/storage/BUILD.bazel b/pkg/statistics/handle/storage/BUILD.bazel index 416e4ff2c4096..52834e9c9d718 100644 --- a/pkg/statistics/handle/storage/BUILD.bazel +++ b/pkg/statistics/handle/storage/BUILD.bazel @@ -20,7 +20,6 @@ go_library( "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/sessionctx", - "//pkg/sessionctx/stmtctx", "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/statistics/handle/cache", diff --git a/pkg/statistics/handle/storage/json.go b/pkg/statistics/handle/storage/json.go index f98882f2350b1..3d441e20b281a 100644 --- a/pkg/statistics/handle/storage/json.go +++ b/pkg/statistics/handle/storage/json.go @@ -18,14 +18,12 @@ import ( "bytes" "encoding/json" "io" - "time" "github.com/klauspost/compress/gzip" "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/types" @@ -103,8 +101,7 @@ func GenJSONTableFromStats(sctx sessionctx.Context, dbName string, tableInfo *mo Version: tbl.Version, } for _, col := range tbl.Columns { - sc := stmtctx.NewStmtCtxWithTimeZone(time.UTC) - hist, err := col.ConvertTo(sc, types.NewFieldType(mysql.TypeBlob)) + hist, err := col.ConvertTo(statistics.UTCWithAllowInvalidDateCtx, types.NewFieldType(mysql.TypeBlob)) if err != nil { return nil, errors.Trace(err) } @@ -183,7 +180,6 @@ func TableStatsFromJSON(tableInfo *model.TableInfo, physicalID int64, jsonTbl *u continue } hist := statistics.HistogramFromProto(jsonCol.Histogram) - sc := stmtctx.NewStmtCtxWithTimeZone(time.UTC) tmpFT := colInfo.FieldType // For new collation data, when storing the bounds of the histogram, we store the collate key instead of the // original value. @@ -195,7 +191,7 @@ func TableStatsFromJSON(tableInfo *model.TableInfo, physicalID int64, jsonTbl *u if colInfo.FieldType.EvalType() == types.ETString && colInfo.FieldType.GetType() != mysql.TypeEnum && colInfo.FieldType.GetType() != mysql.TypeSet { tmpFT = *types.NewFieldType(mysql.TypeBlob) } - hist, err := hist.ConvertTo(sc, &tmpFT) + hist, err := hist.ConvertTo(statistics.UTCWithAllowInvalidDateCtx, &tmpFT) if err != nil { return nil, errors.Trace(err) } diff --git a/pkg/statistics/handle/storage/read.go b/pkg/statistics/handle/storage/read.go index 7586726d3c5f8..c23468116e4b9 100644 --- a/pkg/statistics/handle/storage/read.go +++ b/pkg/statistics/handle/storage/read.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/statistics" statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil" statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types" @@ -99,10 +98,6 @@ func HistogramFromStorage(sctx sessionctx.Context, tableID int64, colID int64, t lowerBound = rows[i].GetDatum(2, &fields[2].Column.FieldType) upperBound = rows[i].GetDatum(3, &fields[3].Column.FieldType) } else { - // Invalid date values may be inserted into table under some relaxed sql mode. Those values may exist in statistics. - // Hence, when reading statistics, we should skip invalid date check. See #39336. - sc := stmtctx.NewStmtCtxWithTimeZone(time.UTC) - sc.SetTypeFlags(sc.TypeFlags().WithIgnoreInvalidDateErr(true).WithIgnoreZeroInDate(true)) d := rows[i].GetDatum(2, &fields[2].Column.FieldType) // For new collation data, when storing the bounds of the histogram, we store the collate key instead of the // original value. @@ -114,12 +109,12 @@ func HistogramFromStorage(sctx sessionctx.Context, tableID int64, colID int64, t if tp.EvalType() == types.ETString && tp.GetType() != mysql.TypeEnum && tp.GetType() != mysql.TypeSet { tp = types.NewFieldType(mysql.TypeBlob) } - lowerBound, err = d.ConvertTo(sc.TypeCtx(), tp) + lowerBound, err = d.ConvertTo(statistics.UTCWithAllowInvalidDateCtx, tp) if err != nil { return nil, errors.Trace(err) } d = rows[i].GetDatum(3, &fields[3].Column.FieldType) - upperBound, err = d.ConvertTo(sc.TypeCtx(), tp) + upperBound, err = d.ConvertTo(statistics.UTCWithAllowInvalidDateCtx, tp) if err != nil { return nil, errors.Trace(err) } diff --git a/pkg/statistics/histogram.go b/pkg/statistics/histogram.go index 642110ee2621f..000dd80c5be2c 100644 --- a/pkg/statistics/histogram.go +++ b/pkg/statistics/histogram.go @@ -200,13 +200,13 @@ func (hg *Histogram) DecodeTo(tp *types.FieldType, timeZone *time.Location) erro } // ConvertTo converts the histogram bucket values into `tp`. -func (hg *Histogram) ConvertTo(sc *stmtctx.StatementContext, tp *types.FieldType) (*Histogram, error) { +func (hg *Histogram) ConvertTo(tctx types.Context, tp *types.FieldType) (*Histogram, error) { hist := NewHistogram(hg.ID, hg.NDV, hg.NullCount, hg.LastUpdateVersion, tp, hg.Len(), hg.TotColSize) hist.Correlation = hg.Correlation iter := chunk.NewIterator4Chunk(hg.Bounds) for row := iter.Begin(); row != iter.End(); row = iter.Next() { d := row.GetDatum(0, hg.Tp) - d, err := d.ConvertTo(sc.TypeCtx(), tp) + d, err := d.ConvertTo(tctx, tp) if err != nil { return nil, errors.Trace(err) } diff --git a/pkg/statistics/scalar.go b/pkg/statistics/scalar.go index 105dda35cd77b..3b9568dd23cb9 100644 --- a/pkg/statistics/scalar.go +++ b/pkg/statistics/scalar.go @@ -20,8 +20,8 @@ import ( "time" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/context" ) // calcFraction is used to calculate the fraction of the interval [lower, upper] that lies within the [lower, value] @@ -43,6 +43,12 @@ func calcFraction(lower, upper, value float64) float64 { return frac } +// UTCWithAllowInvalidDateCtx is introduced for the following reason: +// +// Invalid date values may be inserted into table under some relaxed sql mode. Those values may exist in statistics. +// Hence, when reading statistics, we should skip invalid date check. See #39336. +var UTCWithAllowInvalidDateCtx = types.NewContext(types.DefaultStmtFlags|types.FlagIgnoreInvalidDateErr|types.FlagIgnoreZeroInDateErr, time.UTC, context.IgnoreWarn) + func convertDatumToScalar(value *types.Datum, commonPfxLen int) float64 { switch value.Kind() { case types.KindFloat32: @@ -72,8 +78,7 @@ func convertDatumToScalar(value *types.Datum, commonPfxLen int) float64 { case mysql.TypeTimestamp: minTime = types.MinTimestamp } - sc := stmtctx.NewStmtCtxWithTimeZone(types.BoundTimezone) - return float64(valueTime.Sub(sc.TypeCtx(), &minTime).Duration) + return float64(valueTime.Sub(UTCWithAllowInvalidDateCtx, &minTime).Duration) case types.KindString, types.KindBytes: bytes := value.GetBytes() if len(bytes) <= commonPfxLen {