Skip to content

Commit

Permalink
statistics: reduce allocation of types.Context (#52615) (#52963)
Browse files Browse the repository at this point in the history
close #52342
  • Loading branch information
ti-chi-bot authored Apr 29, 2024
1 parent 015448e commit f37e5f3
Show file tree
Hide file tree
Showing 8 changed files with 17 additions and 28 deletions.
1 change: 1 addition & 0 deletions pkg/statistics/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
1 change: 0 additions & 1 deletion pkg/statistics/handle/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
10 changes: 2 additions & 8 deletions pkg/statistics/handle/bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (
"context"
"sync"
"sync/atomic"
"time"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/config"
Expand All @@ -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"
Expand Down Expand Up @@ -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)
Expand Down
1 change: 0 additions & 1 deletion pkg/statistics/handle/storage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
8 changes: 2 additions & 6 deletions pkg/statistics/handle/storage/json.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)
}
Expand Down Expand Up @@ -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.
Expand All @@ -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)
}
Expand Down
9 changes: 2 additions & 7 deletions pkg/statistics/handle/storage/read.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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.
Expand All @@ -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)
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/statistics/histogram.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
Expand Down
11 changes: 8 additions & 3 deletions pkg/statistics/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -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]
Expand All @@ -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:
Expand Down Expand Up @@ -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 {
Expand Down

0 comments on commit f37e5f3

Please sign in to comment.