diff --git a/infoschema/perfschema/tables.go b/infoschema/perfschema/tables.go index e0712aad7514f..a8dd09f2f72f2 100644 --- a/infoschema/perfschema/tables.go +++ b/infoschema/perfschema/tables.go @@ -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 } diff --git a/infoschema/tiniub/const.go b/infoschema/tiniub/const.go deleted file mode 100644 index 5c0b332637852..0000000000000 --- a/infoschema/tiniub/const.go +++ /dev/null @@ -1,28 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tiniub - -const tableSlowQuery = "CREATE TABLE if not exists slow_query (" + - "`SQL` VARCHAR(4096)," + - "`START` TIMESTAMP," + - "`DURATION` TIME," + - "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);" diff --git a/infoschema/tiniub/init.go b/infoschema/tiniub/init.go deleted file mode 100644 index 5212068246e9a..0000000000000 --- a/infoschema/tiniub/init.go +++ /dev/null @@ -1,59 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tiniub - -import ( - "sync" - - "github.com/pingcap/parser" - "github.com/pingcap/parser/ast" - "github.com/pingcap/parser/model" - "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/ddl" - "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/meta/autoid" -) - -var once sync.Once - -// Init initializes the TiNiuB database. -func Init() { - initOnce := func() { - p := parser.New() - stmt, err := p.ParseOneStmt(tableSlowQuery, "", "") - if err != nil { - panic(err) - } - meta, err := ddl.BuildTableInfoFromAST(stmt.(*ast.CreateTableStmt)) - if err != nil { - panic(err) - } - meta.ID = autoid.GenLocalSchemaID() - for _, c := range meta.Columns { - c.ID = autoid.GenLocalSchemaID() - } - dbInfo := &model.DBInfo{ - ID: autoid.GenLocalSchemaID(), - Name: model.NewCIStr("TiNiuB"), - Charset: mysql.DefaultCharset, - Collate: mysql.DefaultCollationName, - Tables: []*model.TableInfo{meta}, - } - infoschema.RegisterVirtualTable(dbInfo, tableFromMeta) - } - if expression.EvalAstExpr != nil { - once.Do(initOnce) - } -} diff --git a/infoschema/tiniub/tables.go b/infoschema/tiniub/tables.go index 1a4ed62995a5f..bcdccd1ebdb1a 100644 --- a/infoschema/tiniub/tables.go +++ b/infoschema/tiniub/tables.go @@ -18,6 +18,8 @@ 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" @@ -25,17 +27,33 @@ import ( "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) @@ -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. @@ -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 { @@ -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 } diff --git a/infoschema/tiniub/tables_test.go b/infoschema/tiniub/tables_test.go index e807be897f884..4c34ff5a84341 100644 --- a/infoschema/tiniub/tables_test.go +++ b/infoschema/tiniub/tables_test.go @@ -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()) } diff --git a/statistics/bootstrap.go b/statistics/bootstrap.go index 0cda1a965fbc0..f04fd1f8f1c6b 100644 --- a/statistics/bootstrap.go +++ b/statistics/bootstrap.go @@ -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" @@ -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, diff --git a/statistics/update.go b/statistics/update.go index b65c4cdbb7734..fb351bded91b2 100644 --- a/statistics/update.go +++ b/statistics/update.go @@ -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" @@ -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 + "`"