Skip to content

Commit

Permalink
sql: add SHOW TRANSACTIONS
Browse files Browse the repository at this point in the history
Release note (sql change): add the SHOW TRANSACTIONS statement, similar
to SHOW SESSIONS and variants. It shows the currently active
transactions in the node or cluster, and some information about them.
  • Loading branch information
jordanlewis committed Aug 26, 2020
1 parent bae332d commit 4ec8fe2
Show file tree
Hide file tree
Showing 11 changed files with 105 additions and 7 deletions.
14 changes: 10 additions & 4 deletions docs/generated/http/full.md
Original file line number Diff line number Diff line change
Expand Up @@ -786,12 +786,11 @@ The underlying response type is something we're looking to get rid of.
| application_name | [string](#cockroach.server.serverpb.ListSessionsResponse-string) | | Application name specified by the client. |
| active_queries | [ActiveQuery](#cockroach.server.serverpb.ListSessionsResponse-cockroach.server.serverpb.ActiveQuery) | repeated | Queries in progress on this session. |
| start | [google.protobuf.Timestamp](#cockroach.server.serverpb.ListSessionsResponse-google.protobuf.Timestamp) | | Timestamp of session's start. |
| kv_txn_id | [bytes](#cockroach.server.serverpb.ListSessionsResponse-bytes) | | ID of the current KV transaction for this session. Nil if the session doesn't currently have a transaction. |
| active_txn | [TxnInfo](#cockroach.server.serverpb.ListSessionsResponse-cockroach.server.serverpb.TxnInfo) | | Information about the txn in progress on this session. Nil if the session doesn't currently have a transaction. |
| last_active_query | [string](#cockroach.server.serverpb.ListSessionsResponse-string) | | SQL string of the last query executed on this session. |
| id | [bytes](#cockroach.server.serverpb.ListSessionsResponse-bytes) | | ID of the session (uint128 represented as raw bytes). |
| alloc_bytes | [int64](#cockroach.server.serverpb.ListSessionsResponse-int64) | | Number of currently allocated bytes in the session memory monitor. |
| max_alloc_bytes | [int64](#cockroach.server.serverpb.ListSessionsResponse-int64) | | High water mark of allocated bytes in the session memory monitor. |
| active_txn | [TxnInfo](#cockroach.server.serverpb.ListSessionsResponse-cockroach.server.serverpb.TxnInfo) | | Information about the txn in progress on this session. Nil if the session doesn't currently have a transaction. |



Expand Down Expand Up @@ -822,6 +821,10 @@ The underlying response type is something we're looking to get rid of.
| id | [bytes](#cockroach.server.serverpb.ListSessionsResponse-bytes) | | |
| start | [google.protobuf.Timestamp](#cockroach.server.serverpb.ListSessionsResponse-google.protobuf.Timestamp) | | The start timestamp of the transaction. |
| txn_description | [string](#cockroach.server.serverpb.ListSessionsResponse-string) | | txn_description is a text description of the underlying kv.Txn, intended for troubleshooting purposes. |
| num_statements_executed | [int32](#cockroach.server.serverpb.ListSessionsResponse-int32) | | num_statements_executed is the number of statements that were executed so far on this transaction. |
| num_retries | [int32](#cockroach.server.serverpb.ListSessionsResponse-int32) | | num_retries is the number of times that this transaction was retried. |
| num_auto_retries | [int32](#cockroach.server.serverpb.ListSessionsResponse-int32) | | num_retries is the number of times that this transaction was automatically retried by the SQL executor. |
| deadline | [google.protobuf.Timestamp](#cockroach.server.serverpb.ListSessionsResponse-google.protobuf.Timestamp) | | The deadline by which the transaction must be committed. |



Expand Down Expand Up @@ -887,12 +890,11 @@ The underlying response type is something we're looking to get rid of.
| application_name | [string](#cockroach.server.serverpb.ListSessionsResponse-string) | | Application name specified by the client. |
| active_queries | [ActiveQuery](#cockroach.server.serverpb.ListSessionsResponse-cockroach.server.serverpb.ActiveQuery) | repeated | Queries in progress on this session. |
| start | [google.protobuf.Timestamp](#cockroach.server.serverpb.ListSessionsResponse-google.protobuf.Timestamp) | | Timestamp of session's start. |
| kv_txn_id | [bytes](#cockroach.server.serverpb.ListSessionsResponse-bytes) | | ID of the current KV transaction for this session. Nil if the session doesn't currently have a transaction. |
| active_txn | [TxnInfo](#cockroach.server.serverpb.ListSessionsResponse-cockroach.server.serverpb.TxnInfo) | | Information about the txn in progress on this session. Nil if the session doesn't currently have a transaction. |
| last_active_query | [string](#cockroach.server.serverpb.ListSessionsResponse-string) | | SQL string of the last query executed on this session. |
| id | [bytes](#cockroach.server.serverpb.ListSessionsResponse-bytes) | | ID of the session (uint128 represented as raw bytes). |
| alloc_bytes | [int64](#cockroach.server.serverpb.ListSessionsResponse-int64) | | Number of currently allocated bytes in the session memory monitor. |
| max_alloc_bytes | [int64](#cockroach.server.serverpb.ListSessionsResponse-int64) | | High water mark of allocated bytes in the session memory monitor. |
| active_txn | [TxnInfo](#cockroach.server.serverpb.ListSessionsResponse-cockroach.server.serverpb.TxnInfo) | | Information about the txn in progress on this session. Nil if the session doesn't currently have a transaction. |



Expand Down Expand Up @@ -923,6 +925,10 @@ The underlying response type is something we're looking to get rid of.
| id | [bytes](#cockroach.server.serverpb.ListSessionsResponse-bytes) | | |
| start | [google.protobuf.Timestamp](#cockroach.server.serverpb.ListSessionsResponse-google.protobuf.Timestamp) | | The start timestamp of the transaction. |
| txn_description | [string](#cockroach.server.serverpb.ListSessionsResponse-string) | | txn_description is a text description of the underlying kv.Txn, intended for troubleshooting purposes. |
| num_statements_executed | [int32](#cockroach.server.serverpb.ListSessionsResponse-int32) | | num_statements_executed is the number of statements that were executed so far on this transaction. |
| num_retries | [int32](#cockroach.server.serverpb.ListSessionsResponse-int32) | | num_retries is the number of times that this transaction was retried. |
| num_auto_retries | [int32](#cockroach.server.serverpb.ListSessionsResponse-int32) | | num_retries is the number of times that this transaction was automatically retried by the SQL executor. |
| deadline | [google.protobuf.Timestamp](#cockroach.server.serverpb.ListSessionsResponse-google.protobuf.Timestamp) | | The deadline by which the transaction must be committed. |



Expand Down
1 change: 1 addition & 0 deletions docs/generated/sql/bnf/show_var.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -24,5 +24,6 @@ show_stmt ::=
| show_stats_stmt
| show_tables_stmt
| show_trace_stmt
| show_transactions_stmt
| show_users_stmt
| show_zone_stmt
6 changes: 6 additions & 0 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -215,6 +215,7 @@ show_stmt ::=
| show_stats_stmt
| show_tables_stmt
| show_trace_stmt
| show_transactions_stmt
| show_users_stmt
| show_zone_stmt

Expand Down Expand Up @@ -665,6 +666,10 @@ show_trace_stmt ::=
'SHOW' opt_compact 'TRACE' 'FOR' 'SESSION'
| 'SHOW' opt_compact 'KV' 'TRACE' 'FOR' 'SESSION'

show_transactions_stmt ::=
'SHOW' opt_cluster 'TRANSACTIONS'
| 'SHOW' 'ALL' opt_cluster 'TRANSACTIONS'

show_users_stmt ::=
'SHOW' 'USERS'

Expand Down Expand Up @@ -993,6 +998,7 @@ unreserved_keyword ::=
| 'TIES'
| 'TRACE'
| 'TRANSACTION'
| 'TRANSACTIONS'
| 'TRIGGER'
| 'TRUNCATE'
| 'TRUSTED'
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1252,6 +1252,9 @@ func (ex *connExecutor) resetExtraTxnState(
if ex.extraTxnState.onTxnRestart != nil {
ex.extraTxnState.onTxnRestart()
}
ex.state.mu.Lock()
defer ex.state.mu.Unlock()
ex.state.mu.stmtCount = 0
}
// NOTE: on txnRestart we don't need to muck with the savepoints stack. It's either a
// a ROLLBACK TO SAVEPOINT that generated the event, and that statement deals with the
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/delegate/delegate.go
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,9 @@ func TryDelegate(
case *tree.ShowTables:
return d.delegateShowTables(t)

case *tree.ShowTransactions:
return d.delegateShowTransactions(t)

case *tree.ShowUsers:
return d.delegateShowRoles()

Expand Down
29 changes: 29 additions & 0 deletions pkg/sql/delegate/show_transactions.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package delegate

import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
)

func (d *delegator) delegateShowTransactions(n *tree.ShowTransactions) (tree.Statement, error) {
const query = `SELECT node_id, id AS txn_id, application_name, num_stmts, num_retries, num_auto_retries FROM `
table := `"".crdb_internal.node_transactions`
if n.Cluster {
table = `"".crdb_internal.cluster_transactions`
}
var filter string
if !n.All {
filter = " WHERE application_name NOT LIKE '" + catconstants.InternalAppNamePrefix + "%'"
}
return parse(query + table + filter)
}
3 changes: 3 additions & 0 deletions pkg/sql/parser/help_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -270,6 +270,9 @@ func TestContextualHelp(t *testing.T) {
{`SHOW SESSIONS ??`, `SHOW SESSIONS`},
{`SHOW LOCAL SESSIONS ??`, `SHOW SESSIONS`},

{`SHOW TRANSACTIONS ??`, `SHOW TRANSACTIONS`},
{`SHOW LOCAL TRANSACTIONS ??`, `SHOW TRANSACTIONS`},

{`SHOW STATISTICS ??`, `SHOW STATISTICS`},
{`SHOW STATISTICS FOR TABLE ??`, `SHOW STATISTICS`},

Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/parser/parse_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2177,6 +2177,8 @@ $function$`,

{`SHOW SESSIONS`, `SHOW CLUSTER SESSIONS`},
{`SHOW ALL SESSIONS`, `SHOW ALL CLUSTER SESSIONS`},
{`SHOW TRANSACTIONS`, `SHOW CLUSTER TRANSACTIONS`},
{`SHOW ALL TRANSACTIONS`, `SHOW ALL CLUSTER TRANSACTIONS`},
{`SHOW QUERIES`, `SHOW CLUSTER QUERIES`},
{`SHOW ALL QUERIES`, `SHOW ALL CLUSTER QUERIES`},

Expand Down
25 changes: 22 additions & 3 deletions pkg/sql/parser/sql.y
Original file line number Diff line number Diff line change
Expand Up @@ -650,7 +650,8 @@ func (u *sqlSymUnion) refreshDataOption() tree.RefreshDataOption {
%token <str> SYMMETRIC SYNTAX SYSTEM SQRT SUBSCRIPTION

%token <str> TABLE TABLES TEMP TEMPLATE TEMPORARY TENANT TESTING_RELOCATE EXPERIMENTAL_RELOCATE TEXT THEN
%token <str> TIES TIME TIMETZ TIMESTAMP TIMESTAMPTZ TO THROTTLING TRAILING TRACE TRANSACTION TREAT TRIGGER TRIM TRUE
%token <str> TIES TIME TIMETZ TIMESTAMP TIMESTAMPTZ TO THROTTLING TRAILING TRACE
%token <str> TRANSACTION TRANSACTIONS TREAT TRIGGER TRIM TRUE
%token <str> TRUNCATE TRUSTED TYPE TYPES
%token <str> TRACING

Expand Down Expand Up @@ -860,6 +861,7 @@ func (u *sqlSymUnion) refreshDataOption() tree.RefreshDataOption {
%type <tree.Statement> show_tables_stmt
%type <tree.Statement> show_trace_stmt
%type <tree.Statement> show_transaction_stmt
%type <tree.Statement> show_transactions_stmt
%type <tree.Statement> show_types_stmt
%type <tree.Statement> show_users_stmt
%type <tree.Statement> show_zone_stmt
Expand Down Expand Up @@ -4173,8 +4175,8 @@ zone_value:
// SHOW CREATE, SHOW DATABASES, SHOW ENUMS, 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 TYPES,
// SHOW USERS, SHOW LAST QUERY STATISTICS, SHOW SCHEDULES
// SHOW STATISTICS, SHOW SYNTAX, SHOW TABLES, SHOW TRACE, SHOW TRANSACTION,
// SHOW TRANSACTIONS, SHOW TYPES, SHOW USERS, SHOW LAST QUERY STATISTICS, SHOW SCHEDULES
show_stmt:
show_backup_stmt // EXTEND WITH HELP: SHOW BACKUP
| show_columns_stmt // EXTEND WITH HELP: SHOW COLUMNS
Expand Down Expand Up @@ -4205,6 +4207,7 @@ show_stmt:
| show_tables_stmt // EXTEND WITH HELP: SHOW TABLES
| show_trace_stmt // EXTEND WITH HELP: SHOW TRACE
| show_transaction_stmt // EXTEND WITH HELP: SHOW TRANSACTION
| show_transactions_stmt // EXTEND WITH HELP: SHOW TRANSACTIONS
| show_users_stmt // EXTEND WITH HELP: SHOW USERS
| show_zone_stmt
| SHOW error // SHOW HELP: SHOW
Expand Down Expand Up @@ -4729,6 +4732,21 @@ show_tables_stmt:
}
| SHOW TABLES error // SHOW HELP: SHOW TABLES

// %Help: SHOW TRANSACTIONS - list open client transactions across the cluster
// %Category: Misc
// %Text: SHOW [ALL] [CLUSTER | LOCAL] TRANSACTIONS
show_transactions_stmt:
SHOW opt_cluster TRANSACTIONS
{
$$.val = &tree.ShowTransactions{Cluster: $2.bool()}
}
| SHOW opt_cluster TRANSACTIONS error // SHOW HELP: SHOW TRANSACTIONS
| SHOW ALL opt_cluster TRANSACTIONS
{
$$.val = &tree.ShowTransactions{All: true, Cluster: $3.bool()}
}
| SHOW ALL opt_cluster TRANSACTIONS error // SHOW HELP: SHOW TRANSACTIONS

with_comment:
WITH COMMENT { $$.val = true }
| /* EMPTY */ { $$.val = false }
Expand Down Expand Up @@ -11538,6 +11556,7 @@ unreserved_keyword:
| TIES
| TRACE
| TRANSACTION
| TRANSACTIONS
| TRIGGER
| TRUNCATE
| TRUSTED
Expand Down
19 changes: 19 additions & 0 deletions pkg/sql/sem/tree/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -344,6 +344,25 @@ func (node *ShowTables) Format(ctx *FmtCtx) {
}
}

// ShowTransactions represents a SHOW TRANSACTIONS statement
type ShowTransactions struct {
All bool
Cluster bool
}

// Format implements the NodeFormatter interface.
func (node *ShowTransactions) Format(ctx *FmtCtx) {
ctx.WriteString("SHOW ")
if node.All {
ctx.WriteString("ALL ")
}
if node.Cluster {
ctx.WriteString("CLUSTER TRANSACTIONS")
} else {
ctx.WriteString("LOCAL TRANSACTIONS")
}
}

// ShowConstraints represents a SHOW CONSTRAINTS statement.
type ShowConstraints struct {
Table *UnresolvedObjectName
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/sem/tree/stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -945,6 +945,12 @@ func (*ShowTables) StatementType() StatementType { return Rows }
// StatementTag returns a short string identifying the type of statement.
func (*ShowTables) StatementTag() string { return "SHOW TABLES" }

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

// StatementTag returns a short string identifying the type of statement.
func (*ShowTransactions) StatementTag() string { return "SHOW TRANSACTIONS" }

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

Expand Down Expand Up @@ -1114,6 +1120,7 @@ func (n *ShowTables) String() string { return AsString(n) }
func (n *ShowTypes) String() string { return AsString(n) }
func (n *ShowTraceForSession) String() string { return AsString(n) }
func (n *ShowTransactionStatus) String() string { return AsString(n) }
func (n *ShowTransactions) 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) }
Expand Down

0 comments on commit 4ec8fe2

Please sign in to comment.