From 98c4e9f3c6cfde03a1d95f0bad99b3baa55613b3 Mon Sep 17 00:00:00 2001 From: flipped <19flipped@gmail.com> Date: Wed, 28 Mar 2018 14:53:56 +0800 Subject: [PATCH 1/8] stats: dynamically update the average column size --- executor/executor.go | 2 +- executor/show_stats.go | 2 +- executor/write.go | 15 +++++++-- session/session.go | 2 +- sessionctx/variable/session.go | 15 +++++++-- statistics/ddl_test.go | 2 +- statistics/dump.go | 2 +- statistics/handle_test.go | 38 +++++++++++++++-------- statistics/histogram.go | 13 ++++---- statistics/table.go | 29 +++++++++++++---- statistics/update.go | 57 +++++++++++++++++++++++++++++----- statistics/update_test.go | 23 +++++++++++--- table/tables/tables.go | 8 ++++- 13 files changed, 159 insertions(+), 49 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index 59a1157790b27..9c714ab12f040 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -472,7 +472,7 @@ func (e *SelectLockExec) Open(ctx context.Context) error { txnCtx.ForUpdate = true for id := range e.Schema().TblID2Handle { // This operation is only for schema validator check. - txnCtx.UpdateDeltaForTable(id, 0, 0) + txnCtx.UpdateDeltaForTable(id, 0, 0, nil) } return nil } diff --git a/executor/show_stats.go b/executor/show_stats.go index bce7fb50b8f72..16eb7872fafde 100644 --- a/executor/show_stats.go +++ b/executor/show_stats.go @@ -55,7 +55,7 @@ func (e *ShowExec) fetchShowStatsHistogram() error { statsTbl := h.GetTableStats(tbl.ID) if !statsTbl.Pseudo { for _, col := range statsTbl.Columns { - e.histogramToRow(db.Name.O, tbl.Name.O, col.Info.Name.O, 0, col.Histogram, col.AvgColSize()) + e.histogramToRow(db.Name.O, tbl.Name.O, col.Info.Name.O, 0, col.Histogram, col.AvgColSize(statsTbl.Count)) } for _, idx := range statsTbl.Indices { e.histogramToRow(db.Name.O, tbl.Name.O, idx.Info.Name.O, 1, idx.Histogram, 0) diff --git a/executor/write.go b/executor/write.go index 2cde566b5c1df..ee33402805424 100644 --- a/executor/write.go +++ b/executor/write.go @@ -158,8 +158,11 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu } else { sc.AddAffectedRows(1) } - - ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, 0, 1) + colSize := make(map[int64]int64) + for id, col := range t.WritableCols() { + colSize[col.ID] = int64(len(newData[id].GetBytes()) - len(oldData[id].GetBytes())) + } + ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, 0, 1, &colSize) return true, nil } @@ -385,7 +388,13 @@ func (e *DeleteExec) removeRow(ctx sessionctx.Context, t table.Table, h int64, d } ctx.StmtAddDirtyTableOP(DirtyTableDeleteRow, t.Meta().ID, h, nil) ctx.GetSessionVars().StmtCtx.AddAffectedRows(1) - ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, -1, 1) + colSize := make(map[int64]int64) + for id, col := range t.WritableCols() { + if col.State == model.StatePublic { + colSize[col.ID] = -int64(len(data[id].GetBytes())) + } + } + ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, -1, 1, &colSize) return nil } diff --git a/session/session.go b/session/session.go index 1e0ab0c920380..002d39c0523ae 100644 --- a/session/session.go +++ b/session/session.go @@ -357,7 +357,7 @@ func (s *session) doCommitWithRetry(ctx context.Context) error { mapper := s.GetSessionVars().TxnCtx.TableDeltaMap if s.statsCollector != nil && mapper != nil { for id, item := range mapper { - s.statsCollector.Update(id, item.Delta, item.Count) + s.statsCollector.Update(id, item.Delta, item.Count, &item.ColSize) } } return nil diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index cfac03a7ab2c9..39f9210ccbfeb 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -101,13 +101,21 @@ type TransactionContext struct { } // UpdateDeltaForTable updates the delta info for some table. -func (tc *TransactionContext) UpdateDeltaForTable(tableID int64, delta int64, count int64) { +func (tc *TransactionContext) UpdateDeltaForTable(tableID int64, delta int64, count int64, colSize *map[int64]int64) { if tc.TableDeltaMap == nil { tc.TableDeltaMap = make(map[int64]TableDelta) } item := tc.TableDeltaMap[tableID] + if item.ColSize == nil { + item.ColSize = make(map[int64]int64) + } item.Delta += delta item.Count += count + if colSize != nil { + for key, val := range *colSize { + item.ColSize[key] += val + } + } tc.TableDeltaMap[tableID] = item } @@ -525,6 +533,7 @@ const ( // TableDelta stands for the changed count for one table. type TableDelta struct { - Delta int64 - Count int64 + Delta int64 + Count int64 + ColSize map[int64]int64 } diff --git a/statistics/ddl_test.go b/statistics/ddl_test.go index 46ee0f6ed12a1..2515eed3efb27 100644 --- a/statistics/ddl_test.go +++ b/statistics/ddl_test.go @@ -151,7 +151,7 @@ func (s *testStatsCacheSuite) TestDDLHistogram(c *C) { statsTbl = do.StatsHandle().GetTableStats(tableInfo.ID) c.Assert(statsTbl.Pseudo, IsFalse) sc = new(stmtctx.StatementContext) - c.Check(statsTbl.Columns[tableInfo.Columns[5].ID].AvgColSize(), Equals, 3.0) + c.Check(statsTbl.Columns[tableInfo.Columns[5].ID].AvgColSize(statsTbl.Count), Equals, 3.0) testKit.MustExec("create index i on t(c2, c1)") testKit.MustExec("analyze table t") diff --git a/statistics/dump.go b/statistics/dump.go index 16fa0c116c84e..19bd55ad38044 100644 --- a/statistics/dump.go +++ b/statistics/dump.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" - tipb "github.com/pingcap/tipb/go-tipb" + "github.com/pingcap/tipb/go-tipb" ) // JSONTable is used for dumping statistics. diff --git a/statistics/handle_test.go b/statistics/handle_test.go index 4dc0e387c5164..d05451c2be5fa 100644 --- a/statistics/handle_test.go +++ b/statistics/handle_test.go @@ -220,19 +220,19 @@ func (s *testStatsCacheSuite) TestAvgColLen(c *C) { c.Assert(err, IsNil) tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo.ID) - c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(), Equals, 8.0) + c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.Count), Equals, 8.0) // The size of varchar type is LEN + BYTE, here is 1 + 7 = 8 - c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(), Equals, 8.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(), Equals, 4.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(), Equals, 16.0) + 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) 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.ID) - c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(), Equals, 8.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(), Equals, 10.5) - c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(), Equals, 4.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(), Equals, 16.0) + 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) } func (s *testStatsCacheSuite) TestVersion(c *C) { @@ -319,14 +319,14 @@ func (s *testStatsCacheSuite) TestLoadHist(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") - testKit.MustExec("create table t (c1 int, c2 int)") + testKit.MustExec("create table t (c1 varchar(12), c2 char(12))") do := s.do h := do.StatsHandle() err := h.HandleDDLEvent(<-h.DDLEventCh()) c.Assert(err, IsNil) rowCount := 10 for i := 0; i < rowCount; i++ { - testKit.MustExec("insert into t values(1,2)") + testKit.MustExec("insert into t values('a','ddd')") } testKit.MustExec("analyze table t") is := do.InfoSchema() @@ -335,16 +335,28 @@ func (s *testStatsCacheSuite) TestLoadHist(c *C) { tableInfo := tbl.Meta() oldStatsTbl := h.GetTableStats(tableInfo.ID) for i := 0; i < rowCount; i++ { - testKit.MustExec("insert into t values(1,2)") + testKit.MustExec("insert into t values('bb','sdfga')") } h.DumpStatsDeltaToKV() h.Update(do.InfoSchema()) newStatsTbl := h.GetTableStats(tableInfo.ID) // The stats table is updated. c.Assert(oldStatsTbl == newStatsTbl, IsFalse) - // The histograms is not updated. + // Only the TotColSize of histograms is updated. for id, hist := range oldStatsTbl.Columns { - c.Assert(hist, Equals, newStatsTbl.Columns[id]) + c.Assert(hist.TotColSize, Less, newStatsTbl.Columns[id].TotColSize) + + temp := hist.TotColSize + hist.TotColSize = newStatsTbl.Columns[id].TotColSize + c.Assert(statistics.HistogramEqual(&hist.Histogram, &newStatsTbl.Columns[id].Histogram, false), IsTrue) + hist.TotColSize = temp + + c.Assert(hist.CMSketch, NotNil) + c.Assert(newStatsTbl.Columns[id].CMSketch, NotNil) + c.Assert(hist.CMSketch.Equal(newStatsTbl.Columns[id].CMSketch), IsTrue) + + c.Assert(hist.Count, Equals, newStatsTbl.Columns[id].Count) + c.Assert(hist.Info, Equals, newStatsTbl.Columns[id].Info) } // Add column c3, we only update c3. testKit.MustExec("alter table t add column c3 int") diff --git a/statistics/histogram.go b/statistics/histogram.go index 5102118c67d96..832d10e95e581 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -105,7 +105,10 @@ func (hg *Histogram) GetUpper(idx int) *types.Datum { } // AvgColSize is the average column size of the histogram. -func (c *Column) AvgColSize() float64 { +func (c *Column) AvgColSize(count int64) float64 { + if count == 0 { + return 0 + } switch c.Histogram.tp.Tp { case mysql.TypeFloat: return 4 @@ -117,10 +120,8 @@ func (c *Column) AvgColSize() float64 { case mysql.TypeNewDecimal: return types.MyDecimalStructSize default: - if c.Count == 0 { - return 0 - } - return float64(c.TotColSize) / float64(c.Count) + // Keep two decimal place. + return math.Round(float64(c.TotColSize)/float64(count)*100) / 100 } } @@ -292,7 +293,7 @@ func (hg *Histogram) toString(isIndex bool) string { if isIndex { strs = append(strs, fmt.Sprintf("index:%d ndv:%d", hg.ID, hg.NDV)) } else { - strs = append(strs, fmt.Sprintf("column:%d ndv:%d", hg.ID, hg.NDV)) + strs = append(strs, fmt.Sprintf("column:%d ndv:%d totColSize:%d", hg.ID, hg.NDV, hg.TotColSize)) } for i := 0; i < hg.Len(); i++ { upperVal, err := hg.GetUpper(i).ToString() diff --git a/statistics/table.go b/statistics/table.go index a12ea08c91054..4dd8769df5704 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -125,18 +125,30 @@ func (h *Handle) columnStatsFromStorage(row types.Row, table *Table, tableInfo * continue } isHandle := tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag) - needNotLoad := col == nil || (col.Len() == 0 && col.LastUpdateVersion < histVer) - if h.Lease > 0 && !isHandle && needNotLoad && !loadAll { + // If the stats will be analyzed automatically and this column is no handle and we don't specify to load all, + // and the column doesn't has buckets before and the version is newer than last update version, + // or it is the first time to load this column, then we load the column without buckets. + DontLoadBuckets := h.Lease > 0 && !isHandle && (col == nil || col.Len() == 0 && col.LastUpdateVersion < histVer) + if DontLoadBuckets && !loadAll { count, err := columnCountFromStorage(h.ctx, table.TableID, histID) if err != nil { return errors.Trace(err) } col = &Column{ - Histogram: Histogram{ID: histID, NDV: distinct, NullCount: nullCount, tp: &colInfo.FieldType, LastUpdateVersion: histVer, TotColSize: totColSize}, - Info: colInfo, - Count: count} + Histogram: Histogram{ + ID: histID, + NDV: distinct, + NullCount: nullCount, + tp: &colInfo.FieldType, + LastUpdateVersion: histVer, + TotColSize: totColSize, + }, + Info: colInfo, + Count: count, + } break } + // Otherwise, if the version is newer than last update version or it is the first time to load this column or we want to load all, then we load the column with the buckets of histogram. if col == nil || col.LastUpdateVersion < histVer || loadAll { hg, err := histogramFromStorage(h.ctx, tableInfo.ID, histID, &colInfo.FieldType, distinct, 0, histVer, nullCount, totColSize) if err != nil { @@ -146,7 +158,12 @@ func (h *Handle) columnStatsFromStorage(row types.Row, table *Table, tableInfo * if err != nil { return errors.Trace(err) } - col = &Column{Histogram: *hg, Info: colInfo, CMSketch: cms, Count: int64(hg.totalRowCount())} + col = &Column{ + Histogram: *hg, + Info: colInfo, + CMSketch: cms, + Count: int64(hg.totalRowCount()), + } } break } diff --git a/statistics/update.go b/statistics/update.go index 86a82fe52b79e..3af48e7ef589d 100644 --- a/statistics/update.go +++ b/statistics/update.go @@ -32,10 +32,18 @@ import ( type tableDeltaMap map[int64]variable.TableDelta -func (m tableDeltaMap) update(id int64, delta int64, count int64) { +func (m tableDeltaMap) update(id int64, delta int64, count int64, colSize *map[int64]int64) { item := m[id] item.Delta += delta item.Count += count + if item.ColSize == nil { + item.ColSize = make(map[int64]int64) + } + if colSize != nil { + for key, val := range *colSize { + item.ColSize[key] += val + } + } m[id] = item } @@ -43,7 +51,7 @@ func (h *Handle) merge(s *SessionStatsCollector) { s.Lock() defer s.Unlock() for id, item := range s.mapper { - h.globalMap.update(id, item.Delta, item.Count) + h.globalMap.update(id, item.Delta, item.Count, &item.ColSize) } h.feedback = mergeQueryFeedback(h.feedback, s.feedback) s.mapper = make(tableDeltaMap) @@ -70,10 +78,10 @@ func (s *SessionStatsCollector) Delete() { } // Update will updates the delta and count for one table id. -func (s *SessionStatsCollector) Update(id int64, delta int64, count int64) { +func (s *SessionStatsCollector) Update(id int64, delta int64, count int64, colSize *map[int64]int64) { s.Lock() defer s.Unlock() - s.mapper.update(id, delta, count) + s.mapper.update(id, delta, count, colSize) } func mergeQueryFeedback(lq []*QueryFeedback, rq []*QueryFeedback) []*QueryFeedback { @@ -154,11 +162,23 @@ func (h *Handle) DumpStatsDeltaToKV() error { } h.listHead.Unlock() for id, item := range h.globalMap { - updated, err := h.dumpTableStatDeltaToKV(id, item) + updatedCount, err := h.dumpTableStatCountToKV(id, item) + if err != nil { + return errors.Trace(err) + } + if updatedCount { + h.globalMap.update(id, -item.Delta, -item.Count, nil) + } + updatedColSize, err := h.dumpTableStatColSizeToKV(id, item) if err != nil { return errors.Trace(err) } - if updated { + if updatedColSize { + m := h.globalMap[id] + m.ColSize = nil + h.globalMap[id] = m + } + if updatedCount && updatedColSize { delete(h.globalMap, id) } } @@ -166,7 +186,7 @@ func (h *Handle) DumpStatsDeltaToKV() error { } // dumpTableStatDeltaToKV dumps a single delta with some table to KV and updates the version. -func (h *Handle) dumpTableStatDeltaToKV(id int64, delta variable.TableDelta) (bool, error) { +func (h *Handle) dumpTableStatCountToKV(id int64, delta variable.TableDelta) (bool, error) { if delta.Count == 0 { return true, nil } @@ -190,6 +210,29 @@ func (h *Handle) dumpTableStatDeltaToKV(id int64, delta variable.TableDelta) (bo return updated, errors.Trace(err) } +func (h *Handle) dumpTableStatColSizeToKV(id int64, delta variable.TableDelta) (bool, error) { + if delta.ColSize == nil { + return true, nil + } + ctx := context.TODO() + _, err := h.ctx.(sqlexec.SQLExecutor).Execute(ctx, "begin") + if err != nil { + return false, errors.Trace(err) + } + version := h.ctx.Txn().StartTS() + + for key, val := range delta.ColSize { + sql := fmt.Sprintf("update mysql.stats_histograms set version = %d, tot_col_size = tot_col_size + %d where hist_id = %d and table_id = %d", version, val, key, id) + _, err = h.ctx.(sqlexec.SQLExecutor).Execute(ctx, sql) + if err != nil { + return false, errors.Trace(err) + } + } + updated := h.ctx.GetSessionVars().StmtCtx.AffectedRows() > 0 + _, err = h.ctx.(sqlexec.SQLExecutor).Execute(ctx, "commit") + return updated, errors.Trace(err) +} + const ( // StatsOwnerKey is the stats owner path that is saved to etcd. StatsOwnerKey = "/tidb/stats/owner" diff --git a/statistics/update_test.go b/statistics/update_test.go index 8b5c401f506d4..1af20da220892 100644 --- a/statistics/update_test.go +++ b/statistics/update_test.go @@ -139,6 +139,9 @@ func (s *testStatsUpdateSuite) 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")) } func (s *testStatsUpdateSuite) TestRollback(c *C) { @@ -271,7 +274,7 @@ func (s *testStatsUpdateSuite) TestAutoUpdate(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") - testKit.MustExec("create table t (a int)") + testKit.MustExec("create table t (a varchar(20))") statistics.AutoAnalyzeMinCnt = 0 defer func() { @@ -290,7 +293,7 @@ func (s *testStatsUpdateSuite) TestAutoUpdate(c *C) { stats := h.GetTableStats(tableInfo.ID) c.Assert(stats.Count, Equals, int64(0)) - _, err = testKit.Exec("insert into t values (1)") + _, err = testKit.Exec("insert into t values ('ss')") c.Assert(err, IsNil) h.DumpStatsDeltaToKV() h.Update(is) @@ -300,8 +303,13 @@ func (s *testStatsUpdateSuite) TestAutoUpdate(c *C) { stats = h.GetTableStats(tableInfo.ID) c.Assert(stats.Count, Equals, int64(1)) c.Assert(stats.ModifyCount, Equals, int64(0)) + for _, item := range stats.Columns { + // TotColSize = 2(length of 'ss') + 1(size of len byte). + c.Assert(item.TotColSize, Equals, int64(3)) + break + } - _, err = testKit.Exec("insert into t values (1)") + _, err = testKit.Exec("insert into t values ('eee')") c.Assert(err, IsNil) h.DumpStatsDeltaToKV() h.Clear() @@ -316,6 +324,11 @@ func (s *testStatsUpdateSuite) TestAutoUpdate(c *C) { c.Assert(stats.Count, Equals, int64(2)) // Modify count is non-zero means that we do not analyze the table. 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(6)) + break + } _, err = testKit.Exec("create index idx on t(a)") c.Assert(err, IsNil) @@ -330,8 +343,8 @@ func (s *testStatsUpdateSuite) TestAutoUpdate(c *C) { c.Assert(stats.ModifyCount, Equals, int64(0)) hg, ok := stats.Indices[tableInfo.Indices[0].ID] c.Assert(ok, IsTrue) - c.Assert(hg.NDV, Equals, int64(1)) - c.Assert(hg.Len(), Equals, 1) + c.Assert(hg.NDV, Equals, int64(2)) + c.Assert(hg.Len(), Equals, 2) } func appendBucket(h *statistics.Histogram, l, r int64) { diff --git a/table/tables/tables.go b/table/tables/tables.go index 0a7c2e94fba71..6121615bcfb15 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -434,7 +434,13 @@ func (t *Table) AddRecord(ctx sessionctx.Context, r []types.Datum, skipHandleChe } } sessVars.StmtCtx.AddAffectedRows(1) - sessVars.TxnCtx.UpdateDeltaForTable(t.ID, 1, 1) + colSize := make(map[int64]int64) + for id, col := range t.WritableCols() { + if col.State == model.StatePublic { + colSize[col.ID] = int64(len(r[id].GetBytes())) + } + } + sessVars.TxnCtx.UpdateDeltaForTable(t.ID, 1, 1, &colSize) return recordID, nil } From 94b0c8614513ff5b8ef06cdcb77a4a36ce00e4f9 Mon Sep 17 00:00:00 2001 From: flipped <19flipped@gmail.com> Date: Wed, 28 Mar 2018 19:15:01 +0800 Subject: [PATCH 2/8] address comments --- executor/write.go | 6 ++++-- statistics/ddl.go | 10 +--------- statistics/handle_test.go | 3 --- statistics/sample.go | 2 +- statistics/table.go | 16 ++++++++++------ statistics/update.go | 30 +++++++++++++----------------- table/tables/tables.go | 2 +- types/datum.go | 7 ++++++- 8 files changed, 36 insertions(+), 40 deletions(-) diff --git a/executor/write.go b/executor/write.go index ee33402805424..eb7bc38d6eb4b 100644 --- a/executor/write.go +++ b/executor/write.go @@ -160,7 +160,9 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu } colSize := make(map[int64]int64) for id, col := range t.WritableCols() { - colSize[col.ID] = int64(len(newData[id].GetBytes()) - len(oldData[id].GetBytes())) + if col.State == model.StatePublic && newData[id].Size() != 0 { + colSize[col.ID] = int64(newData[id].Size() - oldData[id].Size()) + } } ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, 0, 1, &colSize) return true, nil @@ -391,7 +393,7 @@ func (e *DeleteExec) removeRow(ctx sessionctx.Context, t table.Table, h int64, d colSize := make(map[int64]int64) for id, col := range t.WritableCols() { if col.State == model.StatePublic { - colSize[col.ID] = -int64(len(data[id].GetBytes())) + colSize[col.ID] = -int64(data[id].Size()) } } ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, -1, 1, &colSize) diff --git a/statistics/ddl.go b/statistics/ddl.go index 145cb0b282906..b19c7add35b3e 100644 --- a/statistics/ddl.go +++ b/statistics/ddl.go @@ -118,16 +118,8 @@ func (h *Handle) insertColStats2KV(tableID int64, colInfo *model.ColumnInfo) err return errors.Trace(err) } } else { - var totColSize int64 - switch colInfo.Tp { - case mysql.TypeFloat, mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, - mysql.TypeDouble, mysql.TypeYear, mysql.TypeDuration, mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDecimal: - totColSize = 0 - default: - totColSize = int64(len(value.GetBytes())) - } // If this stats exists, we insert histogram meta first, the distinct_count will always be one. - _, err = exec.Execute(ctx, fmt.Sprintf("insert into mysql.stats_histograms (version, table_id, is_index, hist_id, distinct_count, tot_col_size) values (%d, %d, 0, %d, 1, %d)", h.ctx.Txn().StartTS(), tableID, colInfo.ID, totColSize*count)) + _, err = exec.Execute(ctx, fmt.Sprintf("insert into mysql.stats_histograms (version, table_id, is_index, hist_id, distinct_count, tot_col_size) values (%d, %d, 0, %d, 1, %d)", h.ctx.Txn().StartTS(), tableID, colInfo.ID, int64(value.Size())*count)) if err != nil { return errors.Trace(err) } diff --git a/statistics/handle_test.go b/statistics/handle_test.go index d05451c2be5fa..4a403e2288580 100644 --- a/statistics/handle_test.go +++ b/statistics/handle_test.go @@ -351,10 +351,7 @@ func (s *testStatsCacheSuite) TestLoadHist(c *C) { c.Assert(statistics.HistogramEqual(&hist.Histogram, &newStatsTbl.Columns[id].Histogram, false), IsTrue) hist.TotColSize = temp - c.Assert(hist.CMSketch, NotNil) - c.Assert(newStatsTbl.Columns[id].CMSketch, NotNil) c.Assert(hist.CMSketch.Equal(newStatsTbl.Columns[id].CMSketch), IsTrue) - c.Assert(hist.Count, Equals, newStatsTbl.Columns[id].Count) c.Assert(hist.Info, Equals, newStatsTbl.Columns[id].Info) } diff --git a/statistics/sample.go b/statistics/sample.go index 33065fa35f986..bf4bff1b0e5c8 100644 --- a/statistics/sample.go +++ b/statistics/sample.go @@ -104,7 +104,7 @@ func (c *SampleCollector) collect(sc *stmtctx.StatementContext, d types.Datum) e c.CMSketch.InsertBytes(d.GetBytes()) } // Minus one is to remove the flag byte. - c.TotalSize += int64(len(d.GetBytes()) - 1) + c.TotalSize += int64(d.Size() - 1) } c.seenValues++ // The following code use types.CopyDatum(d) because d may have a deep reference diff --git a/statistics/table.go b/statistics/table.go index 4dd8769df5704..73f92a4770e01 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -125,11 +125,16 @@ func (h *Handle) columnStatsFromStorage(row types.Row, table *Table, tableInfo * continue } isHandle := tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag) - // If the stats will be analyzed automatically and this column is no handle and we don't specify to load all, - // and the column doesn't has buckets before and the version is newer than last update version, - // or it is the first time to load this column, then we load the column without buckets. - DontLoadBuckets := h.Lease > 0 && !isHandle && (col == nil || col.Len() == 0 && col.LastUpdateVersion < histVer) - if DontLoadBuckets && !loadAll { + // We will not load buckets if: + // 1. Lease > 0, and: + // 2. this column is not handle, and: + // 3. the column doesn't has buckets before, and: + // 4. loadAll is false. + notNeedLoad := h.Lease > 0 && + !isHandle && + (col == nil || col.Len() == 0 && col.LastUpdateVersion < histVer) && + !loadAll + if notNeedLoad { count, err := columnCountFromStorage(h.ctx, table.TableID, histID) if err != nil { return errors.Trace(err) @@ -148,7 +153,6 @@ func (h *Handle) columnStatsFromStorage(row types.Row, table *Table, tableInfo * } break } - // Otherwise, if the version is newer than last update version or it is the first time to load this column or we want to load all, then we load the column with the buckets of histogram. if col == nil || col.LastUpdateVersion < histVer || loadAll { hg, err := histogramFromStorage(h.ctx, tableInfo.ID, histID, &colInfo.FieldType, distinct, 0, histVer, nullCount, totColSize) if err != nil { diff --git a/statistics/update.go b/statistics/update.go index 3af48e7ef589d..6d274f67340e6 100644 --- a/statistics/update.go +++ b/statistics/update.go @@ -162,23 +162,20 @@ func (h *Handle) DumpStatsDeltaToKV() error { } h.listHead.Unlock() for id, item := range h.globalMap { - updatedCount, err := h.dumpTableStatCountToKV(id, item) + updated, err := h.dumpTableStatCountToKV(id, item) if err != nil { return errors.Trace(err) } - if updatedCount { + if updated { h.globalMap.update(id, -item.Delta, -item.Count, nil) } - updatedColSize, err := h.dumpTableStatColSizeToKV(id, item) - if err != nil { + if err = h.dumpTableStatColSizeToKV(id, item); err != nil { return errors.Trace(err) } - if updatedColSize { - m := h.globalMap[id] - m.ColSize = nil - h.globalMap[id] = m - } - if updatedCount && updatedColSize { + m := h.globalMap[id] + m.ColSize = nil + h.globalMap[id] = m + if updated { delete(h.globalMap, id) } } @@ -210,27 +207,26 @@ func (h *Handle) dumpTableStatCountToKV(id int64, delta variable.TableDelta) (bo return updated, errors.Trace(err) } -func (h *Handle) dumpTableStatColSizeToKV(id int64, delta variable.TableDelta) (bool, error) { +func (h *Handle) dumpTableStatColSizeToKV(id int64, delta variable.TableDelta) error { if delta.ColSize == nil { - return true, nil + return nil } ctx := context.TODO() _, err := h.ctx.(sqlexec.SQLExecutor).Execute(ctx, "begin") if err != nil { - return false, errors.Trace(err) + return errors.Trace(err) } version := h.ctx.Txn().StartTS() for key, val := range delta.ColSize { - sql := fmt.Sprintf("update mysql.stats_histograms set version = %d, tot_col_size = tot_col_size + %d where hist_id = %d and table_id = %d", version, val, key, id) + sql := fmt.Sprintf("update mysql.stats_histograms set version = %d, tot_col_size = tot_col_size + %d where hist_id = %d and table_id = %d and is_index = 0", version, val, key, id) _, err = h.ctx.(sqlexec.SQLExecutor).Execute(ctx, sql) if err != nil { - return false, errors.Trace(err) + return errors.Trace(err) } } - updated := h.ctx.GetSessionVars().StmtCtx.AffectedRows() > 0 _, err = h.ctx.(sqlexec.SQLExecutor).Execute(ctx, "commit") - return updated, errors.Trace(err) + return errors.Trace(err) } const ( diff --git a/table/tables/tables.go b/table/tables/tables.go index 6121615bcfb15..1f5320d0c32f0 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -437,7 +437,7 @@ func (t *Table) AddRecord(ctx sessionctx.Context, r []types.Datum, skipHandleChe colSize := make(map[int64]int64) for id, col := range t.WritableCols() { if col.State == model.StatePublic { - colSize[col.ID] = int64(len(r[id].GetBytes())) + colSize[col.ID] = int64(r[id].Size()) } } sessVars.TxnCtx.UpdateDeltaForTable(t.ID, 1, 1, &colSize) diff --git a/types/datum.go b/types/datum.go index 1605a03ed3546..1e9a9d3541fce 100644 --- a/types/datum.go +++ b/types/datum.go @@ -112,7 +112,7 @@ func (d *Datum) Length() int { return int(d.length) } -// SetLength sets the length of the datum +// SetLength sets the length of the datum. func (d *Datum) SetLength(l int) { d.length = uint32(l) } @@ -122,6 +122,11 @@ func (d *Datum) IsNull() bool { return d.k == KindNull } +// Size gets the length of byte of the datum. +func (d *Datum) Size() int { + return len(d.b) +} + // GetInt64 gets int64 value. func (d *Datum) GetInt64() int64 { return d.i From 30174aae17673f2a7f8898feba4a16fc64679883 Mon Sep 17 00:00:00 2001 From: flipped <19flipped@gmail.com> Date: Wed, 28 Mar 2018 19:48:44 +0800 Subject: [PATCH 3/8] no update when colSize is 0 --- executor/write.go | 12 +++++++++--- statistics/update.go | 5 ++++- table/tables/tables.go | 5 ++++- 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/executor/write.go b/executor/write.go index eb7bc38d6eb4b..cc42593ca86da 100644 --- a/executor/write.go +++ b/executor/write.go @@ -160,8 +160,11 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu } colSize := make(map[int64]int64) for id, col := range t.WritableCols() { - if col.State == model.StatePublic && newData[id].Size() != 0 { - colSize[col.ID] = int64(newData[id].Size() - oldData[id].Size()) + if col.State == model.StatePublic { + val := int64(newData[id].Size() - oldData[id].Size()) + if val != 0 { + colSize[col.ID] = val + } } } ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, 0, 1, &colSize) @@ -393,7 +396,10 @@ func (e *DeleteExec) removeRow(ctx sessionctx.Context, t table.Table, h int64, d colSize := make(map[int64]int64) for id, col := range t.WritableCols() { if col.State == model.StatePublic { - colSize[col.ID] = -int64(data[id].Size()) + val := -int64(data[id].Size()) + if val != 0 { + colSize[col.ID] = val + } } } ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, -1, 1, &colSize) diff --git a/statistics/update.go b/statistics/update.go index 6d274f67340e6..a5b9d3e366c75 100644 --- a/statistics/update.go +++ b/statistics/update.go @@ -208,7 +208,7 @@ func (h *Handle) dumpTableStatCountToKV(id int64, delta variable.TableDelta) (bo } func (h *Handle) dumpTableStatColSizeToKV(id int64, delta variable.TableDelta) error { - if delta.ColSize == nil { + if len(delta.ColSize) == 0 { return nil } ctx := context.TODO() @@ -219,6 +219,9 @@ func (h *Handle) dumpTableStatColSizeToKV(id int64, delta variable.TableDelta) e version := h.ctx.Txn().StartTS() for key, val := range delta.ColSize { + if val == 0 { + continue + } sql := fmt.Sprintf("update mysql.stats_histograms set version = %d, tot_col_size = tot_col_size + %d where hist_id = %d and table_id = %d and is_index = 0", version, val, key, id) _, err = h.ctx.(sqlexec.SQLExecutor).Execute(ctx, sql) if err != nil { diff --git a/table/tables/tables.go b/table/tables/tables.go index 1f5320d0c32f0..39f92df8b2ca9 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -437,7 +437,10 @@ func (t *Table) AddRecord(ctx sessionctx.Context, r []types.Datum, skipHandleChe colSize := make(map[int64]int64) for id, col := range t.WritableCols() { if col.State == model.StatePublic { - colSize[col.ID] = int64(r[id].Size()) + val := int64(r[id].Size()) + if val != 0 { + colSize[col.ID] = val + } } } sessVars.TxnCtx.UpdateDeltaForTable(t.ID, 1, 1, &colSize) From 31e3ecf5681c360d1763d6c5818ca7c10eb13cf6 Mon Sep 17 00:00:00 2001 From: flipped <19flipped@gmail.com> Date: Tue, 3 Apr 2018 15:48:37 +0800 Subject: [PATCH 4/8] remove .Size() --- executor/write.go | 4 ++-- statistics/ddl.go | 2 +- statistics/feedback_test.go | 14 +++++++------- statistics/sample.go | 2 +- statistics/update_test.go | 2 +- table/tables/tables.go | 2 +- types/datum.go | 5 ----- 7 files changed, 13 insertions(+), 18 deletions(-) diff --git a/executor/write.go b/executor/write.go index 567caf36b4ae3..bef068eb7c7fb 100644 --- a/executor/write.go +++ b/executor/write.go @@ -161,7 +161,7 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu colSize := make(map[int64]int64) for id, col := range t.WritableCols() { if col.State == model.StatePublic { - val := int64(newData[id].Size() - oldData[id].Size()) + val := int64(len(newData[id].GetBytes()) - len(oldData[id].GetBytes())) if val != 0 { colSize[col.ID] = val } @@ -396,7 +396,7 @@ func (e *DeleteExec) removeRow(ctx sessionctx.Context, t table.Table, h int64, d colSize := make(map[int64]int64) for id, col := range t.WritableCols() { if col.State == model.StatePublic { - val := -int64(data[id].Size()) + val := -int64(len(data[id].GetBytes())) if val != 0 { colSize[col.ID] = val } diff --git a/statistics/ddl.go b/statistics/ddl.go index b19c7add35b3e..69bb8899eabca 100644 --- a/statistics/ddl.go +++ b/statistics/ddl.go @@ -119,7 +119,7 @@ func (h *Handle) insertColStats2KV(tableID int64, colInfo *model.ColumnInfo) err } } else { // If this stats exists, we insert histogram meta first, the distinct_count will always be one. - _, err = exec.Execute(ctx, fmt.Sprintf("insert into mysql.stats_histograms (version, table_id, is_index, hist_id, distinct_count, tot_col_size) values (%d, %d, 0, %d, 1, %d)", h.ctx.Txn().StartTS(), tableID, colInfo.ID, int64(value.Size())*count)) + _, err = exec.Execute(ctx, fmt.Sprintf("insert into mysql.stats_histograms (version, table_id, is_index, hist_id, distinct_count, tot_col_size) values (%d, %d, 0, %d, 1, %d)", h.ctx.Txn().StartTS(), tableID, colInfo.ID, int64(len(value.GetBytes()))*count)) if err != nil { return errors.Trace(err) } diff --git a/statistics/feedback_test.go b/statistics/feedback_test.go index 69eb0f60eb379..4ce5692170106 100644 --- a/statistics/feedback_test.go +++ b/statistics/feedback_test.go @@ -69,7 +69,7 @@ func (s *testFeedbackSuite) TestUpdateHistogram(c *C) { defaultBucketCount = 5 defer func() { defaultBucketCount = originBucketCount }() c.Assert(UpdateHistogram(q.Hist(), []*QueryFeedback{q}).ToString(0), Equals, - "column:0 ndv:0\n"+ + "column:0 ndv:0 totColSize:0\n"+ "num: 10000\tlower_bound: 0\tupper_bound: 1\trepeats: 0\n"+ "num: 10003\tlower_bound: 2\tupper_bound: 3\trepeats: 0\n"+ "num: 10021\tlower_bound: 4\tupper_bound: 20\trepeats: 0\n"+ @@ -87,7 +87,7 @@ func (s *testFeedbackSuite) TestSplitBuckets(c *C) { q.feedback = feedbacks buckets, isNewBuckets, totalCount := splitBuckets(q.Hist(), []*QueryFeedback{q}) c.Assert(buildNewHistogram(q.Hist(), buckets).ToString(0), Equals, - "column:0 ndv:0\n"+ + "column:0 ndv:0 totColSize:0\n"+ "num: 1\tlower_bound: 0\tupper_bound: 1\trepeats: 0\n"+ "num: 1\tlower_bound: 2\tupper_bound: 3\trepeats: 0\n"+ "num: 1\tlower_bound: 5\tupper_bound: 7\trepeats: 0\n"+ @@ -106,7 +106,7 @@ func (s *testFeedbackSuite) TestSplitBuckets(c *C) { q.feedback = feedbacks buckets, isNewBuckets, totalCount = splitBuckets(q.Hist(), []*QueryFeedback{q}) c.Assert(buildNewHistogram(q.Hist(), buckets).ToString(0), Equals, - "column:0 ndv:0\n"+ + "column:0 ndv:0 totColSize:0\n"+ "num: 100000\tlower_bound: 0\tupper_bound: 1\trepeats: 0\n"+ "num: 100000\tlower_bound: 2\tupper_bound: 3\trepeats: 0\n"+ "num: 100000\tlower_bound: 5\tupper_bound: 7\trepeats: 0\n"+ @@ -126,7 +126,7 @@ func (s *testFeedbackSuite) TestSplitBuckets(c *C) { q.feedback = feedbacks buckets, isNewBuckets, totalCount = splitBuckets(q.Hist(), []*QueryFeedback{q}) c.Assert(buildNewHistogram(q.Hist(), buckets).ToString(0), Equals, - "column:0 ndv:0\n"+ + "column:0 ndv:0 totColSize:0\n"+ "num: 1000000\tlower_bound: 0\tupper_bound: 1000000\trepeats: 0") c.Assert(isNewBuckets, DeepEquals, []bool{false}) c.Assert(totalCount, Equals, int64(1000000)) @@ -147,14 +147,14 @@ func (s *testFeedbackSuite) TestMergeBuckets(c *C) { counts: []int64{1}, isNewBuckets: []bool{false}, bucketCount: 1, - result: "column:0 ndv:0\nnum: 1\tlower_bound: 1\tupper_bound: 2\trepeats: 0", + result: "column:0 ndv:0 totColSize:0\nnum: 1\tlower_bound: 1\tupper_bound: 2\trepeats: 0", }, { points: []int64{1, 2, 2, 3, 3, 4}, counts: []int64{100000, 1, 1}, isNewBuckets: []bool{false, false, false}, bucketCount: 2, - result: "column:0 ndv:0\n" + + result: "column:0 ndv:0 totColSize:0\n" + "num: 100000\tlower_bound: 1\tupper_bound: 2\trepeats: 0\n" + "num: 100002\tlower_bound: 2\tupper_bound: 4\trepeats: 0", }, @@ -164,7 +164,7 @@ func (s *testFeedbackSuite) TestMergeBuckets(c *C) { counts: []int64{1, 1, 100000, 100000}, isNewBuckets: []bool{false, false, false, false}, bucketCount: 3, - result: "column:0 ndv:0\n" + + result: "column:0 ndv:0 totColSize:0\n" + "num: 2\tlower_bound: 1\tupper_bound: 3\trepeats: 0\n" + "num: 100002\tlower_bound: 3\tupper_bound: 4\trepeats: 0\n" + "num: 200002\tlower_bound: 4\tupper_bound: 5\trepeats: 0", diff --git a/statistics/sample.go b/statistics/sample.go index bf4bff1b0e5c8..33065fa35f986 100644 --- a/statistics/sample.go +++ b/statistics/sample.go @@ -104,7 +104,7 @@ func (c *SampleCollector) collect(sc *stmtctx.StatementContext, d types.Datum) e c.CMSketch.InsertBytes(d.GetBytes()) } // Minus one is to remove the flag byte. - c.TotalSize += int64(d.Size() - 1) + c.TotalSize += int64(len(d.GetBytes()) - 1) } c.seenValues++ // The following code use types.CopyDatum(d) because d may have a deep reference diff --git a/statistics/update_test.go b/statistics/update_test.go index 031ece6546514..5e29f320e2421 100644 --- a/statistics/update_test.go +++ b/statistics/update_test.go @@ -430,7 +430,7 @@ func (s *testStatsUpdateSuite) TestQueryFeedback(c *C) { }{ { sql: "select * from t where t.a <= 5", - hist: "column:1 ndv:3\n" + + hist: "column:1 ndv:3 totColSize:0\n" + "num: 1\tlower_bound: 1\tupper_bound: 1\trepeats: 1\n" + "num: 2\tlower_bound: 2\tupper_bound: 2\trepeats: 1\n" + "num: 4\tlower_bound: 3\tupper_bound: 6\trepeats: 0", diff --git a/table/tables/tables.go b/table/tables/tables.go index 39f92df8b2ca9..abdcfbeaac49d 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -437,7 +437,7 @@ func (t *Table) AddRecord(ctx sessionctx.Context, r []types.Datum, skipHandleChe colSize := make(map[int64]int64) for id, col := range t.WritableCols() { if col.State == model.StatePublic { - val := int64(r[id].Size()) + val := int64(len(r[id].GetBytes())) if val != 0 { colSize[col.ID] = val } diff --git a/types/datum.go b/types/datum.go index dceb10beb5b84..549c3127cd21d 100644 --- a/types/datum.go +++ b/types/datum.go @@ -122,11 +122,6 @@ func (d *Datum) IsNull() bool { return d.k == KindNull } -// Size gets the length of byte of the datum. -func (d *Datum) Size() int { - return len(d.b) -} - // GetInt64 gets int64 value. func (d *Datum) GetInt64() int64 { return d.i From 6955210428c74830490e9b1eb968153a101e6d6e Mon Sep 17 00:00:00 2001 From: flipped <19flipped@gmail.com> Date: Wed, 4 Apr 2018 17:38:17 +0800 Subject: [PATCH 5/8] address comment --- statistics/table.go | 2 +- statistics/update.go | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/statistics/table.go b/statistics/table.go index 650b0dbc0d992..9c74a0887d2ac 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -146,7 +146,7 @@ func (h *Handle) columnStatsFromStorage(row types.Row, table *Table, tableInfo * TotColSize: totColSize, }, Info: colInfo, - Count: count + nullCount + Count: count + nullCount, } break } diff --git a/statistics/update.go b/statistics/update.go index e992d502ceea6..4bb6bb9d15189 100644 --- a/statistics/update.go +++ b/statistics/update.go @@ -172,11 +172,12 @@ func (h *Handle) DumpStatsDeltaToKV() error { if err = h.dumpTableStatColSizeToKV(id, item); err != nil { return errors.Trace(err) } - m := h.globalMap[id] - m.ColSize = nil - h.globalMap[id] = m if updated { delete(h.globalMap, id) + } else { + m := h.globalMap[id] + m.ColSize = nil + h.globalMap[id] = m } } return nil From 516d6f29513e947feed5cfa63cb50b8f659167b4 Mon Sep 17 00:00:00 2001 From: flipped <19flipped@gmail.com> Date: Wed, 11 Apr 2018 11:06:48 +0800 Subject: [PATCH 6/8] fix CI --- statistics/feedback_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/statistics/feedback_test.go b/statistics/feedback_test.go index 4b5aaac4cafcd..d72742727ca11 100644 --- a/statistics/feedback_test.go +++ b/statistics/feedback_test.go @@ -69,7 +69,7 @@ func (s *testFeedbackSuite) TestUpdateHistogram(c *C) { originBucketCount := defaultBucketCount defaultBucketCount = 5 defer func() { defaultBucketCount = originBucketCount }() - c.Assert(UpdateHistogram(q.Hist(), []*QueryFeedback{q}).ToString(0), Equals, + c.Assert(UpdateHistogram(q.Hist(), q).ToString(0), Equals, "column:0 ndv:0 totColSize:0\n"+ "num: 10000\tlower_bound: 0\tupper_bound: 1\trepeats: 0\n"+ "num: 10003\tlower_bound: 2\tupper_bound: 3\trepeats: 0\n"+ From 414ac9756a330e72c3f08977a89e434384d7fa07 Mon Sep 17 00:00:00 2001 From: flipped <19flipped@gmail.com> Date: Fri, 13 Apr 2018 11:20:53 +0800 Subject: [PATCH 7/8] address comment --- executor/executor.go | 2 +- executor/write.go | 14 ++++++-------- sessionctx/variable/session.go | 8 +++----- table/tables/tables.go | 12 +++++------- 4 files changed, 15 insertions(+), 21 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index 1f0527aa2f83c..882db36685ebf 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -464,7 +464,7 @@ func (e *SelectLockExec) Open(ctx context.Context) error { txnCtx.ForUpdate = true for id := range e.Schema().TblID2Handle { // This operation is only for schema validator check. - txnCtx.UpdateDeltaForTable(id, 0, 0, nil) + txnCtx.UpdateDeltaForTable(id, 0, 0, map[int64]int64{}) } return nil } diff --git a/executor/write.go b/executor/write.go index 00bb8ed0dd67d..280c7f346ccad 100644 --- a/executor/write.go +++ b/executor/write.go @@ -166,15 +166,13 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu sc.AddAffectedRows(1) } colSize := make(map[int64]int64) - for id, col := range t.WritableCols() { - if col.State == model.StatePublic { - val := int64(len(newData[id].GetBytes()) - len(oldData[id].GetBytes())) - if val != 0 { - colSize[col.ID] = val - } + for id, col := range t.Cols() { + val := int64(len(newData[id].GetBytes()) - len(oldData[id].GetBytes())) + if val != 0 { + colSize[col.ID] = val } } - ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, 0, 1, &colSize) + ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, 0, 1, colSize) return true, handleChanged, newHandle, nil } @@ -409,7 +407,7 @@ func (e *DeleteExec) removeRow(ctx sessionctx.Context, t table.Table, h int64, d } } } - ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, -1, 1, &colSize) + ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, -1, 1, colSize) return nil } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index b78f4bbc86804..279cf65b8322d 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -101,7 +101,7 @@ type TransactionContext struct { } // UpdateDeltaForTable updates the delta info for some table. -func (tc *TransactionContext) UpdateDeltaForTable(tableID int64, delta int64, count int64, colSize *map[int64]int64) { +func (tc *TransactionContext) UpdateDeltaForTable(tableID int64, delta int64, count int64, colSize map[int64]int64) { if tc.TableDeltaMap == nil { tc.TableDeltaMap = make(map[int64]TableDelta) } @@ -111,10 +111,8 @@ func (tc *TransactionContext) UpdateDeltaForTable(tableID int64, delta int64, co } item.Delta += delta item.Count += count - if colSize != nil { - for key, val := range *colSize { - item.ColSize[key] += val - } + for key, val := range colSize { + item.ColSize[key] += val } tc.TableDeltaMap[tableID] = item } diff --git a/table/tables/tables.go b/table/tables/tables.go index 41289a869fe37..45689fbf15da1 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -435,15 +435,13 @@ func (t *Table) AddRecord(ctx sessionctx.Context, r []types.Datum, skipHandleChe } sessVars.StmtCtx.AddAffectedRows(1) colSize := make(map[int64]int64) - for id, col := range t.WritableCols() { - if col.State == model.StatePublic { - val := int64(len(r[id].GetBytes())) - if val != 0 { - colSize[col.ID] = val - } + for id, col := range t.Cols() { + val := int64(len(r[id].GetBytes())) + if val != 0 { + colSize[col.ID] = val } } - sessVars.TxnCtx.UpdateDeltaForTable(t.ID, 1, 1, &colSize) + sessVars.TxnCtx.UpdateDeltaForTable(t.ID, 1, 1, colSize) return recordID, nil } From 4aca56400b082875a3c7762401e12df5afce12e1 Mon Sep 17 00:00:00 2001 From: flipped <19flipped@gmail.com> Date: Fri, 13 Apr 2018 13:45:55 +0800 Subject: [PATCH 8/8] address comment --- executor/write.go | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/executor/write.go b/executor/write.go index 280c7f346ccad..563a8e5d24e89 100644 --- a/executor/write.go +++ b/executor/write.go @@ -399,12 +399,10 @@ func (e *DeleteExec) removeRow(ctx sessionctx.Context, t table.Table, h int64, d ctx.StmtAddDirtyTableOP(DirtyTableDeleteRow, t.Meta().ID, h, nil) ctx.GetSessionVars().StmtCtx.AddAffectedRows(1) colSize := make(map[int64]int64) - for id, col := range t.WritableCols() { - if col.State == model.StatePublic { - val := -int64(len(data[id].GetBytes())) - if val != 0 { - colSize[col.ID] = val - } + for id, col := range t.Cols() { + val := -int64(len(data[id].GetBytes())) + if val != 0 { + colSize[col.ID] = val } } ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, -1, 1, colSize)