Skip to content

Commit

Permalink
sql: add a new observer statement to show last query statistics
Browse files Browse the repository at this point in the history
Adds a new observer statement `SHOW LAST QUERY STATISTICS` which
returns service, latency, planning, and parsing timigns for the last
executed statement. By exposing these timings this PR builds a bridge
towards providing more in-depth metrics to improve timings displayed by
the CLI.

Note that this new statement should probably not be exposed to users,
and should only be used for internal use (and therefore I'm leaving
the release note blank here). This is because the CLI issues some
"under the hood" queries (eg. retriving the current db so that it can
be displayed). If a user would use this statement without insight into
these "under the hood" queries it could prove to be confusing.

Informs #49450

Release note: none
  • Loading branch information
arulajmani committed Jul 7, 2020
1 parent 0b6e118 commit 056d5c5
Show file tree
Hide file tree
Showing 12 changed files with 173 additions and 16 deletions.
1 change: 1 addition & 0 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1026,6 +1026,7 @@ type connExecutor struct {
// safe to use when a statement is not being parallelized. It must be reset
// before using.
planner planner

// phaseTimes tracks session- and transaction-level phase times. It is
// copied-by-value when resetting statsCollector before executing each
// statement.
Expand Down
25 changes: 24 additions & 1 deletion pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/duration"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/fsm"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -69,6 +70,7 @@ func (ex *connExecutor) execStmt(
// Run observer statements in a separate code path; their execution does not
// depend on the current transaction state.
if _, ok := stmt.AST.(tree.ObserverStatement); ok {
ex.statsCollector.reset(&ex.server.sqlStats, ex.appStats, &ex.phaseTimes)
err := ex.runObserverStatement(ctx, stmt, res)
// Note that regardless of res.Err(), these observer statements don't
// generate error events; transactions are always allowed to continue.
Expand Down Expand Up @@ -493,7 +495,6 @@ func (ex *connExecutor) execStmtInOpenState(
}
p.extendedEvalCtx.Placeholders = &p.semaCtx.Placeholders
p.extendedEvalCtx.Annotations = &p.semaCtx.Annotations
ex.phaseTimes[plannerStartExecStmt] = timeutil.Now()
p.stmt = &stmt
p.cancelChecker = sqlbase.NewCancelChecker(ctx)
p.autoCommit = os.ImplicitTxn.Get() && !ex.server.cfg.TestingKnobs.DisableAutoCommit
Expand Down Expand Up @@ -1117,6 +1118,8 @@ func (ex *connExecutor) runObserverStatement(
case *tree.SetTracing:
ex.runSetTracing(ctx, sqlStmt, res)
return nil
case *tree.ShowLastQueryStatistics:
return ex.runShowLastQueryStatistics(ctx, res)
default:
res.SetError(errors.AssertionFailedf("unrecognized observer statement type %T", stmt.AST))
return nil
Expand Down Expand Up @@ -1154,6 +1157,26 @@ func (ex *connExecutor) runShowTransactionState(
return res.AddRow(ctx, tree.Datums{tree.NewDString(state)})
}

func (ex *connExecutor) runShowLastQueryStatistics(
ctx context.Context, res RestrictedCommandResult,
) error {
res.SetColumns(ctx, sqlbase.ShowLastQueryStatisticsColumns)

phaseTimes := &ex.statsCollector.previousPhaseTimes
runLat := phaseTimes.getRunLatency().Seconds()
parseLat := phaseTimes.getParsingLatency().Seconds()
planLat := phaseTimes.getPlanningLatency().Seconds()
svcLat := phaseTimes.getServiceLatency().Seconds()

return res.AddRow(ctx,
tree.Datums{
tree.NewDInterval(duration.FromFloat64(parseLat), types.DefaultIntervalTypeMetadata),
tree.NewDInterval(duration.FromFloat64(planLat), types.DefaultIntervalTypeMetadata),
tree.NewDInterval(duration.FromFloat64(runLat), types.DefaultIntervalTypeMetadata),
tree.NewDInterval(duration.FromFloat64(svcLat), types.DefaultIntervalTypeMetadata),
})
}

func (ex *connExecutor) runSetTracing(
ctx context.Context, n *tree.SetTracing, res RestrictedCommandResult,
) {
Expand Down
67 changes: 67 additions & 0 deletions pkg/sql/conn_executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowexec"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
Expand Down Expand Up @@ -811,6 +812,72 @@ func TestTrimFlushedStatements(t *testing.T) {
require.NoError(t, tx.Commit())
}

func TestShowLastQueryStatistics(t *testing.T) {
defer leaktest.AfterTest(t)()

ctx := context.Background()
params := base.TestServerArgs{}
s, sqlConn, _ := serverutils.StartServer(t, params)
defer s.Stopper().Stop(ctx)

if _, err := sqlConn.Exec("CREATE TABLE t(a INT)"); err != nil {
t.Fatal(err)
}

rows, err := sqlConn.Query("SHOW LAST QUERY STATISTICS")
if err != nil {
t.Fatalf("show last query statistics failed: %v", err)
}
defer rows.Close()

var parseLatency string
var planLatency string
var execLatency string
var serviceLatency string

rows.Next()
if err := rows.Scan(&parseLatency, &planLatency, &execLatency, &serviceLatency); err != nil {
t.Fatalf("unexpected error while reading last query statistics: %v", err)
}

parseInterval, err := tree.ParseDInterval(parseLatency)
if err != nil {
t.Fatal(err)
}
planInterval, err := tree.ParseDInterval(planLatency)
if err != nil {
t.Fatal(err)
}
execInterval, err := tree.ParseDInterval(execLatency)
if err != nil {
t.Fatal(err)
}
serviceInterval, err := tree.ParseDInterval(serviceLatency)
if err != nil {
t.Fatal(err)
}

if parseInterval.AsFloat64() <= 0 || parseInterval.AsFloat64() > 1 {
t.Fatalf("unexpected parse latency: %v", parseInterval.AsFloat64())
}

if planInterval.AsFloat64() <= 0 || planInterval.AsFloat64() > 1 {
t.Fatalf("unexpected plan latency: %v", planInterval.AsFloat64())
}

if serviceInterval.AsFloat64() <= 0 || serviceInterval.AsFloat64() > 1 {
t.Fatalf("unexpected service latency: %v", serviceInterval.AsFloat64())
}

if execInterval.AsFloat64() <= 0 || execInterval.AsFloat64() > 1 {
t.Fatalf("unexpected execution latency: %v", execInterval.AsFloat64())
}

if rows.Next() {
t.Fatalf("unexpected number of rows returned by last query statistics: %v", err)
}
}

// dynamicRequestFilter exposes a filter method which is a
// kvserverbase.ReplicaRequestFilter but can be set dynamically.
type dynamicRequestFilter struct {
Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/delegate/delegate.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,12 @@ func TryDelegate(
case *tree.ShowTransactionStatus:
return d.delegateShowVar(&tree.ShowVar{Name: "transaction_status"})

case *tree.ShowLastQueryStatistics:
return nil, unimplemented.New(
"show last query statistics",
"cannot use SHOW LAST QUERY STATISTICS as a statement source",
)

case *tree.ShowSavepointStatus:
return nil, unimplemented.NewWithIssue(47333, "cannot use SHOW SAVEPOINT STATUS as a statement source")

Expand Down
11 changes: 8 additions & 3 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -2144,6 +2144,9 @@ type sqlStatsCollector struct {
appStats *appStats
// phaseTimes tracks session-level phase times.
phaseTimes phaseTimes
// previousPhaseTimes tracks the session-level phase times for the previous
// query. This enables the `SHOW LAST QUERY STATISTICS` observer statement.
previousPhaseTimes phaseTimes
}

// newSQLStatsCollector creates an instance of sqlStatsCollector. Note that
Expand Down Expand Up @@ -2185,9 +2188,11 @@ func (s *sqlStatsCollector) recordTransaction(txnTimeSec float64, ev txnEvent, i
}

func (s *sqlStatsCollector) reset(sqlStats *sqlStats, appStats *appStats, phaseTimes *phaseTimes) {
previousPhaseTimes := &s.phaseTimes
*s = sqlStatsCollector{
sqlStats: sqlStats,
appStats: appStats,
phaseTimes: *phaseTimes,
sqlStats: sqlStats,
appStats: appStats,
previousPhaseTimes: *previousPhaseTimes,
phaseTimes: *phaseTimes,
}
}
36 changes: 25 additions & 11 deletions pkg/sql/executor_statement_metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,27 @@ const (
// copy behavior.
type phaseTimes [sessionNumPhases]time.Time

// getServiceLatency returns the time between a query being received and the end
// of run.
func (p *phaseTimes) getServiceLatency() time.Duration {
return p[plannerEndExecStmt].Sub(p[sessionQueryReceived])
}

// getRunLatency returns the time between a query execution starting and ending.
func (p *phaseTimes) getRunLatency() time.Duration {
return p[plannerEndExecStmt].Sub(p[plannerStartExecStmt])
}

// getPlanningLatency returns the time it takes for a query to be planned.
func (p *phaseTimes) getPlanningLatency() time.Duration {
return p[plannerEndLogicalPlan].Sub(p[plannerStartLogicalPlan])
}

// getParsingLatency returns the time it takes for a query to be parsed.
func (p *phaseTimes) getParsingLatency() time.Duration {
return p[sessionEndParse].Sub(p[sessionStartParse])
}

// EngineMetrics groups a set of SQL metrics.
type EngineMetrics struct {
// The subset of SELECTs that are processed through DistSQL.
Expand Down Expand Up @@ -108,19 +129,12 @@ func (ex *connExecutor) recordStatementSummary(
) {
phaseTimes := &ex.statsCollector.phaseTimes

// Compute the run latency. This is always recorded in the
// server metrics.
runLatRaw := phaseTimes[plannerEndExecStmt].Sub(phaseTimes[plannerStartExecStmt])

// Collect the statistics.
runLatRaw := phaseTimes.getRunLatency()
runLat := runLatRaw.Seconds()

parseLat := phaseTimes[sessionEndParse].
Sub(phaseTimes[sessionStartParse]).Seconds()
planLat := phaseTimes[plannerEndLogicalPlan].
Sub(phaseTimes[plannerStartLogicalPlan]).Seconds()
// service latency: time query received to end of run
svcLatRaw := phaseTimes[plannerEndExecStmt].Sub(phaseTimes[sessionQueryReceived])
parseLat := phaseTimes.getParsingLatency().Seconds()
planLat := phaseTimes.getPlanningLatency().Seconds()
svcLatRaw := phaseTimes.getServiceLatency()
svcLat := svcLatRaw.Seconds()

// processing latency: contributing towards SQL results.
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/parser/help_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -320,6 +320,7 @@ func TestContextualHelp(t *testing.T) {
{`SHOW SYNTAX ??`, `SHOW SYNTAX`},
{`SHOW SYNTAX 'foo' ??`, `SHOW SYNTAX`},
{`SHOW SAVEPOINT STATUS ??`, `SHOW SAVEPOINT`},
{`SHOW LAST QUERY STATISTICS ??`, `SHOW LAST QUERY STATISTICS`},

{`SHOW RANGE ??`, `SHOW RANGE`},

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/parser/parse_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -572,6 +572,7 @@ func TestParse(t *testing.T) {
{`EXPLAIN SHOW TRANSACTION STATUS`},
{`SHOW SAVEPOINT STATUS`},
{`EXPLAIN SHOW SAVEPOINT STATUS`},
{`SHOW LAST QUERY STATISTICS`},

{`SHOW SYNTAX 'select 1'`},
{`EXPLAIN SHOW SYNTAX 'select 1'`},
Expand Down
15 changes: 14 additions & 1 deletion pkg/sql/parser/sql.y
Original file line number Diff line number Diff line change
Expand Up @@ -822,6 +822,7 @@ func (u *sqlSymUnion) alterTypeAddValuePlacement() *tree.AlterTypeAddValuePlacem
%type <tree.Statement> show_savepoint_stmt
%type <tree.Statement> show_stats_stmt
%type <tree.Statement> show_syntax_stmt
%type <tree.Statement> show_last_query_stats_stmt
%type <tree.Statement> show_tables_stmt
%type <tree.Statement> show_trace_stmt
%type <tree.Statement> show_transaction_stmt
Expand Down Expand Up @@ -3668,7 +3669,8 @@ zone_value:
// SHOW CREATE, SHOW DATABASES, SHOW HISTOGRAM, SHOW INDEXES, SHOW
// PARTITIONS, SHOW JOBS, SHOW QUERIES, SHOW RANGE, SHOW RANGES,
// SHOW ROLES, SHOW SCHEMAS, SHOW SEQUENCES, SHOW SESSION, SHOW SESSIONS,
// SHOW STATISTICS, SHOW SYNTAX, SHOW TABLES, SHOW TRACE SHOW TRANSACTION, SHOW USERS
// SHOW STATISTICS, SHOW SYNTAX, SHOW TABLES, SHOW TRACE SHOW TRANSACTION, SHOW USERS,
// SHOW LAST QUERY STATISTICS
show_stmt:
show_backup_stmt // EXTEND WITH HELP: SHOW BACKUP
| show_columns_stmt // EXTEND WITH HELP: SHOW COLUMNS
Expand Down Expand Up @@ -3699,6 +3701,7 @@ show_stmt:
| show_users_stmt // EXTEND WITH HELP: SHOW USERS
| show_zone_stmt
| SHOW error // SHOW HELP: SHOW
| show_last_query_stats_stmt // EXTEND WITH HELP: SHOW LAST QUERY STATISTICS

// Cursors are not yet supported by CockroachDB. CLOSE ALL is safe to no-op
// since there will be no open cursors.
Expand Down Expand Up @@ -4170,6 +4173,16 @@ show_syntax_stmt:
}
| SHOW SYNTAX error // SHOW HELP: SHOW SYNTAX

// %Help: SHOW LAST QUERY STATISTICS - display statistics for the last query issued
// %Category: Misc
// %Text: SHOW LAST QUERY STATISTICS
show_last_query_stats_stmt:
SHOW LAST QUERY STATISTICS
{
/* SKIP DOC */
$$.val = &tree.ShowLastQueryStatistics{}
}

// %Help: SHOW SAVEPOINT - display current savepoint properties
// %Category: Cfg
// %Text: SHOW SAVEPOINT STATUS
Expand Down
8 changes: 8 additions & 0 deletions pkg/sql/sem/tree/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -395,6 +395,14 @@ func (node *ShowTransactionStatus) Format(ctx *FmtCtx) {
ctx.WriteString("SHOW TRANSACTION STATUS")
}

// ShowLastQueryStatistics represents a SHOW LAST QUERY STATS statement.
type ShowLastQueryStatistics struct{}

// Format implements the NodeFormatter interface.
func (node *ShowLastQueryStatistics) Format(ctx *FmtCtx) {
ctx.WriteString("SHOW LAST QUERY STATISTICS")
}

// ShowSavepointStatus represents a SHOW SAVEPOINT STATUS statement.
type ShowSavepointStatus struct {
}
Expand Down
9 changes: 9 additions & 0 deletions pkg/sql/sem/tree/stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -811,6 +811,14 @@ func (*ShowSavepointStatus) StatementTag() string { return "SHOW SAVEPOINT STATU

func (*ShowSavepointStatus) observerStatement() {}

// StatementType implements the Statement interface.
func (*ShowLastQueryStatistics) StatementType() StatementType { return Rows }

// StatementTag returns a short string identifying the type of statement.
func (*ShowLastQueryStatistics) StatementTag() string { return "SHOW LAST QUERY STATISTICS" }

func (*ShowLastQueryStatistics) observerStatement() {}

// StatementType implements the Statement interface.
func (*ShowUsers) StatementType() StatementType { return Rows }

Expand Down Expand Up @@ -1016,6 +1024,7 @@ func (n *ShowTableStats) String() string { return AsString(n) }
func (n *ShowTables) String() string { return AsString(n) }
func (n *ShowTraceForSession) String() string { return AsString(n) }
func (n *ShowTransactionStatus) String() string { return AsString(n) }
func (n *ShowLastQueryStatistics) String() string { return AsString(n) }
func (n *ShowUsers) String() string { return AsString(n) }
func (n *ShowVar) String() string { return AsString(n) }
func (n *ShowZoneConfig) String() string { return AsString(n) }
Expand Down
9 changes: 9 additions & 0 deletions pkg/sql/sqlbase/result_columns.go
Original file line number Diff line number Diff line change
Expand Up @@ -206,6 +206,15 @@ var ShowSyntaxColumns = ResultColumns{
{Name: "message", Typ: types.String},
}

// ShowLastQueryStatisticsColumns are the columns of a
// SHOW LAST QUERY STATISTICS statement.
var ShowLastQueryStatisticsColumns = ResultColumns{
{Name: "parse_latency", Typ: types.Interval},
{Name: "plan_latency", Typ: types.Interval},
{Name: "exec_latency", Typ: types.Interval},
{Name: "service_latency", Typ: types.Interval},
}

// ShowFingerprintsColumns are the result columns of a
// SHOW EXPERIMENTAL_FINGERPRINTS statement.
var ShowFingerprintsColumns = ResultColumns{
Expand Down

0 comments on commit 056d5c5

Please sign in to comment.