Skip to content

Commit

Permalink
*: move the Handle of stats to single package. (#10149)
Browse files Browse the repository at this point in the history
  • Loading branch information
winoros authored Apr 17, 2019
1 parent 44cbb23 commit e5f734e
Show file tree
Hide file tree
Showing 34 changed files with 1,613 additions and 1,468 deletions.
5 changes: 3 additions & 2 deletions cmd/importer/stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/pingcap/log"
"github.com/pingcap/parser/model"
stats "github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
Expand All @@ -34,12 +35,12 @@ func loadStats(tblInfo *model.TableInfo, path string) (*stats.Table, error) {
if err != nil {
return nil, errors.Trace(err)
}
jsTable := &stats.JSONTable{}
jsTable := &handle.JSONTable{}
err = json.Unmarshal(data, jsTable)
if err != nil {
return nil, errors.Trace(err)
}
return stats.TableStatsFromJSON(tblInfo, tblInfo.ID, jsTable)
return handle.TableStatsFromJSON(tblInfo, tblInfo.ID, jsTable)
}

type histogram struct {
Expand Down
6 changes: 3 additions & 3 deletions distsql/request_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -179,7 +179,7 @@ func (builder *RequestBuilder) SetConcurrency(concurrency int) *RequestBuilder {

// TableRangesToKVRanges converts table ranges to "KeyRange".
func TableRangesToKVRanges(tid int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) []kv.KeyRange {
if fb == nil || fb.Hist() == nil {
if fb == nil || fb.Hist == nil {
return tableRangesToKVRangesWithoutSplit(tid, ranges)
}
krs := make([]kv.KeyRange, 0, len(ranges))
Expand Down Expand Up @@ -256,7 +256,7 @@ func TableHandlesToKVRanges(tid int64, handles []int64) []kv.KeyRange {

// IndexRangesToKVRanges converts index ranges to "KeyRange".
func IndexRangesToKVRanges(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) ([]kv.KeyRange, error) {
if fb == nil || fb.Hist() == nil {
if fb == nil || fb.Hist == nil {
return indexRangesToKVWithoutSplit(sc, tid, idxID, ranges)
}
feedbackRanges := make([]*ranger.Range, 0, len(ranges))
Expand All @@ -268,7 +268,7 @@ func IndexRangesToKVRanges(sc *stmtctx.StatementContext, tid, idxID int64, range
feedbackRanges = append(feedbackRanges, &ranger.Range{LowVal: []types.Datum{types.NewBytesDatum(low)},
HighVal: []types.Datum{types.NewBytesDatum(high)}, LowExclude: false, HighExclude: true})
}
feedbackRanges = fb.Hist().SplitRange(sc, feedbackRanges, true)
feedbackRanges = fb.Hist.SplitRange(sc, feedbackRanges, true)
krs := make([]kv.KeyRange, 0, len(feedbackRanges))
for _, ran := range feedbackRanges {
low, high := ran.LowVal[0].GetBytes(), ran.HighVal[0].GetBytes()
Expand Down
14 changes: 7 additions & 7 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ import (
"github.com/pingcap/tidb/privilege/privileges"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/sqlexec"
Expand Down Expand Up @@ -822,13 +822,13 @@ func (do *Domain) LoadBindInfoLoop(ctx sessionctx.Context, parser *parser.Parser
}

// StatsHandle returns the statistic handle.
func (do *Domain) StatsHandle() *statistics.Handle {
return (*statistics.Handle)(atomic.LoadPointer(&do.statsHandle))
func (do *Domain) StatsHandle() *handle.Handle {
return (*handle.Handle)(atomic.LoadPointer(&do.statsHandle))
}

// CreateStatsHandle is used only for test.
func (do *Domain) CreateStatsHandle(ctx sessionctx.Context) {
atomic.StorePointer(&do.statsHandle, unsafe.Pointer(statistics.NewHandle(ctx, do.statsLease)))
atomic.StorePointer(&do.statsHandle, unsafe.Pointer(handle.NewHandle(ctx, do.statsLease)))
}

// StatsUpdating checks if the stats worker is updating.
Expand All @@ -853,7 +853,7 @@ var RunAutoAnalyze = true
// It should be called only once in BootstrapSession.
func (do *Domain) UpdateTableStatsLoop(ctx sessionctx.Context) error {
ctx.GetSessionVars().InRestrictedSQL = true
statsHandle := statistics.NewHandle(ctx, do.statsLease)
statsHandle := handle.NewHandle(ctx, do.statsLease)
atomic.StorePointer(&do.statsHandle, unsafe.Pointer(statsHandle))
do.ddl.RegisterEventCh(statsHandle.DDLEventCh())
if do.statsLease <= 0 {
Expand All @@ -877,7 +877,7 @@ func (do *Domain) newStatsOwner() owner.Manager {
if do.etcdClient == nil {
statsOwner = owner.NewMockManager(id, cancelFunc)
} else {
statsOwner = owner.NewOwnerManager(do.etcdClient, statistics.StatsPrompt, id, statistics.StatsOwnerKey, cancelFunc)
statsOwner = owner.NewOwnerManager(do.etcdClient, handle.StatsPrompt, id, handle.StatsOwnerKey, cancelFunc)
}
// TODO: Need to do something when err is not nil.
err := statsOwner.CampaignOwner(cancelCtx)
Expand Down Expand Up @@ -932,7 +932,7 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager)
logutil.Logger(context.Background()).Debug("handle ddl event failed", zap.Error(err))
}
case <-deltaUpdateTicker.C:
err = statsHandle.DumpStatsDeltaToKV(statistics.DumpDelta)
err = statsHandle.DumpStatsDeltaToKV(handle.DumpDelta)
if err != nil {
logutil.Logger(context.Background()).Debug("dump stats delta failed", zap.Error(err))
}
Expand Down
4 changes: 2 additions & 2 deletions executor/load_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/util/chunk"
)

Expand Down Expand Up @@ -76,7 +76,7 @@ func (e *LoadStatsExec) Open(ctx context.Context) error {

// Update updates the stats of the corresponding table according to the data.
func (e *LoadStatsInfo) Update(data []byte) error {
jsonTbl := &statistics.JSONTable{}
jsonTbl := &handle.JSONTable{}
if err := json.Unmarshal(data, jsonTbl); err != nil {
return errors.Trace(err)
}
Expand Down
8 changes: 4 additions & 4 deletions executor/seqtest/seq_executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ import (
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/mockstore/mocktikv"
"github.com/pingcap/tidb/store/tikv"
Expand Down Expand Up @@ -599,17 +599,17 @@ func (s *seqTestSuite) TestShowStatsHealthy(c *C) {
tk.MustQuery("show stats_healthy").Check(testkit.Rows("test t 100"))
tk.MustExec("insert into t values (1), (2)")
do, _ := session.GetDomain(s.store)
do.StatsHandle().DumpStatsDeltaToKV(statistics.DumpAll)
do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)
tk.MustExec("analyze table t")
tk.MustQuery("show stats_healthy").Check(testkit.Rows("test t 100"))
tk.MustExec("insert into t values (3), (4), (5), (6), (7), (8), (9), (10)")
do.StatsHandle().DumpStatsDeltaToKV(statistics.DumpAll)
do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)
do.StatsHandle().Update(do.InfoSchema())
tk.MustQuery("show stats_healthy").Check(testkit.Rows("test t 19"))
tk.MustExec("analyze table t")
tk.MustQuery("show stats_healthy").Check(testkit.Rows("test t 100"))
tk.MustExec("delete from t")
do.StatsHandle().DumpStatsDeltaToKV(statistics.DumpAll)
do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)
do.StatsHandle().Update(do.InfoSchema())
tk.MustQuery("show stats_healthy").Check(testkit.Rows("test t 0"))
}
Expand Down
4 changes: 2 additions & 2 deletions executor/table_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,9 +99,9 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error {
}

e.resultHandler = &tableResultHandler{}
if e.feedback != nil && e.feedback.Hist() != nil {
if e.feedback != nil && e.feedback.Hist != nil {
// EncodeInt don't need *statement.Context.
e.ranges = e.feedback.Hist().SplitRange(nil, e.ranges, false)
e.ranges = e.feedback.Hist.SplitRange(nil, e.ranges, false)
}
firstPartRanges, secondPartRanges := splitRanges(e.ranges, e.keepOrder)
firstResult, err := e.buildResp(ctx, firstPartRanges)
Expand Down
10 changes: 5 additions & 5 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import (
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/testkit"
Expand Down Expand Up @@ -137,22 +137,22 @@ func (s *testTableSuite) TestDataForTableStatsField(c *C) {
tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check(
testkit.Rows("0 0 0 0"))
tk.MustExec(`insert into t(c, d, e) values(1, 2, "c"), (2, 3, "d"), (3, 4, "e")`)
h.DumpStatsDeltaToKV(statistics.DumpAll)
h.DumpStatsDeltaToKV(handle.DumpAll)
h.Update(is)
tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check(
testkit.Rows("3 17 51 3"))
tk.MustExec(`insert into t(c, d, e) values(4, 5, "f")`)
h.DumpStatsDeltaToKV(statistics.DumpAll)
h.DumpStatsDeltaToKV(handle.DumpAll)
h.Update(is)
tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check(
testkit.Rows("4 17 68 4"))
tk.MustExec("delete from t where c >= 3")
h.DumpStatsDeltaToKV(statistics.DumpAll)
h.DumpStatsDeltaToKV(handle.DumpAll)
h.Update(is)
tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check(
testkit.Rows("2 17 34 2"))
tk.MustExec("delete from t where c=3")
h.DumpStatsDeltaToKV(statistics.DumpAll)
h.DumpStatsDeltaToKV(handle.DumpAll)
h.Update(is)
tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check(
testkit.Rows("2 17 34 2"))
Expand Down
15 changes: 8 additions & 7 deletions planner/core/cbo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/util/testkit"
"github.com/pingcap/tidb/util/testleak"
Expand All @@ -47,7 +48,7 @@ func (s *testAnalyzeSuite) loadTableStats(fileName string, dom *domain.Domain) e
if err != nil {
return err
}
statsTbl := &statistics.JSONTable{}
statsTbl := &handle.JSONTable{}
err = json.Unmarshal(bytes, statsTbl)
if err != nil {
return err
Expand Down Expand Up @@ -105,7 +106,7 @@ func (s *testAnalyzeSuite) TestCBOWithoutAnalyze(c *C) {
c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil)
testKit.MustExec("insert into t1 values (1), (2), (3), (4), (5), (6)")
testKit.MustExec("insert into t2 values (1), (2), (3), (4), (5), (6)")
h.DumpStatsDeltaToKV(statistics.DumpAll)
h.DumpStatsDeltaToKV(handle.DumpAll)
c.Assert(h.Update(dom.InfoSchema()), IsNil)
testKit.MustQuery("explain select * from t1, t2 where t1.a = t2.a").Check(testkit.Rows(
"HashLeftJoin_8 7.49 root inner join, inner:TableReader_15, equal:[eq(test.t1.a, test.t2.a)]",
Expand Down Expand Up @@ -195,7 +196,7 @@ func (s *testAnalyzeSuite) TestTableDual(c *C) {
testKit.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10)")
c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil)

h.DumpStatsDeltaToKV(statistics.DumpAll)
h.DumpStatsDeltaToKV(handle.DumpAll)
c.Assert(h.Update(dom.InfoSchema()), IsNil)

testKit.MustQuery(`explain select * from t where 1 = 0`).Check(testkit.Rows(
Expand Down Expand Up @@ -225,12 +226,12 @@ func (s *testAnalyzeSuite) TestEstimation(c *C) {
testKit.MustExec("insert into t select * from t")
h := dom.StatsHandle()
h.HandleDDLEvent(<-h.DDLEventCh())
h.DumpStatsDeltaToKV(statistics.DumpAll)
h.DumpStatsDeltaToKV(handle.DumpAll)
testKit.MustExec("analyze table t")
for i := 1; i <= 8; i++ {
testKit.MustExec("delete from t where a = ?", i)
}
h.DumpStatsDeltaToKV(statistics.DumpAll)
h.DumpStatsDeltaToKV(handle.DumpAll)
c.Assert(h.Update(dom.InfoSchema()), IsNil)
testKit.MustQuery("explain select count(*) from t group by a").Check(testkit.Rows(
"HashAgg_9 2.00 root group by:col_1, funcs:count(col_0)",
Expand Down Expand Up @@ -567,12 +568,12 @@ func (s *testAnalyzeSuite) TestOutdatedAnalyze(c *C) {
}
h := dom.StatsHandle()
h.HandleDDLEvent(<-h.DDLEventCh())
h.DumpStatsDeltaToKV(statistics.DumpAll)
h.DumpStatsDeltaToKV(handle.DumpAll)
testKit.MustExec("analyze table t")
testKit.MustExec("insert into t select * from t")
testKit.MustExec("insert into t select * from t")
testKit.MustExec("insert into t select * from t")
h.DumpStatsDeltaToKV(statistics.DumpAll)
h.DumpStatsDeltaToKV(handle.DumpAll)
c.Assert(h.Update(dom.InfoSchema()), IsNil)
statistics.RatioOfPseudoEstimate = 10.0
testKit.MustQuery("explain select * from t where a <= 5 and b <= 5").Check(testkit.Rows(
Expand Down
6 changes: 3 additions & 3 deletions server/statistics_handler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import (
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/mockstore/mocktikv"
)
Expand Down Expand Up @@ -120,11 +120,11 @@ func (ds *testDumpStatsSuite) prepareData(c *C) {
h.HandleDDLEvent(<-h.DDLEventCh())
dbt.mustExec("create index c on test (a, b)")
dbt.mustExec("insert test values (1, 's')")
c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil)
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
dbt.mustExec("analyze table test")
dbt.mustExec("insert into test(a,b) values (1, 'v'),(3, 'vvv'),(5, 'vv')")
is := ds.sh.do.InfoSchema()
c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil)
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
c.Assert(h.Update(is), IsNil)
}

Expand Down
4 changes: 2 additions & 2 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ import (
"github.com/pingcap/tidb/sessionctx/binloginfo"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
Expand Down Expand Up @@ -152,7 +152,7 @@ type session struct {
sessionVars *variable.SessionVars
sessionManager util.SessionManager

statsCollector *statistics.SessionStatsCollector
statsCollector *handle.SessionStatsCollector
// ddlOwnerChecker is used in `select tidb_is_ddl_owner()` statement;
ddlOwnerChecker owner.DDLOwnerChecker
}
Expand Down
9 changes: 6 additions & 3 deletions statistics/cmsketch.go
Original file line number Diff line number Diff line change
Expand Up @@ -145,15 +145,17 @@ func CMSketchFromProto(protoSketch *tipb.CMSketch) *CMSketch {
return c
}

func encodeCMSketch(c *CMSketch) ([]byte, error) {
// EncodeCMSketch encodes the given CMSketch to byte slice.
func EncodeCMSketch(c *CMSketch) ([]byte, error) {
if c == nil || c.count == 0 {
return nil, nil
}
p := CMSketchToProto(c)
return p.Marshal()
}

func decodeCMSketch(data []byte) (*CMSketch, error) {
// DecodeCMSketch decode a CMSketch from the given byte slice.
func DecodeCMSketch(data []byte) (*CMSketch, error) {
if data == nil {
return nil, nil
}
Expand Down Expand Up @@ -191,7 +193,8 @@ func (c *CMSketch) Equal(rc *CMSketch) bool {
return true
}

func (c *CMSketch) copy() *CMSketch {
// Copy makes a copy for current CMSketch.
func (c *CMSketch) Copy() *CMSketch {
if c == nil {
return nil
}
Expand Down
4 changes: 2 additions & 2 deletions statistics/cmsketch_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -120,10 +120,10 @@ func (s *testStatisticsSuite) TestCMSketchCoding(c *C) {
lSketch.table[i][j] = math.MaxUint32
}
}
bytes, err := encodeCMSketch(lSketch)
bytes, err := EncodeCMSketch(lSketch)
c.Assert(err, IsNil)
c.Assert(len(bytes), Equals, 61455)
rSketch, err := decodeCMSketch(bytes)
rSketch, err := DecodeCMSketch(bytes)
c.Assert(err, IsNil)
c.Assert(lSketch.Equal(rSketch), IsTrue)
}
Loading

0 comments on commit e5f734e

Please sign in to comment.