Skip to content

Commit

Permalink
util/tracing,sql: add builtin to set trace spans' verbosity
Browse files Browse the repository at this point in the history
Previously there was no way to change a span's verbosity via the
SQL shell. We want to be able to set a specific long-running
span's verbosity on to retrieve its recordings. This patch adds a
builtin, `crdb_internal.set_trace_verbose` that takes in a
trace ID and a bool representing verbose or not
verbose. It sets the verbosity of all spans in this trace.

Note that we would prefer to toggle individual span verbosity,
but this would require a registry of Span objects that
is not added to the 21.1 release. If this Span registry were added
in the future, we could access a Span given its span ID.

Release justification: Adds a crdb_internal tool meant for on-call
engineers, TSEs, etc to debug.

Release note (sql change): Adds a new builtin that sets the verbosity
of all spans in a given trace. Syntax:
crdb_internal.set_trace_verbose($traceID,$verbosityAsBool).
  • Loading branch information
angelapwen committed Mar 5, 2021
1 parent 9ccda03 commit d4f9b02
Show file tree
Hide file tree
Showing 9 changed files with 456 additions and 229 deletions.
2 changes: 2 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -2706,6 +2706,8 @@ SELECT * FROM crdb_internal.check_consistency(true, ‘\x02’, ‘\x04’)</p>
</span></td></tr>
<tr><td><a name="crdb_internal.round_decimal_values"></a><code>crdb_internal.round_decimal_values(val: <a href="decimal.html">decimal</a>[], scale: <a href="int.html">int</a>) &rarr; <a href="decimal.html">decimal</a>[]</code></td><td><span class="funcdesc"><p>This function is used internally to round decimal array values during mutations.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.set_trace_verbose"></a><code>crdb_internal.set_trace_verbose(trace_id: <a href="int.html">int</a>, verbosity: <a href="bool.html">bool</a>) &rarr; <a href="bool.html">bool</a></code></td><td><span class="funcdesc"><p>Returns true if root span was found and verbosity was set, false otherwise.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.set_vmodule"></a><code>crdb_internal.set_vmodule(vmodule_string: <a href="string.html">string</a>) &rarr; <a href="int.html">int</a></code></td><td><span class="funcdesc"><p>Set the equivalent of the <code>--vmodule</code> flag on the gateway node processing this request; it affords control over the logging verbosity of different files. Example syntax: <code>crdb_internal.set_vmodule('recordio=2,file=1,gfs*=3')</code>. Reset with: <code>crdb_internal.set_vmodule('')</code>. Raising the verbosity can severely affect performance.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.trace_id"></a><code>crdb_internal.trace_id() &rarr; <a href="int.html">int</a></code></td><td><span class="funcdesc"><p>Returns the current trace ID or an error if no trace is open.</p>
Expand Down
13 changes: 13 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/builtin_function
Original file line number Diff line number Diff line change
Expand Up @@ -2960,3 +2960,16 @@ SELECT * FROM crdb_internal.payloads_for_trace(0)
WHERE false
----
span_id payload_type payload_jsonb

# switch users -- this one has no permissions so expect errors
user testuser

query error insufficient privilege
SELECT * FROM crdb_internal.set_trace_verbose(0, false)

user root

query B
SELECT * FROM crdb_internal.set_trace_verbose(0, false)
WHERE false
----
60 changes: 60 additions & 0 deletions pkg/sql/sem/builtins/builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/sql/sqlliveness"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/streaming"
"github.com/cockroachdb/cockroach/pkg/util/duration"
Expand Down Expand Up @@ -3637,6 +3638,65 @@ may increase either contention or retry errors, or both.`,
},
),

// Toggles all spans of the requested trace to verbose or non-verbose.
"crdb_internal.set_trace_verbose": makeBuiltin(
tree.FunctionProperties{Category: categorySystemInfo},
tree.Overload{
Types: tree.ArgTypes{
{"trace_id", types.Int},
{"verbosity", types.Bool},
},
ReturnType: tree.FixedReturnType(types.Bool),
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
// The user must be an admin to use this builtin.
isAdmin, err := ctx.SessionAccessor.HasAdminRole(ctx.Context)
if err != nil {
return nil, err
}
if !isAdmin {
if err := checkPrivilegedUser(ctx); err != nil {
return nil, err
}
}

traceID := uint64(*(args[0].(*tree.DInt)))
verbosity := bool(*(args[1].(*tree.DBool)))

const query = `SELECT span_id
FROM crdb_internal.node_inflight_trace_spans
WHERE trace_id = $1
AND parent_span_id = 0`

ie := ctx.InternalExecutor.(sqlutil.InternalExecutor)
row, err := ie.QueryRowEx(
ctx.Ctx(),
"crdb_internal.set_trace_verbose",
ctx.Txn,
sessiondata.NoSessionDataOverride,
query,
traceID,
)
if err != nil {
return nil, err
}
if row == nil {
return tree.DBoolFalse, nil
}
rootSpanID := uint64(*row[0].(*tree.DInt))

rootSpan, found := ctx.Settings.Tracer.GetActiveSpanFromID(rootSpanID)
if !found {
return tree.DBoolFalse, nil
}

rootSpan.SetVerboseRecursively(verbosity)
return tree.DBoolTrue, nil
},
Info: "Returns true if root span was found and verbosity was set, false otherwise.",
Volatility: tree.VolatilityVolatile,
},
),

"crdb_internal.locality_value": makeBuiltin(
tree.FunctionProperties{Category: categorySystemInfo},
tree.Overload{
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/tests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ go_test(
"split_test.go",
"system_table_test.go",
"table_split_test.go",
"tracing_sql_test.go",
"virtual_table_test.go",
],
data = glob(["testdata/**"]),
Expand Down Expand Up @@ -89,6 +90,7 @@ go_test(
"//pkg/util/randutil",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"//pkg/util/tracing",
"//pkg/util/uuid",
"@com_github_cockroachdb_cockroach_go//crdb",
"@com_github_cockroachdb_datadriven//:datadriven",
Expand Down
91 changes: 91 additions & 0 deletions pkg/sql/tests/tracing_sql_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,91 @@
// Copyright 2021 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 tests

import (
"context"
"fmt"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/stretchr/testify/require"
)

func TestSetTraceSpansVerbosityBuiltin(t *testing.T) {
defer leaktest.AfterTest(t)()
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(context.Background())
r := sqlutils.MakeSQLRunner(db)

tr := s.Tracer().(*tracing.Tracer)

// Try to toggle the verbosity of a trace that doesn't exist, returns false.
// NB: Technically this could return true in the unlikely scenario that there
// is a trace with ID of 0.
r.CheckQueryResults(
t,
"SELECT * FROM crdb_internal.set_trace_verbose(0, true)",
[][]string{{`false`}},
)

root := tr.StartSpan("root", tracing.WithForceRealSpan())
defer root.Finish()
require.False(t, root.IsVerbose())

child := tr.StartSpan("root.child", tracing.WithParentAndAutoCollection(root))
defer child.Finish()
require.False(t, child.IsVerbose())

childChild := tr.StartSpan("root.child.child", tracing.WithParentAndAutoCollection(child))
defer childChild.Finish()
require.False(t, childChild.IsVerbose())

// Toggle the trace's verbosity and confirm all spans are verbose.
traceID := root.TraceID()
query := fmt.Sprintf(
"SELECT * FROM crdb_internal.set_trace_verbose(%d, true)",
traceID,
)
r.CheckQueryResults(
t,
query,
[][]string{{`true`}},
)

require.True(t, root.IsVerbose())
require.True(t, child.IsVerbose())
require.True(t, childChild.IsVerbose())

// New child of verbose child span should also be verbose by default.
childNewChild := tr.StartSpan("root.child.newchild", tracing.WithParentAndAutoCollection(child))
defer childNewChild.Finish()
require.True(t, childNewChild.IsVerbose())

// Toggle the trace's verbosity and confirm none of the spans are verbose.
query = fmt.Sprintf(
"SELECT * FROM crdb_internal.set_trace_verbose(%d, false)",
traceID,
)
r.CheckQueryResults(
t,
query,
[][]string{{`true`}},
)

require.False(t, root.IsVerbose())
require.False(t, child.IsVerbose())
require.False(t, childChild.IsVerbose())
require.False(t, childNewChild.IsVerbose())
}
1 change: 1 addition & 0 deletions pkg/util/tracing/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ go_library(
"recording.go",
"shadow.go",
"span.go",
"span_inner.go",
"span_options.go",
"tags.go",
"test_utils.go",
Expand Down
23 changes: 22 additions & 1 deletion pkg/util/tracing/crdbspan.go
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ func (s *crdbSpan) enableRecording(parent *crdbSpan, recType RecordingType) {
if parent != nil {
parent.addChild(s)
}
if recType == RecordingOff {
if recType == RecordingOff || s.recordingType() == recType {
return
}

Expand All @@ -128,6 +128,9 @@ func (s *crdbSpan) resetRecording() {
}

func (s *crdbSpan) disableRecording() {
if s.recordingType() == RecordingOff {
return
}
s.mu.Lock()
defer s.mu.Unlock()
oldRecType := s.mu.recording.recordingType.swap(RecordingOff)
Expand Down Expand Up @@ -354,6 +357,24 @@ func (s *crdbSpan) addChild(child *crdbSpan) {
s.mu.Unlock()
}

// setVerboseRecursively sets the verbosity of the crdbSpan appropriately and
// recurses on its list of children.
func (s *crdbSpan) setVerboseRecursively(to bool) {
if to {
s.enableRecording(nil /* parent */, RecordingVerbose)
} else {
s.disableRecording()
}

s.mu.Lock()
children := s.mu.recording.children
s.mu.Unlock()

for _, child := range children {
child.setVerboseRecursively(to)
}
}

var sortPool = sync.Pool{
New: func() interface{} {
return &Recording{}
Expand Down
Loading

0 comments on commit d4f9b02

Please sign in to comment.