Skip to content

Commit

Permalink
*: Implement perfschema with MemoryTable
Browse files Browse the repository at this point in the history
  • Loading branch information
shenli committed Feb 29, 2016
1 parent 73d6ce5 commit 64912c8
Show file tree
Hide file tree
Showing 17 changed files with 257 additions and 868 deletions.
38 changes: 15 additions & 23 deletions executor/compiler.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,34 +35,26 @@ type Compiler struct {
// a plan, and we wrap the plan in an adapter as stmt.Statement.
// If it is not supported, the node will be converted to old statement.
func (c *Compiler) Compile(ctx context.Context, node ast.StmtNode) (stmt.Statement, error) {
if optimizer.IsSupported(node) {
ast.SetFlag(node)
ast.SetFlag(node)

is := sessionctx.GetDomain(ctx).InfoSchema()
if err := optimizer.Preprocess(node, is, ctx); err != nil {
return nil, errors.Trace(err)
}
// Validate should be after NameResolve.
if err := optimizer.Validate(node, false); err != nil {
return nil, errors.Trace(err)
}
sb := NewSubQueryBuilder(is)
p, err := optimizer.Optimize(ctx, node, sb)
if err != nil {
return nil, errors.Trace(err)
}
sa := &statementAdapter{
is: is,
plan: p,
}
return sa, nil
is := sessionctx.GetDomain(ctx).InfoSchema()
if err := optimizer.Preprocess(node, is, ctx); err != nil {
return nil, errors.Trace(err)
}
// Validate should be after NameResolve.
if err := optimizer.Validate(node, false); err != nil {
return nil, errors.Trace(err)
}
c.converter = &converter.Converter{}
s, err := c.converter.Convert(node)
sb := NewSubQueryBuilder(is)
p, err := optimizer.Optimize(ctx, node, sb)
if err != nil {
return nil, errors.Trace(err)
}
return s, nil
sa := &statementAdapter{
is: is,
plan: p,
}
return sa, nil
}

// NewSubQueryBuilder builds and returns a new SubQuery builder.
Expand Down
60 changes: 24 additions & 36 deletions executor/prepared.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ import (
"github.com/juju/errors"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/executor/converter"
"github.com/pingcap/tidb/field"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/optimizer"
Expand Down Expand Up @@ -143,15 +142,13 @@ func (e *PrepareExec) DoPrepare() {
SchemaVersion: e.IS.SchemaMetaVersion(),
}

if optimizer.IsSupported(stmt) {
err := optimizer.Prepare(e.IS, e.Ctx, stmt)
if err != nil {
e.Err = errors.Trace(err)
return
}
if resultSetNode, ok := stmt.(ast.ResultSetNode); ok {
e.ResultFields = convertResultFields(resultSetNode.GetResultFields())
}
err = optimizer.Prepare(e.IS, e.Ctx, stmt)
if err != nil {
e.Err = errors.Trace(err)
return
}
if resultSetNode, ok := stmt.(ast.ResultSetNode); ok {
e.ResultFields = convertResultFields(resultSetNode.GetResultFields())
}

if e.ID == 0 {
Expand Down Expand Up @@ -217,35 +214,26 @@ func (e *ExecuteExec) Build() error {
prepared.Params[i].SetValue(val)
}

if optimizer.IsSupported(prepared.Stmt) {
if prepared.SchemaVersion != e.IS.SchemaMetaVersion() {
// If the schema version has changed we need to prepare it again,
// if this time it failed, the real reason for the error is schema changed.
err := optimizer.Prepare(e.IS, e.Ctx, prepared.Stmt)
if err != nil {
return ErrSchemaChanged.Gen("Schema change casued error: %s", err.Error())
}
prepared.SchemaVersion = e.IS.SchemaMetaVersion()
}
sb := &subqueryBuilder{is: e.IS}
plan, err := optimizer.Optimize(e.Ctx, prepared.Stmt, sb)
if prepared.SchemaVersion != e.IS.SchemaMetaVersion() {
// If the schema version has changed we need to prepare it again,
// if this time it failed, the real reason for the error is schema changed.
err := optimizer.Prepare(e.IS, e.Ctx, prepared.Stmt)
if err != nil {
return errors.Trace(err)
}
b := newExecutorBuilder(e.Ctx, e.IS)
stmtExec := b.build(plan)
if b.err != nil {
return errors.Trace(b.err)
}
e.StmtExec = stmtExec
} else {
conv := converter.Converter{}
oStmt, err := conv.Convert(prepared.Stmt)
if err != nil {
return errors.Trace(err)
return ErrSchemaChanged.Gen("Schema change casued error: %s", err.Error())
}
e.OldStmt = oStmt
prepared.SchemaVersion = e.IS.SchemaMetaVersion()
}
sb := &subqueryBuilder{is: e.IS}
plan, err := optimizer.Optimize(e.Ctx, prepared.Stmt, sb)
if err != nil {
return errors.Trace(err)
}
b := newExecutorBuilder(e.Ctx, e.IS)
stmtExec := b.build(plan)
if b.err != nil {
return errors.Trace(b.err)
}
e.StmtExec = stmtExec
return nil
}

Expand Down
26 changes: 26 additions & 0 deletions infoschema/infoschema.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/perfschema"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/terror"
// import table implementation to init table.TableFromMeta
Expand Down Expand Up @@ -204,9 +205,14 @@ func NewHandle(store kv.Storage) *Handle {
}
// init memory tables
initMemoryTables(store)
initPerfSchema(store)
return h
}

func initPerfSchema(store kv.Storage) {
perfHandle = perfschema.NewPerfHandle(store)
}

func genGlobalID(store kv.Storage) (int64, error) {
var globalID int64
err := kv.RunInNewTxn(store, true, func(txn kv.Transaction) error {
Expand All @@ -230,6 +236,8 @@ var (
defTbl table.Table
profilingTbl table.Table
nameToTable map[string]table.Table

perfHandle perfschema.PerfSchema
)

func setColumnID(meta *model.TableInfo, store kv.Storage) error {
Expand Down Expand Up @@ -373,6 +381,24 @@ func (h *Handle) Set(newInfo []*model.DBInfo, schemaMetaVersion int64) error {
info.columnNameToID[columnName{tname, c.Name.L}] = c.ID
}
}

// Add Performance_Schema
psDB := perfHandle.GetDBMeta()
info.schemaNameToID[psDB.Name.L] = psDB.ID
info.schemas[psDB.ID] = psDB
for _, t := range psDB.Tables {
tbl, ok := perfHandle.GetTable(t.Name.O)
if !ok {
return errors.Errorf("table `%s` is missing.", t.Name)
}
info.tables[t.ID] = tbl
tname := tableName{psDB.Name.L, t.Name.L}
info.tableNameToID[tname] = t.ID
for _, c := range t.Columns {
info.columns[c.ID] = c
info.columnNameToID[columnName{tname, c.Name.L}] = c.ID
}
}
// Should refill some tables in Information_Schema.
// schemata/tables/columns/statistics
dbNames := make([]string, 0, len(info.schemas))
Expand Down
15 changes: 8 additions & 7 deletions infoschema/infoschema_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/perfschema"
"github.com/pingcap/tidb/store/localstore"
"github.com/pingcap/tidb/store/localstore/goleveldb"
"github.com/pingcap/tidb/util/testutil"
Expand Down Expand Up @@ -101,13 +102,13 @@ func (*testSuite) TestT(c *C) {
is := handle.Get()

schemaNames := is.AllSchemaNames()
c.Assert(len(schemaNames), Equals, 2)
c.Assert(testutil.CompareUnorderedStringSlice(schemaNames, []string{infoschema.Name, "Test"}), IsTrue)
c.Assert(schemaNames, HasLen, 3)
c.Assert(testutil.CompareUnorderedStringSlice(schemaNames, []string{infoschema.Name, perfschema.Name, "Test"}), IsTrue)

schemas := is.AllSchemas()
c.Assert(len(schemas), Equals, 2)
c.Assert(schemas, HasLen, 3)
schemas = is.Clone()
c.Assert(len(schemas), Equals, 2)
c.Assert(schemas, HasLen, 3)

c.Assert(is.SchemaExists(dbName), IsTrue)
c.Assert(is.SchemaExists(noexist), IsFalse)
Expand Down Expand Up @@ -167,13 +168,13 @@ func (*testSuite) TestT(c *C) {

indices, ok := is.ColumnIndicesByID(colID)
c.Assert(ok, IsTrue)
c.Assert(len(indices), Equals, 1)
c.Assert(indices, HasLen, 1)

tbs := is.SchemaTables(dbName)
c.Assert(len(tbs), Equals, 1)
c.Assert(tbs, HasLen, 1)

tbs = is.SchemaTables(noexist)
c.Assert(len(tbs), Equals, 0)
c.Assert(tbs, HasLen, 0)

idx, ok := is.IndexByName(dbName, tbName, idxName)
c.Assert(ok, IsTrue)
Expand Down
46 changes: 0 additions & 46 deletions optimizer/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,15 +14,12 @@
package optimizer

import (
"strings"

"github.com/juju/errors"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/optimizer/plan"
"github.com/pingcap/tidb/perfschema"
"github.com/pingcap/tidb/terror"
)

Expand Down Expand Up @@ -61,49 +58,6 @@ func Prepare(is infoschema.InfoSchema, ctx context.Context, node ast.Node) error
return nil
}

type supportChecker struct {
unsupported bool
}

func (c *supportChecker) Enter(in ast.Node) (ast.Node, bool) {
switch x := in.(type) {
case *ast.TableName:
if strings.EqualFold(x.Schema.O, perfschema.Name) {
c.unsupported = true
}
}
return in, c.unsupported
}

func (c *supportChecker) Leave(in ast.Node) (ast.Node, bool) {
return in, !c.unsupported
}

// IsSupported checks if the node is supported to use new plan.
// We first support single table select statement without group by clause or aggregate functions.
// TODO: 1. insert/update/delete. 2. join tables. 3. subquery. 4. group by and aggregate function.
func IsSupported(node ast.Node) bool {
switch node.(type) {
case *ast.SelectStmt, *ast.PrepareStmt, *ast.ExecuteStmt, *ast.DeallocateStmt,
*ast.AdminStmt, *ast.UpdateStmt, *ast.DeleteStmt, *ast.UnionStmt, *ast.InsertStmt:
case *ast.UseStmt, *ast.SetStmt, *ast.SetCharsetStmt:
case *ast.BeginStmt, *ast.CommitStmt, *ast.RollbackStmt:
case *ast.DoStmt:
case *ast.CreateUserStmt, *ast.SetPwdStmt, *ast.GrantStmt:
case *ast.TruncateTableStmt, *ast.AlterTableStmt:
case *ast.CreateDatabaseStmt, *ast.CreateTableStmt, *ast.CreateIndexStmt:
case *ast.DropDatabaseStmt, *ast.DropIndexStmt, *ast.DropTableStmt:
case *ast.ShowStmt:
case *ast.ExplainStmt:
default:
return false
}

var checker supportChecker
node.Accept(&checker)
return !checker.unsupported
}

// Optimizer error codes.
const (
CodeOneColumn terror.ErrCode = 1
Expand Down
Loading

0 comments on commit 64912c8

Please sign in to comment.