Skip to content

Commit

Permalink
statistics/handle: use StatementContext with TimeZone in initStatsBuc…
Browse files Browse the repository at this point in the history
…kets4Chunk (#41993) (#42004)

close #41938
  • Loading branch information
ti-chi-bot authored May 23, 2023
1 parent de80b6a commit fee5061
Show file tree
Hide file tree
Showing 2 changed files with 25 additions and 3 deletions.
11 changes: 8 additions & 3 deletions statistics/handle/bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ package handle
import (
"context"
"strconv"
"time"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/config"
Expand All @@ -25,6 +26,7 @@ import (
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/parser/terror"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
Expand Down Expand Up @@ -310,15 +312,18 @@ func (h *Handle) initStatsBuckets4Chunk(cache *statsCache, iter *chunk.Iterator4
}
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.StatementContext{TimeZone: time.UTC, AllowInvalidDate: true, IgnoreZeroInDate: true}
var err error
lower, err = d.ConvertTo(h.mu.ctx.GetSessionVars().StmtCtx, &column.Info.FieldType)
lower, err = d.ConvertTo(sc, &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(h.mu.ctx.GetSessionVars().StmtCtx, &column.Info.FieldType)
upper, err = d.ConvertTo(sc, &column.Info.FieldType)
if err != nil {
logutil.BgLogger().Debug("decode bucket upper bound failed", zap.Error(err))
delete(table.Columns, histID)
Expand All @@ -335,7 +340,7 @@ func (h *Handle) initTopNCountSum(tableID, colID int64) (int64, error) {
// So we need to add TopN total count here.
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats)
selSQL := "select sum(count) from mysql.stats_top_n where table_id = %? and is_index = 0 and hist_id = %?"
rs, err := h.mu.ctx.(sqlexec.SQLExecutor).ExecuteInternal(ctx, selSQL, tableID, colID)
rs, err := h.initStatsCtx.(sqlexec.SQLExecutor).ExecuteInternal(ctx, selSQL, tableID, colID)
if rs != nil {
defer terror.Call(rs.Close)
}
Expand Down
17 changes: 17 additions & 0 deletions statistics/handle/handle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -544,6 +544,23 @@ func TestInitStatsVer2(t *testing.T) {
h.SetLease(0)
}

func TestInitStatsIssue41938(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set @@global.tidb_analyze_version=1")
tk.MustExec("set @@session.tidb_analyze_version=1")
tk.MustExec("create table t1 (a timestamp primary key)")
tk.MustExec("insert into t1 values ('2023-03-07 14:24:30'), ('2023-03-07 14:24:31'), ('2023-03-07 14:24:32'), ('2023-03-07 14:24:33')")
tk.MustExec("analyze table t1 with 0 topn")
h := dom.StatsHandle()
// `InitStats` is only called when `Lease` is not 0, so here we just change it.
h.SetLease(time.Millisecond)
h.Clear()
require.NoError(t, h.InitStats(dom.InfoSchema()))
h.SetLease(0)
}

func TestReloadExtStatsLockRelease(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
Expand Down

0 comments on commit fee5061

Please sign in to comment.