Skip to content

Commit

Permalink
*: improve column size calculation of statistics (#11091)
Browse files Browse the repository at this point in the history
  • Loading branch information
eurekaka authored and zz-jason committed Jul 15, 2019
1 parent a090e6b commit 9b82445
Show file tree
Hide file tree
Showing 11 changed files with 113 additions and 67 deletions.
2 changes: 1 addition & 1 deletion executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -1002,7 +1002,7 @@ func (e *AnalyzeFastExec) buildColumnStats(ID int64, collector *statistics.Sampl
collector.NullCount++
continue
}
bytes, err := tablecodec.EncodeValue(e.ctx.GetSessionVars().StmtCtx, sample.Value)
bytes, err := tablecodec.EncodeValue(e.ctx.GetSessionVars().StmtCtx, nil, sample.Value)
if err != nil {
return nil, nil, err
}
Expand Down
2 changes: 1 addition & 1 deletion executor/show_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ func (e *ShowExec) appendTableForStatsHistograms(dbName, tblName, partitionName
if col.IsInvalid(nil, false) {
continue
}
e.histogramToRow(dbName, tblName, partitionName, col.Info.Name.O, 0, col.Histogram, col.AvgColSize(statsTbl.Count))
e.histogramToRow(dbName, tblName, partitionName, col.Info.Name.O, 0, col.Histogram, col.AvgColSize(statsTbl.Count, false))
}
for _, idx := range statsTbl.Indices {
e.histogramToRow(dbName, tblName, partitionName, idx.Info.Name.O, 1, idx.Histogram, 0)
Expand Down
8 changes: 4 additions & 4 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -143,22 +143,22 @@ func (s *testTableSuite) TestDataForTableStatsField(c *C) {
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
c.Assert(h.Update(is), IsNil)
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"))
testkit.Rows("3 18 54 6"))
tk.MustExec(`insert into t(c, d, e) values(4, 5, "f")`)
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
c.Assert(h.Update(is), IsNil)
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"))
testkit.Rows("4 18 72 8"))
tk.MustExec("delete from t where c >= 3")
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
c.Assert(h.Update(is), IsNil)
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"))
testkit.Rows("2 18 36 4"))
tk.MustExec("delete from t where c=3")
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
c.Assert(h.Update(is), IsNil)
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"))
testkit.Rows("2 18 36 4"))
}

func (s *testTableSuite) TestCharacterSetCollations(c *C) {
Expand Down
2 changes: 1 addition & 1 deletion planner/core/plan_to_pb.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,7 +152,7 @@ func SetPBColumnsDefaultValue(ctx sessionctx.Context, pbColumns []*tipb.ColumnIn
return err
}

pbColumns[i].DefaultVal, err = tablecodec.EncodeValue(ctx.GetSessionVars().StmtCtx, d)
pbColumns[i].DefaultVal, err = tablecodec.EncodeValue(sessVars.StmtCtx, nil, d)
if err != nil {
return err
}
Expand Down
4 changes: 2 additions & 2 deletions statistics/handle/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,7 +160,7 @@ func (s *testStatsSuite) TestDDLHistogram(c *C) {
tableInfo = tbl.Meta()
statsTbl = do.StatsHandle().GetTableStats(tableInfo)
c.Assert(statsTbl.Pseudo, IsFalse)
c.Check(statsTbl.Columns[tableInfo.Columns[5].ID].AvgColSize(statsTbl.Count), Equals, 3.0)
c.Check(statsTbl.Columns[tableInfo.Columns[5].ID].AvgColSize(statsTbl.Count, false), Equals, 3.0)

testKit.MustExec("create index i on t(c2, c1)")
testKit.MustExec("analyze table t")
Expand Down Expand Up @@ -212,6 +212,6 @@ PARTITION BY RANGE ( a ) (
for _, def := range pi.Definitions {
statsTbl := h.GetPartitionStats(tableInfo, def.ID)
c.Assert(statsTbl.Pseudo, IsFalse)
c.Check(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count), Equals, 3.0)
c.Check(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count, false), Equals, 3.0)
}
}
16 changes: 8 additions & 8 deletions statistics/handle/handle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -208,19 +208,19 @@ func (s *testStatsSuite) TestAvgColLen(c *C) {
c.Assert(err, IsNil)
tableInfo := tbl.Meta()
statsTbl := do.StatsHandle().GetTableStats(tableInfo)
c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.Count), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.Count, false), Equals, 1.0)

// The size of varchar type is LEN + BYTE, here is 1 + 7 = 8
c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.Count), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count), Equals, 4.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count), Equals, 16.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0)
testKit.MustExec("insert into t values(132, '123456789112', 1232.3, '2018-03-07 19:17:29')")
testKit.MustExec("analyze table t")
statsTbl = do.StatsHandle().GetTableStats(tableInfo)
c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.Count), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.Count), Equals, 10.5)
c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count), Equals, 4.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count), Equals, 16.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.Count, false), Equals, 1.5)
c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.Count, false), Equals, 10.5)
c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0)
}

func (s *testStatsSuite) TestDurationToTS(c *C) {
Expand Down
33 changes: 17 additions & 16 deletions statistics/handle/update_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,8 +160,8 @@ func (s *testStatsSuite) TestSingleSessionInsert(c *C) {
rs := testKit.MustQuery("select modify_count from mysql.stats_meta")
rs.Check(testkit.Rows("40", "70"))

rs = testKit.MustQuery("select tot_col_size from mysql.stats_histograms")
rs.Check(testkit.Rows("0", "0", "10", "10"))
rs = testKit.MustQuery("select tot_col_size from mysql.stats_histograms").Sort()
rs.Check(testkit.Rows("0", "0", "20", "20"))

// test dump delta only when `modify count / count` is greater than the ratio.
originValue := handle.DumpStatsDeltaRatio
Expand Down Expand Up @@ -343,7 +343,7 @@ func (s *testStatsSuite) TestUpdatePartition(c *C) {
statsTbl := h.GetPartitionStats(tableInfo, def.ID)
c.Assert(statsTbl.ModifyCount, Equals, int64(1))
c.Assert(statsTbl.Count, Equals, int64(1))
c.Assert(statsTbl.Columns[bColID].TotColSize, Equals, int64(1))
c.Assert(statsTbl.Columns[bColID].TotColSize, Equals, int64(2))
}

testKit.MustExec(`update t set a = a + 1, b = "aa"`)
Expand All @@ -353,7 +353,7 @@ func (s *testStatsSuite) TestUpdatePartition(c *C) {
statsTbl := h.GetPartitionStats(tableInfo, def.ID)
c.Assert(statsTbl.ModifyCount, Equals, int64(2))
c.Assert(statsTbl.Count, Equals, int64(1))
c.Assert(statsTbl.Columns[bColID].TotColSize, Equals, int64(2))
c.Assert(statsTbl.Columns[bColID].TotColSize, Equals, int64(3))
}

testKit.MustExec("delete from t")
Expand Down Expand Up @@ -442,7 +442,7 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) {
c.Assert(stats.ModifyCount, Equals, int64(1))
for _, item := range stats.Columns {
// TotColSize = 6, because the table has not been analyzed, and insert statement will add 3(length of 'eee') to TotColSize.
c.Assert(item.TotColSize, Equals, int64(14))
c.Assert(item.TotColSize, Equals, int64(15))
break
}

Expand Down Expand Up @@ -1307,7 +1307,7 @@ func (s *testStatsSuite) TestIndexQueryFeedback(c *C) {
},
{
sql: "select * from t use index(idx_ac) where a = 1 and c < 21",
hist: "column:3 ndv:20 totColSize:20\n" +
hist: "column:3 ndv:20 totColSize:40\n" +
"num: 13 lower_bound: -9223372036854775808 upper_bound: 6 repeats: 0\n" +
"num: 13 lower_bound: 7 upper_bound: 13 repeats: 0\n" +
"num: 12 lower_bound: 14 upper_bound: 21 repeats: 0",
Expand All @@ -1318,7 +1318,7 @@ func (s *testStatsSuite) TestIndexQueryFeedback(c *C) {
},
{
sql: "select * from t use index(idx_ad) where a = 1 and d < 21",
hist: "column:4 ndv:20 totColSize:160\n" +
hist: "column:4 ndv:20 totColSize:320\n" +
"num: 13 lower_bound: -10000000000000 upper_bound: 6 repeats: 0\n" +
"num: 12 lower_bound: 7 upper_bound: 13 repeats: 0\n" +
"num: 10 lower_bound: 14 upper_bound: 21 repeats: 0",
Expand All @@ -1329,7 +1329,7 @@ func (s *testStatsSuite) TestIndexQueryFeedback(c *C) {
},
{
sql: "select * from t use index(idx_ae) where a = 1 and e < 21",
hist: "column:5 ndv:20 totColSize:160\n" +
hist: "column:5 ndv:20 totColSize:320\n" +
"num: 13 lower_bound: -100000000000000000000000 upper_bound: 6 repeats: 0\n" +
"num: 12 lower_bound: 7 upper_bound: 13 repeats: 0\n" +
"num: 10 lower_bound: 14 upper_bound: 21 repeats: 0",
Expand All @@ -1340,7 +1340,7 @@ func (s *testStatsSuite) TestIndexQueryFeedback(c *C) {
},
{
sql: "select * from t use index(idx_af) where a = 1 and f < 21",
hist: "column:6 ndv:20 totColSize:200\n" +
hist: "column:6 ndv:20 totColSize:400\n" +
"num: 13 lower_bound: -999999999999999.99 upper_bound: 6.00 repeats: 0\n" +
"num: 12 lower_bound: 7.00 upper_bound: 13.00 repeats: 0\n" +
"num: 10 lower_bound: 14.00 upper_bound: 21.00 repeats: 0",
Expand All @@ -1351,7 +1351,7 @@ func (s *testStatsSuite) TestIndexQueryFeedback(c *C) {
},
{
sql: "select * from t use index(idx_ag) where a = 1 and g < 21",
hist: "column:7 ndv:20 totColSize:98\n" +
hist: "column:7 ndv:20 totColSize:196\n" +
"num: 13 lower_bound: -838:59:59 upper_bound: 00:00:06 repeats: 0\n" +
"num: 11 lower_bound: 00:00:07 upper_bound: 00:00:13 repeats: 0\n" +
"num: 10 lower_bound: 00:00:14 upper_bound: 00:00:21 repeats: 0",
Expand All @@ -1362,7 +1362,7 @@ func (s *testStatsSuite) TestIndexQueryFeedback(c *C) {
},
{
sql: `select * from t use index(idx_ah) where a = 1 and h < "1000-01-21"`,
hist: "column:8 ndv:20 totColSize:180\n" +
hist: "column:8 ndv:20 totColSize:360\n" +
"num: 13 lower_bound: 1000-01-01 upper_bound: 1000-01-07 repeats: 0\n" +
"num: 11 lower_bound: 1000-01-08 upper_bound: 1000-01-14 repeats: 0\n" +
"num: 10 lower_bound: 1000-01-15 upper_bound: 1000-01-21 repeats: 0",
Expand Down Expand Up @@ -1504,7 +1504,7 @@ func (s *testStatsSuite) TestFeedbackRanges(c *C) {
},
{
sql: "select * from t use index(idx) where a = 1 and (b <= 50 or (b > 130 and b < 140))",
hist: "column:2 ndv:20 totColSize:20\n" +
hist: "column:2 ndv:20 totColSize:30\n" +
"num: 7 lower_bound: -128 upper_bound: 6 repeats: 0\n" +
"num: 7 lower_bound: 7 upper_bound: 13 repeats: 1\n" +
"num: 6 lower_bound: 14 upper_bound: 19 repeats: 1",
Expand Down Expand Up @@ -1561,38 +1561,39 @@ func (s *testStatsSuite) TestUnsignedFeedbackRanges(c *C) {
}{
{
sql: "select * from t where a <= 50",
hist: "column:1 ndv:30 totColSize:0\n" +
hist: "column:1 ndv:30 totColSize:10\n" +
"num: 8 lower_bound: 0 upper_bound: 7 repeats: 0\n" +
"num: 8 lower_bound: 8 upper_bound: 15 repeats: 0\n" +
"num: 14 lower_bound: 16 upper_bound: 50 repeats: 0",
tblName: "t",
},
{
sql: "select count(*) from t",
hist: "column:1 ndv:30 totColSize:0\n" +
hist: "column:1 ndv:30 totColSize:10\n" +
"num: 8 lower_bound: 0 upper_bound: 7 repeats: 0\n" +
"num: 8 lower_bound: 8 upper_bound: 15 repeats: 0\n" +
"num: 14 lower_bound: 16 upper_bound: 255 repeats: 0",
tblName: "t",
},
{
sql: "select * from t1 where a <= 50",
hist: "column:1 ndv:30 totColSize:0\n" +
hist: "column:1 ndv:30 totColSize:10\n" +
"num: 8 lower_bound: 0 upper_bound: 7 repeats: 0\n" +
"num: 8 lower_bound: 8 upper_bound: 15 repeats: 0\n" +
"num: 14 lower_bound: 16 upper_bound: 50 repeats: 0",
tblName: "t1",
},
{
sql: "select count(*) from t1",
hist: "column:1 ndv:30 totColSize:0\n" +
hist: "column:1 ndv:30 totColSize:10\n" +
"num: 8 lower_bound: 0 upper_bound: 7 repeats: 0\n" +
"num: 8 lower_bound: 8 upper_bound: 15 repeats: 0\n" +
"num: 14 lower_bound: 16 upper_bound: 18446744073709551615 repeats: 0",
tblName: "t1",
},
}
is := s.do.InfoSchema()
c.Assert(h.Update(is), IsNil)
for i, t := range tests {
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr(t.tblName))
c.Assert(err, IsNil)
Expand Down
37 changes: 23 additions & 14 deletions statistics/histogram.go
Original file line number Diff line number Diff line change
Expand Up @@ -108,25 +108,34 @@ func (hg *Histogram) GetUpper(idx int) *types.Datum {
return &d
}

// AvgColSize is the average column size of the histogram.
func (c *Column) AvgColSize(count int64) float64 {
// AvgColSize is the average column size of the histogram. These sizes are derived from function `encode`
// and `Datum::ConvertTo`, so we need to update them if those 2 functions are changed.
func (c *Column) AvgColSize(count int64, isKey bool) float64 {
if count == 0 {
return 0
}
switch c.Histogram.Tp.Tp {
case mysql.TypeFloat:
return 4
case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong,
mysql.TypeDouble, mysql.TypeYear:
// Note that, if the handle column is encoded as value, instead of key, i.e,
// when the handle column is in a unique index, the real column size may be
// smaller than 8 because it is encoded using `EncodeVarint`. Since we don't
// know the exact value size now, use 8 as approximation.
if c.IsHandle {
return 8
case mysql.TypeDuration, mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp:
return 16
case mysql.TypeNewDecimal:
return types.MyDecimalStructSize
default:
// Keep two decimal place.
return math.Round(float64(c.TotColSize)/float64(count)*100) / 100
}
histCount := c.TotalRowCount()
notNullRatio := 1.0
if histCount > 0 {
notNullRatio = 1.0 - float64(c.NullCount)/histCount
}
switch c.Histogram.Tp.Tp {
case mysql.TypeFloat, mysql.TypeDouble, mysql.TypeDuration, mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp:
return 8 * notNullRatio
case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeYear, mysql.TypeEnum, mysql.TypeBit, mysql.TypeSet:
if isKey {
return 8 * notNullRatio
}
}
// Keep two decimal place.
return math.Round(float64(c.TotColSize)/float64(count)*100) / 100
}

// AppendBucket appends a bucket into `hg`.
Expand Down
42 changes: 29 additions & 13 deletions table/tables/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -314,7 +314,9 @@ func (t *tableCommon) UpdateRecord(ctx sessionctx.Context, h int64, oldData, new
}

key := t.RecordKey(h)
value, err := tablecodec.EncodeRow(ctx.GetSessionVars().StmtCtx, row, colIDs, nil, nil)
sessVars := ctx.GetSessionVars()
sc := sessVars.StmtCtx
value, err := tablecodec.EncodeRow(sc, row, colIDs, nil, nil)
if err != nil {
return err
}
Expand All @@ -338,13 +340,21 @@ func (t *tableCommon) UpdateRecord(ctx sessionctx.Context, h int64, oldData, new
}
}
colSize := make(map[int64]int64)
encodedCol := make([]byte, 0, 16)
for id, col := range t.Cols() {
val := int64(len(newData[id].GetBytes()) - len(oldData[id].GetBytes()))
if val != 0 {
colSize[col.ID] = val
encodedCol, err = tablecodec.EncodeValue(sc, encodedCol[:0], newData[id])
if err != nil {
continue
}
newLen := len(encodedCol) - 1
encodedCol, err = tablecodec.EncodeValue(sc, encodedCol[:0], oldData[id])
if err != nil {
continue
}
oldLen := len(encodedCol) - 1
colSize[col.ID] = int64(newLen - oldLen)
}
ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.physicalTableID, 0, 1, colSize)
sessVars.TxnCtx.UpdateDeltaForTable(t.physicalTableID, 0, 1, colSize)
return nil
}

Expand Down Expand Up @@ -504,7 +514,8 @@ func (t *tableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ..
writeBufs := sessVars.GetWriteStmtBufs()
adjustRowValuesBuf(writeBufs, len(row))
key := t.RecordKey(recordID)
writeBufs.RowValBuf, err = tablecodec.EncodeRow(ctx.GetSessionVars().StmtCtx, row, colIDs, writeBufs.RowValBuf, writeBufs.AddRowValues)
sc := sessVars.StmtCtx
writeBufs.RowValBuf, err = tablecodec.EncodeRow(sc, row, colIDs, writeBufs.RowValBuf, writeBufs.AddRowValues)
if err != nil {
return 0, err
}
Expand Down Expand Up @@ -532,13 +543,15 @@ func (t *tableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ..
return 0, err
}
}
sessVars.StmtCtx.AddAffectedRows(1)
sc.AddAffectedRows(1)
colSize := make(map[int64]int64)
encodedCol := make([]byte, 0, 16)
for id, col := range t.Cols() {
val := int64(len(r[id].GetBytes()))
if val != 0 {
colSize[col.ID] = val
encodedCol, err = tablecodec.EncodeValue(sc, encodedCol[:0], r[id])
if err != nil {
continue
}
colSize[col.ID] = int64(len(encodedCol) - 1)
}
sessVars.TxnCtx.UpdateDeltaForTable(t.physicalTableID, 1, 1, colSize)
return recordID, nil
Expand Down Expand Up @@ -714,11 +727,14 @@ func (t *tableCommon) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Da
err = t.addDeleteBinlog(ctx, binlogRow, colIDs)
}
colSize := make(map[int64]int64)
encodedCol := make([]byte, 0, 16)
sc := ctx.GetSessionVars().StmtCtx
for id, col := range t.Cols() {
val := -int64(len(r[id].GetBytes()))
if val != 0 {
colSize[col.ID] = val
encodedCol, err = tablecodec.EncodeValue(sc, encodedCol[:0], r[id])
if err != nil {
continue
}
colSize[col.ID] = -int64(len(encodedCol) - 1)
}
ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.physicalTableID, -1, 1, colSize)
return err
Expand Down
7 changes: 3 additions & 4 deletions tablecodec/tablecodec.go
Original file line number Diff line number Diff line change
Expand Up @@ -209,14 +209,13 @@ func DecodeRowKey(key kv.Key) (int64, error) {
}

// EncodeValue encodes a go value to bytes.
func EncodeValue(sc *stmtctx.StatementContext, raw types.Datum) ([]byte, error) {
func EncodeValue(sc *stmtctx.StatementContext, b []byte, raw types.Datum) ([]byte, error) {
var v types.Datum
err := flatten(sc, raw, &v)
if err != nil {
return nil, errors.Trace(err)
return nil, err
}
b, err := codec.EncodeValue(sc, nil, v)
return b, errors.Trace(err)
return codec.EncodeValue(sc, b, v)
}

// EncodeRow encode row data and column ids into a slice of byte.
Expand Down
Loading

0 comments on commit 9b82445

Please sign in to comment.