Skip to content

Commit

Permalink
add some test.
Browse files Browse the repository at this point in the history
  • Loading branch information
hanfei1991 committed Mar 22, 2017
1 parent 67e1a81 commit 4d46d80
Show file tree
Hide file tree
Showing 4 changed files with 89 additions and 31 deletions.
10 changes: 10 additions & 0 deletions bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -177,6 +177,7 @@ const (
version2 = 2
version3 = 3
version4 = 4
version5 = 5
)

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

if ver < version5 {
upgradeToVer5(s)
}

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

Expand Down Expand Up @@ -303,6 +308,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
57 changes: 28 additions & 29 deletions statistics/statistics.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,44 +66,37 @@ type Column struct {
Repeats []int64
}

func (c *Column) SaveToStorage(ctx context.Context, table table.Table, isIndex bool) error {
var column_name string
var col_index int
func (c *Column) saveToStorage(ctx context.Context, table table.Table, isIndex bool) error {
var colName string
if isIndex {
column_name = "index_id"
col_index = 2
colName = "index_id"
} else {
column_name = "col_id"
col_index = 1
colName = "col_id"
}
tableID := table.Meta().ID
insertSQL := fmt.Sprintf("insert into mysql.stats_columns (table_id, %s, distinct_count) values (%d, %d, %d)", column_name, tableID, c.ID, c.NDV)
insertSQL := fmt.Sprintf("insert into mysql.stats_columns (table_id, %s, distinct_count) values (%d, %d, %d)", colName, tableID, c.ID, c.NDV)
_, err := ctx.(sqlexec.SQLExecutor).Execute(insertSQL)
if err != nil {
return errors.Trace(err)
}
for i := 0; i < len(c.Numbers); i++ {
row := make([]types.Datum, 7)
// set table id
row[0] = types.NewDatum(tableID)
// set column or index id
row[col_index] = types.NewDatum(c.ID)
// set bucket it
row[3] = types.NewDatum(i)
// set count
var count int64
if i == 0 {
row[4] = types.NewDatum(c.Numbers[i])
count = c.Numbers[i]
} else {
row[4] = types.NewDatum(c.Numbers[i] - c.Numbers[i-1])
count = c.Numbers[i] - c.Numbers[i-1]
}
// set repeats
row[5] = types.NewDatum(c.Repeats[i])
// set value
row[6], err = c.Values[i].ConvertTo(ctx.GetSessionVars().StmtCtx, types.NewFieldType(mysql.TypeVarString))
val, err := c.Values[i].ConvertTo(ctx.GetSessionVars().StmtCtx, types.NewFieldType(mysql.TypeBlob))
if err != nil {
return errors.Trace(err)
}
_, err = table.AddRecord(ctx, row)
if isIndex {
insertSQL = fmt.Sprintf("insert into mysql.stats_buckets values(%d, NULL, %d, %d, %d, %d, X'%X')", tableID, c.ID, i, count, c.Repeats[i], val.GetBytes())
} else {
insertSQL = fmt.Sprintf("insert into mysql.stats_buckets values(%d, %d, NULL, %d, %d, %d, X'%X')", tableID, c.ID, i, count, c.Repeats[i], val.GetBytes())
}
_, err = ctx.(sqlexec.SQLExecutor).Execute(insertSQL)
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -265,6 +258,7 @@ type Table struct {
Table table.Table
}

// SaveToStorage saves stats table to storage.
func (t *Table) SaveToStorage(ctx context.Context) error {
_, err := ctx.(sqlexec.SQLExecutor).Execute("begin")
txn := ctx.Txn()
Expand All @@ -291,29 +285,33 @@ func (t *Table) SaveToStorage(ctx context.Context) error {
return errors.Trace(err)
}
for _, col := range t.Columns {
col.SaveToStorage(ctx, t.Table, false)
col.saveToStorage(ctx, t.Table, false)
}
for _, idx := range t.Indices {
idx.SaveToStorage(ctx, t.Table, true)
idx.saveToStorage(ctx, t.Table, true)
}
_, err = ctx.(sqlexec.SQLExecutor).Execute("commit")
return errors.Trace(err)
}

func colStatsFromStorage(ctx context.Context, tableId int64, colID int64, tp *types.FieldType, distinct int64, isIndex bool) (*Column, error) {
func colStatsFromStorage(ctx context.Context, tableID int64, colID int64, tp *types.FieldType, distinct int64, isIndex bool) (*Column, error) {
var column_name string
if isIndex {
column_name = "index_id"
} else {
column_name = "col_id"
}
selSQL := fmt.Sprintf("select bucket_id, count, repeats, value from stats_buckets where table_id = %d and %s = %d", tableId, column_name, colID)
selSQL := fmt.Sprintf("select bucket_id, count, repeats, value from mysql.stats_buckets where table_id = %d and %s = %d", tableID, column_name, colID)
rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, selSQL)
if err != nil {
return nil, errors.Trace(err)
}
bucketSize := len(rows)
if bucketSize == 0 {
return nil, nil
}
colStats := &Column{
ID: colID,
NDV: distinct,
Numbers: make([]int64, bucketSize),
Repeats: make([]int64, bucketSize),
Expand Down Expand Up @@ -342,18 +340,19 @@ func colStatsFromStorage(ctx context.Context, tableId int64, colID int64, tp *ty
return colStats, nil
}

func tableStatsFromStorage(ctx context.Context, info *model.TableInfo, count int64) (*Table, error) {
// TableStatsFromStorage load table stats info from storage.
func TableStatsFromStorage(ctx context.Context, info *model.TableInfo, count int64) (*Table, error) {
table := &Table{
Info: info,
Count: count,
}
selSQL := fmt.Sprintf("select * from stats_columns where table_id = %d", info.ID)
selSQL := fmt.Sprintf("select * from mysql.stats_columns where table_id = %d", info.ID)
rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, selSQL)
if err != nil {
return nil, errors.Trace(err)
}
for _, row := range rows {
distinct := row.Data[4].GetInt64()
distinct := row.Data[3].GetInt64()
if row.Data[1].IsNull() {
// process index
colID := row.Data[2].GetInt64()
Expand Down
2 changes: 1 addition & 1 deletion statistics/statscache.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ func (h *Handle) Update(m *meta.Meta, is infoschema.InfoSchema) error {
continue
}
tableInfo := table.Meta()
tbl, err := tableStatsFromStorage(h.ctx, tableInfo, count)
tbl, err := TableStatsFromStorage(h.ctx, tableInfo, count)
// Error is not nil may mean that there are some ddl changes on this table, so the origin
// statistics can not be used any more, we give it a nil one.
if err != nil {
Expand Down
51 changes: 50 additions & 1 deletion statistics/statscache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,56 @@ func (s *testStatsCacheSuite) TestStatsCache(c *C) {
do.StatsHandle().Clear()
do.StatsHandle().Update(m, is)
statsTbl = statistics.GetStatisticsTableCache(tableInfo)
c.Assert(statsTbl.Pseudo, IsTrue)
c.Assert(statsTbl.Pseudo, IsFalse)
}

func compareTwoColumnsStatsSlice(cols0 []*statistics.Column, cols1 []*statistics.Column, c *C) {
c.Assert(len(cols0), Equals, len(cols1))
for _, col0 := range cols0 {
find := false
for _, col1 := range cols1 {
if col0.ID == col1.ID {
c.Assert(col0.NDV, Equals, col1.NDV)
c.Assert(len(col0.Numbers), Equals, len(col1.Numbers))
for j := 0; j < len(col0.Numbers); j++ {
c.Assert(col0.Numbers[j], Equals, col1.Numbers[j])
c.Assert(col0.Repeats[j], Equals, col1.Repeats[j])
c.Assert(col0.Values[j], DeepEquals, col1.Values[j])
}
find = true
break
}
}
c.Assert(find, IsTrue)
}
}

func (s *testStatsCacheSuite) TestStatsStoreAndLoad(c *C) {
store, do, err := newStoreWithBootstrap()
c.Assert(err, IsNil)
defer store.Close()
testKit := testkit.NewTestKit(c, store)
testKit.MustExec("use test")
testKit.MustExec("create table t (c1 int, c2 int)")
for i := 0; i < 100000; i++ {
testKit.MustExec("insert into t values (?, ?)", i, i+1)
}
testKit.MustExec("create index idx_t on t(c2)")
is := do.InfoSchema()
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
c.Assert(err, IsNil)
tableInfo := tbl.Meta()

testKit.MustExec("analyze table t")
statsTbl1 := statistics.GetStatisticsTableCache(tableInfo)

se, err := tidb.CreateSession(store)
c.Assert(err, IsNil)
statsTbl2, err := statistics.TableStatsFromStorage(se, tableInfo, 100000)
c.Assert(err, IsNil)

compareTwoColumnsStatsSlice(statsTbl1.Columns, statsTbl2.Columns, c)
compareTwoColumnsStatsSlice(statsTbl1.Indices, statsTbl2.Indices, c)
}

func newStoreWithBootstrap() (kv.Storage, *domain.Domain, error) {
Expand Down

0 comments on commit 4d46d80

Please sign in to comment.