Skip to content

Commit

Permalink
address comment
Browse files Browse the repository at this point in the history
  • Loading branch information
tiancaiamao committed Jan 10, 2019
1 parent bbb6a22 commit 345e067
Show file tree
Hide file tree
Showing 7 changed files with 84 additions and 216 deletions.
12 changes: 12 additions & 0 deletions infoschema/perfschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,19 @@ type perfSchemaTable struct {
cols []*table.Column
}

var pluginTable = make(map[string]func(autoid.Allocator, *model.TableInfo) (table.Table, error))

func RegisterTable(tableName, sql string,
tableFromMeta func(autoid.Allocator, *model.TableInfo) (table.Table, error)) {
perfSchemaTables = append(perfSchemaTables, sql)
pluginTable[tableName] = tableFromMeta
}

func tableFromMeta(alloc autoid.Allocator, meta *model.TableInfo) (table.Table, error) {
if f, ok := pluginTable[meta.Name.L]; ok {
ret, err := f(alloc, meta)
return ret, err
}
return createPerfSchemaTable(meta), nil
}

Expand Down
28 changes: 0 additions & 28 deletions infoschema/tiniub/const.go

This file was deleted.

59 changes: 0 additions & 59 deletions infoschema/tiniub/init.go

This file was deleted.

190 changes: 62 additions & 128 deletions infoschema/tiniub/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,24 +18,42 @@ import (
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/infoschema/perfschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
)

// Init should be called before perfschema.Init()
func Init() {
perfschema.RegisterTable("slow_query", tableSlowQuery, tableFromMeta)
}

type slowQueryTable struct {
infoschema.VirtualTable
meta *model.TableInfo
cols []*table.Column
}

const tableSlowQuery = "CREATE TABLE if not exists slow_query (" +
"`SQL` VARCHAR(4096)," +
"`START` TIMESTAMP (6)," +
"`DURATION` TIME (6)," +
"DETAILS VARCHAR(256)," +
"SUCC TINYINT," +
"CONN_ID BIGINT," +
"TRANSACTION_TS BIGINT," +
"USER VARCHAR(32) NOT NULL," +
"DB VARCHAR(64) NOT NULL," +
"TABLE_IDS VARCHAR(256)," +
"INDEX_IDS VARCHAR(256)," +
"INTERNAL TINYINT);"

// tableFromMeta creates the slow query table.
func tableFromMeta(alloc autoid.Allocator, meta *model.TableInfo) (table.Table, error) {
return createSlowQueryTable(meta), nil
}

// createSlowQueryTable creates all slowQueryTables
func createSlowQueryTable(meta *model.TableInfo) *slowQueryTable {
columns := make([]*table.Column, 0, len(meta.Columns))
for _, colInfo := range meta.Columns {
col := table.ToColumn(colInfo)
Expand All @@ -45,7 +63,7 @@ func createSlowQueryTable(meta *model.TableInfo) *slowQueryTable {
meta: meta,
cols: columns,
}
return t
return t, nil
}

// IterRecords rewrites the IterRecords method of slowQueryTable.
Expand All @@ -55,20 +73,35 @@ func (s slowQueryTable) IterRecords(ctx sessionctx.Context, startKey kv.Key, col

for i, item := range result {
row := make([]types.Datum, 0, len(cols))
row = append(row, types.NewDatum(item.SQL))

ts := types.NewTimeDatum(types.Time{types.FromGoTime(item.Start), mysql.TypeTimestamp, types.MaxFsp})
row = append(row, ts)
row = append(row, types.NewDurationDatum(types.Duration{item.Duration, types.MaxFsp}))
row = append(row, types.NewDatum(item.Detail.String()))
row = append(row, types.NewDatum(item.Succ))
row = append(row, types.NewDatum(item.ConnID))
row = append(row, types.NewDatum(item.TxnTS))
row = append(row, types.NewDatum(item.User))
row = append(row, types.NewDatum(item.DB))
row = append(row, types.NewDatum(item.TableIDs))
row = append(row, types.NewDatum(item.IndexIDs))
row = append(row, types.NewDatum(item.Internal))
for _, col := range cols {
switch col.Name.L {
case "sql":
row = append(row, types.NewDatum(item.SQL))
case "start":
ts := types.NewTimeDatum(types.Time{types.FromGoTime(item.Start), mysql.TypeTimestamp, types.MaxFsp})
row = append(row, ts)
case "duration":
row = append(row, types.NewDurationDatum(types.Duration{item.Duration, types.MaxFsp}))
case "details":
row = append(row, types.NewDatum(item.Detail.String()))
case "succ":
row = append(row, types.NewDatum(item.Succ))
case "conn_id":
row = append(row, types.NewDatum(item.ConnID))
case "transaction_ts":
row = append(row, types.NewDatum(item.TxnTS))
case "user":
row = append(row, types.NewDatum(item.User))
case "db":
row = append(row, types.NewDatum(item.DB))
case "table_ids":
row = append(row, types.NewDatum(item.TableIDs))
case "index_ids":
row = append(row, types.NewDatum(item.IndexIDs))
case "internal":
row = append(row, types.NewDatum(item.Internal))
}
}

more, err := fn(int64(i), row, cols)
if err != nil {
Expand All @@ -81,121 +114,22 @@ func (s slowQueryTable) IterRecords(ctx sessionctx.Context, startKey kv.Key, col
return nil
}

// createPerfSchemaTable creates all slowQueryTables
func createPerfSchemaTable(meta *model.TableInfo) *slowQueryTable {
columns := make([]*table.Column, 0, len(meta.Columns))
for _, colInfo := range meta.Columns {
col := table.ToColumn(colInfo)
columns = append(columns, col)
}
t := &slowQueryTable{
meta: meta,
cols: columns,
}
return t
}

// RowWithCols implements table.Table Type interface.
func (s *slowQueryTable) RowWithCols(ctx sessionctx.Context, h int64, cols []*table.Column) ([]types.Datum, error) {
return nil, table.ErrUnsupportedOp
}

// Row implements table.Table Type interface.
func (s *slowQueryTable) Row(ctx sessionctx.Context, h int64) ([]types.Datum, error) {
return nil, table.ErrUnsupportedOp
}

// Cols implements table.Table Type interface.
func (s *slowQueryTable) Cols() []*table.Column {
return s.cols
func (vt *slowQueryTable) Cols() []*table.Column {
return vt.cols
}

// WritableCols implements table.Table Type interface.
func (s *slowQueryTable) WritableCols() []*table.Column {
return s.cols
}

// Indices implements table.Table Type interface.
func (s *slowQueryTable) Indices() []table.Index {
return nil
}

// WritableIndices implements table.Table Type interface.
func (s *slowQueryTable) WritableIndices() []table.Index {
return nil
}

// DeletableIndices implements table.Table Type interface.
func (s *slowQueryTable) DeletableIndices() []table.Index {
return nil
}

// RecordPrefix implements table.Table Type interface.
func (s *slowQueryTable) RecordPrefix() kv.Key {
return nil
}

// IndexPrefix implements table.Table Type interface.
func (s *slowQueryTable) IndexPrefix() kv.Key {
return nil
}

// FirstKey implements table.Table Type interface.
func (s *slowQueryTable) FirstKey() kv.Key {
return nil
}

// RecordKey implements table.Table Type interface.
func (s *slowQueryTable) RecordKey(h int64) kv.Key {
return nil
}

// AddRecord implements table.Table Type interface.
func (s *slowQueryTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...*table.AddRecordOpt) (recordID int64, err error) {
return 0, table.ErrUnsupportedOp
}

// RemoveRecord implements table.Table Type interface.
func (s *slowQueryTable) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Datum) error {
return table.ErrUnsupportedOp
}

// UpdateRecord implements table.Table Type interface.
func (s *slowQueryTable) UpdateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datum, touched []bool) error {
return table.ErrUnsupportedOp
}

// AllocAutoID implements table.Table Type interface.
func (s *slowQueryTable) AllocAutoID(ctx sessionctx.Context) (int64, error) {
return 0, table.ErrUnsupportedOp
}

// Allocator implements table.Table Type interface.
func (s *slowQueryTable) Allocator(ctx sessionctx.Context) autoid.Allocator {
return nil
}

// RebaseAutoID implements table.Table Type interface.
func (s *slowQueryTable) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool) error {
return table.ErrUnsupportedOp
}

// Meta implements table.Table Type interface.
func (s *slowQueryTable) Meta() *model.TableInfo {
return s.meta
func (vt *slowQueryTable) WritableCols() []*table.Column {
return vt.cols
}

// GetID implements table.Table GetID interface.
func (s *slowQueryTable) GetPhysicalID() int64 {
return s.meta.ID
func (vt *slowQueryTable) GetPhysicalID() int64 {
return vt.meta.ID
}

// Seek implements table.Table Type interface.
func (s *slowQueryTable) Seek(ctx sessionctx.Context, h int64) (int64, bool, error) {
return 0, false, table.ErrUnsupportedOp
}

// Type implements table.Table Type interface.
func (s *slowQueryTable) Type() table.Type {
return table.VirtualTable
// Meta implements table.Table Type interface.
func (vt *slowQueryTable) Meta() *model.TableInfo {
return vt.meta
}
2 changes: 1 addition & 1 deletion infoschema/tiniub/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,6 @@ func (s *testSuite) TestTiNiuBTables(c *C) {

tk := testkit.NewTestKit(c, store)

tk.MustExec("use tiniub")
tk.MustExec("use PERFORMANCE_SCHEMA")
tk.MustQuery("select * from slow_query").Check(testkit.Rows())
}
5 changes: 5 additions & 0 deletions statistics/bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/sessionctx"
tt "github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/sqlexec"
Expand All @@ -37,6 +38,10 @@ func (h *Handle) initStatsMeta4Chunk(is infoschema.InfoSchema, tables statsCache
log.Debugf("Unknown physical ID %d in stats meta table, maybe it has been dropped", physicalID)
continue
}
if table.Type() != tt.NormalTable {
continue
}

tableInfo := table.Meta()
newHistColl := HistColl{
PhysicalID: physicalID,
Expand Down
4 changes: 4 additions & 0 deletions statistics/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/tikv/oracle"
tt "github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/sqlexec"
log "github.com/sirupsen/logrus"
Expand Down Expand Up @@ -711,6 +712,9 @@ func (h *Handle) HandleAutoAnalyze(is infoschema.InfoSchema) error {
for _, db := range dbs {
tbls := is.SchemaTables(model.NewCIStr(db))
for _, tbl := range tbls {
if tbl.Type() != tt.NormalTable {
continue
}
tblInfo := tbl.Meta()
pi := tblInfo.GetPartitionInfo()
tblName := "`" + db + "`.`" + tblInfo.Name.O + "`"
Expand Down

0 comments on commit 345e067

Please sign in to comment.