Skip to content

Commit

Permalink
Merge pull request #16860 from knz/20170705-show-improvs
Browse files Browse the repository at this point in the history
sql: miscellaneous SHOW improvements (part 2)
  • Loading branch information
knz authored Jul 6, 2017
2 parents 985b5d7 + e217250 commit 4b98593
Show file tree
Hide file tree
Showing 9 changed files with 415 additions and 346 deletions.
288 changes: 288 additions & 0 deletions pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,17 +17,23 @@
package sql

import (
"bytes"
"fmt"
"sort"
"strings"
"time"

"github.com/pkg/errors"
"golang.org/x/net/context"

"github.com/cockroachdb/cockroach/pkg/build"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/jobs"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/log"
)

const crdbInternalName = "crdb_internal"
Expand All @@ -42,6 +48,13 @@ var crdbInternal = virtualSchema{
crdbInternalStmtStatsTable,
crdbInternalJobsTable,
crdbInternalSessionTraceTable,
crdbInternalClusterSettingsTable,
crdbInternalSessionVariablesTable,
crdbInternalLocalQueriesTable,
crdbInternalClusterQueriesTable,
crdbInternalLocalSessionsTable,
crdbInternalClusterSessionsTable,
crdbInternalBuiltinFunctionsTable,
},
}

Expand Down Expand Up @@ -490,3 +503,278 @@ CREATE TABLE crdb_internal.session_trace(
return nil
},
}

// crdbInternalClusterSettingsTable exposes the list of current
// cluster settings.
var crdbInternalClusterSettingsTable = virtualSchemaTable{
schema: `
CREATE TABLE crdb_internal.cluster_settings (
name STRING NOT NULL,
current_value STRING NOT NULL,
type STRING NOT NULL,
description STRING NOT NULL
);
`,
populate: func(ctx context.Context, p *planner, _ string, addRow func(...parser.Datum) error) error {
for _, k := range settings.Keys() {
setting, _ := settings.Lookup(k)
if err := addRow(
parser.NewDString(k),
parser.NewDString(setting.String()),
parser.NewDString(setting.Typ()),
parser.NewDString(setting.Description()),
); err != nil {
return err
}
}
return nil
},
}

// crdbInternalSessionVariablesTable exposes the session variables.
var crdbInternalSessionVariablesTable = virtualSchemaTable{
schema: `
CREATE TABLE crdb_internal.session_variables (
variable STRING NOT NULL,
value STRING NOT NULL
);
`,
populate: func(ctx context.Context, p *planner, _ string, addRow func(...parser.Datum) error) error {
for _, vName := range varNames {
gen := varGen[vName]
value := gen.Get(p.session)
if err := addRow(
parser.NewDString(vName),
parser.NewDString(value),
); err != nil {
return err
}
}
return nil
},
}

const queriesSchemaPattern = `
CREATE TABLE crdb_internal.%s (
node_id INT NOT NULL, -- the node on which the query is running
username STRING, -- the user running the query
start TIMESTAMP, -- the start time of the query
query STRING, -- the SQL code of the query
client_address STRING, -- the address of the client that issued the query
application_name STRING, -- the name of the application as per SET application_name
distributed BOOL, -- whether the query is running distributed
phase STRING -- the current execution phase
);
`

// crdbInternalLocalQueriesTable exposes the list of running queries
// on the current node. The results are dependent on the current user.
var crdbInternalLocalQueriesTable = virtualSchemaTable{
schema: fmt.Sprintf(queriesSchemaPattern, "node_queries"),
populate: func(ctx context.Context, p *planner, _ string, addRow func(...parser.Datum) error) error {
req := serverpb.ListSessionsRequest{Username: p.session.User}
response, err := p.session.execCfg.StatusServer.ListLocalSessions(ctx, &req)
if err != nil {
return err
}
return populateQueriesTable(ctx, addRow, response)
},
}

// crdbInternalClusterQueriesTable exposes the list of running queries
// on the entire cluster. The result is dependent on the current user.
var crdbInternalClusterQueriesTable = virtualSchemaTable{
schema: fmt.Sprintf(queriesSchemaPattern, "cluster_queries"),
populate: func(ctx context.Context, p *planner, _ string, addRow func(...parser.Datum) error) error {
req := serverpb.ListSessionsRequest{Username: p.session.User}
response, err := p.session.execCfg.StatusServer.ListSessions(ctx, &req)
if err != nil {
return err
}
return populateQueriesTable(ctx, addRow, response)
},
}

func populateQueriesTable(
ctx context.Context, addRow func(...parser.Datum) error, response *serverpb.ListSessionsResponse,
) error {
for _, session := range response.Sessions {
for _, query := range session.ActiveQueries {
isDistributedDatum := parser.DNull
phase := strings.ToLower(query.Phase.String())
if phase == "executing" {
isDistributedDatum = parser.DBoolFalse
if query.IsDistributed {
isDistributedDatum = parser.DBoolTrue
}
}
if err := addRow(
parser.NewDInt(parser.DInt(session.NodeID)),
parser.NewDString(session.Username),
parser.MakeDTimestamp(query.Start, time.Microsecond),
parser.NewDString(query.Sql),
parser.NewDString(session.ClientAddress),
parser.NewDString(session.ApplicationName),
isDistributedDatum,
parser.NewDString(phase),
); err != nil {
return err
}
}
}

for _, rpcErr := range response.Errors {
log.Warning(ctx, rpcErr.Message)
if rpcErr.NodeID != 0 {
// Add a row with this node ID, and nulls for all other columns
if err := addRow(
parser.NewDInt(parser.DInt(rpcErr.NodeID)),
parser.DNull,
parser.DNull,
parser.DNull,
parser.DNull,
parser.DNull,
parser.DNull,
parser.DNull,
); err != nil {
return err
}
}
}
return nil
}

const sessionsSchemaPattern = `
CREATE TABLE crdb_internal.%s (
node_id INT NOT NULL, -- the node on which the query is running
username STRING, -- the user running the query
client_address STRING, -- the address of the client that issued the query
application_name STRING, -- the name of the application as per SET application_name
active_queries STRING, -- the currently running queries as SQL
session_start TIMESTAMP, -- the time when the session was opened
oldest_query_start TIMESTAMP, -- the time when the oldest query in the session was started
kv_txn STRING -- the ID of the current KV transaction
);
`

// crdbInternalLocalSessionsTable exposes the list of running sessions
// on the current node. The results are dependent on the current user.
var crdbInternalLocalSessionsTable = virtualSchemaTable{
schema: fmt.Sprintf(sessionsSchemaPattern, "node_sessions"),
populate: func(ctx context.Context, p *planner, _ string, addRow func(...parser.Datum) error) error {
req := serverpb.ListSessionsRequest{Username: p.session.User}
response, err := p.session.execCfg.StatusServer.ListLocalSessions(ctx, &req)
if err != nil {
return err
}
return populateSessionsTable(ctx, addRow, response)
},
}

// crdbInternalClusterSessionsTable exposes the list of running sessions
// on the entire cluster. The result is dependent on the current user.
var crdbInternalClusterSessionsTable = virtualSchemaTable{
schema: fmt.Sprintf(sessionsSchemaPattern, "cluster_sessions"),
populate: func(ctx context.Context, p *planner, _ string, addRow func(...parser.Datum) error) error {
req := serverpb.ListSessionsRequest{Username: p.session.User}
response, err := p.session.execCfg.StatusServer.ListSessions(ctx, &req)
if err != nil {
return err
}
return populateSessionsTable(ctx, addRow, response)
},
}

func populateSessionsTable(
ctx context.Context, addRow func(...parser.Datum) error, response *serverpb.ListSessionsResponse,
) error {
for _, session := range response.Sessions {
// Generate active_queries and oldest_query_start
var activeQueries bytes.Buffer
var oldestStart time.Time
var oldestStartDatum parser.Datum

for _, query := range session.ActiveQueries {
activeQueries.WriteString(query.Sql)
activeQueries.WriteString("; ")

if oldestStart.IsZero() || query.Start.Before(oldestStart) {
oldestStart = query.Start
}
}

if oldestStart.IsZero() {
oldestStartDatum = parser.DNull
} else {
oldestStartDatum = parser.MakeDTimestamp(oldestStart, time.Microsecond)
}

kvTxnIDDatum := parser.DNull
if session.KvTxnID != nil {
kvTxnIDDatum = parser.NewDString(session.KvTxnID.String())
}

if err := addRow(
parser.NewDInt(parser.DInt(session.NodeID)),
parser.NewDString(session.Username),
parser.NewDString(session.ClientAddress),
parser.NewDString(session.ApplicationName),
parser.NewDString(activeQueries.String()),
parser.MakeDTimestamp(session.Start, time.Microsecond),
oldestStartDatum,
kvTxnIDDatum,
); err != nil {
return err
}
}

for _, rpcErr := range response.Errors {
log.Warning(ctx, rpcErr.Message)
if rpcErr.NodeID != 0 {
// Add a row with this node ID, and nulls for all other columns
if err := addRow(
parser.NewDInt(parser.DInt(rpcErr.NodeID)),
parser.DNull,
parser.DNull,
parser.DNull,
parser.DNull,
parser.DNull,
parser.DNull,
parser.DNull,
); err != nil {
return err
}
}
}

return nil
}

// crdbInternalBuiltinFunctionsTable exposes the built-in function
// metadata.
var crdbInternalBuiltinFunctionsTable = virtualSchemaTable{
schema: `
CREATE TABLE crdb_internal.builtin_functions (
function STRING NOT NULL,
signature STRING NOT NULL,
category STRING NOT NULL,
details STRING NOT NULL
);
`,
populate: func(ctx context.Context, _ *planner, _ string, addRow func(...parser.Datum) error) error {
for _, name := range parser.AllBuiltinNames {
overloads := parser.Builtins[name]
for _, f := range overloads {
if err := addRow(
parser.NewDString(name),
parser.NewDString(f.Signature()),
parser.NewDString(f.Category()),
parser.NewDString(f.Info),
); err != nil {
return err
}
}
}
return nil
},
}
34 changes: 22 additions & 12 deletions pkg/sql/logictest/testdata/logic_test/explain
Original file line number Diff line number Diff line change
Expand Up @@ -136,37 +136,47 @@ EXPLAIN SHOW TABLES
1 render
2 filter
3 values
3 size 5 columns, 52 rows
3 size 5 columns, 59 rows

query ITTT
EXPLAIN SHOW DATABASE
----
0 values
0 size 1 column, 1 row
0 render
1 filter
2 values
2 size 2 columns, 19 rows

query ITTT
EXPLAIN SHOW TIME ZONE
----
0 values
0 size 1 column, 1 row
0 render
1 filter
2 values
2 size 2 columns, 19 rows

query ITTT
EXPLAIN SHOW DEFAULT_TRANSACTION_ISOLATION
----
0 values
0 size 1 column, 1 row
0 render
1 filter
2 values
2 size 2 columns, 19 rows

query ITTT
EXPLAIN SHOW TRANSACTION ISOLATION LEVEL
----
0 values
0 size 1 column, 1 row
0 render
1 filter
2 values
2 size 2 columns, 19 rows

query ITTT
EXPLAIN SHOW TRANSACTION PRIORITY
----
0 values
0 size 1 column, 1 row
0 render
1 filter
2 values
2 size 2 columns, 19 rows

query ITTT
EXPLAIN SHOW COLUMNS FROM foo
Expand All @@ -182,7 +192,7 @@ EXPLAIN SHOW COLUMNS FROM foo
5 render
6 filter
7 values
7 size 13 columns, 459 rows
7 size 13 columns, 501 rows
5 render
6 filter
7 values
Expand Down
Loading

0 comments on commit 4b98593

Please sign in to comment.