Skip to content

Commit

Permalink
planner,executor: support admin show slow command (#7785)
Browse files Browse the repository at this point in the history
  • Loading branch information
tiancaiamao committed Sep 26, 2018
1 parent af3b782 commit 081920d
Show file tree
Hide file tree
Showing 6 changed files with 126 additions and 1 deletion.
6 changes: 6 additions & 0 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -381,6 +381,12 @@ func (a *ExecStmt) logSlowQuery(txnTS uint64, succ bool) {
if user != nil {
userString = user.String()
}
if len(tableIDs) > 10 {
tableIDs = tableIDs[10 : len(tableIDs)-1] // Remove "table_ids:" and the last ","
}
if len(indexIDs) > 10 {
indexIDs = indexIDs[10 : len(indexIDs)-1] // Remove "index_ids:" and the last ","
}
domain.GetDomain(a.Ctx).LogSlowQuery(&domain.SlowQueryInfo{
SQL: sql,
Start: a.startTime,
Expand Down
10 changes: 10 additions & 0 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,8 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor {
return b.buildShowDDLJobs(v)
case *plannercore.ShowDDLJobQueries:
return b.buildShowDDLJobQueries(v)
case *plannercore.ShowSlow:
return b.buildShowSlow(v)
case *plannercore.Show:
return b.buildShow(v)
case *plannercore.Simple:
Expand Down Expand Up @@ -226,6 +228,14 @@ func (b *executorBuilder) buildShowDDLJobQueries(v *plannercore.ShowDDLJobQuerie
return e
}

func (b *executorBuilder) buildShowSlow(v *plannercore.ShowSlow) Executor {
e := &ShowSlowExec{
baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()),
ShowSlow: v.ShowSlow,
}
return e
}

func (b *executorBuilder) buildCheckIndex(v *plannercore.CheckIndex) Executor {
readerExec, err := buildNoRangeIndexLookUpReader(b, v.IndexLookUpReader)
if err != nil {
Expand Down
63 changes: 63 additions & 0 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,12 @@ import (

"github.com/cznic/mathutil"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/table"
Expand Down Expand Up @@ -472,6 +474,67 @@ func (e *CheckIndexExec) Next(ctx context.Context, chk *chunk.Chunk) error {
return nil
}

// ShowSlowExec represents the executor of showing the slow queries.
// It is build from the "admin show slow" statement:
// admin show slow top [internal | all] N
// admin show slow recent N
type ShowSlowExec struct {
baseExecutor

ShowSlow *ast.ShowSlow
result []*domain.SlowQueryInfo
cursor int
}

// Open implements the Executor Open interface.
func (e *ShowSlowExec) Open(ctx context.Context) error {
if err := e.baseExecutor.Open(ctx); err != nil {
return errors.Trace(err)
}

dom := domain.GetDomain(e.ctx)
e.result = dom.ShowSlowQuery(e.ShowSlow)
return nil
}

// Next implements the Executor Next interface.
func (e *ShowSlowExec) Next(ctx context.Context, chk *chunk.Chunk) error {
chk.Reset()
if e.cursor >= len(e.result) {
return nil
}

for e.cursor < len(e.result) && chk.NumRows() < e.maxChunkSize {
slow := e.result[e.cursor]
chk.AppendString(0, slow.SQL)
chk.AppendTime(1, types.Time{
Time: types.FromGoTime(slow.Start),
Type: mysql.TypeTimestamp,
Fsp: types.MaxFsp,
})
chk.AppendDuration(2, types.Duration{Duration: slow.Duration, Fsp: types.MaxFsp})
chk.AppendString(3, slow.Detail.String())
if slow.Succ {
chk.AppendInt64(4, 1)
} else {
chk.AppendInt64(4, 0)
}
chk.AppendUint64(5, slow.ConnID)
chk.AppendUint64(6, slow.TxnTS)
chk.AppendString(7, slow.User)
chk.AppendString(8, slow.DB)
chk.AppendString(9, slow.TableIDs)
chk.AppendString(10, slow.IndexIDs)
if slow.Internal {
chk.AppendInt64(11, 0)
} else {
chk.AppendInt64(11, 1)
}
e.cursor++
}
return nil
}

// SelectLockExec represents a select lock executor.
// It is built from the "SELECT .. FOR UPDATE" or the "SELECT .. LOCK IN SHARE MODE" statement.
// For "SELECT .. FOR UPDATE" statement, it locks every row key from source Executor.
Expand Down
15 changes: 14 additions & 1 deletion executor/show_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -646,11 +646,24 @@ func (s *testSuite) TestShowTableStatus(c *C) {
partition by range(a)
( partition p0 values less than (10),
partition p1 values less than (20),
partition p2 values less than (maxvalue)
partition p2 values less than (maxvalue)
);`)
rs, err = tk.Exec("show table status from test like 'tp';")
c.Assert(errors.ErrorStack(err), Equals, "")
rows, err = session.GetRows4Test(context.Background(), tk.Se, rs)
c.Assert(errors.ErrorStack(err), Equals, "")
c.Assert(rows[0].GetString(16), Equals, "partitioned")
}

func (s *testSuite) TestShowSlow(c *C) {
tk := testkit.NewTestKit(c, s.store)
// The test result is volatile, because
// 1. Slow queries is stored in domain, which may be affected by other tests.
// 2. Collecting slow queries is a asynchronous process, check immediately may not get the expected result.
// 3. Make slow query like "select sleep(1)" would slow the CI.
// So, we just cover the code but do not check the result.
tk.MustQuery(`admin show slow recent 3`)
tk.MustQuery(`admin show slow top 3`)
tk.MustQuery(`admin show slow top internal 3`)
tk.MustQuery(`admin show slow top all 3`)
}
7 changes: 7 additions & 0 deletions planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,13 @@ type ShowDDLJobs struct {
JobNumber int64
}

// ShowSlow is for showing slow queries.
type ShowSlow struct {
baseSchemaProducer

*ast.ShowSlow
}

// ShowDDLJobQueries is for showing DDL job queries sql.
type ShowDDLJobQueries struct {
baseSchemaProducer
Expand Down
26 changes: 26 additions & 0 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -501,6 +501,10 @@ func (b *planBuilder) buildAdmin(as *ast.AdminStmt) (Plan, error) {
p := &ShowDDLJobQueries{JobIDs: as.JobIDs}
p.SetSchema(buildShowDDLJobQueriesFields())
ret = p
case ast.AdminShowSlow:
p := &ShowSlow{ShowSlow: as.ShowSlow}
p.SetSchema(buildShowSlowSchema())
ret = p
default:
return nil, ErrUnsupportedType.GenWithStack("Unsupported ast.AdminStmt(%T) for buildAdmin", as)
}
Expand Down Expand Up @@ -746,6 +750,28 @@ func buildShowDDLJobQueriesFields() *expression.Schema {
return schema
}

func buildShowSlowSchema() *expression.Schema {
longlongSize, _ := mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeLonglong)
tinySize, _ := mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeTiny)
timestampSize, _ := mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeTimestamp)
durationSize, _ := mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeDuration)

schema := expression.NewSchema(make([]*expression.Column, 0, 11)...)
schema.Append(buildColumn("", "SQL", mysql.TypeVarchar, 4096))
schema.Append(buildColumn("", "START", mysql.TypeTimestamp, timestampSize))
schema.Append(buildColumn("", "DURATION", mysql.TypeDuration, durationSize))
schema.Append(buildColumn("", "DETAILS", mysql.TypeVarchar, 256))
schema.Append(buildColumn("", "SUCC", mysql.TypeTiny, tinySize))
schema.Append(buildColumn("", "CONN_ID", mysql.TypeLonglong, longlongSize))
schema.Append(buildColumn("", "TRANSACTION_TS", mysql.TypeLonglong, longlongSize))
schema.Append(buildColumn("", "USER", mysql.TypeVarchar, 32))
schema.Append(buildColumn("", "DB", mysql.TypeVarchar, 64))
schema.Append(buildColumn("", "TABLE_IDS", mysql.TypeVarchar, 256))
schema.Append(buildColumn("", "INDEX_IDS", mysql.TypeVarchar, 256))
schema.Append(buildColumn("", "INTERNAL", mysql.TypeTiny, tinySize))
return schema
}

func buildCancelDDLJobsFields() *expression.Schema {
schema := expression.NewSchema(make([]*expression.Column, 0, 2)...)
schema.Append(buildColumn("", "JOB_ID", mysql.TypeVarchar, 64))
Expand Down

0 comments on commit 081920d

Please sign in to comment.