Skip to content

Commit

Permalink
statistics: refactor package. (pingcap#2913)
Browse files Browse the repository at this point in the history
  • Loading branch information
hanfei1991 authored Mar 29, 2017
1 parent 1253d2b commit ee7df05
Show file tree
Hide file tree
Showing 16 changed files with 413 additions and 508 deletions.
44 changes: 42 additions & 2 deletions bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,13 +116,39 @@ const (
UNIQUE KEY name (name)
) ENGINE=InnoDB DEFAULT CHARSET=utf8 STATS_PERSISTENT=0 COMMENT='help topics';`

// CreateStatsMetaTable store's the meta of table statistics.
// CreateStatsMetaTable stores the meta of table statistics.
CreateStatsMetaTable = `CREATE TABLE if not exists mysql.stats_meta (
version bigint(64) unsigned NOT NULL,
table_id bigint(64) NOT NULL,
modify_count bigint(64) NOT NULL DEFAULT 0,
count bigint(64) unsigned NOT NULL DEFAULT 0,
index idx_ver(version)
index idx_ver(version),
unique index tbl(table_id)
);`

// CreateStatsColsTable stores the statistics of table columns.
CreateStatsColsTable = `CREATE TABLE if not exists mysql.stats_histograms (
table_id bigint(64) NOT NULL,
is_index tinyint(2) NOT NULL,
hist_id bigint(64) NOT NULL,
distinct_count bigint(64) NOT NULL,
distinct_ratio double(64) NOT NULL DEFAULT 0,
use_count_to_estimate tinyint(2) NOT NULL DEFAULT 0,
modify_count bigint(64) NOT NULL DEFAULT 0,
version bigint(64) unsigned NOT NULL DEFAULT 0,
unique index tbl(table_id, is_index, hist_id)
);`

// CreateStatsBucketsTable stores the histogram info for every table columns.
CreateStatsBucketsTable = `CREATE TABLE if not exists mysql.stats_buckets (
table_id bigint(64) NOT NULL,
is_index tinyint(2) NOT NULL,
hist_id bigint(64) NOT NULL,
bucket_id bigint(64) NOT NULL,
count bigint(64) NOT NULL,
repeats bigint(64) NOT NULL,
value blob NOT NULL,
unique index tbl(table_id, is_index, hist_id, bucket_id)
);`
)

Expand Down Expand Up @@ -153,6 +179,7 @@ const (
version2 = 2
version3 = 3
version4 = 4
version5 = 5
)

func checkBootstrapped(s Session) (bool, error) {
Expand Down Expand Up @@ -226,6 +253,10 @@ func upgrade(s Session) {
upgradeToVer4(s)
}

if ver < version5 {
upgradeToVer5(s)
}

updateBootstrapVer(s)
_, err = s.Execute("COMMIT")

Expand Down Expand Up @@ -279,6 +310,11 @@ func upgradeToVer4(s Session) {
mustExecute(s, sql)
}

func upgradeToVer5(s Session) {
mustExecute(s, CreateStatsColsTable)
mustExecute(s, CreateStatsBucketsTable)
}

// Update boostrap version variable in mysql.TiDB table.
func updateBootstrapVer(s Session) {
// Update bootstrap version.
Expand Down Expand Up @@ -319,6 +355,10 @@ func doDDLWorks(s Session) {
mustExecute(s, CreateHelpTopic)
// Create stats_meta table.
mustExecute(s, CreateStatsMetaTable)
// Create stats_columns table.
mustExecute(s, CreateStatsColsTable)
// Create stats_buckets table.
mustExecute(s, CreateStatsBucketsTable)
}

// Execute DML statements in bootstrap stage.
Expand Down
55 changes: 22 additions & 33 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,9 +26,9 @@ import (
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/perfschema"
"github.com/pingcap/tidb/plan/statscache"
"github.com/pingcap/tidb/privilege/privileges"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/terror"
)

Expand All @@ -38,7 +38,7 @@ type Domain struct {
store kv.Storage
infoHandle *infoschema.Handle
privHandle *privileges.Handle
statsHandle *statscache.Handle
statsHandle *statistics.Handle
ddl ddl.DDL
m sync.Mutex
SchemaValidator SchemaValidator
Expand Down Expand Up @@ -408,49 +408,38 @@ func (do *Domain) PrivilegeHandle() *privileges.Handle {
}

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

func (do *Domain) loadTableStats() error {
ver, err := do.store.CurrentVersion()
if err != nil {
return errors.Trace(err)
}
snapshot, err := do.store.GetSnapshot(kv.NewVersion(ver.Ver))
if err != nil {
return errors.Trace(err)
}
m := meta.NewSnapshotMeta(snapshot)
err = do.statsHandle.Update(m, do.InfoSchema())
return errors.Trace(err)
}

// LoadTableStatsLoop creates a goroutine loads stats info in a loop, it
// should be called only once in BootstrapSession.
func (do *Domain) LoadTableStatsLoop(ctx context.Context) error {
do.statsHandle = statscache.NewHandle(ctx)
err := do.loadTableStats()
do.statsHandle = statistics.NewHandle(ctx)
err := do.statsHandle.Update(do.InfoSchema())
if err != nil {
return errors.Trace(err)
}
lease := do.DDL().GetLease()
if lease > 0 {
go func(do *Domain) {
ticker := time.NewTicker(lease)
for {
select {
case <-ticker.C:
err := do.loadTableStats()
if err != nil {
log.Error(errors.ErrorStack(err))
}
case <-do.exit:
return
if lease <= 0 {
return nil
}
go func(do *Domain) {
ticker := time.NewTicker(lease)
defer ticker.Stop()

for {
select {
case <-ticker.C:
err := do.statsHandle.Update(do.InfoSchema())
if err != nil {
log.Error(errors.ErrorStack(err))
}
case <-do.exit:
return
}
}(do)
}
}
}(do)
return nil
}

Expand Down
2 changes: 1 addition & 1 deletion executor/aggregate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -270,7 +270,7 @@ func (s *testSuite) TestAggregation(c *C) {

result = tk.MustQuery("select count(*) from information_schema.columns")
// When adding new memory table in information_schema, please update this variable.
columnCountOfAllInformationSchemaTables := "561"
columnCountOfAllInformationSchemaTables := "576"
result.Check(testkit.Rows(columnCountOfAllInformationSchemaTables))

tk.MustExec("drop table if exists t1")
Expand Down
27 changes: 3 additions & 24 deletions executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,18 +14,14 @@
package executor

import (
"fmt"
"math/rand"

"github.com/juju/errors"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/plan/statistics"
"github.com/pingcap/tidb/plan/statscache"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/util/types"
)

Expand Down Expand Up @@ -99,11 +95,9 @@ func (e *AnalyzeExec) Next() (*Row, error) {
}

func (e *AnalyzeExec) buildStatisticsAndSaveToKV(count int64, columnSamples [][]types.Datum, idxRS []ast.RecordSet, pkRS ast.RecordSet) error {
txn := e.ctx.Txn()
statBuilder := &statistics.Builder{
Ctx: e.ctx,
TblInfo: e.tblInfo,
StartTS: int64(txn.StartTS()),
Count: count,
NumBuckets: defaultBucketCount,
ColumnSamples: columnSamples,
Expand All @@ -117,23 +111,8 @@ func (e *AnalyzeExec) buildStatisticsAndSaveToKV(count int64, columnSamples [][]
if err != nil {
return errors.Trace(err)
}
version := e.ctx.Txn().StartTS()
statscache.SetStatisticsTableCache(e.tblInfo.ID, t, version)
tpb, err := t.ToPB()
if err != nil {
return errors.Trace(err)
}
m := meta.NewMeta(txn)
err = m.SetTableStats(e.tblInfo.ID, tpb)
if err != nil {
return errors.Trace(err)
}
insertSQL := fmt.Sprintf("insert into mysql.stats_meta (version, table_id) values (%d, %d) on duplicate key update version = %d", version, e.tblInfo.ID, version)
_, _, err = e.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(e.ctx, insertSQL)
if err != nil {
return errors.Trace(err)
}
return nil
err = t.SaveToStorage(e.ctx)
return errors.Trace(err)
}

// collectSamples collects sample from the result set, using Reservoir Sampling algorithm.
Expand Down
34 changes: 0 additions & 34 deletions meta/meta.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,10 @@ import (
"sync"
"time"

"github.com/golang/protobuf/proto"
"github.com/juju/errors"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/plan/statistics"
"github.com/pingcap/tidb/structure"
"github.com/pingcap/tidb/terror"
)
Expand Down Expand Up @@ -704,38 +702,6 @@ func (m *Meta) tableStatsKey(tableID int64) []byte {
return []byte(fmt.Sprintf("%s:%d", mTableStatsPrefix, tableID))
}

// SetTableStats sets table statistics.
func (m *Meta) SetTableStats(tableID int64, tpb *statistics.TablePB) error {
key := m.tableStatsKey(tableID)
data, err := proto.Marshal(tpb)
if err != nil {
return errors.Trace(err)
}
err = m.txn.Set(key, data)
if err != nil {
return errors.Trace(err)
}
return nil
}

// GetTableStats gets table statistics.
func (m *Meta) GetTableStats(tableID int64) (*statistics.TablePB, error) {
key := m.tableStatsKey(tableID)
data, err := m.txn.Get(key)
if err != nil {
return nil, errors.Trace(err)
}
if len(data) == 0 {
return nil, nil
}
tpb := &statistics.TablePB{}
err = proto.Unmarshal(data, tpb)
if err != nil {
return nil, errors.Trace(err)
}
return tpb, nil
}

func (m *Meta) schemaDiffKey(schemaVersion int64) []byte {
return []byte(fmt.Sprintf("%s:%d", mSchemaDiffPrefix, schemaVersion))
}
Expand Down
4 changes: 2 additions & 2 deletions plan/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,8 @@ import (
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/plan/statscache"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/util/types"
)

Expand Down Expand Up @@ -990,7 +990,7 @@ func (b *planBuilder) buildTableDual() LogicalPlan {
}

func (b *planBuilder) buildDataSource(tn *ast.TableName) LogicalPlan {
statisticTable := statscache.GetStatisticsTableCache(tn.TableInfo)
statisticTable := statistics.GetStatisticsTableCache(tn.TableInfo)
if b.err != nil {
return nil
}
Expand Down
2 changes: 1 addition & 1 deletion plan/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ import (
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/plan/statistics"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/util/types"
)

Expand Down
Loading

0 comments on commit ee7df05

Please sign in to comment.