From b5c50a13289a408b7df145104686bf961770d897 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Mon, 10 Sep 2018 16:11:26 +0800 Subject: [PATCH 1/3] infoschema: fill data length fields for tables --- infoschema/tables.go | 108 ++++++++++++++++++++++++++++++-------- infoschema/tables_test.go | 88 +++++++++++++++---------------- statistics/histogram.go | 18 ++----- types/field_type.go | 20 +++++++ util/chunk/chunk.go | 4 +- util/chunk/codec.go | 22 +------- 6 files changed, 159 insertions(+), 101 deletions(-) diff --git a/infoschema/tables.go b/infoschema/tables.go index bd2bda00cbd09..ca6f58c4ece81 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -692,6 +692,62 @@ func getRowCountAllTable(ctx sessionctx.Context) (map[int64]uint64, error) { return rowCountMap, nil } +type tableHistID struct { + tableID int64 + histID int64 +} + +func getColLengthAllTables(ctx sessionctx.Context) (map[tableHistID]int64, error) { + rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, "select table_id, hist_id, tot_col_size from mysql.stats_histograms where is_index = 0") + if err != nil { + return nil, errors.Trace(err) + } + colLengthMap := make(map[tableHistID]int64, len(rows)) + for _, row := range rows { + tableID := row.GetInt64(0) + histID := row.GetInt64(1) + totalSize := row.GetInt64(2) + if totalSize < 0 { + totalSize = 0 + } + colLengthMap[tableHistID{tableID: tableID, histID: histID}] = totalSize + } + return colLengthMap, nil +} + +func getDataAndIndexLength(info *model.TableInfo, rowCount uint64, columnLengthMap map[tableHistID]int64) (uint64, uint64) { + columnLength := make(map[string]uint64) + for _, col := range info.Columns { + if col.State != model.StatePublic { + continue + } + length := col.FieldType.Length() + if length != types.VarElemLen { + columnLength[col.Name.L] = rowCount * uint64(length) + } else { + length := columnLengthMap[tableHistID{tableID: info.ID, histID: col.ID}] + columnLength[col.Name.L] = uint64(length) + } + } + dataLength, indexLength := uint64(0), uint64(0) + for _, length := range columnLength { + dataLength += length + } + for _, idx := range info.Indices { + if idx.State != model.StatePublic { + continue + } + for _, col := range idx.Columns { + if col.Length == types.UnspecifiedLength { + indexLength += columnLength[col.Name.L] + } else { + indexLength += rowCount * uint64(col.Length) + } + } + } + return dataLength, indexLength +} + func getAutoIncrementID(ctx sessionctx.Context, schema *model.DBInfo, tblInfo *model.TableInfo) (int64, error) { hasAutoIncID := false for _, col := range tblInfo.Cols() { @@ -720,6 +776,10 @@ func dataForTables(ctx sessionctx.Context, schemas []*model.DBInfo) ([][]types.D if err != nil { return nil, errors.Trace(err) } + colLengthMap, err := getColLengthAllTables(ctx) + if err != nil { + return nil, errors.Trace(err) + } checker := privilege.GetPrivilegeManager(ctx) @@ -744,28 +804,34 @@ func dataForTables(ctx sessionctx.Context, schemas []*model.DBInfo) ([][]types.D if err != nil { return nil, errors.Trace(err) } + rowCount := tableRowsMap[table.ID] + dataLength, indexLength := getDataAndIndexLength(table, rowCount, colLengthMap) + avgRowLength := uint64(0) + if rowCount != 0 { + avgRowLength = dataLength / rowCount + } record := types.MakeDatums( - catalogVal, // TABLE_CATALOG - schema.Name.O, // TABLE_SCHEMA - table.Name.O, // TABLE_NAME - "BASE TABLE", // TABLE_TYPE - "InnoDB", // ENGINE - uint64(10), // VERSION - "Compact", // ROW_FORMAT - tableRowsMap[table.ID], // TABLE_ROWS - uint64(0), // AVG_ROW_LENGTH - uint64(16384), // DATA_LENGTH - uint64(0), // MAX_DATA_LENGTH - uint64(0), // INDEX_LENGTH - uint64(0), // DATA_FREE - autoIncID, // AUTO_INCREMENT - createTime, // CREATE_TIME - nil, // UPDATE_TIME - nil, // CHECK_TIME - collation, // TABLE_COLLATION - nil, // CHECKSUM - "", // CREATE_OPTIONS - table.Comment, // TABLE_COMMENT + catalogVal, // TABLE_CATALOG + schema.Name.O, // TABLE_SCHEMA + table.Name.O, // TABLE_NAME + "BASE TABLE", // TABLE_TYPE + "InnoDB", // ENGINE + uint64(10), // VERSION + "Compact", // ROW_FORMAT + rowCount, // TABLE_ROWS + avgRowLength, // AVG_ROW_LENGTH + dataLength, // DATA_LENGTH + uint64(0), // MAX_DATA_LENGTH + indexLength, // INDEX_LENGTH + uint64(0), // DATA_FREE + autoIncID, // AUTO_INCREMENT + createTime, // CREATE_TIME + nil, // UPDATE_TIME + nil, // CHECK_TIME + collation, // TABLE_COLLATION + nil, // CHECKSUM + "", // CREATE_OPTIONS + table.Comment, // TABLE_COMMENT ) rows = append(rows, record) } diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 82f08876f5982..ecf849e607af3 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -47,52 +47,9 @@ func (s *testSuite) TestInfoschemaFielValue(c *C) { Check(testkit.Rows(" ", " 3", " 3", " 4", " ")) tk.MustQuery("select CHARACTER_MAXIMUM_LENGTH,CHARACTER_OCTET_LENGTH,NUMERIC_PRECISION,NUMERIC_SCALE,DATETIME_PRECISION from information_schema.COLUMNS where table_name='strschema'"). Check(testkit.Rows("3 3 ", "3 3 ", "3 3 ", "3 3 ")) // FIXME: for mysql last two will be "255 255 ", "255 255 " -} - -func (s *testSuite) TestDataForTableRowsCountField(c *C) { - testleak.BeforeTest() - defer testleak.AfterTest(c)() - store, err := mockstore.NewMockTikvStore() - c.Assert(err, IsNil) - defer store.Close() - session.SetStatsLease(0) - do, err := session.BootstrapSession(store) - c.Assert(err, IsNil) - defer do.Close() - - h := do.StatsHandle() - is := do.InfoSchema() - tk := testkit.NewTestKit(c, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (c int, d int)") - h.HandleDDLEvent(<-h.DDLEventCh()) - tk.MustQuery("select table_rows from information_schema.tables where table_name='t'").Check( - testkit.Rows("0")) - tk.MustExec("insert into t(c, d) values(1, 2), (2, 3), (3, 4)") - h.DumpStatsDeltaToKV(statistics.DumpAll) - h.Update(is) - tk.MustQuery("select table_rows from information_schema.tables where table_name='t'").Check( - testkit.Rows("3")) - tk.MustExec("insert into t(c, d) values(4, 5)") - h.DumpStatsDeltaToKV(statistics.DumpAll) - h.Update(is) - tk.MustQuery("select table_rows from information_schema.tables where table_name='t'").Check( - testkit.Rows("4")) - tk.MustExec("delete from t where c >= 3") - h.DumpStatsDeltaToKV(statistics.DumpAll) - h.Update(is) - tk.MustQuery("select table_rows from information_schema.tables where table_name='t'").Check( - testkit.Rows("2")) - tk.MustExec("delete from t where c=3") - h.DumpStatsDeltaToKV(statistics.DumpAll) - h.Update(is) - tk.MustQuery("select table_rows from information_schema.tables where table_name='t'").Check( - testkit.Rows("2")) // Test for auto increment ID. - tk.MustExec("drop table t") + tk.MustExec("drop table if exists t") tk.MustExec("create table t (c int auto_increment primary key, d int)") tk.MustQuery("select auto_increment from information_schema.tables where table_name='t'").Check( testkit.Rows("1")) @@ -122,3 +79,46 @@ func (s *testSuite) TestDataForTableRowsCountField(c *C) { tk1.MustQuery("select distinct(table_schema) from information_schema.tables").Check(testkit.Rows("INFORMATION_SCHEMA")) } + +func (s *testSuite) TestDataForTableStatsField(c *C) { + testleak.BeforeTest() + defer testleak.AfterTest(c)() + store, err := mockstore.NewMockTikvStore() + c.Assert(err, IsNil) + defer store.Close() + session.SetStatsLease(0) + do, err := session.BootstrapSession(store) + c.Assert(err, IsNil) + defer do.Close() + + h := do.StatsHandle() + is := do.InfoSchema() + tk := testkit.NewTestKit(c, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (c int, d int, e char(5), index idx(e))") + h.HandleDDLEvent(<-h.DDLEventCh()) + tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check( + testkit.Rows("0 0 0 0")) + tk.MustExec(`insert into t(c, d, e) values(1, 2, "c"), (2, 3, "d"), (3, 4, "e")`) + h.DumpStatsDeltaToKV(statistics.DumpAll) + h.Update(is) + 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")) + tk.MustExec(`insert into t(c, d, e) values(4, 5, "f")`) + h.DumpStatsDeltaToKV(statistics.DumpAll) + h.Update(is) + 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")) + tk.MustExec("delete from t where c >= 3") + h.DumpStatsDeltaToKV(statistics.DumpAll) + h.Update(is) + 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")) + tk.MustExec("delete from t where c=3") + h.DumpStatsDeltaToKV(statistics.DumpAll) + h.Update(is) + 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")) +} diff --git a/statistics/histogram.go b/statistics/histogram.go index 372a2ff8f96f7..756f7853ec06b 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -108,20 +108,12 @@ func (c *Column) AvgColSize(count int64) 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: - 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 + len := c.Histogram.tp.Length() + if len != types.VarElemLen { + return float64(len) } + // Keep two decimal place. + return math.Round(float64(c.TotColSize)/float64(count)*100) / 100 } // AppendBucket appends a bucket into `hg`. diff --git a/types/field_type.go b/types/field_type.go index 29d3a3fd1c9d5..6c896b4922342 100644 --- a/types/field_type.go +++ b/types/field_type.go @@ -1421,3 +1421,23 @@ func SetBinChsClnFlag(ft *FieldType) { ft.Collate = charset.CollationBin ft.Flag |= mysql.BinaryFlag } + +// VarElemLen indicates this column is a variable length column. +const VarElemLen = -1 + +// Length is the length of value for the type. +func (ft *FieldType) Length() int { + switch ft.Tp { + case mysql.TypeFloat: + return 4 + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, + mysql.TypeLonglong, mysql.TypeDouble, mysql.TypeYear, mysql.TypeDuration: + return 8 + case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: + return 16 + case mysql.TypeNewDecimal: + return MyDecimalStructSize + default: + return VarElemLen + } +} diff --git a/util/chunk/chunk.go b/util/chunk/chunk.go index a9211cb79a416..c5cae0a3005a0 100644 --- a/util/chunk/chunk.go +++ b/util/chunk/chunk.go @@ -53,8 +53,8 @@ func New(fields []*types.FieldType, cap, maxChunkSize int) *Chunk { chk.columns = make([]*column, 0, len(fields)) chk.capacity = mathutil.Min(cap, maxChunkSize) for _, f := range fields { - elemLen := getFixedLen(f) - if elemLen == varElemLen { + elemLen := f.Length() + if elemLen == types.VarElemLen { chk.columns = append(chk.columns, newVarLenColumn(chk.capacity, nil)) } else { chk.columns = append(chk.columns, newFixedLenColumn(elemLen, chk.capacity)) diff --git a/util/chunk/codec.go b/util/chunk/codec.go index 20401f95f762b..d92516ca4973b 100644 --- a/util/chunk/codec.go +++ b/util/chunk/codec.go @@ -19,7 +19,6 @@ import ( "unsafe" "github.com/cznic/mathutil" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/types" ) @@ -124,7 +123,7 @@ func (c *Codec) decodeColumn(buffer []byte, col *column, ordinal int) (remained } // decode offsets. - numFixedBytes := getFixedLen(c.colTypes[ordinal]) + numFixedBytes := c.colTypes[ordinal].Length() numDataBytes := numFixedBytes * col.length if numFixedBytes == -1 { numOffsetBytes := (col.length + 1) * 4 @@ -163,25 +162,6 @@ func (c *Codec) bytesToI32Slice(b []byte) (i32s []int32) { return i32s } -// varElemLen indicates this column is a variable length column. -const varElemLen = -1 - -func getFixedLen(colType *types.FieldType) int { - switch colType.Tp { - case mysql.TypeFloat: - return 4 - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, - mysql.TypeLonglong, mysql.TypeDouble, mysql.TypeYear, mysql.TypeDuration: - return 8 - case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: - return 16 - case mysql.TypeNewDecimal: - return types.MyDecimalStructSize - default: - return varElemLen - } -} - func init() { for i := 0; i < 128; i++ { allNotNullBitmap[i] = 0xFF From bb1fdf60fb5efbc074a278b2385c4b18548304ec Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Tue, 11 Sep 2018 15:56:31 +0800 Subject: [PATCH 2/3] address comments --- infoschema/tables.go | 53 ++++++++++++++++++++++++++++++--------- infoschema/tables_test.go | 4 +++ statistics/histogram.go | 18 +++++++++---- types/field_type.go | 22 ++++++++-------- util/chunk/chunk.go | 4 +-- util/chunk/codec.go | 22 +++++++++++++++- 6 files changed, 92 insertions(+), 31 deletions(-) diff --git a/infoschema/tables.go b/infoschema/tables.go index ca6f58c4ece81..8ffd136969f88 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -16,6 +16,7 @@ package infoschema import ( "fmt" "sort" + "sync" "time" "github.com/juju/errors" @@ -697,12 +698,12 @@ type tableHistID struct { histID int64 } -func getColLengthAllTables(ctx sessionctx.Context) (map[tableHistID]int64, error) { +func getColLengthAllTables(ctx sessionctx.Context) (map[tableHistID]uint64, error) { rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, "select table_id, hist_id, tot_col_size from mysql.stats_histograms where is_index = 0") if err != nil { return nil, errors.Trace(err) } - colLengthMap := make(map[tableHistID]int64, len(rows)) + colLengthMap := make(map[tableHistID]uint64, len(rows)) for _, row := range rows { tableID := row.GetInt64(0) histID := row.GetInt64(1) @@ -710,23 +711,23 @@ func getColLengthAllTables(ctx sessionctx.Context) (map[tableHistID]int64, error if totalSize < 0 { totalSize = 0 } - colLengthMap[tableHistID{tableID: tableID, histID: histID}] = totalSize + colLengthMap[tableHistID{tableID: tableID, histID: histID}] = uint64(totalSize) } return colLengthMap, nil } -func getDataAndIndexLength(info *model.TableInfo, rowCount uint64, columnLengthMap map[tableHistID]int64) (uint64, uint64) { +func getDataAndIndexLength(info *model.TableInfo, rowCount uint64, columnLengthMap map[tableHistID]uint64) (uint64, uint64) { columnLength := make(map[string]uint64) for _, col := range info.Columns { if col.State != model.StatePublic { continue } - length := col.FieldType.Length() - if length != types.VarElemLen { + length := col.FieldType.StorageLength() + if length != types.VarStorageLen { columnLength[col.Name.L] = rowCount * uint64(length) } else { length := columnLengthMap[tableHistID{tableID: info.ID, histID: col.ID}] - columnLength[col.Name.L] = uint64(length) + columnLength[col.Name.L] = length } } dataLength, indexLength := uint64(0), uint64(0) @@ -748,6 +749,38 @@ func getDataAndIndexLength(info *model.TableInfo, rowCount uint64, columnLengthM return dataLength, indexLength } +type statsCache struct { + mu sync.Mutex + modifyTime time.Time + tableRows map[int64]uint64 + colLength map[tableHistID]uint64 +} + +var tableStatsCache = &statsCache{} + +// TableStatsCacheExpiry is the expiry time for table stats cache. +var TableStatsCacheExpiry = 3 * time.Second + +func (c *statsCache) get(ctx sessionctx.Context) (map[int64]uint64, map[tableHistID]uint64, error) { + c.mu.Lock() + defer c.mu.Unlock() + if time.Now().Sub(c.modifyTime) < TableStatsCacheExpiry { + return c.tableRows, c.colLength, nil + } + tableRows, err := getRowCountAllTable(ctx) + if err != nil { + return nil, nil, errors.Trace(err) + } + colLength, err := getColLengthAllTables(ctx) + if err != nil { + return nil, nil, errors.Trace(err) + } + c.tableRows = tableRows + c.colLength = colLength + c.modifyTime = time.Now() + return tableRows, colLength, nil +} + func getAutoIncrementID(ctx sessionctx.Context, schema *model.DBInfo, tblInfo *model.TableInfo) (int64, error) { hasAutoIncID := false for _, col := range tblInfo.Cols() { @@ -772,11 +805,7 @@ func getAutoIncrementID(ctx sessionctx.Context, schema *model.DBInfo, tblInfo *m } func dataForTables(ctx sessionctx.Context, schemas []*model.DBInfo) ([][]types.Datum, error) { - tableRowsMap, err := getRowCountAllTable(ctx) - if err != nil { - return nil, errors.Trace(err) - } - colLengthMap, err := getColLengthAllTables(ctx) + tableRowsMap, colLengthMap, err := tableStatsCache.get(ctx) if err != nil { return nil, errors.Trace(err) } diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index ecf849e607af3..5004f2167d12f 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -15,6 +15,7 @@ package infoschema_test import ( . "github.com/pingcap/check" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/mockstore" @@ -90,6 +91,9 @@ func (s *testSuite) TestDataForTableStatsField(c *C) { do, err := session.BootstrapSession(store) c.Assert(err, IsNil) defer do.Close() + oldExpiryTime := infoschema.TableStatsCacheExpiry + infoschema.TableStatsCacheExpiry = 0 + defer func() { infoschema.TableStatsCacheExpiry = oldExpiryTime }() h := do.StatsHandle() is := do.InfoSchema() diff --git a/statistics/histogram.go b/statistics/histogram.go index 756f7853ec06b..372a2ff8f96f7 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -108,12 +108,20 @@ func (c *Column) AvgColSize(count int64) float64 { if count == 0 { return 0 } - len := c.Histogram.tp.Length() - if len != types.VarElemLen { - return float64(len) + 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: + 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 } - // Keep two decimal place. - return math.Round(float64(c.TotColSize)/float64(count)*100) / 100 } // AppendBucket appends a bucket into `hg`. diff --git a/types/field_type.go b/types/field_type.go index 6c896b4922342..0a71e6dae7895 100644 --- a/types/field_type.go +++ b/types/field_type.go @@ -1422,22 +1422,22 @@ func SetBinChsClnFlag(ft *FieldType) { ft.Flag |= mysql.BinaryFlag } -// VarElemLen indicates this column is a variable length column. -const VarElemLen = -1 +// VarStorageLen indicates this column is a variable length column. +const VarStorageLen = -1 -// Length is the length of value for the type. -func (ft *FieldType) Length() int { +// StorageLength is the length of stored value for the type. +func (ft *FieldType) StorageLength() int { switch ft.Tp { - case mysql.TypeFloat: - return 4 case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, - mysql.TypeLonglong, mysql.TypeDouble, mysql.TypeYear, mysql.TypeDuration: + mysql.TypeLonglong, mysql.TypeDouble, mysql.TypeFloat, mysql.TypeYear, mysql.TypeDuration, + mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp, mysql.TypeEnum, mysql.TypeSet, + mysql.TypeBit: + // This may not be the accurate length, because we may encode them as varint. return 8 - case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: - return 16 case mysql.TypeNewDecimal: - return MyDecimalStructSize + precision, frac := ft.Flen-ft.Decimal, ft.Decimal + return precision/digitsPerWord + dig2bytes[precision%digitsPerWord] + frac/digitsPerWord + dig2bytes[frac%digitsPerWord] default: - return VarElemLen + return VarStorageLen } } diff --git a/util/chunk/chunk.go b/util/chunk/chunk.go index c5cae0a3005a0..a9211cb79a416 100644 --- a/util/chunk/chunk.go +++ b/util/chunk/chunk.go @@ -53,8 +53,8 @@ func New(fields []*types.FieldType, cap, maxChunkSize int) *Chunk { chk.columns = make([]*column, 0, len(fields)) chk.capacity = mathutil.Min(cap, maxChunkSize) for _, f := range fields { - elemLen := f.Length() - if elemLen == types.VarElemLen { + elemLen := getFixedLen(f) + if elemLen == varElemLen { chk.columns = append(chk.columns, newVarLenColumn(chk.capacity, nil)) } else { chk.columns = append(chk.columns, newFixedLenColumn(elemLen, chk.capacity)) diff --git a/util/chunk/codec.go b/util/chunk/codec.go index d92516ca4973b..20401f95f762b 100644 --- a/util/chunk/codec.go +++ b/util/chunk/codec.go @@ -19,6 +19,7 @@ import ( "unsafe" "github.com/cznic/mathutil" + "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/types" ) @@ -123,7 +124,7 @@ func (c *Codec) decodeColumn(buffer []byte, col *column, ordinal int) (remained } // decode offsets. - numFixedBytes := c.colTypes[ordinal].Length() + numFixedBytes := getFixedLen(c.colTypes[ordinal]) numDataBytes := numFixedBytes * col.length if numFixedBytes == -1 { numOffsetBytes := (col.length + 1) * 4 @@ -162,6 +163,25 @@ func (c *Codec) bytesToI32Slice(b []byte) (i32s []int32) { return i32s } +// varElemLen indicates this column is a variable length column. +const varElemLen = -1 + +func getFixedLen(colType *types.FieldType) int { + switch colType.Tp { + case mysql.TypeFloat: + return 4 + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, + mysql.TypeLonglong, mysql.TypeDouble, mysql.TypeYear, mysql.TypeDuration: + return 8 + case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: + return 16 + case mysql.TypeNewDecimal: + return types.MyDecimalStructSize + default: + return varElemLen + } +} + func init() { for i := 0; i < 128; i++ { allNotNullBitmap[i] = 0xFF From 1cf30ef38b978d66eb31e2bd72b6c9a4febb8c40 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Wed, 12 Sep 2018 19:27:04 +0800 Subject: [PATCH 3/3] address comment --- infoschema/tables.go | 23 ++++++++++++++++++++--- 1 file changed, 20 insertions(+), 3 deletions(-) diff --git a/infoschema/tables.go b/infoschema/tables.go index 64c2825330f53..d7f8e2816ed06 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -770,6 +770,7 @@ func getDataAndIndexLength(info *model.TableInfo, rowCount uint64, columnLengthM type statsCache struct { mu sync.Mutex + loading bool modifyTime time.Time tableRows map[int64]uint64 colLength map[tableHistID]uint64 @@ -780,23 +781,39 @@ var tableStatsCache = &statsCache{} // TableStatsCacheExpiry is the expiry time for table stats cache. var TableStatsCacheExpiry = 3 * time.Second +func (c *statsCache) setLoading(loading bool) { + c.mu.Lock() + c.loading = loading + c.mu.Unlock() +} + func (c *statsCache) get(ctx sessionctx.Context) (map[int64]uint64, map[tableHistID]uint64, error) { c.mu.Lock() - defer c.mu.Unlock() - if time.Now().Sub(c.modifyTime) < TableStatsCacheExpiry { - return c.tableRows, c.colLength, nil + if time.Now().Sub(c.modifyTime) < TableStatsCacheExpiry || c.loading { + tableRows, colLength := c.tableRows, c.colLength + c.mu.Unlock() + return tableRows, colLength, nil } + c.loading = true + c.mu.Unlock() + tableRows, err := getRowCountAllTable(ctx) if err != nil { + c.setLoading(false) return nil, nil, errors.Trace(err) } colLength, err := getColLengthAllTables(ctx) if err != nil { + c.setLoading(false) return nil, nil, errors.Trace(err) } + + c.mu.Lock() + c.loading = false c.tableRows = tableRows c.colLength = colLength c.modifyTime = time.Now() + c.mu.Unlock() return tableRows, colLength, nil }