Skip to content

Commit

Permalink
statistics: refactor stats package.
Browse files Browse the repository at this point in the history
  • Loading branch information
hanfei1991 committed Mar 22, 2017
1 parent b4f985f commit 6fefdd7
Show file tree
Hide file tree
Showing 14 changed files with 233 additions and 385 deletions.
30 changes: 29 additions & 1 deletion bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,14 +116,38 @@ 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)
);`

// CreateStatsColsTable stores the statistics of table columns.
CreateStatsColsTable = `CREATE TABLE if not exists mysql.stats_columns (
table_id bigint(64) NOT NULL,
col_id bigint(64),
index_id bigint(64),
distinct_count bigint(64) NOT NULL,
distinct_ratio double(64) NOT NULL DEFAULT 0,
use_count_to_estimate tinyint(2) NOT NULL DEFAULT 0,
version bigint(64) unsigned NOT NULL DEFAULT 0,
index tbl(table_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,
col_id bigint(64),
index_id bigint(64),
bucket_id bigint(64) NOT NULL,
count bigint(64) NOT NULL,
repeats bigint(64) NOT NULL,
value blob NOT NULL,
index tbl(table_id, col_id, index_id, bucket_id)
);`
)

// Bootstrap initiates system DB for a store.
Expand Down Expand Up @@ -319,6 +343,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
6 changes: 3 additions & 3 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 @@ -424,7 +424,7 @@ func (do *Domain) loadTableStats() error {
// 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)
do.statsHandle = statistics.NewHandle(ctx)
err := do.loadTableStats()
if err != nil {
return errors.Trace(err)
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 := "575"
result.Check(testkit.Rows(columnCountOfAllInformationSchemaTables))

tk.MustExec("drop table if exists t1")
Expand Down
33 changes: 9 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,14 @@ 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()
is := GetInfoSchema(e.ctx)
table, ok := is.TableByID(e.tblInfo.ID)
if !ok {
return errors.Errorf("Can't find table id %d", e.tblInfo.ID)
}
statBuilder := &statistics.Builder{
Sc: e.ctx.GetSessionVars().StmtCtx,
TblInfo: e.tblInfo,
StartTS: int64(txn.StartTS()),
Count: count,
NumBuckets: defaultBucketCount,
ColumnSamples: columnSamples,
Expand All @@ -112,28 +111,14 @@ func (e *AnalyzeExec) buildStatisticsAndSaveToKV(count int64, columnSamples [][]
IdxOffsets: e.idxOffsets,
PkRecords: pkRS,
PkOffset: e.pkOffset,
Table: table,
}
t, err := statBuilder.NewTable()
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 @@ -927,7 +927,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
146 changes: 0 additions & 146 deletions plan/statistics/statistics.pb.go

This file was deleted.

Loading

0 comments on commit 6fefdd7

Please sign in to comment.