Skip to content

Commit

Permalink
sql: small but essential tracing fixes
Browse files Browse the repository at this point in the history
1. Require admin role for `crdb_internal.node_inflight_trace_spans`
2. Add num_structured column to that table (this can be seen as
   temporary for testing; we can remove it once cockroachdb#55733 is fully
   addressed and payloads can be introspected separately).
3. Add workaround for failure to finish sql txn spans (cockroachdb#59315)
   for use in logic tests, which together with 2.) and 4.) enables
   a better `contention_event` test.
4. Add `crdb_internal.trace_id` builtin.

Release note: None
  • Loading branch information
tbg authored and angelapwen committed Feb 17, 2021
1 parent 87ee57b commit 483a6d6
Show file tree
Hide file tree
Showing 10 changed files with 93 additions and 44 deletions.
2 changes: 2 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -2648,6 +2648,8 @@ SELECT * FROM crdb_internal.check_consistency(true, ‘\x02’, ‘\x04’)</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>
</span></td></tr>
<tr><td><a name="current_database"></a><code>current_database() &rarr; <a href="string.html">string</a></code></td><td><span class="funcdesc"><p>Returns the current database.</p>
</span></td></tr>
<tr><td><a name="current_schema"></a><code>current_schema() &rarr; <a href="string.html">string</a></code></td><td><span class="funcdesc"><p>Returns the current schema.</p>
Expand Down
7 changes: 4 additions & 3 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -592,9 +592,10 @@ func (s *Server) newConnExecutor(
nodeIDOrZero: nodeIDOrZero,
clock: s.cfg.Clock,
// Future transaction's monitors will inherits from sessionRootMon.
connMon: sessionRootMon,
tracer: s.cfg.AmbientCtx.Tracer,
settings: s.cfg.Settings,
connMon: sessionRootMon,
tracer: s.cfg.AmbientCtx.Tracer,
settings: s.cfg.Settings,
execTestingKnobs: s.GetExecutorConfig().TestingKnobs,
},
memMetrics: memMetrics,
planner: planner{execCfg: s.cfg, alloc: &rowenc.DatumAlloc{}},
Expand Down
24 changes: 21 additions & 3 deletions pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
ptypes "github.com/gogo/protobuf/types"
)

// CrdbInternalName is the name of the crdb_internal schema.
Expand Down Expand Up @@ -1145,12 +1146,21 @@ CREATE TABLE crdb_internal.node_inflight_trace_spans (
goroutine_id INT NOT NULL, -- The ID of the goroutine on which the span was created.
finished BOOL NOT NULL, -- True if the span has been Finish()ed, false otherwise.
start_time TIMESTAMPTZ, -- The span's start time.
duration INTERVAL, -- The span's duration, measured from start to Finish().
-- A span whose recording is collected before it's finished will
-- have the duration set as the "time of collection - start time".
duration INTERVAL, -- The span's duration, measured from start to Finish().
-- A span whose recording is collected before it's finished will
-- have the duration set as the "time of collection - start time".
num_payloads INT NOT NULL, -- The number of structured payloads in this span.
operation STRING NULL -- The span's operation.
)`,
populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error {
hasAdmin, err := p.HasAdminRole(ctx)
if err != nil {
return err
}
if !hasAdmin {
return pgerror.Newf(pgcode.InsufficientPrivilege,
"only users with the admin role are allowed to read crdb_internal.node_inflight_trace_spans")
}
return p.ExecCfg().Settings.Tracer.VisitSpans(func(span *tracing.Span) error {
for _, rec := range span.GetRecording() {
traceID := rec.TraceID
Expand All @@ -1167,7 +1177,14 @@ CREATE TABLE crdb_internal.node_inflight_trace_spans (
spanDuration := rec.Duration
operation := rec.Operation

var numStructured int
rec.Structured(func(any *ptypes.Any) {
numStructured++
})

if err := addRow(
// TODO(angelapwen): we're casting uint64s to int64 here,
// is that ok?
tree.NewDInt(tree.DInt(traceID)),
tree.NewDInt(tree.DInt(parentSpanID)),
tree.NewDInt(tree.DInt(spanID)),
Expand All @@ -1178,6 +1195,7 @@ CREATE TABLE crdb_internal.node_inflight_trace_spans (
duration.MakeDuration(spanDuration.Nanoseconds(), 0, 0),
types.DefaultIntervalTypeMetadata,
),
tree.NewDInt(tree.DInt(numStructured)),
tree.NewDString(operation),
); err != nil {
return err
Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -921,6 +921,12 @@ type ExecutorTestingKnobs struct {
// DeterministicExplainAnalyze, if set, will result in overriding fields in
// EXPLAIN ANALYZE (PLAN) that can vary between runs (like elapsed times).
DeterministicExplainAnalyze bool

// Pretend59315IsFixed pretends that this issue is fixed:
// https://github.com/cockroachdb/cockroach/issues/59315
// which means that we don't need the WithBypassRegistry option
// in resetForNewSQLTxn.
Pretend59315IsFixed bool
}

// PGWireTestingKnobs contains knobs for the pgwire module.
Expand Down
11 changes: 10 additions & 1 deletion pkg/sql/logictest/logic.go
Original file line number Diff line number Diff line change
Expand Up @@ -465,7 +465,8 @@ type testClusterConfig struct {
isCCLConfig bool
// localities is set if nodes should be set to a particular locality.
// Nodes are 1-indexed.
localities map[int]roachpb.Locality
localities map[int]roachpb.Locality
pretend59315IsFixed bool
}

const threeNodeTenantConfigName = "3node-tenant"
Expand Down Expand Up @@ -562,6 +563,13 @@ var logicTestConfigs = []testClusterConfig{
overrideDistSQLMode: "on",
overrideAutoStats: "false",
},
{
name: "5node-pretend59315IsFixed",
numNodes: 5,
overrideDistSQLMode: "on",
overrideAutoStats: "false",
pretend59315IsFixed: true,
},
{
name: "5node-metadata",
numNodes: 5,
Expand Down Expand Up @@ -1332,6 +1340,7 @@ func (t *logicTest) newCluster(serverArgs TestServerArgs) {
},
SQLExecutor: &sql.ExecutorTestingKnobs{
DeterministicExplainAnalyze: true,
Pretend59315IsFixed: t.cfg.pretend59315IsFixed,
},
},
ClusterName: "testclustername",
Expand Down
48 changes: 16 additions & 32 deletions pkg/sql/logictest/testdata/logic_test/contention_event
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
# LogicTest: 5node
# LogicTest: 5node-pretend59315IsFixed
#
# Verify that ContentionEvents are emitted. This is mostly a sanity check - look
# for the datadriven tests in `pkg/kv/kvserver/concurrency` for the actual events
Expand All @@ -11,7 +11,6 @@ statement ok
CREATE TABLE kv (k VARCHAR PRIMARY KEY, v VARCHAR);
ALTER TABLE kv SPLIT AT VALUES ('b'), ('d'), ('q'), ('z');


query TT
SELECT * FROM kv
----
Expand All @@ -26,49 +25,34 @@ INSERT INTO kv VALUES('k', 'v')

user root

statement ok
SET tracing=on

# Scan all ranges of the table (note that we have intentionally
# split it into at least six ranges). This is better than a point
# lookup in that it gives tracing more of a chance to get things
# wrong due to DistSender parallelism.
statement ok
SET TRACING=on;
BEGIN;
SET TRANSACTION PRIORITY HIGH;
SELECT * FROM kv;
COMMIT;
SET TRACING=off;

# Check for the message emitted by (roachpb.ContentionEvent).String() as proof that a ContentionEvent
# payload was added to the trace.
#
# NB: it looks like there can be multiple (probably when there's an internal restart
# or something like that). Unfortunately, it is difficult in logic tests to log something
# that is not part of the scripted output.
#
# NB: if the matcher here ever gets updated, make sure to update throughout this file.
query B
SELECT count(message) > 0 FROM [ SHOW TRACE FOR SESSION ] WHERE message LIKE '%conflicted with % on % for %'
----
true

# Clean up.
user testuser

statement ok
ROLLBACK

# Do a straightforward non-conflicting insert; we want to check that there's *no* ContentionEvent
# emitted for those. There's no reason to believe it would be - this is a pure sanity check.

statement ok
SET TRACING=on;
INSERT INTO kv VALUES ('l', 'lll');
SET TRACING=off;
user root

query I
SELECT count(message) FROM [ SHOW TRACE FOR SESSION ] WHERE message LIKE '%conflicted with % on % for %'
# Check that the number of payloads in the open trace is at least 1.
# TODO(angelapwen): when we have a way to pull particular payloads for
# a trace, we should verify that we're seeing the right contention event.
# As is, the payloads could be something else (though we verified manually
# and there is a contention event).
#
# NB: the contention event is not in our trace span but in one of its
# children, so it wouldn't be found if we filtered by the trace span ID.
#
# NB: this needs the 5node-pretend59315 config because otherwise the span is not
# tracked.
query B
SELECT count(num_payloads) > 0 FROM crdb_internal.node_inflight_trace_spans WHERE trace_id = crdb_internal.trace_id();
----
0
true
7 changes: 5 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/crdb_internal
Original file line number Diff line number Diff line change
Expand Up @@ -250,10 +250,10 @@ SELECT * FROM crdb_internal.zones WHERE false
zone_id subzone_id target range_name database_name table_name index_name partition_name
raw_config_yaml raw_config_sql raw_config_protobuf full_config_yaml full_config_sql

query IIIIBTTT colnames
query IIIIBTTIT colnames
SELECT * FROM crdb_internal.node_inflight_trace_spans WHERE span_id < 0
----
trace_id parent_span_id span_id goroutine_id finished start_time duration operation
trace_id parent_span_id span_id goroutine_id finished start_time duration num_payloads operation

query ITTTTITTTTTTTTTI colnames
SELECT * FROM crdb_internal.ranges WHERE range_id < 0
Expand Down Expand Up @@ -477,6 +477,9 @@ select * from crdb_internal.kv_store_status
query error pq: only users with the admin role are allowed to read crdb_internal.gossip_alerts
select * from crdb_internal.gossip_alerts

query error pq: only users with the admin role are allowed to read crdb_internal.node_inflight_trace_spans
select * from crdb_internal.node_inflight_trace_spans

# Anyone can see the executable version.
query T
select regexp_replace(crdb_internal.node_executable_version()::string, '(-\d+)?$', '');
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/sem/builtins/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,7 @@ go_library(
"//pkg/util/timeofday",
"//pkg/util/timetz",
"//pkg/util/timeutil",
"//pkg/util/tracing",
"//pkg/util/unaccent",
"//pkg/util/uuid",
"@com_github_cockroachdb_apd_v2//:apd",
Expand Down
20 changes: 20 additions & 0 deletions pkg/sql/sem/builtins/builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/timeofday"
"github.com/cockroachdb/cockroach/pkg/util/timetz"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/unaccent"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -3578,6 +3579,25 @@ may increase either contention or retry errors, or both.`,
},
),

// Get the current trace ID.
"crdb_internal.trace_id": makeBuiltin(
tree.FunctionProperties{Category: categorySystemInfo},
tree.Overload{
Types: tree.ArgTypes{},
ReturnType: tree.FixedReturnType(types.Int),
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
sp := tracing.SpanFromContext(ctx.Context)
if sp == nil {
return nil, errors.New("no active trace span")
}
return tree.NewDInt(tree.DInt(sp.GetRecording()[0].TraceID)), nil
},
Info: "Returns the current trace ID or an error if no trace is open.",
// NB: possibly this is or could be made stable, but it's not worth it.
Volatility: tree.VolatilityVolatile,
},
),

"crdb_internal.locality_value": makeBuiltin(
tree.FunctionProperties{Category: categorySystemInfo},
tree.Overload{
Expand Down
11 changes: 8 additions & 3 deletions pkg/sql/txn_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -157,7 +157,11 @@ func (ts *txnState) resetForNewSQLTxn(
// TODO(andrei): figure out how to close these spans on server shutdown? Ties
// into a larger discussion about how to drain SQL and rollback open txns.
opName := sqlTxnName
txnCtx, sp := createRootOrChildSpan(connCtx, opName, tranCtx.tracer, tracing.WithBypassRegistry())
var traceOpts []tracing.SpanOption
if !tranCtx.execTestingKnobs.Pretend59315IsFixed {
traceOpts = append(traceOpts, tracing.WithBypassRegistry())
}
txnCtx, sp := createRootOrChildSpan(connCtx, opName, tranCtx.tracer, traceOpts...)
if txnType == implicitTxn {
sp.SetTag("implicit", "true")
}
Expand Down Expand Up @@ -384,8 +388,9 @@ type transitionCtx struct {
tracer *tracing.Tracer
// sessionTracing provides access to the session's tracing interface. The
// state machine needs to see if session tracing is enabled.
sessionTracing *SessionTracing
settings *cluster.Settings
sessionTracing *SessionTracing
settings *cluster.Settings
execTestingKnobs ExecutorTestingKnobs
}

var noRewind = rewindCapability{}
Expand Down

0 comments on commit 483a6d6

Please sign in to comment.