Skip to content

Commit

Permalink
sql: make 'set tracing' just work
Browse files Browse the repository at this point in the history
Before this patch, starting tracing while tracing was ongoing, as well
as stopping tracing while it wasn't resulted in errors. Also, like all
statements, running set tracing in an aborted transaction was an error.
All these facts colluded to make tracing from a program around some code
that might result in errors hilariously frustrating - you start tracing,
then you may or may not get an error, then you want to stop tracing -
but how do you stop?

This patch makes it so that starting twice, stopping twice, or running
set tracing after an error, is a-OK.
Note that `set tracing=cluster;set tracing=cluster;set tracing=off`
results in tracing being off (there's no counter for the nesting).

Fixes cockroachdb#23558

Release note(sql): `set tracing=<mode>` became less finicky about the
current session state, so that a client can more easily trace around
statement that produce errors.
  • Loading branch information
andreimatei committed May 3, 2018
1 parent cdd0909 commit 37f7bcf
Show file tree
Hide file tree
Showing 11 changed files with 182 additions and 69 deletions.
58 changes: 58 additions & 0 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,11 @@ import (
"context"
"fmt"
"strconv"
"strings"
"time"

"github.com/pkg/errors"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/coltypes"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
Expand All @@ -30,6 +33,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
)

var errSavepointNotUsed = pgerror.NewErrorf(
Expand Down Expand Up @@ -899,6 +903,9 @@ func (ex *connExecutor) runObserverStatement(
return ex.runShowTransactionState(ctx, res)
case *tree.ShowSyntax:
return ex.runShowSyntax(ctx, sqlStmt.Statement, res)
case *tree.SetTracing:
ex.runSetTracing(ctx, sqlStmt, res)
return nil
default:
res.SetError(pgerror.NewErrorf(pgerror.CodeInternalError,
"programming error: unrecognized observer statement type %T", stmt.AST))
Expand Down Expand Up @@ -941,6 +948,57 @@ func (ex *connExecutor) runShowTransactionState(
return res.AddRow(ctx, tree.Datums{tree.NewDString(state)})
}

func (ex *connExecutor) runSetTracing(
ctx context.Context, n *tree.SetTracing, res RestrictedCommandResult,
) {
if len(n.Values) == 0 {
res.SetError(fmt.Errorf("set tracing missing argument"))
return
}

modes := make([]string, len(n.Values))
for i, v := range n.Values {
v = unresolvedNameToStrVal(v)
strVal, ok := v.(*tree.StrVal)
if !ok {
res.SetError(fmt.Errorf("expected string for set tracing argument, not %T", v))
return
}
modes[i] = strVal.RawString()
}

if err := ex.enableTracing(modes); err != nil {
res.SetError(err)
}
}

func (ex *connExecutor) enableTracing(modes []string) error {
traceKV := false
recordingType := tracing.SnowballRecording
enableMode := true

for _, s := range modes {
switch strings.ToLower(s) {
case "on":
enableMode = true
case "off":
enableMode = false
case "kv":
traceKV = true
case "local":
recordingType = tracing.SingleNodeRecording
case "cluster":
recordingType = tracing.SnowballRecording
default:
return errors.Errorf("set tracing: unknown mode %q", s)
}
}
if !enableMode {
return stopTracing(&ex.dataMutator)
}
return ex.dataMutator.StartSessionTracing(recordingType, traceKV)
}

// addActiveQuery adds a running query to the list of running queries.
//
// It returns a cleanup function that needs to be run when the query is no
Expand Down
44 changes: 33 additions & 11 deletions pkg/sql/logictest/testdata/logic_test/show_trace
Original file line number Diff line number Diff line change
Expand Up @@ -12,18 +12,15 @@ SET tracing = on; BEGIN; SELECT 1; COMMIT; SELECT 2; SET tracing = off;
query ITT
SELECT span, message, operation FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%SPAN START%' OR message LIKE '%pos%executing%';
----
0 === SPAN START: sql txn === sql txn
1 === SPAN START: session recording === session recording
1 [NoTxn pos:5] executing ExecStmt: BEGIN TRANSACTION session recording
0 === SPAN START: session recording === session recording
0 [NoTxn pos:5] executing ExecStmt: BEGIN TRANSACTION session recording
1 === SPAN START: sql txn === sql txn
1 [Open pos:6] executing ExecStmt: SELECT 1 sql txn
1 [Open pos:7] executing ExecStmt: COMMIT TRANSACTION sql txn
0 [NoTxn pos:8] executing ExecStmt: SELECT 2 session recording
2 === SPAN START: sql txn === sql txn
2 [Open pos:6] executing ExecStmt: SELECT 1 sql txn
2 [Open pos:7] executing ExecStmt: COMMIT TRANSACTION sql txn
1 [NoTxn pos:8] executing ExecStmt: SELECT 2 session recording
3 === SPAN START: sql txn === sql txn
3 [Open pos:8] executing ExecStmt: SELECT 2 sql txn
1 [NoTxn pos:9] executing ExecStmt: SET tracing = off session recording
4 === SPAN START: sql txn === sql txn
4 [Open pos:9] executing ExecStmt: SET tracing = off sql txn
2 [Open pos:8] executing ExecStmt: SELECT 2 sql txn
0 [NoTxn pos:9] executing ExecStmt: SET TRACING = off session recording

# Same, with SHOW TRACE FOR.
# This also tests that sub-spans are reported properly.
Expand Down Expand Up @@ -827,3 +824,28 @@ query T
SELECT message FROM [ SHOW TRACE FOR SELECT * FROM t.enginestats ] WHERE message LIKE '%InternalDelete%'
----
engine stats: {InternalDeleteSkippedCount:0 TimeBoundNumSSTs:0}

# Check that we can run set tracing regardless of the current tracing state.
# This is convenient; sometimes it's unclear, for example, if you previously
# stopped tracing or not, so issuing a set tracing=off should just work.
subtest idempotent

statement ok
SET tracing = on; SET tracing = on;

statement ok
SET tracing = off; SET tracing = off;

# Check that we can run set tracing in the aborted state (this is implemented by
# making set tracing an ObserverStmt). This is very convenient for clients that
# start tracing, then might get an error, then want to stop tracing.
subtest aborted_txn

query error pq: foo
BEGIN; SELECT crdb_internal.force_error('', 'foo')

statement ok
SET tracing = off

statement ok
ROLLBACK
1 change: 1 addition & 0 deletions pkg/sql/parser/parse_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -785,6 +785,7 @@ func TestParse(t *testing.T) {
{`SET TRANSACTION PRIORITY NORMAL`},
{`SET TRANSACTION PRIORITY HIGH`},
{`SET TRANSACTION ISOLATION LEVEL SNAPSHOT, PRIORITY HIGH`},
{`SET TRACING = off`},
{`SET SESSION CHARACTERISTICS AS TRANSACTION ISOLATION LEVEL SERIALIZABLE`},
{`SET SESSION CHARACTERISTICS AS TRANSACTION ISOLATION LEVEL SNAPSHOT`},
{`SET CLUSTER SETTING a = 3`},
Expand Down
19 changes: 18 additions & 1 deletion pkg/sql/parser/sql.y
Original file line number Diff line number Diff line change
Expand Up @@ -525,6 +525,7 @@ func newNameFromStr(s string) *tree.Name {
%token <str> TABLE TABLES TEMP TEMPLATE TEMPORARY TESTING_RANGES EXPERIMENTAL_RANGES TESTING_RELOCATE EXPERIMENTAL_RELOCATE TEXT THAN THEN
%token <str> TIME TIMETZ TIMESTAMP TIMESTAMPTZ TO TRAILING TRACE TRANSACTION TREAT TRIM TRUE
%token <str> TRUNCATE TYPE
%token <str> TRACING

%token <str> UNBOUNDED UNCOMMITTED UNION UNIQUE UNKNOWN
%token <str> UPDATE UPSERT USE USER USERS USING UUID
Expand Down Expand Up @@ -2516,6 +2517,7 @@ set_exprs_internal:
// SET [SESSION] <var> { TO | = } <values...>
// SET [SESSION] TIME ZONE <tz>
// SET [SESSION] CHARACTERISTICS AS TRANSACTION ISOLATION LEVEL { SNAPSHOT | SERIALIZABLE }
// SET [SESSION] TRACING { TO | = } { off | cluster | on | kv | local } [,...]
//
// %SeeAlso: SHOW SESSION, RESET, DISCARD, SHOW, SET CLUSTER SETTING, SET TRANSACTION,
// WEBDOCS/set-vars.html
Expand Down Expand Up @@ -2564,8 +2566,23 @@ generic_set:
}
| var_name '=' var_list
{
$$.val = &tree.SetVar{Name: strings.Join($1.strs(), "."), Values: $3.exprs()}
if $1.strs()[0] == "tracing" {
$$.val = &tree.SetTracing{Values: $3.exprs()}
} else {
$$.val = &tree.SetVar{Name: strings.Join($1.strs(), "."), Values: $3.exprs()}
}
}
/*
* | TRACING '=' var_list
{
$$.val = &tree.SetTracing{Values: $3.exprs()}
}
| TRACING TO var_list
{
$$.val = &tree.SetTracing{Values: $3.exprs()}
}
*/


set_rest_more:
// Generic SET syntaxes:
Expand Down
11 changes: 11 additions & 0 deletions pkg/sql/sem/tree/set.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,3 +82,14 @@ func (node *SetSessionCharacteristics) Format(ctx *FmtCtx) {
ctx.WriteString("SET SESSION CHARACTERISTICS AS TRANSACTION")
node.Modes.Format(ctx)
}

// SetTracing represents a SET TRACING statement.
type SetTracing struct {
Values Exprs
}

// Format implements the NodeFormatter interface.
func (node *SetTracing) Format(ctx *FmtCtx) {
ctx.WriteString("SET TRACING = ")
ctx.FormatNode(&node.Values)
}
28 changes: 23 additions & 5 deletions pkg/sql/sem/tree/stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,11 +151,17 @@ func (l *StatementList) Format(ctx *FmtCtx) {
}
}

// ObserverStatement is a marker interface to be implemented by
// statements which are always valid and do not modify neither the
// session state, the txn state or the database. They are to be
// allowed at any point, including when a current transaction is in a
// transient state like retry_wait.
// ObserverStatement is a marker interface for statements which are allowed to
// run regardless of the current transaction state: statements other than
// rollback are generally rejected if the session is in a failed transaction
// state, but it's convenient to allow some statements (e.g. "show syntax; set
// tracing").
// Such statements are not expected to modify the database, the transaction or
// session state (other than special cases such as enabling/disabling tracing).
//
// These statements short-circuit the regular execution - they don't get planned
// (there's no corresponding planNodes). The connExecutor recognizes them and
// handles them.
type ObserverStatement interface {
observerStatement()
}
Expand Down Expand Up @@ -584,6 +590,17 @@ func (*SetTransaction) StatementTag() string { return "SET TRANSACTION" }

func (*SetTransaction) hiddenFromStats() {}

// StatementType implements the Statement interface.
func (*SetTracing) StatementType() StatementType { return Ack }

// StatementTag returns a short string identifying the type of statement.
func (*SetTracing) StatementTag() string { return "SET TRACING" }

func (*SetTracing) hiddenFromStats() {}

// observerStatement implements the ObserverStatement interface.
func (*SetTracing) observerStatement() {}

// StatementType implements the Statement interface.
func (*SetZoneConfig) StatementType() StatementType { return RowsAffected }

Expand Down Expand Up @@ -933,6 +950,7 @@ func (n *SetClusterSetting) String() string { return AsString(n) }
func (n *SetZoneConfig) String() string { return AsString(n) }
func (n *SetSessionCharacteristics) String() string { return AsString(n) }
func (n *SetTransaction) String() string { return AsString(n) }
func (n *SetTracing) String() string { return AsString(n) }
func (n *SetVar) String() string { return AsString(n) }
func (n *ShowBackup) String() string { return AsString(n) }
func (n *ShowClusterSetting) String() string { return AsString(n) }
Expand Down
22 changes: 22 additions & 0 deletions pkg/sql/sem/tree/walk.go
Original file line number Diff line number Diff line change
Expand Up @@ -1113,6 +1113,28 @@ func (stmt *SetVar) WalkStmt(v Visitor) Statement {
return ret
}

// CopyNode makes a copy of this Statement without recursing in any child Statements.
func (stmt *SetTracing) CopyNode() *SetTracing {
stmtCopy := *stmt
stmtCopy.Values = append(Exprs(nil), stmt.Values...)
return &stmtCopy
}

// WalkStmt is part of the WalkableStmt interface.
func (stmt *SetTracing) WalkStmt(v Visitor) Statement {
ret := stmt
for i, expr := range stmt.Values {
e, changed := WalkExpr(v, expr)
if changed {
if ret == stmt {
ret = stmt.CopyNode()
}
ret.Values[i] = e
}
}
return ret
}

// CopyNode makes a copy of this Statement without recursing in any child Statements.
func (stmt *SetClusterSetting) CopyNode() *SetClusterSetting {
stmtCopy := *stmt
Expand Down
6 changes: 4 additions & 2 deletions pkg/sql/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1564,7 +1564,8 @@ func (st *SessionTracing) getRecording() ([]traceRow, error) {
// are per-row.
func (st *SessionTracing) StartTracing(recType tracing.RecordingType, kvTracingEnabled bool) error {
if st.enabled {
return errors.Errorf("already tracing")
// We're already tracing. No-op.
return nil
}

// If we're inside a transaction, start recording on the txn span.
Expand Down Expand Up @@ -1607,7 +1608,8 @@ func (st *SessionTracing) StartTracing(recType tracing.RecordingType, kvTracingE
// An error is returned if tracing was not active.
func (st *SessionTracing) StopTracing() error {
if !st.enabled {
return errors.Errorf("not tracing")
// We're not currently tracing. No-op.
return nil
}
st.enabled = false

Expand Down
6 changes: 1 addition & 5 deletions pkg/sql/set_cluster_setting.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,11 +58,7 @@ func (p *planner) SetClusterSetting(
// For DEFAULT, let the value reference be nil. That's a RESET in disguise.
if _, ok := n.Value.(tree.DefaultVal); !ok {
expr := n.Value
if s, ok := expr.(*tree.UnresolvedName); ok {
// Special rule for SET: because SET doesn't apply in the context
// of a table, SET ... = IDENT really means SET ... = 'IDENT'.
expr = tree.NewStrVal(tree.AsStringWithFlags(s, tree.FmtBareIdentifiers))
}
expr = unresolvedNameToStrVal(expr)

var requiredType types.T
switch setting.(type) {
Expand Down
15 changes: 10 additions & 5 deletions pkg/sql/set_var.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,11 +63,7 @@ func (p *planner) SetVar(ctx context.Context, n *tree.SetVar) (planNode, error)
if !isReset {
typedValues = make([]tree.TypedExpr, len(n.Values))
for i, expr := range n.Values {
// Special rule for SET: because SET doesn't apply in the context
// of a table, SET ... = IDENT really means SET ... = 'IDENT'.
if s, ok := expr.(*tree.UnresolvedName); ok {
expr = tree.NewStrVal(tree.AsStringWithFlags(s, tree.FmtBareIdentifiers))
}
expr = unresolvedNameToStrVal(expr)

var dummyHelper tree.IndexedVarHelper
typedValue, err := p.analyzeExpr(
Expand Down Expand Up @@ -98,6 +94,15 @@ func (p *planner) SetVar(ctx context.Context, n *tree.SetVar) (planNode, error)
return &setVarNode{v: v, typedValues: typedValues}, nil
}

// Special rule for SET: because SET doesn't apply in the context
// of a table, SET ... = IDENT really means SET ... = 'IDENT'.
func unresolvedNameToStrVal(expr tree.Expr) tree.Expr {
if s, ok := expr.(*tree.UnresolvedName); ok {
return tree.NewStrVal(tree.AsStringWithFlags(s, tree.FmtBareIdentifiers))
}
return expr
}

func (n *setVarNode) startExec(params runParams) error {
if n.typedValues != nil {
for i, v := range n.typedValues {
Expand Down
41 changes: 1 addition & 40 deletions pkg/sql/vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -582,49 +582,10 @@ var varGen = map[string]sessionVar{
}
return stopTracing(m)
},
Set: func(
_ context.Context, m *sessionDataMutator,
evalCtx *extendedEvalContext, values []tree.TypedExpr,
) error {
return enableTracing(&evalCtx.EvalContext, m, values)
},
// Setting is done by the SetTracing statement.
},
}

func enableTracing(
evalCtx *tree.EvalContext, m *sessionDataMutator, values []tree.TypedExpr,
) error {
traceKV := false
recordingType := tracing.SnowballRecording
enableMode := true

for _, v := range values {
s, err := datumAsString(evalCtx, "trace", v)
if err != nil {
return err
}

switch strings.ToLower(s) {
case "on":
enableMode = true
case "off":
enableMode = false
case "kv":
traceKV = true
case "local":
recordingType = tracing.SingleNodeRecording
case "cluster":
recordingType = tracing.SnowballRecording
default:
return errors.Errorf("set tracing: unknown mode %q", s)
}
}
if !enableMode {
return stopTracing(m)
}
return m.StartSessionTracing(recordingType, traceKV)
}

func stopTracing(m *sessionDataMutator) error {
if err := m.StopSessionTracing(); err != nil {
return errors.Wrapf(err, "error stopping tracing")
Expand Down

0 comments on commit 37f7bcf

Please sign in to comment.