From 6f8be65c6f092b02a76b565286d1ebd1fe84b455 Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Tue, 8 Feb 2022 11:58:00 -0500 Subject: [PATCH] util/tracing: move Recording to tracingpb The Recording type was defined identically in both the tracing and the tracingpb package. This was confusing. This patch removes the tracing definition, leaving only tracingpb.Recording. Release note: None --- pkg/bench/rttanalysis/BUILD.bazel | 1 - pkg/bench/rttanalysis/cluster.go | 8 +- pkg/bench/rttanalysis/rtt_analysis_bench.go | 7 +- .../boundedstaleness_test.go | 4 +- .../kvfollowerreadsccl/followerreads_test.go | 6 +- pkg/ccl/kvccl/kvtenantccl/BUILD.bazel | 2 +- .../kvccl/kvtenantccl/tenant_trace_test.go | 6 +- pkg/ccl/multiregionccl/BUILD.bazel | 1 + pkg/ccl/multiregionccl/datadriven_test.go | 9 +- pkg/ccl/multiregionccl/roundtrips_test.go | 7 +- pkg/cli/BUILD.bazel | 1 + pkg/cli/debug_send_kv_batch.go | 7 +- pkg/jobs/BUILD.bazel | 1 + pkg/jobs/adopt.go | 3 +- pkg/kv/BUILD.bazel | 2 + pkg/kv/kvclient/kvcoord/BUILD.bazel | 1 + pkg/kv/kvclient/kvcoord/transport_test.go | 3 +- pkg/kv/kvclient/kvstreamer/BUILD.bazel | 2 +- pkg/kv/kvclient/kvstreamer/large_keys_test.go | 6 +- pkg/kv/kvnemesis/BUILD.bazel | 1 + pkg/kv/kvnemesis/applier.go | 3 +- pkg/kv/kvserver/BUILD.bazel | 2 + pkg/kv/kvserver/client_protectedts_test.go | 4 +- pkg/kv/kvserver/concurrency/BUILD.bazel | 1 + .../concurrency/concurrency_manager_test.go | 11 +- .../concurrency/lock_table_waiter_test.go | 7 +- pkg/kv/kvserver/kvserverbase/BUILD.bazel | 2 +- pkg/kv/kvserver/kvserverbase/stores.go | 4 +- pkg/kv/kvserver/replica_learner_test.go | 3 +- pkg/kv/kvserver/store.go | 5 +- pkg/kv/kvserver/stores_base.go | 4 +- pkg/kv/test_utils.go | 3 +- pkg/kv/txn_test.go | 5 +- pkg/server/admin.go | 7 +- pkg/server/node.go | 3 +- pkg/server/node_tenant.go | 3 +- pkg/server/node_tenant_test.go | 6 +- pkg/server/status.go | 3 +- pkg/sql/conn_executor_exec.go | 11 +- pkg/sql/crdb_internal.go | 8 +- pkg/sql/crdb_internal_test.go | 6 +- pkg/sql/exec_util.go | 16 +-- pkg/sql/execinfra/BUILD.bazel | 1 + pkg/sql/execinfra/processorsbase.go | 3 +- pkg/sql/execstats/stats.go | 5 +- pkg/sql/executor_statement_metrics.go | 4 +- pkg/sql/explain_bundle.go | 7 +- pkg/sql/instrumentation.go | 10 +- pkg/sql/rowexec/BUILD.bazel | 1 + pkg/sql/rowexec/inverted_joiner_test.go | 3 +- pkg/sql/rowexec/joinreader_blackbox_test.go | 5 +- pkg/sql/rowexec/joinreader_test.go | 3 +- pkg/sql/rowexec/tablereader_test.go | 5 +- pkg/sql/rowexec/zigzagjoiner_test.go | 3 +- pkg/sql/rowflow/BUILD.bazel | 1 + pkg/sql/rowflow/routers_test.go | 3 +- pkg/sql/sem/builtins/builtins.go | 8 +- pkg/sql/sem/builtins/generator_builtins.go | 2 +- .../sem/builtins/generator_probe_ranges.go | 5 +- pkg/sql/trace_test.go | 6 +- pkg/sql/txn_state.go | 3 +- pkg/testutils/testcluster/BUILD.bazel | 1 + pkg/testutils/testcluster/testcluster.go | 3 +- pkg/upgrade/upgrademanager/BUILD.bazel | 1 + .../upgrademanager/manager_external_test.go | 3 +- pkg/util/log/BUILD.bazel | 1 + pkg/util/log/ambient_context_test.go | 5 +- pkg/util/log/clog_test.go | 3 +- pkg/util/log/trace_client_test.go | 9 +- pkg/util/log/trace_test.go | 5 +- pkg/util/tracing/BUILD.bazel | 3 - pkg/util/tracing/bench_test.go | 9 +- pkg/util/tracing/collector/collector.go | 10 +- pkg/util/tracing/collector/collector_test.go | 10 +- pkg/util/tracing/crdbspan.go | 97 ++++++++--------- pkg/util/tracing/grpc_interceptor.go | 5 - pkg/util/tracing/grpc_interceptor_test.go | 8 +- pkg/util/tracing/service/BUILD.bazel | 1 + pkg/util/tracing/service/service.go | 5 +- pkg/util/tracing/service/service_test.go | 4 +- pkg/util/tracing/span.go | 40 +++---- pkg/util/tracing/span_inner.go | 12 ++- pkg/util/tracing/span_options.go | 22 ++-- pkg/util/tracing/span_test.go | 102 +++++++++--------- pkg/util/tracing/tags_test.go | 13 +-- pkg/util/tracing/test_utils.go | 6 +- pkg/util/tracing/tracer.go | 30 +++--- pkg/util/tracing/tracer_snapshots.go | 7 +- pkg/util/tracing/tracer_test.go | 99 ++++++++--------- pkg/util/tracing/tracingpb/BUILD.bazel | 6 ++ pkg/util/tracing/{ => tracingpb}/recording.go | 58 ++++++---- .../tracing/tracingui/span_registry_ui.go | 4 +- pkg/util/tracing/utils.go | 24 +---- pkg/util/tracing/zipper/BUILD.bazel | 2 +- pkg/util/tracing/zipper/zipper.go | 32 +++--- 95 files changed, 476 insertions(+), 437 deletions(-) rename pkg/util/tracing/{ => tracingpb}/recording.go (89%) diff --git a/pkg/bench/rttanalysis/BUILD.bazel b/pkg/bench/rttanalysis/BUILD.bazel index e400f91645db..8370c9ab26d4 100644 --- a/pkg/bench/rttanalysis/BUILD.bazel +++ b/pkg/bench/rttanalysis/BUILD.bazel @@ -23,7 +23,6 @@ go_library( "//pkg/util/quotapool", "//pkg/util/syncutil", "//pkg/util/system", - "//pkg/util/tracing", "//pkg/util/tracing/tracingpb", "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//require", diff --git a/pkg/bench/rttanalysis/cluster.go b/pkg/bench/rttanalysis/cluster.go index cba939055202..332b5022e04f 100644 --- a/pkg/bench/rttanalysis/cluster.go +++ b/pkg/bench/rttanalysis/cluster.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/sql" - "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" ) // ClusterConstructor is used to construct a Cluster for an individual case run. @@ -31,7 +31,7 @@ func MakeClusterConstructor( ) ClusterConstructor { return func(t testing.TB) *Cluster { c := &Cluster{} - beforePlan := func(trace tracing.Recording, stmt string) { + beforePlan := func(trace tracingpb.Recording, stmt string) { if _, ok := c.stmtToKVBatchRequests.Load(stmt); ok { c.stmtToKVBatchRequests.Store(stmt, trace) } @@ -60,9 +60,9 @@ func (c *Cluster) clearStatementTrace(stmt string) { c.stmtToKVBatchRequests.Store(stmt, nil) } -func (c *Cluster) getStatementTrace(stmt string) (tracing.Recording, bool) { +func (c *Cluster) getStatementTrace(stmt string) (tracingpb.Recording, bool) { out, _ := c.stmtToKVBatchRequests.Load(stmt) - r, ok := out.(tracing.Recording) + r, ok := out.(tracingpb.Recording) return r, ok } diff --git a/pkg/bench/rttanalysis/rtt_analysis_bench.go b/pkg/bench/rttanalysis/rtt_analysis_bench.go index a1694b2f2603..e9f333c84be8 100644 --- a/pkg/bench/rttanalysis/rtt_analysis_bench.go +++ b/pkg/bench/rttanalysis/rtt_analysis_bench.go @@ -23,7 +23,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/quotapool" - "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/stretchr/testify/require" ) @@ -116,7 +115,7 @@ func executeRoundTripTest(b testingB, tc RoundTripBenchTestCase, cc ClusterConst roundTrips := 0 b.ResetTimer() b.StopTimer() - var r tracing.Recording + var r tracingpb.Recording // Do an extra iteration and don't record it in order to deal with effects of // running it the first time. @@ -168,12 +167,12 @@ const roundTripsMetric = "roundtrips" // count the number of KvBatchRequests inside a recording, this is done by // counting each "txn coordinator send" operation. -func countKvBatchRequestsInRecording(r tracing.Recording) (sends int, hasRetry bool) { +func countKvBatchRequestsInRecording(r tracingpb.Recording) (sends int, hasRetry bool) { root := r[0] return countKvBatchRequestsInSpan(r, root) } -func countKvBatchRequestsInSpan(r tracing.Recording, sp tracingpb.RecordedSpan) (int, bool) { +func countKvBatchRequestsInSpan(r tracingpb.Recording, sp tracingpb.RecordedSpan) (int, bool) { count := 0 // Count the number of OpTxnCoordSender operations while traversing the // tree of spans. diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go index 0b35b43ea598..2cdfc0a03566 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go @@ -234,7 +234,7 @@ func (bse *boundedStalenessEvents) onTxnRetry( } } -func (bse *boundedStalenessEvents) onStmtTrace(nodeIdx int, rec tracing.Recording, stmt string) { +func (bse *boundedStalenessEvents) onStmtTrace(nodeIdx int, rec tracingpb.Recording, stmt string) { bse.mu.Lock() defer bse.mu.Unlock() @@ -275,7 +275,7 @@ func TestBoundedStalenessDataDriven(t *testing.T) { clusterArgs.ServerArgsPerNode[i] = base.TestServerArgs{ Knobs: base.TestingKnobs{ SQLExecutor: &sql.ExecutorTestingKnobs{ - WithStatementTrace: func(trace tracing.Recording, stmt string) { + WithStatementTrace: func(trace tracingpb.Recording, stmt string) { bse.onStmtTrace(i, trace, stmt) }, OnTxnRetry: func(err error, evalCtx *eval.Context) { diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go index b5ef0920c2fc..ac2bcce288e8 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go @@ -37,7 +37,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" @@ -561,7 +561,7 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) { defer utilccl.TestingEnableEnterprise()() historicalQuery := `SELECT * FROM test AS OF SYSTEM TIME follower_read_timestamp() WHERE k=2` - recCh := make(chan tracing.Recording, 1) + recCh := make(chan tracingpb.Recording, 1) var n2Addr, n3Addr syncutil.AtomicString tc := testcluster.StartTestCluster(t, 4, @@ -591,7 +591,7 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) { }, }, SQLExecutor: &sql.ExecutorTestingKnobs{ - WithStatementTrace: func(trace tracing.Recording, stmt string) { + WithStatementTrace: func(trace tracingpb.Recording, stmt string) { if stmt == historicalQuery { recCh <- trace } diff --git a/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel b/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel index 19ef6f5859c4..5d7970f31ec8 100644 --- a/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel +++ b/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel @@ -88,7 +88,7 @@ go_test( "//pkg/util/randutil", "//pkg/util/retry", "//pkg/util/stop", - "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "//pkg/util/uuid", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", diff --git a/pkg/ccl/kvccl/kvtenantccl/tenant_trace_test.go b/pkg/ccl/kvccl/kvtenantccl/tenant_trace_test.go index dcb8819ce3a1..d0875fc3b56c 100644 --- a/pkg/ccl/kvccl/kvtenantccl/tenant_trace_test.go +++ b/pkg/ccl/kvccl/kvtenantccl/tenant_trace_test.go @@ -25,7 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/redact" "github.com/stretchr/testify/require" ) @@ -50,7 +50,7 @@ func testTenantTracesAreRedactedImpl(t *testing.T, redactable bool) { visibleString = "tenant-can-see-this" ) - recCh := make(chan tracing.Recording, 1) + recCh := make(chan tracingpb.Recording, 1) args := base.TestServerArgs{ Knobs: base.TestingKnobs{ @@ -64,7 +64,7 @@ func testTenantTracesAreRedactedImpl(t *testing.T, redactable bool) { }, }, SQLExecutor: &sql.ExecutorTestingKnobs{ - WithStatementTrace: func(trace tracing.Recording, stmt string) { + WithStatementTrace: func(trace tracingpb.Recording, stmt string) { if stmt == testStmt { recCh <- trace } diff --git a/pkg/ccl/multiregionccl/BUILD.bazel b/pkg/ccl/multiregionccl/BUILD.bazel index 3e497904cf17..0b5993e78cbb 100644 --- a/pkg/ccl/multiregionccl/BUILD.bazel +++ b/pkg/ccl/multiregionccl/BUILD.bazel @@ -75,6 +75,7 @@ go_test( "//pkg/util/randutil", "//pkg/util/syncutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//require", diff --git a/pkg/ccl/multiregionccl/datadriven_test.go b/pkg/ccl/multiregionccl/datadriven_test.go index 924bf579d889..70df53146ff1 100644 --- a/pkg/ccl/multiregionccl/datadriven_test.go +++ b/pkg/ccl/multiregionccl/datadriven_test.go @@ -38,6 +38,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/datadriven" "github.com/cockroachdb/errors" ) @@ -118,7 +119,7 @@ func TestMultiRegionDataDriven(t *testing.T) { defer ds.cleanup(ctx) var mu syncutil.Mutex var traceStmt string - var recCh chan tracing.Recording + var recCh chan tracingpb.Recording datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "sleep-for-follower-read": @@ -135,7 +136,7 @@ func TestMultiRegionDataDriven(t *testing.T) { } serverArgs := make(map[int]base.TestServerArgs) localityNames := strings.Split(localities, ",") - recCh = make(chan tracing.Recording, 1) + recCh = make(chan tracingpb.Recording, 1) for i, localityName := range localityNames { localityCfg := roachpb.Locality{ Tiers: []roachpb.Tier{ @@ -147,7 +148,7 @@ func TestMultiRegionDataDriven(t *testing.T) { Locality: localityCfg, Knobs: base.TestingKnobs{ SQLExecutor: &sql.ExecutorTestingKnobs{ - WithStatementTrace: func(trace tracing.Recording, stmt string) { + WithStatementTrace: func(trace tracingpb.Recording, stmt string) { mu.Lock() defer mu.Unlock() if stmt == traceStmt { @@ -475,7 +476,7 @@ func nodeIdToIdx(t *testing.T, tc serverutils.TestClusterInterface, id roachpb.N // message. An error is returned if more than one (or no) "dist sender send" // messages are found in the recording. func checkReadServedLocallyInSimpleRecording( - rec tracing.Recording, + rec tracingpb.Recording, ) (servedLocally bool, servedUsingFollowerReads bool, err error) { foundDistSenderSend := false for _, sp := range rec { diff --git a/pkg/ccl/multiregionccl/roundtrips_test.go b/pkg/ccl/multiregionccl/roundtrips_test.go index f04e22eac0eb..9249c2f88d0d 100644 --- a/pkg/ccl/multiregionccl/roundtrips_test.go +++ b/pkg/ccl/multiregionccl/roundtrips_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) @@ -36,7 +37,7 @@ func TestEnsureLocalReadsOnGlobalTables(t *testing.T) { // ensureOnlyLocalReads looks at a trace to ensure that reads were served // locally. It returns true if the read was served as a follower read. - ensureOnlyLocalReads := func(t *testing.T, rec tracing.Recording) (servedUsingFollowerReads bool) { + ensureOnlyLocalReads := func(t *testing.T, rec tracingpb.Recording) (servedUsingFollowerReads bool) { for _, sp := range rec { if sp.Operation == "dist sender send" { require.True(t, tracing.LogsContainMsg(sp, kvbase.RoutingRequestLocallyMsg), @@ -57,11 +58,11 @@ func TestEnsureLocalReadsOnGlobalTables(t *testing.T) { } presentTimeRead := `SELECT * FROM t.test_table WHERE k=2` - recCh := make(chan tracing.Recording, 1) + recCh := make(chan tracingpb.Recording, 1) knobs := base.TestingKnobs{ SQLExecutor: &sql.ExecutorTestingKnobs{ - WithStatementTrace: func(trace tracing.Recording, stmt string) { + WithStatementTrace: func(trace tracingpb.Recording, stmt string) { if stmt == presentTimeRead { recCh <- trace } diff --git a/pkg/cli/BUILD.bazel b/pkg/cli/BUILD.bazel index a1060ae379bf..4fcc86fc11d4 100644 --- a/pkg/cli/BUILD.bazel +++ b/pkg/cli/BUILD.bazel @@ -206,6 +206,7 @@ go_library( "//pkg/util/sysutil", "//pkg/util/timeutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "//pkg/util/tracing/zipper", "//pkg/util/uuid", "//pkg/workload", diff --git a/pkg/cli/debug_send_kv_batch.go b/pkg/cli/debug_send_kv_batch.go index 4d6f118b0f28..af679763d842 100644 --- a/pkg/cli/debug_send_kv_batch.go +++ b/pkg/cli/debug_send_kv_batch.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/spf13/cobra" ) @@ -226,7 +227,7 @@ func runSendKVBatch(cmd *cobra.Command, args []string) error { func sendKVBatchRequestWithTracingOption( ctx context.Context, verboseTrace bool, admin serverpb.AdminClient, ba *roachpb.BatchRequest, -) (br *roachpb.BatchResponse, rec tracing.Recording, err error) { +) (br *roachpb.BatchResponse, rec tracingpb.Recording, err error) { var sp *tracing.Span if verboseTrace { // Set up a tracing span and enable verbose tracing if requested by @@ -237,7 +238,7 @@ func sendKVBatchRequestWithTracingOption( // because otherwise the unit test TestSendKVBatch becomes non-deterministic // on the contents of the traceInfo JSON field in the request. _, sp = tracing.NewTracer().StartSpanCtx(ctx, "debug-send-kv-batch", - tracing.WithRecording(tracing.RecordingVerbose)) + tracing.WithRecording(tracingpb.RecordingVerbose)) defer sp.Finish() // Inject the span metadata into the KV request. @@ -252,7 +253,7 @@ func sendKVBatchRequestWithTracingOption( sp.ImportRemoteRecording(br.CollectedSpans) // Extract the recording. - rec = sp.GetRecording(tracing.RecordingVerbose) + rec = sp.GetRecording(tracingpb.RecordingVerbose) } return br, rec, errors.Wrap(err, "request failed") diff --git a/pkg/jobs/BUILD.bazel b/pkg/jobs/BUILD.bazel index 9a298c9e5f7c..05569be3249c 100644 --- a/pkg/jobs/BUILD.bazel +++ b/pkg/jobs/BUILD.bazel @@ -61,6 +61,7 @@ go_library( "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_errors//oserror", "@com_github_cockroachdb_logtags//:logtags", diff --git a/pkg/jobs/adopt.go b/pkg/jobs/adopt.go index 39fbffc7fd2a..c001d9f08323 100644 --- a/pkg/jobs/adopt.go +++ b/pkg/jobs/adopt.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "go.opentelemetry.io/otel/attribute" ) @@ -384,7 +385,7 @@ func (r *Registry) runJob( // A new root span will be created on every resumption of the job. var spanOptions []tracing.SpanOption if tj, ok := resumer.(TraceableJob); ok && tj.ForceRealSpan() { - spanOptions = append(spanOptions, tracing.WithRecording(tracing.RecordingStructured)) + spanOptions = append(spanOptions, tracing.WithRecording(tracingpb.RecordingStructured)) } // TODO(ajwerner): Move this writing up the trace ID down into // stepThroughStateMachine where we're already often (and soon with diff --git a/pkg/kv/BUILD.bazel b/pkg/kv/BUILD.bazel index d6e1b264416f..191220bff2b2 100644 --- a/pkg/kv/BUILD.bazel +++ b/pkg/kv/BUILD.bazel @@ -37,6 +37,7 @@ go_library( "//pkg/util/stop", "//pkg/util/syncutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "//pkg/util/uuid", "@com_github_cockroachdb_apd_v3//:apd", "@com_github_cockroachdb_errors//:errors", @@ -85,6 +86,7 @@ go_test( "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "//pkg/util/uuid", "@com_github_cockroachdb_errors//:errors", "@com_github_gogo_protobuf//proto", diff --git a/pkg/kv/kvclient/kvcoord/BUILD.bazel b/pkg/kv/kvclient/kvcoord/BUILD.bazel index c9327de0172c..819843d7a95c 100644 --- a/pkg/kv/kvclient/kvcoord/BUILD.bazel +++ b/pkg/kv/kvclient/kvcoord/BUILD.bazel @@ -188,6 +188,7 @@ go_test( "//pkg/util/stop", "//pkg/util/syncutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "//pkg/util/uuid", "@com_github_cockroachdb_circuitbreaker//:circuitbreaker", "@com_github_cockroachdb_datadriven//:datadriven", diff --git a/pkg/kv/kvclient/kvcoord/transport_test.go b/pkg/kv/kvclient/kvcoord/transport_test.go index 2f051b2a84bd..e98a0e6da52f 100644 --- a/pkg/kv/kvclient/kvcoord/transport_test.go +++ b/pkg/kv/kvclient/kvcoord/transport_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "google.golang.org/grpc" ) @@ -151,7 +152,7 @@ func (*mockInternalClient) ResetQuorum( func (m *mockInternalClient) Batch( ctx context.Context, in *roachpb.BatchRequest, opts ...grpc.CallOption, ) (*roachpb.BatchResponse, error) { - sp := m.tr.StartSpan("mock", tracing.WithRecording(tracing.RecordingVerbose)) + sp := m.tr.StartSpan("mock", tracing.WithRecording(tracingpb.RecordingVerbose)) defer sp.Finish() ctx = tracing.ContextWithSpan(ctx, sp) diff --git a/pkg/kv/kvclient/kvstreamer/BUILD.bazel b/pkg/kv/kvclient/kvstreamer/BUILD.bazel index 3b53c1e9bd98..3a2221614d8b 100644 --- a/pkg/kv/kvclient/kvstreamer/BUILD.bazel +++ b/pkg/kv/kvclient/kvstreamer/BUILD.bazel @@ -62,7 +62,7 @@ go_test( "//pkg/util/log", "//pkg/util/mon", "//pkg/util/randutil", - "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "@com_github_cockroachdb_errors//:errors", "@com_github_dustin_go_humanize//:go-humanize", "@com_github_stretchr_testify//require", diff --git a/pkg/kv/kvclient/kvstreamer/large_keys_test.go b/pkg/kv/kvclient/kvstreamer/large_keys_test.go index adf01a2aa555..bdfa06caad3a 100644 --- a/pkg/kv/kvclient/kvstreamer/large_keys_test.go +++ b/pkg/kv/kvclient/kvstreamer/large_keys_test.go @@ -23,7 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" - "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/dustin/go-humanize" "github.com/stretchr/testify/require" ) @@ -59,13 +59,13 @@ func TestLargeKeys(t *testing.T) { } rng, _ := randutil.NewTestRand() - recCh := make(chan tracing.Recording, 1) + recCh := make(chan tracingpb.Recording, 1) // We want to capture the trace of the query so that we can count how many // KV requests the Streamer issued. s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{ SQLExecutor: &sql.ExecutorTestingKnobs{ - WithStatementTrace: func(trace tracing.Recording, stmt string) { + WithStatementTrace: func(trace tracingpb.Recording, stmt string) { for _, tc := range testCases { if tc.query == stmt { recCh <- trace diff --git a/pkg/kv/kvnemesis/BUILD.bazel b/pkg/kv/kvnemesis/BUILD.bazel index a48ad400a258..554b1ed879b9 100644 --- a/pkg/kv/kvnemesis/BUILD.bazel +++ b/pkg/kv/kvnemesis/BUILD.bazel @@ -39,6 +39,7 @@ go_library( "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "//pkg/util/uuid", "@com_github_cockroachdb_cockroach_go_v2//crdb", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/kv/kvnemesis/applier.go b/pkg/kv/kvnemesis/applier.go index 2a7f0c804962..2ea04f7eb7ec 100644 --- a/pkg/kv/kvnemesis/applier.go +++ b/pkg/kv/kvnemesis/applier.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "google.golang.org/protobuf/proto" ) @@ -50,7 +51,7 @@ func MakeApplier(env *Env, dbs ...*kv.DB) *Applier { // Apply executes the given Step and mutates it with the result of execution. An // error is only returned from Apply if there is an internal coding error within // Applier, errors from a Step execution are saved in the Step itself. -func (a *Applier) Apply(ctx context.Context, step *Step) (trace tracing.Recording, retErr error) { +func (a *Applier) Apply(ctx context.Context, step *Step) (trace tracingpb.Recording, retErr error) { var db *kv.DB db, step.DBID = a.getNextDBRoundRobin() diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 199a33dcc0fb..8286b6b304ab 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -193,6 +193,7 @@ go_library( "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "//pkg/util/uuid", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_errors//oserror", @@ -414,6 +415,7 @@ go_test( "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "//pkg/util/uuid", "@com_github_cockroachdb_circuitbreaker//:circuitbreaker", "@com_github_cockroachdb_cockroach_go_v2//crdb", diff --git a/pkg/kv/kvserver/client_protectedts_test.go b/pkg/kv/kvserver/client_protectedts_test.go index 6d00cdc09338..0cf206a426f6 100644 --- a/pkg/kv/kvserver/client_protectedts_test.go +++ b/pkg/kv/kvserver/client_protectedts_test.go @@ -31,7 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" - "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" @@ -166,7 +166,7 @@ func TestProtectedTimestamps(t *testing.T) { } thresholdRE := regexp.MustCompile(`(?s).*Threshold:(?P[^\s]*)`) - thresholdFromTrace := func(trace tracing.Recording) hlc.Timestamp { + thresholdFromTrace := func(trace tracingpb.Recording) hlc.Timestamp { threshStr := string(thresholdRE.ExpandString(nil, "$threshold", trace.String(), thresholdRE.FindStringSubmatchIndex(trace.String()))) thresh, err := hlc.ParseTimestamp(threshStr) diff --git a/pkg/kv/kvserver/concurrency/BUILD.bazel b/pkg/kv/kvserver/concurrency/BUILD.bazel index c71fe92632d3..fbbdd6671a80 100644 --- a/pkg/kv/kvserver/concurrency/BUILD.bazel +++ b/pkg/kv/kvserver/concurrency/BUILD.bazel @@ -76,6 +76,7 @@ go_test( "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "//pkg/util/uint128", "//pkg/util/uuid", "@com_github_cockroachdb_datadriven//:datadriven", diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go index 5a3539423639..f79f2b66d2c9 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go @@ -41,6 +41,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/datadriven" "github.com/cockroachdb/errors" @@ -989,7 +990,7 @@ type monitoredGoroutine struct { finished int32 ctx context.Context - collect func() tracing.Recording + collect func() tracingpb.Recording cancel func() prevEvents int } @@ -1002,12 +1003,12 @@ func newMonitor() *monitor { } func (m *monitor) runSync(opName string, fn func(context.Context)) { - ctx, sp := m.tr.StartSpanCtx(context.Background(), opName, tracing.WithRecording(tracing.RecordingVerbose)) + ctx, sp := m.tr.StartSpanCtx(context.Background(), opName, tracing.WithRecording(tracingpb.RecordingVerbose)) g := &monitoredGoroutine{ opSeq: 0, // synchronous opName: opName, ctx: ctx, - collect: func() tracing.Recording { + collect: func() tracingpb.Recording { return sp.GetConfiguredRecording() }, cancel: sp.Finish, @@ -1019,12 +1020,12 @@ func (m *monitor) runSync(opName string, fn func(context.Context)) { func (m *monitor) runAsync(opName string, fn func(context.Context)) (cancel func()) { m.seq++ - ctx, sp := m.tr.StartSpanCtx(context.Background(), opName, tracing.WithRecording(tracing.RecordingVerbose)) + ctx, sp := m.tr.StartSpanCtx(context.Background(), opName, tracing.WithRecording(tracingpb.RecordingVerbose)) g := &monitoredGoroutine{ opSeq: m.seq, opName: opName, ctx: ctx, - collect: func() tracing.Recording { + collect: func() tracingpb.Recording { return sp.GetConfiguredRecording() }, cancel: sp.Finish, diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go index 68d3b1bb5177..0134ef855089 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/stretchr/testify/require" ) @@ -933,7 +934,7 @@ func BenchmarkTxnCache(b *testing.B) { func TestContentionEventTracer(t *testing.T) { tr := tracing.NewTracer() - ctx, sp := tr.StartSpanCtx(context.Background(), "foo", tracing.WithRecording(tracing.RecordingVerbose)) + ctx, sp := tr.StartSpanCtx(context.Background(), "foo", tracing.WithRecording(tracingpb.RecordingVerbose)) defer sp.Finish() clock := hlc.NewClockWithSystemTimeSource(0 /* maxOffset */) @@ -952,7 +953,7 @@ func TestContentionEventTracer(t *testing.T) { require.Zero(t, h.tag.mu.lockWait) require.NotZero(t, h.tag.mu.waitStart) require.Empty(t, events) - rec := sp.GetRecording(tracing.RecordingVerbose) + rec := sp.GetRecording(tracingpb.RecordingVerbose) require.Contains(t, rec[0].Tags, tagNumLocks) require.Equal(t, "1", rec[0].Tags[tagNumLocks]) require.Contains(t, rec[0].Tags, tagWaited) @@ -998,7 +999,7 @@ func TestContentionEventTracer(t *testing.T) { }) require.Len(t, events, 3) require.Less(t, lockWaitBefore, h.tag.mu.lockWait) - rec = sp.GetRecording(tracing.RecordingVerbose) + rec = sp.GetRecording(tracingpb.RecordingVerbose) require.Equal(t, "3", rec[0].Tags[tagNumLocks]) require.Contains(t, rec[0].Tags, tagWaited) require.NotContains(t, rec[0].Tags, tagWaitKey) diff --git a/pkg/kv/kvserver/kvserverbase/BUILD.bazel b/pkg/kv/kvserver/kvserverbase/BUILD.bazel index b4d93a567ab7..3d3d28c8c965 100644 --- a/pkg/kv/kvserver/kvserverbase/BUILD.bazel +++ b/pkg/kv/kvserver/kvserverbase/BUILD.bazel @@ -20,7 +20,7 @@ go_library( "//pkg/util/errorutil", "//pkg/util/hlc", "//pkg/util/quotapool", - "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", ], diff --git a/pkg/kv/kvserver/kvserverbase/stores.go b/pkg/kv/kvserver/kvserverbase/stores.go index 203f05358071..e0a0c0fd2c05 100644 --- a/pkg/kv/kvserver/kvserverbase/stores.go +++ b/pkg/kv/kvserver/kvserverbase/stores.go @@ -15,7 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/errorutil" - "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" ) // StoresIterator is able to iterate over all stores on a given node. @@ -34,7 +34,7 @@ type Store interface { queue string, rangeID roachpb.RangeID, skipShouldQueue bool, - ) (tracing.Recording, error) + ) (tracingpb.Recording, error) // SetQueueActive disables/enables the named queue. SetQueueActive(active bool, queue string) error diff --git a/pkg/kv/kvserver/replica_learner_test.go b/pkg/kv/kvserver/replica_learner_test.go index a965a3651801..3fa6768bb2ee 100644 --- a/pkg/kv/kvserver/replica_learner_test.go +++ b/pkg/kv/kvserver/replica_learner_test.go @@ -39,6 +39,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" "golang.org/x/sync/errgroup" @@ -1285,7 +1286,7 @@ func TestDemotedLearnerRemovalHandlesRace(t *testing.T) { tc.AddVotersOrFatal(t, scratchKey, makeReplicationTargets(2)...) atomic.StoreInt64(&activateTestingKnob, 1) rebalanceCh := make(chan error) - var finishAndGetRecording func() tracing.Recording + var finishAndGetRecording func() tracingpb.Recording err := tc.Stopper().RunAsyncTask(ctx, "test", func(ctx context.Context) { ctx, finishAndGetRecording = tracing.ContextWithRecordingSpan( ctx, tc.Servers[0].Tracer(), "rebalance", diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 581b8a95fa18..98a7f59866f2 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -79,6 +79,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" @@ -3384,7 +3385,7 @@ func (s *Store) ComputeStatsForKeySpan(startKey, endKey roachpb.RKey) (StoreKeyS // AllocatorDryRun runs the given replica through the allocator without actually // carrying out any changes, returning all trace messages collected along the way. // Intended to help power a debug endpoint. -func (s *Store) AllocatorDryRun(ctx context.Context, repl *Replica) (tracing.Recording, error) { +func (s *Store) AllocatorDryRun(ctx context.Context, repl *Replica) (tracingpb.Recording, error) { ctx, collectAndFinish := tracing.ContextWithRecordingSpan(ctx, s.cfg.AmbientCtx.Tracer, "allocator dry run") defer collectAndFinish() canTransferLease := func(ctx context.Context, repl *Replica) bool { return true } @@ -3403,7 +3404,7 @@ func (s *Store) AllocatorDryRun(ctx context.Context, repl *Replica) (tracing.Rec // power an admin debug endpoint. func (s *Store) ManuallyEnqueue( ctx context.Context, queueName string, repl *Replica, skipShouldQueue bool, -) (recording tracing.Recording, processError error, enqueueError error) { +) (recording tracingpb.Recording, processError error, enqueueError error) { ctx = repl.AnnotateCtx(ctx) // Do not enqueue uninitialized replicas. The baseQueue ignores these during diff --git a/pkg/kv/kvserver/stores_base.go b/pkg/kv/kvserver/stores_base.go index 77a5525d0817..bcbfa73c19d4 100644 --- a/pkg/kv/kvserver/stores_base.go +++ b/pkg/kv/kvserver/stores_base.go @@ -12,12 +12,12 @@ package kvserver import ( "context" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "strings" "unsafe" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" ) @@ -58,7 +58,7 @@ func (s *baseStore) StoreID() roachpb.StoreID { // Enqueue is part of kvserverbase.Store. func (s *baseStore) Enqueue( ctx context.Context, queue string, rangeID roachpb.RangeID, skipShouldQueue bool, -) (tracing.Recording, error) { +) (tracingpb.Recording, error) { store := (*Store)(s) repl, err := store.GetReplica(rangeID) if err != nil { diff --git a/pkg/kv/test_utils.go b/pkg/kv/test_utils.go index e75e282edfd7..4c091815d831 100644 --- a/pkg/kv/test_utils.go +++ b/pkg/kv/test_utils.go @@ -16,11 +16,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvbase" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" ) // OnlyFollowerReads looks through all the RPCs and asserts that every single // one resulted in a follower read. Returns false if no RPCs are found. -func OnlyFollowerReads(rec tracing.Recording) bool { +func OnlyFollowerReads(rec tracingpb.Recording) bool { foundFollowerRead := false for _, sp := range rec { if sp.Operation == "/cockroach.roachpb.Internal/Batch" && diff --git a/pkg/kv/txn_test.go b/pkg/kv/txn_test.go index 8c2470a5bfbd..c4dfbaee132a 100644 --- a/pkg/kv/txn_test.go +++ b/pkg/kv/txn_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" @@ -42,7 +43,7 @@ func TestTxnVerboseTrace(t *testing.T) { defer log.Scope(t).Close(t) tracer := tracing.NewTracer() - ctx, sp := tracer.StartSpanCtx(context.Background(), "test-txn", tracing.WithRecording(tracing.RecordingVerbose)) + ctx, sp := tracer.StartSpanCtx(context.Background(), "test-txn", tracing.WithRecording(tracingpb.RecordingVerbose)) stopper := stop.NewStopper() defer stopper.Stop(ctx) clock := hlc.NewClockWithSystemTimeSource(time.Nanosecond /* maxOffset */) @@ -55,7 +56,7 @@ func TestTxnVerboseTrace(t *testing.T) { t.Fatal(err) } log.Event(ctx, "txn complete") - collectedSpans := sp.FinishAndGetRecording(tracing.RecordingVerbose) + collectedSpans := sp.FinishAndGetRecording(tracingpb.RecordingVerbose) dump := collectedSpans.String() // dump: // 0.105ms 0.000ms event:inside txn diff --git a/pkg/server/admin.go b/pkg/server/admin.go index e22ac4ebd909..cb0d48b38a8d 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -62,6 +62,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingui" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" @@ -3647,7 +3648,7 @@ func (s *adminServer) GetTrace( if err != nil { return nil, err } - var recording tracing.Recording + var recording tracingpb.Recording var snapshotID tracing.SnapshotID traceID := req.TraceID @@ -3676,7 +3677,7 @@ func (s *adminServer) GetTrace( } traceStillExists = true if recording == nil { - recording = sp.GetFullRecording(tracing.RecordingVerbose) + recording = sp.GetFullRecording(tracingpb.RecordingVerbose) } return iterutil.StopIteration() }); err != nil { @@ -3708,7 +3709,7 @@ func (s *adminServer) SetTraceRecordingType( return nil } // NB: The recording type propagates to the children, recursively. - sp.SetRecordingType(tracing.RecordingTypeFromProto(req.RecordingMode)) + sp.SetRecordingType(tracingpb.RecordingTypeFromProto(req.RecordingMode)) return nil }) return &serverpb.SetTraceRecordingTypeResponse{}, nil diff --git a/pkg/server/node.go b/pkg/server/node.go index ca2560b37a94..f76e8047b44b 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -55,6 +55,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" @@ -1090,7 +1091,7 @@ type spanForRequest struct { // finish finishes the span. If the span was recording and br is not nil, the // recording is written to br.CollectedSpans. func (sp *spanForRequest) finish(ctx context.Context, br *roachpb.BatchResponse) { - var rec tracing.Recording + var rec tracingpb.Recording // If we don't have a response, there's nothing to attach a trace to. // Nothing more for us to do. sp.needRecording = sp.needRecording && br != nil diff --git a/pkg/server/node_tenant.go b/pkg/server/node_tenant.go index b80735ce5b95..0cfdf26ff6a4 100644 --- a/pkg/server/node_tenant.go +++ b/pkg/server/node_tenant.go @@ -12,7 +12,6 @@ package server import ( "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" @@ -27,7 +26,7 @@ const TraceRedactedMarker = redact.RedactableString("verbose trace message redac // The recording is modified in place. // // tenID is the tenant that will receive this recording. -func redactRecordingForTenant(tenID roachpb.TenantID, rec tracing.Recording) error { +func redactRecordingForTenant(tenID roachpb.TenantID, rec tracingpb.Recording) error { if tenID == roachpb.SystemTenantID { return nil } diff --git a/pkg/server/node_tenant_test.go b/pkg/server/node_tenant_test.go index 26cb2d5900b0..524996cec08c 100644 --- a/pkg/server/node_tenant_test.go +++ b/pkg/server/node_tenant_test.go @@ -40,7 +40,7 @@ func TestRedactRecordingForTenant(t *testing.T) { tagSensitive = "tag-tenant-hidden" ) - mkRec := func() tracing.Recording { + mkRec := func() tracingpb.Recording { t.Helper() tags := (&logtags.Buffer{}). Add("tag_sensitive", tagSensitive). @@ -48,10 +48,10 @@ func TestRedactRecordingForTenant(t *testing.T) { ctx := logtags.WithTags(context.Background(), tags) tracer := tracing.NewTracer() tracer.SetRedactable(true) - ctx, sp := tracer.StartSpanCtx(ctx, "foo", tracing.WithRecording(tracing.RecordingVerbose)) + ctx, sp := tracer.StartSpanCtx(ctx, "foo", tracing.WithRecording(tracingpb.RecordingVerbose)) log.Eventf(ctx, "%s %s", msgSensitive, redact.Safe(msgNotSensitive)) sp.SetTag("all_span_tags_are_stripped", attribute.StringValue("because_no_redactability")) - rec := sp.FinishAndGetRecording(tracing.RecordingVerbose) + rec := sp.FinishAndGetRecording(tracingpb.RecordingVerbose) require.Len(t, rec, 1) return rec } diff --git a/pkg/server/status.go b/pkg/server/status.go index 93a103e59126..fecc55afa76b 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -72,7 +72,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" @@ -717,7 +716,7 @@ func (s *statusServer) Allocator( if !rep.OwnsValidLease(ctx, store.Clock().NowAsClockTimestamp()) { return true // continue. } - var allocatorSpans tracing.Recording + var allocatorSpans tracingpb.Recording allocatorSpans, err = store.AllocatorDryRun(ctx, rep) if err != nil { return false // break and bubble up the error. diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 592f47ec5681..66c2c0c4b847 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -58,6 +58,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/lib/pq/oid" @@ -680,7 +681,7 @@ func (ex *connExecutor) execStmtInOpenState( var stmtCtx context.Context // TODO(andrei): I think we should do this even if alreadyRecording == true. if !alreadyRecording && stmtTraceThreshold > 0 { - stmtCtx, stmtThresholdSpan = tracing.EnsureChildSpan(ctx, ex.server.cfg.AmbientCtx.Tracer, "trace-stmt-threshold", tracing.WithRecording(tracing.RecordingVerbose)) + stmtCtx, stmtThresholdSpan = tracing.EnsureChildSpan(ctx, ex.server.cfg.AmbientCtx.Tracer, "trace-stmt-threshold", tracing.WithRecording(tracingpb.RecordingVerbose)) } else { stmtCtx = ctx } @@ -694,7 +695,7 @@ func (ex *connExecutor) execStmtInOpenState( stmtDur := timeutil.Since(ex.phaseTimes.GetSessionPhaseTime(sessionphase.SessionQueryReceived)) needRecording := stmtTraceThreshold < stmtDur if needRecording { - rec := stmtThresholdSpan.FinishAndGetRecording(tracing.RecordingVerbose) + rec := stmtThresholdSpan.FinishAndGetRecording(tracingpb.RecordingVerbose) // NB: This recording does not include the commit for implicit // transactions if the statement didn't auto-commit. logTraceAboveThreshold( @@ -1982,7 +1983,7 @@ func (ex *connExecutor) runSetTracing( func (ex *connExecutor) enableTracing(modes []string) error { traceKV := false - recordingType := tracing.RecordingVerbose + recordingType := tracingpb.RecordingVerbose enableMode := true showResults := false @@ -1997,7 +1998,7 @@ func (ex *connExecutor) enableTracing(modes []string) error { case "kv": traceKV = true case "cluster": - recordingType = tracing.RecordingVerbose + recordingType = tracingpb.RecordingVerbose default: return pgerror.Newf(pgcode.Syntax, "set tracing: unknown mode %q", s) @@ -2273,7 +2274,7 @@ func (ex *connExecutor) recordTransactionFinish( // given threshold. It is used when txn or stmt threshold tracing is enabled. // This function assumes that sp is non-nil and threshold tracing was enabled. func logTraceAboveThreshold( - ctx context.Context, r tracing.Recording, opName string, threshold, elapsed time.Duration, + ctx context.Context, r tracingpb.Recording, opName string, threshold, elapsed time.Duration, ) { if elapsed < threshold { return diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index ad3790475e0f..9abc4c617df7 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1388,8 +1388,8 @@ CREATE TABLE crdb_internal.session_trace ( // returns rows when accessed with an index constraint specifying the trace_id // for which inflight spans need to be aggregated from all nodes in the cluster. // -// Each row in the virtual table corresponds to a single `tracing.Recording` on -// a particular node. A `tracing.Recording` is the trace of a single operation +// Each row in the virtual table corresponds to a single `tracingpb.Recording` on +// a particular node. A `tracingpb.Recording` is the trace of a single operation // rooted at a root span on that node. Under the hood, the virtual table // contacts all "live" nodes in the cluster via the trace collector which // streams back a recording at a time. @@ -1398,7 +1398,7 @@ CREATE TABLE crdb_internal.session_trace ( // The virtual table also produces rows lazily, i.e. as and when they are // consumed by the consumer. Therefore, the memory overhead of querying this // table will be the size of all the `tracing.Recordings` of a particular -// `trace_id` on a single node in the cluster. Each `tracing.Recording` has its +// `trace_id` on a single node in the cluster. Each `tracingpb.Recording` has its // own memory protections via ring buffers, and so we do not expect this // overhead to grow in an unbounded manner. var crdbInternalClusterInflightTracesTable = virtualSchemaTable{ @@ -1489,7 +1489,7 @@ CREATE TABLE crdb_internal.node_inflight_trace_spans ( "only users with the admin role are allowed to read crdb_internal.node_inflight_trace_spans") } return p.ExecCfg().AmbientCtx.Tracer.VisitSpans(func(span tracing.RegistrySpan) error { - for _, rec := range span.GetFullRecording(tracing.RecordingVerbose) { + for _, rec := range span.GetFullRecording(tracingpb.RecordingVerbose) { traceID := rec.TraceID parentSpanID := rec.ParentSpanID spanID := rec.SpanID diff --git a/pkg/sql/crdb_internal_test.go b/pkg/sql/crdb_internal_test.go index 4df6ba473d93..d5aa68e6f85d 100644 --- a/pkg/sql/crdb_internal_test.go +++ b/pkg/sql/crdb_internal_test.go @@ -745,7 +745,7 @@ func TestDistSQLFlowsVirtualTables(t *testing.T) { // root2.child <-- traceID2 func setupTraces(t1, t2 *tracing.Tracer) (tracingpb.TraceID, func()) { // Start a root span on "node 1". - root := t1.StartSpan("root", tracing.WithRecording(tracing.RecordingVerbose)) + root := t1.StartSpan("root", tracing.WithRecording(tracingpb.RecordingVerbose)) time.Sleep(10 * time.Millisecond) @@ -766,11 +766,11 @@ func setupTraces(t1, t2 *tracing.Tracer) (tracingpb.TraceID, func()) { // Start another remote child span on "node 2" that we finish. childRemoteChildFinished := t2.StartSpan("root.child.remotechilddone", tracing.WithRemoteParentFromSpanMeta(child.Meta())) - child.ImportRemoteRecording(childRemoteChildFinished.FinishAndGetRecording(tracing.RecordingVerbose)) + child.ImportRemoteRecording(childRemoteChildFinished.FinishAndGetRecording(tracingpb.RecordingVerbose)) // Start another remote child span on "node 2" that we finish. This will have // a different trace_id from the spans created above. - root2 := t2.StartSpan("root2", tracing.WithRecording(tracing.RecordingVerbose)) + root2 := t2.StartSpan("root2", tracing.WithRecording(tracingpb.RecordingVerbose)) // Start a child span on "node 2". child2 := t2.StartSpan("root2.child", tracing.WithParent(root2)) diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index f15d5d1e85d7..2b39db839658 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -1418,7 +1418,7 @@ type ExecutorTestingKnobs struct { // WithStatementTrace is called after the statement is executed in // execStmtInOpenState. - WithStatementTrace func(trace tracing.Recording, stmt string) + WithStatementTrace func(trace tracingpb.Recording, stmt string) // RunAfterSCJobsCacheLookup is called after the SchemaChangeJobCache is checked for // a given table id. @@ -2183,7 +2183,7 @@ type SessionTracing struct { // If recording==true, recordingType indicates the type of the current // recording. - recordingType tracing.RecordingType + recordingType tracingpb.RecordingType // ex is the connExecutor to which this SessionTracing is tied. ex *connExecutor @@ -2204,7 +2204,7 @@ func (st *SessionTracing) getSessionTrace() ([]traceRow, error) { return st.lastRecording, nil } - return generateSessionTraceVTable(st.connSpan.GetRecording(tracing.RecordingVerbose)) + return generateSessionTraceVTable(st.connSpan.GetRecording(tracingpb.RecordingVerbose)) } // StartTracing starts "session tracing". From this moment on, everything @@ -2226,7 +2226,7 @@ func (st *SessionTracing) getSessionTrace() ([]traceRow, error) { // are per-row. // showResults: If set, result rows are reported in the trace. func (st *SessionTracing) StartTracing( - recType tracing.RecordingType, kvTracingEnabled, showResults bool, + recType tracingpb.RecordingType, kvTracingEnabled, showResults bool, ) error { if st.enabled { // We're already tracing. Only treat as no-op if the same options @@ -2268,7 +2268,7 @@ func (st *SessionTracing) StartTracing( opName, tracing.WithForceRealSpan(), ) - st.connSpan.SetRecordingType(tracing.RecordingVerbose) + st.connSpan.SetRecordingType(tracingpb.RecordingVerbose) st.ex.ctxHolder.hijack(newConnCtx) } @@ -2305,14 +2305,14 @@ func (st *SessionTracing) StopTracing() error { st.enabled = false st.kvTracingEnabled = false st.showResults = false - st.recordingType = tracing.RecordingOff + st.recordingType = tracingpb.RecordingOff // Accumulate all recordings and finish the tracing spans. - rec := st.connSpan.GetRecording(tracing.RecordingVerbose) + rec := st.connSpan.GetRecording(tracingpb.RecordingVerbose) // We're about to finish this span, but there might be a child that remains // open - the child corresponding to the current transaction. We don't want // that span to be recording any more. - st.connSpan.SetRecordingType(tracing.RecordingOff) + st.connSpan.SetRecordingType(tracingpb.RecordingOff) st.connSpan.Finish() st.connSpan = nil st.ex.ctxHolder.unhijack() diff --git a/pkg/sql/execinfra/BUILD.bazel b/pkg/sql/execinfra/BUILD.bazel index 585c4037d930..45085797cc41 100644 --- a/pkg/sql/execinfra/BUILD.bazel +++ b/pkg/sql/execinfra/BUILD.bazel @@ -69,6 +69,7 @@ go_library( "//pkg/util/stop", "//pkg/util/timeutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "//pkg/util/uuid", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", diff --git a/pkg/sql/execinfra/processorsbase.go b/pkg/sql/execinfra/processorsbase.go index c8b0da496f7f..a771d6c12911 100644 --- a/pkg/sql/execinfra/processorsbase.go +++ b/pkg/sql/execinfra/processorsbase.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/optional" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" "go.opentelemetry.io/otel/attribute" @@ -366,7 +367,7 @@ type ProcessorBaseNoHelper struct { // been called. This is needed in order to provide the access to the // recording after the span has been finished in InternalClose. Only set if // storeExecStatsTrace is true. - ExecStatsTrace tracing.Recording + ExecStatsTrace tracingpb.Recording // trailingMetaCallback, if set, will be called by moveToTrailingMeta(). The // callback is expected to close all inputs, do other cleanup on the processor // (including calling InternalClose()) and generate the trailing meta that diff --git a/pkg/sql/execstats/stats.go b/pkg/sql/execstats/stats.go index 292211084b41..94d2856c8ef8 100644 --- a/pkg/sql/execstats/stats.go +++ b/pkg/sql/execstats/stats.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/util/optional" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" pbtypes "github.com/gogo/protobuf/types" ) @@ -33,7 +34,7 @@ func ShouldCollectStats(ctx context.Context, collectStats bool) bool { // contention time from the given recording or, if the recording is nil, from // the tracing span from the context. All contention events found in the trace // are included. -func GetCumulativeContentionTime(ctx context.Context, recording tracing.Recording) time.Duration { +func GetCumulativeContentionTime(ctx context.Context, recording tracingpb.Recording) time.Duration { var cumulativeContentionTime time.Duration if recording == nil { recording = tracing.SpanFromContext(ctx).GetConfiguredRecording() @@ -84,7 +85,7 @@ func PopulateKVMVCCStats(kvStats *execinfrapb.KVStats, ss *ScanStats) { // GetScanStats is a helper function to calculate scan stats from the given // recording or, if the recording is nil, from the tracing span from the // context. -func GetScanStats(ctx context.Context, recording tracing.Recording) (ss ScanStats) { +func GetScanStats(ctx context.Context, recording tracingpb.Recording) (ss ScanStats) { if recording == nil { recording = tracing.SpanFromContext(ctx).GetConfiguredRecording() } diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index 56aa93a0f153..9545675f1cd3 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -20,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" - "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" ) // EngineMetrics groups a set of SQL metrics. @@ -282,7 +282,7 @@ func getNodesFromPlanner(planner *planner) []int64 { // Retrieve the list of all nodes which the statement was executed on. var nodes []int64 if planner.instrumentation.sp != nil { - trace := planner.instrumentation.sp.GetRecording(tracing.RecordingStructured) + trace := planner.instrumentation.sp.GetRecording(tracingpb.RecordingStructured) // ForEach returns nodes in order. execinfrapb.ExtractNodesFromSpans(planner.EvalContext().Context, trace).ForEach(func(i int) { nodes = append(nodes, int64(i)) diff --git a/pkg/sql/explain_bundle.go b/pkg/sql/explain_bundle.go index 53d8e7d9bef2..b7084ec33154 100644 --- a/pkg/sql/explain_bundle.go +++ b/pkg/sql/explain_bundle.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/memzipper" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" ) @@ -118,7 +119,7 @@ func buildStatementBundle( ie *InternalExecutor, plan *planTop, planString string, - trace tracing.Recording, + trace tracingpb.Recording, placeholders *tree.PlaceholderInfo, ) diagnosticsBundle { if plan == nil { @@ -176,7 +177,7 @@ type stmtBundleBuilder struct { ie *InternalExecutor plan *planTop - trace tracing.Recording + trace tracingpb.Recording placeholders *tree.PlaceholderInfo z memzipper.Zipper @@ -186,7 +187,7 @@ func makeStmtBundleBuilder( db *kv.DB, ie *InternalExecutor, plan *planTop, - trace tracing.Recording, + trace tracingpb.Recording, placeholders *tree.PlaceholderInfo, ) stmtBundleBuilder { b := stmtBundleBuilder{db: db, ie: ie, plan: plan, trace: trace, placeholders: placeholders} diff --git a/pkg/sql/instrumentation.go b/pkg/sql/instrumentation.go index f45a02b9e7e6..3f1bb33915ae 100644 --- a/pkg/sql/instrumentation.go +++ b/pkg/sql/instrumentation.go @@ -97,7 +97,7 @@ type instrumentationHelper struct { diagRequestID stmtdiagnostics.RequestID diagRequest stmtdiagnostics.Request stmtDiagnosticsRecorder *stmtdiagnostics.Registry - withStatementTrace func(trace tracing.Recording, stmt string) + withStatementTrace func(trace tracingpb.Recording, stmt string) sp *tracing.Span // shouldFinishSpan determines whether sp needs to be finished in @@ -227,7 +227,7 @@ func (ih *instrumentationHelper) Setup( // recording. Stats will be added as structured metadata and processed in // Finish. newCtx, ih.sp = tracing.EnsureChildSpan(ctx, cfg.AmbientCtx.Tracer, "traced statement", - tracing.WithRecording(tracing.RecordingStructured)) + tracing.WithRecording(tracingpb.RecordingStructured)) ih.shouldFinishSpan = true return newCtx, true } @@ -237,7 +237,7 @@ func (ih *instrumentationHelper) Setup( ih.collectExecStats = true ih.traceMetadata = make(execNodeTraceMetadata) ih.evalCtx = p.EvalContext() - newCtx, ih.sp = tracing.EnsureChildSpan(ctx, cfg.AmbientCtx.Tracer, "traced statement", tracing.WithRecording(tracing.RecordingVerbose)) + newCtx, ih.sp = tracing.EnsureChildSpan(ctx, cfg.AmbientCtx.Tracer, "traced statement", tracing.WithRecording(tracingpb.RecordingVerbose)) ih.shouldFinishSpan = true return newCtx, true } @@ -260,7 +260,7 @@ func (ih *instrumentationHelper) Finish( // Record the statement information that we've collected. // Note that in case of implicit transactions, the trace contains the auto-commit too. - var trace tracing.Recording + var trace tracingpb.Recording if ih.shouldFinishSpan { trace = ih.sp.FinishAndGetConfiguredRecording() } else { @@ -502,7 +502,7 @@ func (ih *instrumentationHelper) setExplainAnalyzeResult( phaseTimes *sessionphase.Times, queryLevelStats *execstats.QueryLevelStats, distSQLFlowInfos []flowInfo, - trace tracing.Recording, + trace tracingpb.Recording, ) (commErr error) { res.ResetStmtType(&tree.ExplainAnalyze{}) res.SetColumns(ctx, colinfo.ExplainPlanColumns) diff --git a/pkg/sql/rowexec/BUILD.bazel b/pkg/sql/rowexec/BUILD.bazel index 11d2f6b21f2a..805876bc7a99 100644 --- a/pkg/sql/rowexec/BUILD.bazel +++ b/pkg/sql/rowexec/BUILD.bazel @@ -201,6 +201,7 @@ go_test( "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "@com_github_axiomhq_hyperloglog//:hyperloglog", "@com_github_cockroachdb_errors//:errors", "@com_github_gogo_protobuf//types", diff --git a/pkg/sql/rowexec/inverted_joiner_test.go b/pkg/sql/rowexec/inverted_joiner_test.go index d9773c1d4315..a9edc019bf62 100644 --- a/pkg/sql/rowexec/inverted_joiner_test.go +++ b/pkg/sql/rowexec/inverted_joiner_test.go @@ -37,6 +37,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) @@ -758,7 +759,7 @@ func TestInvertedJoinerDrain(t *testing.T) { td := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t") tracer := s.TracerI().(*tracing.Tracer) - ctx, sp := tracer.StartSpanCtx(context.Background(), "test flow ctx", tracing.WithRecording(tracing.RecordingVerbose)) + ctx, sp := tracer.StartSpanCtx(context.Background(), "test flow ctx", tracing.WithRecording(tracingpb.RecordingVerbose)) defer sp.Finish() st := cluster.MakeTestingClusterSettings() tempEngine, _, err := storage.NewTempEngine(ctx, base.DefaultTestTempStorageConfig(st), base.DefaultTestStoreSpec) diff --git a/pkg/sql/rowexec/joinreader_blackbox_test.go b/pkg/sql/rowexec/joinreader_blackbox_test.go index 036259017ef8..3f6a864f67a1 100644 --- a/pkg/sql/rowexec/joinreader_blackbox_test.go +++ b/pkg/sql/rowexec/joinreader_blackbox_test.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/stretchr/testify/require" ) @@ -35,13 +36,13 @@ func TestJoinReaderUsesBatchLimit(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - recCh := make(chan tracing.Recording, 1) + recCh := make(chan tracingpb.Recording, 1) joinQuery := "SELECT count(1) FROM (SELECT * FROM test.b NATURAL INNER LOOKUP JOIN test.a)" s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{ SQLExecutor: &sql.ExecutorTestingKnobs{ // Get a recording for the join query. - WithStatementTrace: func(trace tracing.Recording, stmt string) { + WithStatementTrace: func(trace tracingpb.Recording, stmt string) { if stmt == joinQuery { recCh <- trace } diff --git a/pkg/sql/rowexec/joinreader_test.go b/pkg/sql/rowexec/joinreader_test.go index 3217acb7888e..40a7d4b3656a 100644 --- a/pkg/sql/rowexec/joinreader_test.go +++ b/pkg/sql/rowexec/joinreader_test.go @@ -49,6 +49,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/stretchr/testify/require" ) @@ -1372,7 +1373,7 @@ func TestJoinReaderDrain(t *testing.T) { // Run the flow in a verbose trace so that we can test for tracing info. tracer := s.TracerI().(*tracing.Tracer) - ctx, sp := tracer.StartSpanCtx(context.Background(), "test flow ctx", tracing.WithRecording(tracing.RecordingVerbose)) + ctx, sp := tracer.StartSpanCtx(context.Background(), "test flow ctx", tracing.WithRecording(tracingpb.RecordingVerbose)) defer sp.Finish() evalCtx := eval.MakeTestingEvalContext(st) diff --git a/pkg/sql/rowexec/tablereader_test.go b/pkg/sql/rowexec/tablereader_test.go index 8ec1a2eba68e..a373c7ec066e 100644 --- a/pkg/sql/rowexec/tablereader_test.go +++ b/pkg/sql/rowexec/tablereader_test.go @@ -39,6 +39,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/gogo/protobuf/types" ) @@ -317,7 +318,7 @@ func TestTableReaderDrain(t *testing.T) { // Run the flow in a verbose trace so that we can test for tracing info. tracer := s.TracerI().(*tracing.Tracer) - ctx, sp := tracer.StartSpanCtx(context.Background(), "test flow ctx", tracing.WithRecording(tracing.RecordingVerbose)) + ctx, sp := tracer.StartSpanCtx(context.Background(), "test flow ctx", tracing.WithRecording(tracingpb.RecordingVerbose)) defer sp.Finish() st := s.ClusterSettings() evalCtx := eval.MakeTestingEvalContext(st) @@ -394,7 +395,7 @@ func TestLimitScans(t *testing.T) { // Now we're going to run the tableReader and trace it. tracer := s.TracerI().(*tracing.Tracer) - sp := tracer.StartSpan("root", tracing.WithRecording(tracing.RecordingVerbose)) + sp := tracer.StartSpan("root", tracing.WithRecording(tracingpb.RecordingVerbose)) ctx = tracing.ContextWithSpan(ctx, sp) flowCtx.EvalCtx.Context = ctx flowCtx.CollectStats = true diff --git a/pkg/sql/rowexec/zigzagjoiner_test.go b/pkg/sql/rowexec/zigzagjoiner_test.go index 861fe9fce386..3c9154453d3c 100644 --- a/pkg/sql/rowexec/zigzagjoiner_test.go +++ b/pkg/sql/rowexec/zigzagjoiner_test.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/stretchr/testify/require" ) @@ -763,7 +764,7 @@ func TestZigzagJoinerDrain(t *testing.T) { // Run the flow in a verbose trace so that we can test for tracing info. tracer := s.TracerI().(*tracing.Tracer) - ctx, sp := tracer.StartSpanCtx(context.Background(), "test flow ctx", tracing.WithRecording(tracing.RecordingVerbose)) + ctx, sp := tracer.StartSpanCtx(context.Background(), "test flow ctx", tracing.WithRecording(tracingpb.RecordingVerbose)) defer sp.Finish() evalCtx := eval.MakeTestingEvalContext(s.ClusterSettings()) defer evalCtx.Stop(ctx) diff --git a/pkg/sql/rowflow/BUILD.bazel b/pkg/sql/rowflow/BUILD.bazel index 2a62826a29b7..b1f6789721db 100644 --- a/pkg/sql/rowflow/BUILD.bazel +++ b/pkg/sql/rowflow/BUILD.bazel @@ -63,6 +63,7 @@ go_test( "//pkg/util/mon", "//pkg/util/randutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "@com_github_cockroachdb_errors//:errors", "@com_github_gogo_protobuf//types", "@com_github_stretchr_testify//require", diff --git a/pkg/sql/rowflow/routers_test.go b/pkg/sql/rowflow/routers_test.go index 8c062bb63a01..c4c72098f256 100644 --- a/pkg/sql/rowflow/routers_test.go +++ b/pkg/sql/rowflow/routers_test.go @@ -39,6 +39,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" pbtypes "github.com/gogo/protobuf/types" "github.com/stretchr/testify/require" @@ -755,7 +756,7 @@ func TestRouterDiskSpill(t *testing.T) { // Enable stats recording. tracer := tracing.NewTracer() - sp := tracer.StartSpan("root", tracing.WithRecording(tracing.RecordingVerbose)) + sp := tracer.StartSpan("root", tracing.WithRecording(tracingpb.RecordingVerbose)) ctx := tracing.ContextWithSpan(context.Background(), sp) st := cluster.MakeTestingClusterSettings() diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index d2267cdfff70..1ce7774c40f3 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -4584,11 +4584,11 @@ value if you rely on the HLC for accuracy.`, return tree.DBoolFalse, nil } - var recType tracing.RecordingType + var recType tracingpb.RecordingType if verbosity { - recType = tracing.RecordingVerbose + recType = tracingpb.RecordingVerbose } else { - recType = tracing.RecordingOff + recType = tracingpb.RecordingOff } rootSpan.SetRecordingType(recType) return tree.DBoolTrue, nil @@ -6783,7 +6783,7 @@ store housing the range on the node it's run from. One of 'mvccGC', 'merge', 'sp shouldReturnTrace := bool(tree.MustBeDBool(args[3])) var foundRepl bool - var rec tracing.Recording + var rec tracingpb.Recording if err := ctx.KVStoresIterator.ForEachStore(func(store kvserverbase.Store) error { var err error rec, err = store.Enqueue(ctx.Context, queue, rangeID, skipShouldQueue) diff --git a/pkg/sql/sem/builtins/generator_builtins.go b/pkg/sql/sem/builtins/generator_builtins.go index 06dd655eb484..0459220d8df4 100644 --- a/pkg/sql/sem/builtins/generator_builtins.go +++ b/pkg/sql/sem/builtins/generator_builtins.go @@ -1892,7 +1892,7 @@ func (p *payloadsForSpanGenerator) Start(_ context.Context, _ *kv.Txn) error { // managing the iterator's position needs to start at -1 instead of 0. p.payloadIndex = -1 - rec := p.span.GetFullRecording(tracing.RecordingStructured) + rec := p.span.GetFullRecording(tracingpb.RecordingStructured) if rec == nil { // No structured records. return nil diff --git a/pkg/sql/sem/builtins/generator_probe_ranges.go b/pkg/sql/sem/builtins/generator_probe_ranges.go index fb6b50c09399..62fb25be390c 100644 --- a/pkg/sql/sem/builtins/generator_probe_ranges.go +++ b/pkg/sql/sem/builtins/generator_probe_ranges.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" ) @@ -144,7 +145,7 @@ func makeProbeRangeGenerator(ctx *eval.Context, args tree.Datums) (eval.ValueGen ctx.Context, ctx.Tracer, "meta2scan", tracing.WithForceRealSpan(), ) - sp.SetRecordingType(tracing.RecordingVerbose) + sp.SetRecordingType(tracingpb.RecordingVerbose) defer func() { sp.Finish() }() @@ -197,7 +198,7 @@ func (p *probeRangeGenerator) Next(ctx context.Context) (bool, error) { ctx, p.tracer, opName, tracing.WithForceRealSpan(), ) - sp.SetRecordingType(tracing.RecordingVerbose) + sp.SetRecordingType(tracingpb.RecordingVerbose) defer func() { p.curr.verboseTrace = sp.FinishAndGetConfiguredRecording().String() }() diff --git a/pkg/sql/trace_test.go b/pkg/sql/trace_test.go index bb1a6542e60e..6809db329f0b 100644 --- a/pkg/sql/trace_test.go +++ b/pkg/sql/trace_test.go @@ -29,7 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/logtags" "github.com/stretchr/testify/require" ) @@ -595,7 +595,7 @@ func TestTraceDistSQL(t *testing.T) { ctx := context.Background() countStmt := "SELECT count(1) FROM test.a" - recCh := make(chan tracing.Recording, 2) + recCh := make(chan tracingpb.Recording, 2) const numNodes = 2 cluster := serverutils.StartNewTestCluster(t, numNodes, base.TestClusterArgs{ @@ -604,7 +604,7 @@ func TestTraceDistSQL(t *testing.T) { UseDatabase: "test", Knobs: base.TestingKnobs{ SQLExecutor: &sql.ExecutorTestingKnobs{ - WithStatementTrace: func(trace tracing.Recording, stmt string) { + WithStatementTrace: func(trace tracingpb.Recording, stmt string) { if stmt == countStmt { recCh <- trace } diff --git a/pkg/sql/txn_state.go b/pkg/sql/txn_state.go index 253b756795d8..2b2ac13d08d7 100644 --- a/pkg/sql/txn_state.go +++ b/pkg/sql/txn_state.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "go.opentelemetry.io/otel/attribute" @@ -181,7 +182,7 @@ func (ts *txnState) resetForNewSQLTxn( duration := traceTxnThreshold.Get(&tranCtx.settings.SV) if alreadyRecording || duration > 0 { txnCtx, sp = tracing.EnsureChildSpan(connCtx, tranCtx.tracer, opName, - tracing.WithRecording(tracing.RecordingVerbose)) + tracing.WithRecording(tracingpb.RecordingVerbose)) } else if ts.testingForceRealTracingSpans { txnCtx, sp = tracing.EnsureChildSpan(connCtx, tranCtx.tracer, opName, tracing.WithForceRealSpan()) } else { diff --git a/pkg/testutils/testcluster/BUILD.bazel b/pkg/testutils/testcluster/BUILD.bazel index 558c6c75c5f0..2b2ea7746594 100644 --- a/pkg/testutils/testcluster/BUILD.bazel +++ b/pkg/testutils/testcluster/BUILD.bazel @@ -27,6 +27,7 @@ go_library( "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_logtags//:logtags", "@com_github_stretchr_testify//require", diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index 3ea2b20de595..1cff73bc27b0 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -43,6 +43,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" "github.com/stretchr/testify/require" @@ -151,7 +152,7 @@ func (tc *TestCluster) stopServers(ctx context.Context) { fmt.Fprintf(&buf, "unexpectedly found %d active spans:\n", len(sps)) var ids []uint64 for _, sp := range sps { - rec := sp.GetFullRecording(tracing.RecordingVerbose) + rec := sp.GetFullRecording(tracingpb.RecordingVerbose) for _, rs := range rec { // NB: it would be a sight easier to just include these in the output of // the string formatted recording, but making a change there presumably requires diff --git a/pkg/upgrade/upgrademanager/BUILD.bazel b/pkg/upgrade/upgrademanager/BUILD.bazel index ae2c81664b49..5b62d116c08d 100644 --- a/pkg/upgrade/upgrademanager/BUILD.bazel +++ b/pkg/upgrade/upgrademanager/BUILD.bazel @@ -59,6 +59,7 @@ go_test( "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//require", "@org_golang_x_sync//errgroup", diff --git a/pkg/upgrade/upgrademanager/manager_external_test.go b/pkg/upgrade/upgrademanager/manager_external_test.go index 98b589d8c380..c7aca2fd51f4 100644 --- a/pkg/upgrade/upgrademanager/manager_external_test.go +++ b/pkg/upgrade/upgrademanager/manager_external_test.go @@ -38,6 +38,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" "golang.org/x/sync/errgroup" @@ -150,7 +151,7 @@ RETURNING id;`).Scan(&secondID)) // another job. We'll make sure this happens by polling the trace to see // the log line indicating what we want. tr := tc.Server(0).TracerI().(*tracing.Tracer) - recCtx, sp := tr.StartSpanCtx(ctx, "test", tracing.WithRecording(tracing.RecordingVerbose)) + recCtx, sp := tr.StartSpanCtx(ctx, "test", tracing.WithRecording(tracingpb.RecordingVerbose)) defer sp.Finish() upgrade2Err := make(chan error, 1) go func() { diff --git a/pkg/util/log/BUILD.bazel b/pkg/util/log/BUILD.bazel index 7e3b69075774..c0aa5e3824d3 100644 --- a/pkg/util/log/BUILD.bazel +++ b/pkg/util/log/BUILD.bazel @@ -174,6 +174,7 @@ go_test( "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_logtags//:logtags", diff --git a/pkg/util/log/ambient_context_test.go b/pkg/util/log/ambient_context_test.go index c48160deb81e..aa0949a16c07 100644 --- a/pkg/util/log/ambient_context_test.go +++ b/pkg/util/log/ambient_context_test.go @@ -15,6 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/logtags" "github.com/stretchr/testify/require" ) @@ -47,7 +48,7 @@ func TestAnnotateCtxSpan(t *testing.T) { // Annotate a context that has an open span. - sp1 := tracer.StartSpan("root", tracing.WithRecording(tracing.RecordingVerbose)) + sp1 := tracer.StartSpan("root", tracing.WithRecording(tracingpb.RecordingVerbose)) ctx1 := tracing.ContextWithSpan(context.Background(), sp1) Event(ctx1, "a") @@ -57,7 +58,7 @@ func TestAnnotateCtxSpan(t *testing.T) { Event(ctx1, "c") sp2.Finish() - if err := tracing.CheckRecordedSpans(sp1.FinishAndGetRecording(tracing.RecordingVerbose), ` + if err := tracing.CheckRecordedSpans(sp1.FinishAndGetRecording(tracingpb.RecordingVerbose), ` span: root tags: _verbose=1 event: a diff --git a/pkg/util/log/clog_test.go b/pkg/util/log/clog_test.go index 7c184de1d491..37e37d3e9d84 100644 --- a/pkg/util/log/clog_test.go +++ b/pkg/util/log/clog_test.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log/severity" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/logtags" "github.com/stretchr/testify/require" ) @@ -798,7 +799,7 @@ func BenchmarkEventf_WithVerboseTraceSpan(b *testing.B) { tracer.SetRedactable(redactable) ctx, sp := tracer.StartSpanCtx(ctx, "benchspan", tracing.WithForceRealSpan()) defer sp.Finish() - sp.SetRecordingType(tracing.RecordingVerbose) + sp.SetRecordingType(tracingpb.RecordingVerbose) b.ResetTimer() for i := 0; i < b.N; i++ { Eventf(ctx, "%s %s %s", "foo", "bar", "baz") diff --git a/pkg/util/log/trace_client_test.go b/pkg/util/log/trace_client_test.go index 64ad9db34c44..5e64f9b51348 100644 --- a/pkg/util/log/trace_client_test.go +++ b/pkg/util/log/trace_client_test.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/logtags" "github.com/stretchr/testify/require" ) @@ -31,12 +32,12 @@ func TestTrace(t *testing.T) { name: "verbose", init: func(ctx context.Context) (context.Context, *tracing.Span) { tracer := tracing.NewTracer() - sp := tracer.StartSpan("s", tracing.WithRecording(tracing.RecordingVerbose)) + sp := tracer.StartSpan("s", tracing.WithRecording(tracingpb.RecordingVerbose)) ctxWithSpan := tracing.ContextWithSpan(ctx, sp) return ctxWithSpan, sp }, check: func(t *testing.T, _ context.Context, sp *tracing.Span, _ *tracing.Tracer) { - rec := sp.FinishAndGetRecording(tracing.RecordingVerbose) + rec := sp.FinishAndGetRecording(tracingpb.RecordingVerbose) if err := tracing.CheckRecordedSpans(rec, ` span: s tags: _verbose=1 @@ -95,7 +96,7 @@ func TestTraceWithTags(t *testing.T) { ctx = logtags.AddTag(ctx, "tag", 1) tracer := tracing.NewTracer() - sp := tracer.StartSpan("s", tracing.WithRecording(tracing.RecordingVerbose)) + sp := tracer.StartSpan("s", tracing.WithRecording(tracingpb.RecordingVerbose)) ctxWithSpan := tracing.ContextWithSpan(ctx, sp) log.Event(ctxWithSpan, "test1") @@ -103,7 +104,7 @@ func TestTraceWithTags(t *testing.T) { log.VErrEvent(ctxWithSpan, log.NoLogV(), "testerr") log.Info(ctxWithSpan, "log") - if err := tracing.CheckRecordedSpans(sp.FinishAndGetRecording(tracing.RecordingVerbose), ` + if err := tracing.CheckRecordedSpans(sp.FinishAndGetRecording(tracingpb.RecordingVerbose), ` span: s tags: _verbose=1 event: [tag=1] test1 diff --git a/pkg/util/log/trace_test.go b/pkg/util/log/trace_test.go index 19353619a838..2b419be14005 100644 --- a/pkg/util/log/trace_test.go +++ b/pkg/util/log/trace_test.go @@ -17,6 +17,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "golang.org/x/net/trace" ) @@ -116,7 +117,7 @@ func TestEventLogAndTrace(t *testing.T) { VErrEvent(ctxWithEventLog, NoLogV(), "testerr") tracer := tracing.NewTracer() - sp := tracer.StartSpan("s", tracing.WithRecording(tracing.RecordingVerbose)) + sp := tracer.StartSpan("s", tracing.WithRecording(tracingpb.RecordingVerbose)) ctxWithBoth := tracing.ContextWithSpan(ctxWithEventLog, sp) // Events should only go to the trace. Event(ctxWithBoth, "test3") @@ -126,7 +127,7 @@ func TestEventLogAndTrace(t *testing.T) { // Events to parent context should still go to the event log. Event(ctxWithEventLog, "test6") - rec := sp.FinishAndGetRecording(tracing.RecordingVerbose) + rec := sp.FinishAndGetRecording(tracingpb.RecordingVerbose) el.Finish() if err := tracing.CheckRecordedSpans(rec, ` diff --git a/pkg/util/tracing/BUILD.bazel b/pkg/util/tracing/BUILD.bazel index a4dda5583e3d..00a79beda475 100644 --- a/pkg/util/tracing/BUILD.bazel +++ b/pkg/util/tracing/BUILD.bazel @@ -7,7 +7,6 @@ go_library( "crdbspan.go", "doc.go", "grpc_interceptor.go", - "recording.go", "span.go", "span_finalizer_race_off.go", "span_finalizer_race_on.go", @@ -38,9 +37,7 @@ go_library( "@com_github_cockroachdb_logtags//:logtags", "@com_github_cockroachdb_redact//:redact", "@com_github_gogo_protobuf//jsonpb", - "@com_github_gogo_protobuf//proto", "@com_github_gogo_protobuf//types", - "@com_github_jaegertracing_jaeger//model/json", "@com_github_petermattis_goid//:goid", "@com_github_pmezard_go_difflib//difflib", "@io_opentelemetry_go_otel//attribute", diff --git a/pkg/util/tracing/bench_test.go b/pkg/util/tracing/bench_test.go index 0db8d10b772e..7a3db23ae3cf 100644 --- a/pkg/util/tracing/bench_test.go +++ b/pkg/util/tracing/bench_test.go @@ -16,6 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/logtags" "github.com/gogo/protobuf/types" ) @@ -97,7 +98,7 @@ func BenchmarkSpan_GetRecording(b *testing.B) { run := func(b *testing.B, sp *Span) { b.ReportAllocs() for i := 0; i < b.N; i++ { - _ = sp.GetRecording(RecordingStructured) + _ = sp.GetRecording(tracingpb.RecordingStructured) } } @@ -137,10 +138,10 @@ func BenchmarkRecordingWithStructuredEvent(b *testing.B) { for i := 0; i < b.N; i++ { var root *Span if tc.withEventListener { - root = tr.StartSpan("foo", WithRecording(RecordingStructured), + root = tr.StartSpan("foo", WithRecording(tracingpb.RecordingStructured), WithEventListeners(mockListener)) } else { - root = tr.StartSpan("foo", WithRecording(RecordingStructured)) + root = tr.StartSpan("foo", WithRecording(tracingpb.RecordingStructured)) } root.RecordStructured(ev) @@ -149,7 +150,7 @@ func BenchmarkRecordingWithStructuredEvent(b *testing.B) { child := tr.StartSpan("bar", WithParent(root)) child.RecordStructured(ev) child.Finish() - _ = root.FinishAndGetRecording(RecordingStructured) + _ = root.FinishAndGetRecording(tracingpb.RecordingStructured) } }) } diff --git a/pkg/util/tracing/collector/collector.go b/pkg/util/tracing/collector/collector.go index 19dc59db8178..26dbd5cddb31 100644 --- a/pkg/util/tracing/collector/collector.go +++ b/pkg/util/tracing/collector/collector.go @@ -75,13 +75,13 @@ type Iterator struct { curNode roachpb.NodeID // recordingIndex maintains the current position of the iterator in the list - // of tracing.Recordings. The tracing.Recording that the iterator points to is + // of tracing.Recordings. The tracingpb.Recording that the iterator points to is // buffered in `recordings`. recordingIndex int // recordings represent all the tracing.Recordings for a given node currently // accessed by the iterator. - recordings []tracing.Recording + recordings []tracingpb.Recording iterErr error } @@ -158,7 +158,7 @@ func (i *Iterator) Next() { } // Value returns the current value pointed to by the Iterator. -func (i *Iterator) Value() (roachpb.NodeID, tracing.Recording) { +func (i *Iterator) Value() (roachpb.NodeID, tracingpb.Recording) { return i.curNode, i.recordings[i.recordingIndex] } @@ -174,7 +174,7 @@ func (i *Iterator) Error() error { // inflight spans, and relies on gRPC short circuiting local requests. func (t *TraceCollector) getTraceSpanRecordingsForNode( ctx context.Context, traceID tracingpb.TraceID, nodeID roachpb.NodeID, -) ([]tracing.Recording, error) { +) ([]tracingpb.Recording, error) { log.Infof(ctx, "getting span recordings from node %s", nodeID.String()) conn, err := t.dialer.Dial(ctx, nodeID, rpc.DefaultClass) if err != nil { @@ -187,7 +187,7 @@ func (t *TraceCollector) getTraceSpanRecordingsForNode( return nil, err } - var res []tracing.Recording + var res []tracingpb.Recording for _, recording := range resp.Recordings { if recording.RecordedSpans == nil { continue diff --git a/pkg/util/tracing/collector/collector_test.go b/pkg/util/tracing/collector/collector_test.go index 8bf916fad4b0..7427e685a297 100644 --- a/pkg/util/tracing/collector/collector_test.go +++ b/pkg/util/tracing/collector/collector_test.go @@ -66,7 +66,7 @@ func newTestStructured(i string) *testStructuredImpl { // root2.child <-- traceID2 func setupTraces(t1, t2 *tracing.Tracer) (tracingpb.TraceID, tracingpb.TraceID, func()) { // Start a root span on "node 1". - root := t1.StartSpan("root", tracing.WithRecording(tracing.RecordingVerbose)) + root := t1.StartSpan("root", tracing.WithRecording(tracingpb.RecordingVerbose)) root.RecordStructured(newTestStructured("root")) time.Sleep(10 * time.Millisecond) @@ -86,10 +86,10 @@ func setupTraces(t1, t2 *tracing.Tracer) (tracingpb.TraceID, tracingpb.TraceID, // Start another remote child span on "node 2" that we finish. childRemoteChildFinished := t2.StartSpan("root.child.remotechilddone", tracing.WithRemoteParentFromSpanMeta(child.Meta())) - child.ImportRemoteRecording(childRemoteChildFinished.FinishAndGetRecording(tracing.RecordingVerbose)) + child.ImportRemoteRecording(childRemoteChildFinished.FinishAndGetRecording(tracingpb.RecordingVerbose)) // Start a root span on "node 2". - root2 := t2.StartSpan("root2", tracing.WithRecording(tracing.RecordingVerbose)) + root2 := t2.StartSpan("root2", tracing.WithRecording(tracingpb.RecordingVerbose)) root2.RecordStructured(newTestStructured("root2")) // Start a child span on "node 2". @@ -127,8 +127,8 @@ func TestTracingCollectorGetSpanRecordings(t *testing.T) { localTraceID, remoteTraceID, cleanup := setupTraces(localTracer, remoteTracer) defer cleanup() - getSpansFromAllNodes := func(traceID tracingpb.TraceID) map[roachpb.NodeID][]tracing.Recording { - res := make(map[roachpb.NodeID][]tracing.Recording) + getSpansFromAllNodes := func(traceID tracingpb.TraceID) map[roachpb.NodeID][]tracingpb.Recording { + res := make(map[roachpb.NodeID][]tracingpb.Recording) var iter *collector.Iterator var err error diff --git a/pkg/util/tracing/crdbspan.go b/pkg/util/tracing/crdbspan.go index 792c51e2d45c..67e8392944e6 100644 --- a/pkg/util/tracing/crdbspan.go +++ b/pkg/util/tracing/crdbspan.go @@ -234,7 +234,7 @@ func (s *crdbSpan) finish() bool { } s.mu.finished = true - if s.recordingType() != RecordingOff { + if s.recordingType() != tracingpb.RecordingOff { duration := timeutil.Since(s.startTime) if duration == 0 { duration = time.Nanosecond @@ -314,17 +314,17 @@ func (s *crdbSpan) finish() bool { return true } -func (s *crdbSpan) recordingType() RecordingType { +func (s *crdbSpan) recordingType() tracingpb.RecordingType { if s == nil { - return RecordingOff + return tracingpb.RecordingOff } return s.mu.recording.recordingType.load() } // enableRecording start recording on the Span. From now on, log events and // child spans will be stored. -func (s *crdbSpan) enableRecording(recType RecordingType) { - if recType == RecordingOff || s.recordingType() == recType { +func (s *crdbSpan) enableRecording(recType tracingpb.RecordingType) { + if recType == tracingpb.RecordingOff || s.recordingType() == recType { return } @@ -347,12 +347,14 @@ func (s *crdbSpan) SpanID() tracingpb.SpanID { // // finishing indicates whether s is in the process of finishing. If it isn't, // the recording will include an "_unfinished" tag. -func (s *crdbSpan) GetRecording(recType RecordingType, finishing bool) Recording { +func (s *crdbSpan) GetRecording( + recType tracingpb.RecordingType, finishing bool, +) tracingpb.Recording { return s.getRecordingImpl(recType, false /* includeDetachedChildren */, finishing) } // GetFullRecording is part of the RegistrySpan interface. -func (s *crdbSpan) GetFullRecording(recType RecordingType) Recording { +func (s *crdbSpan) GetFullRecording(recType tracingpb.RecordingType) tracingpb.Recording { return s.getRecordingImpl(recType, true /* includeDetachedChildren */, false /* finishing */) } @@ -361,14 +363,14 @@ func (s *crdbSpan) GetFullRecording(recType RecordingType) Recording { // finishing indicates whether s is in the process of finishing. If it isn't, // the recording will include an "_unfinished" tag. func (s *crdbSpan) getRecordingImpl( - recType RecordingType, includeDetachedChildren bool, finishing bool, -) Recording { + recType tracingpb.RecordingType, includeDetachedChildren bool, finishing bool, +) tracingpb.Recording { switch recType { - case RecordingVerbose: + case tracingpb.RecordingVerbose: return s.getVerboseRecording(includeDetachedChildren, finishing) - case RecordingStructured: + case tracingpb.RecordingStructured: return s.getStructuredRecording(includeDetachedChildren) - case RecordingOff: + case tracingpb.RecordingOff: return nil default: panic("unreachable") @@ -379,7 +381,9 @@ func (s *crdbSpan) getRecordingImpl( // // finishing indicates whether s is in the process of finishing. If it isn't, // the recording will include an "_unfinished" tag. -func (s *crdbSpan) getVerboseRecording(includeDetachedChildren bool, finishing bool) Recording { +func (s *crdbSpan) getVerboseRecording( + includeDetachedChildren bool, finishing bool, +) tracingpb.Recording { if s == nil { return nil // noop span } @@ -387,8 +391,8 @@ func (s *crdbSpan) getVerboseRecording(includeDetachedChildren bool, finishing b s.mu.Lock() // The capacity here is approximate since we don't know how many // grandchildren there are. - result := make(Recording, 0, 1+len(s.mu.openChildren)+len(s.mu.recording.finishedChildren)) - result = append(result, s.getRecordingNoChildrenLocked(RecordingVerbose, finishing)) + result := make(tracingpb.Recording, 0, 1+len(s.mu.openChildren)+len(s.mu.recording.finishedChildren)) + result = append(result, s.getRecordingNoChildrenLocked(tracingpb.RecordingVerbose, finishing)) result = append(result, s.mu.recording.finishedChildren...) for _, child := range s.mu.openChildren { @@ -401,7 +405,7 @@ func (s *crdbSpan) getVerboseRecording(includeDetachedChildren bool, finishing b // Sort the spans by StartTime, except the first Span (the root of this // recording) which stays in place. - toSort := sortPool.Get().(*Recording) // avoids allocations in sort.Sort + toSort := sortPool.Get().(*tracingpb.Recording) // avoids allocations in sort.Sort *toSort = result[1:] sort.Sort(toSort) *toSort = nil @@ -417,7 +421,7 @@ func (s *crdbSpan) getVerboseRecording(includeDetachedChildren bool, finishing b // this span (even if the events had been recorded on different spans). // // The caller does not take ownership of the events. -func (s *crdbSpan) getStructuredRecording(includeDetachedChildren bool) Recording { +func (s *crdbSpan) getStructuredRecording(includeDetachedChildren bool) tracingpb.Recording { s.mu.Lock() defer s.mu.Unlock() buffer := make([]*tracingpb.StructuredRecord, 0, 3) @@ -439,7 +443,7 @@ func (s *crdbSpan) getStructuredRecording(includeDetachedChildren bool) Recordin } res := s.getRecordingNoChildrenLocked( - RecordingStructured, + tracingpb.RecordingStructured, false, // finishing - since we're only asking for the structured recording, the argument doesn't matter ) // If necessary, grow res.StructuredRecords to have space for buffer. @@ -462,13 +466,13 @@ func (s *crdbSpan) getStructuredRecording(includeDetachedChildren bool) Recordin for i, e := range buffer { reservedSpace[i] = *e } - return Recording{res} + return tracingpb.Recording{res} } // recordFinishedChildren adds the spans in childRecording to s' recording. // // s takes ownership of childRecording; the caller is not allowed to use them anymore. -func (s *crdbSpan) recordFinishedChildren(childRecording Recording) { +func (s *crdbSpan) recordFinishedChildren(childRecording tracingpb.Recording) { if len(childRecording) == 0 { return } @@ -488,7 +492,7 @@ func (s *crdbSpan) recordFinishedChildren(childRecording Recording) { // s takes ownership of childRecording; the caller is not allowed to use them // anymore. -func (s *crdbSpan) recordFinishedChildrenLocked(childRecording Recording) { +func (s *crdbSpan) recordFinishedChildrenLocked(childRecording tracingpb.Recording) { if len(childRecording) == 0 { return } @@ -496,7 +500,7 @@ func (s *crdbSpan) recordFinishedChildrenLocked(childRecording Recording) { // Depending on the type of recording, we either keep all the information // received, or only the structured events. switch s.recordingType() { - case RecordingVerbose: + case tracingpb.RecordingVerbose: // Change the root of the recording to be a child of this Span. This is // usually already the case, except with DistSQL traces where remote // processors run in spans that FollowFrom an RPC Span that we don't @@ -511,14 +515,14 @@ func (s *crdbSpan) recordFinishedChildrenLocked(childRecording Recording) { // We don't have space for this recording. Let's collect just the structured // records by falling through. fallthrough - case RecordingStructured: + case tracingpb.RecordingStructured: for ci := range childRecording { child := &childRecording[ci] for i := range child.StructuredRecords { s.recordInternalLocked(&child.StructuredRecords[i], &s.mu.recording.structured) } } - case RecordingOff: + case tracingpb.RecordingOff: break default: panic(fmt.Sprintf("unrecognized recording mode: %v", s.recordingType())) @@ -604,7 +608,7 @@ func (s *crdbSpan) notifyEventListeners(item Structured) { // record includes a log message in s' recording. func (s *crdbSpan) record(msg redact.RedactableString) { - if s.recordingType() != RecordingVerbose { + if s.recordingType() != tracingpb.RecordingVerbose { return } @@ -628,7 +632,7 @@ func (s *crdbSpan) record(msg redact.RedactableString) { // recordStructured includes a structured event in s' recording. func (s *crdbSpan) recordStructured(item Structured) { - if s.recordingType() == RecordingOff { + if s.recordingType() == tracingpb.RecordingOff { return } @@ -737,7 +741,7 @@ func (s *crdbSpan) getStructuredEventsLocked( // finishing indicates whether s is in the process of finishing. If it isn't, // the recording will include an "_unfinished" tag. func (s *crdbSpan) getRecordingNoChildrenLocked( - recordingType RecordingType, finishing bool, + recordingType tracingpb.RecordingType, finishing bool, ) tracingpb.RecordedSpan { rs := tracingpb.RecordedSpan{ TraceID: s.traceID, @@ -748,7 +752,7 @@ func (s *crdbSpan) getRecordingNoChildrenLocked( StartTime: s.startTime, Duration: s.mu.duration, RedactableLogs: true, - Verbose: s.recordingType() == RecordingVerbose, + Verbose: s.recordingType() == tracingpb.RecordingVerbose, RecordingMode: s.recordingType().ToProto(), } @@ -776,12 +780,12 @@ func (s *crdbSpan) getRecordingNoChildrenLocked( // stringifying a {NodeID,StoreID}Container (a very common tag) was expensive. // That has become cheap since, so this optimization might not be worth it any // more. - wantTags := recordingType == RecordingVerbose + wantTags := recordingType == tracingpb.RecordingVerbose if wantTags { if !finishing && !s.mu.finished { addTag("_unfinished", "1") } - if s.recordingType() == RecordingVerbose { + if s.recordingType() == tracingpb.RecordingVerbose { addTag("_verbose", "1") } if s.mu.recording.dropped { @@ -928,7 +932,7 @@ func (s *crdbSpan) childFinished(child *crdbSpan) { // Collect the child's recording. - if s.recordingType() == RecordingOff || !collectChildRec { + if s.recordingType() == tracingpb.RecordingOff || !collectChildRec { return } @@ -957,7 +961,7 @@ func (s *crdbSpan) visitOpenChildren(visitor func(child *Span)) { } // SetRecordingType is part of the RegistrySpan interface. -func (s *crdbSpan) SetRecordingType(to RecordingType) { +func (s *crdbSpan) SetRecordingType(to tracingpb.RecordingType) { s.mu.recording.recordingType.swap(to) s.mu.Lock() @@ -968,7 +972,7 @@ func (s *crdbSpan) SetRecordingType(to RecordingType) { } // RecordingType is part of the RegistrySpan interface. -func (s *crdbSpan) RecordingType() RecordingType { +func (s *crdbSpan) RecordingType() tracingpb.RecordingType { return s.recordingType() } @@ -995,33 +999,18 @@ func (s *crdbSpan) setGoroutineID(gid int64) { var sortPool = sync.Pool{ New: func() interface{} { - return &Recording{} + return &tracingpb.Recording{} }, } -// Less implements sort.Interface. -func (r Recording) Less(i, j int) bool { - return r[i].StartTime.Before(r[j].StartTime) -} - -// Swap implements sort.Interface. -func (r Recording) Swap(i, j int) { - r[i], r[j] = r[j], r[i] -} - -// Len implements sort.Interface. -func (r Recording) Len() int { - return len(r) -} - -type atomicRecordingType RecordingType +type atomicRecordingType tracingpb.RecordingType // load returns the recording type. -func (art *atomicRecordingType) load() RecordingType { - return RecordingType(atomic.LoadInt32((*int32)(art))) +func (art *atomicRecordingType) load() tracingpb.RecordingType { + return tracingpb.RecordingType(atomic.LoadInt32((*int32)(art))) } // swap stores the new recording type and returns the old one. -func (art *atomicRecordingType) swap(recType RecordingType) RecordingType { - return RecordingType(atomic.SwapInt32((*int32)(art), int32(recType))) +func (art *atomicRecordingType) swap(recType tracingpb.RecordingType) tracingpb.RecordingType { + return tracingpb.RecordingType(atomic.SwapInt32((*int32)(art), int32(recType))) } diff --git a/pkg/util/tracing/grpc_interceptor.go b/pkg/util/tracing/grpc_interceptor.go index 8ccdc171ba7e..be4c4ee8fc83 100644 --- a/pkg/util/tracing/grpc_interceptor.go +++ b/pkg/util/tracing/grpc_interceptor.go @@ -16,7 +16,6 @@ import ( "strings" "sync/atomic" - "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/codes" "google.golang.org/grpc" @@ -446,7 +445,3 @@ func (cs *tracingClientStream) CloseSend() error { } return err } - -// Recording represents a group of RecordedSpans rooted at a fixed root span, as -// returned by GetRecording. Spans are sorted by StartTime. -type Recording []tracingpb.RecordedSpan diff --git a/pkg/util/tracing/grpc_interceptor_test.go b/pkg/util/tracing/grpc_interceptor_test.go index b2b3b472db7e..c78127122164 100644 --- a/pkg/util/tracing/grpc_interceptor_test.go +++ b/pkg/util/tracing/grpc_interceptor_test.go @@ -63,7 +63,7 @@ func TestGRPCInterceptors(t *testing.T) { return nil, errors.New("no span in ctx") } sp.RecordStructured(newTestStructured(magicValue)) - recs := sp.GetRecording(tracing.RecordingVerbose) + recs := sp.GetRecording(tracingpb.RecordingVerbose) if len(recs) != 1 { return nil, errors.Newf("expected exactly one recorded span, not %+v", recs) } @@ -246,7 +246,7 @@ func TestGRPCInterceptors(t *testing.T) { c := grpcutils.NewGRPCTestClient(conn) require.NoError(t, err) - ctx, sp := tr.StartSpanCtx(bgCtx, "root", tracing.WithRecording(tracing.RecordingVerbose)) + ctx, sp := tr.StartSpanCtx(bgCtx, "root", tracing.WithRecording(tracingpb.RecordingVerbose)) recAny, err := tc.do(ctx, c) require.NoError(t, err) var rec tracingpb.RecordedSpan @@ -254,7 +254,7 @@ func TestGRPCInterceptors(t *testing.T) { require.Len(t, rec.StructuredRecords, 1) sp.ImportRemoteRecording([]tracingpb.RecordedSpan{rec}) var n int - finalRecs := sp.FinishAndGetRecording(tracing.RecordingVerbose) + finalRecs := sp.FinishAndGetRecording(tracingpb.RecordingVerbose) for _, rec := range finalRecs { n += len(rec.StructuredRecords) // Remove all of the _unfinished tags. These crop up because @@ -286,7 +286,7 @@ func TestGRPCInterceptors(t *testing.T) { // immediate) in the ctx cancellation subtest. testutils.SucceedsSoon(t, func() error { return tr.VisitSpans(func(sp tracing.RegistrySpan) error { - rec := sp.GetFullRecording(tracing.RecordingVerbose)[0] + rec := sp.GetFullRecording(tracingpb.RecordingVerbose)[0] return errors.Newf("leaked span: %s %s", rec.Operation, rec.Tags) }) }) diff --git a/pkg/util/tracing/service/BUILD.bazel b/pkg/util/tracing/service/BUILD.bazel index 7abfed6af271..142ac05fa5d8 100644 --- a/pkg/util/tracing/service/BUILD.bazel +++ b/pkg/util/tracing/service/BUILD.bazel @@ -7,6 +7,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "//pkg/util/tracing/tracingservicepb", ], ) diff --git a/pkg/util/tracing/service/service.go b/pkg/util/tracing/service/service.go index 61d314d011ac..17f0841bd4d3 100644 --- a/pkg/util/tracing/service/service.go +++ b/pkg/util/tracing/service/service.go @@ -24,6 +24,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingservicepb" ) @@ -43,7 +44,7 @@ func New(tracer *tracing.Tracer) *Service { // GetSpanRecordings implements the tracingpb.TraceServer interface. // // This method iterates over all active root spans registered with the nodes' -// local inflight span registry, and returns a tracing.Recording for each root +// local inflight span registry, and returns a tracingpb.Recording for each root // span with a matching trace_id. func (s *Service) GetSpanRecordings( _ context.Context, request *tracingservicepb.GetSpanRecordingsRequest, @@ -53,7 +54,7 @@ func (s *Service) GetSpanRecordings( if span.TraceID() != request.TraceID { return nil } - recording := span.GetFullRecording(tracing.RecordingVerbose) + recording := span.GetFullRecording(tracingpb.RecordingVerbose) if recording != nil { resp.Recordings = append(resp.Recordings, tracingservicepb.GetSpanRecordingsResponse_Recording{RecordedSpans: recording}) diff --git a/pkg/util/tracing/service/service_test.go b/pkg/util/tracing/service/service_test.go index 6eb9073c8039..9f1e70672976 100644 --- a/pkg/util/tracing/service/service_test.go +++ b/pkg/util/tracing/service/service_test.go @@ -29,7 +29,7 @@ func TestTracingServiceGetSpanRecordings(t *testing.T) { tracer1 := tracing.NewTracer() setupTraces := func() (tracingpb.TraceID, func()) { // Start a root span. - root1 := tracer1.StartSpan("root1", tracing.WithRecording(tracing.RecordingVerbose)) + root1 := tracer1.StartSpan("root1", tracing.WithRecording(tracingpb.RecordingVerbose)) child1 := tracer1.StartSpan("root1.child", tracing.WithParent(root1)) child2 := tracer1.StartSpan("root1.child.detached", tracing.WithParent(child1), tracing.WithDetachedRecording()) // Create a span that will be added to the tracers' active span map, but @@ -39,7 +39,7 @@ func TestTracingServiceGetSpanRecordings(t *testing.T) { time.Sleep(10 * time.Millisecond) // Start span with different trace ID. - root2 := tracer1.StartSpan("root2", tracing.WithRecording(tracing.RecordingVerbose)) + root2 := tracer1.StartSpan("root2", tracing.WithRecording(tracingpb.RecordingVerbose)) root2.Record("root2") return root1.TraceID(), func() { diff --git a/pkg/util/tracing/span.go b/pkg/util/tracing/span.go index b2c123e84dce..e769dd892b67 100644 --- a/pkg/util/tracing/span.go +++ b/pkg/util/tracing/span.go @@ -274,9 +274,9 @@ func (sp *Span) finishInternal() { // // Returns nil if the span is not currently recording (even if it had been // recording in the past). -func (sp *Span) FinishAndGetRecording(recType RecordingType) Recording { - rec := Recording(nil) - if sp.RecordingType() != RecordingOff { +func (sp *Span) FinishAndGetRecording(recType tracingpb.RecordingType) tracingpb.Recording { + rec := tracingpb.Recording(nil) + if sp.RecordingType() != tracingpb.RecordingOff { rec = sp.i.GetRecording(recType, true /* finishing */) } // Reach directly into sp.i to pass the finishing argument. @@ -290,10 +290,10 @@ func (sp *Span) FinishAndGetRecording(recType RecordingType) Recording { // // Returns nil if the span is not currently recording (even if it had been // recording in the past). -func (sp *Span) FinishAndGetConfiguredRecording() Recording { - rec := Recording(nil) +func (sp *Span) FinishAndGetConfiguredRecording() tracingpb.Recording { + rec := tracingpb.Recording(nil) recType := sp.RecordingType() - if recType != RecordingOff { + if recType != tracingpb.RecordingOff { rec = sp.i.GetRecording(recType, true /* finishing */) } // Reach directly into sp.i to pass the finishing argument. @@ -326,11 +326,11 @@ func (sp *Span) FinishAndGetConfiguredRecording() Recording { // // If recType is RecordingStructured, the return value will be nil if the span // doesn't have any structured events. -func (sp *Span) GetRecording(recType RecordingType) Recording { +func (sp *Span) GetRecording(recType tracingpb.RecordingType) tracingpb.Recording { if sp.detectUseAfterFinish() { return nil } - if sp.RecordingType() == RecordingOff { + if sp.RecordingType() == tracingpb.RecordingOff { return nil } return sp.i.GetRecording(recType, false /* finishing */) @@ -341,12 +341,12 @@ func (sp *Span) GetRecording(recType RecordingType) Recording { // // Returns nil if the span is not currently recording (even if it had been // recording in the past). -func (sp *Span) GetConfiguredRecording() Recording { +func (sp *Span) GetConfiguredRecording() tracingpb.Recording { if sp.detectUseAfterFinish() { return nil } recType := sp.RecordingType() - if recType == RecordingOff { + if recType == tracingpb.RecordingOff { return nil } return sp.i.GetRecording(recType, false /* finishing */) @@ -357,7 +357,7 @@ func (sp *Span) GetConfiguredRecording() Recording { // GetRecording() output for the receiver. // // This function is used to import a recording from another node. -func (sp *Span) ImportRemoteRecording(remoteRecording Recording) { +func (sp *Span) ImportRemoteRecording(remoteRecording tracingpb.Recording) { if !sp.detectUseAfterFinish() { sp.i.ImportRemoteRecording(remoteRecording) } @@ -377,7 +377,7 @@ func (sp *Span) Meta() SpanMeta { // SetRecordingType sets the recording mode of the span and its children, // recursively. Setting it to RecordingOff disables further recording. // Everything recorded so far remains in memory. -func (sp *Span) SetRecordingType(to RecordingType) { +func (sp *Span) SetRecordingType(to tracingpb.RecordingType) { if sp.detectUseAfterFinish() { return } @@ -385,16 +385,16 @@ func (sp *Span) SetRecordingType(to RecordingType) { } // RecordingType returns the range's current recording mode. -func (sp *Span) RecordingType() RecordingType { +func (sp *Span) RecordingType() tracingpb.RecordingType { if sp.detectUseAfterFinish() { - return RecordingOff + return tracingpb.RecordingOff } return sp.i.RecordingType() } // IsVerbose returns true if the Span is verbose. See SetVerbose for details. func (sp *Span) IsVerbose() bool { - return sp.RecordingType() == RecordingVerbose + return sp.RecordingType() == tracingpb.RecordingVerbose } // Record provides a way to record free-form text into verbose spans. Recordings @@ -812,7 +812,7 @@ type SpanMeta struct { otelCtx oteltrace.SpanContext // If set, all spans derived from this context are being recorded. - recordingType RecordingType + recordingType tracingpb.RecordingType // sterile is set if this span does not want to have children spans. In that // case, trying to create a child span will result in the would-be child being @@ -882,13 +882,13 @@ func SpanMetaFromProto(info tracingpb.TraceInfo) SpanMeta { } switch info.RecordingMode { case tracingpb.RecordingMode_OFF: - sm.recordingType = RecordingOff + sm.recordingType = tracingpb.RecordingOff case tracingpb.RecordingMode_STRUCTURED: - sm.recordingType = RecordingStructured + sm.recordingType = tracingpb.RecordingStructured case tracingpb.RecordingMode_VERBOSE: - sm.recordingType = RecordingVerbose + sm.recordingType = tracingpb.RecordingVerbose default: - sm.recordingType = RecordingOff + sm.recordingType = tracingpb.RecordingOff } return sm } diff --git a/pkg/util/tracing/span_inner.go b/pkg/util/tracing/span_inner.go index d1f66b656ac2..1e27f245ed81 100644 --- a/pkg/util/tracing/span_inner.go +++ b/pkg/util/tracing/span_inner.go @@ -65,11 +65,11 @@ func (s *spanInner) isSterile() bool { return s.sterile } -func (s *spanInner) RecordingType() RecordingType { +func (s *spanInner) RecordingType() tracingpb.RecordingType { return s.crdb.recordingType() } -func (s *spanInner) SetRecordingType(to RecordingType) { +func (s *spanInner) SetRecordingType(to tracingpb.RecordingType) { if s.isNoop() { panic(errors.AssertionFailedf("SetVerbose called on NoopSpan; use the WithForceRealSpan option for StartSpan")) } @@ -80,7 +80,9 @@ func (s *spanInner) SetRecordingType(to RecordingType) { // // finishing indicates whether s is in the process of finishing. If it isn't, // the recording will include an "_unfinished" tag. -func (s *spanInner) GetRecording(recType RecordingType, finishing bool) Recording { +func (s *spanInner) GetRecording( + recType tracingpb.RecordingType, finishing bool, +) tracingpb.Recording { if s.isNoop() { return nil } @@ -112,7 +114,7 @@ func (s *spanInner) Finish() { func (s *spanInner) Meta() SpanMeta { var traceID tracingpb.TraceID var spanID tracingpb.SpanID - var recordingType RecordingType + var recordingType tracingpb.RecordingType var sterile bool if s.crdb != nil { @@ -237,7 +239,7 @@ func (s *spanInner) Recordf(format string, args ...interface{}) { // hasVerboseSink returns false if there is no reason to even evaluate Record // because the result wouldn't be used for anything. func (s *spanInner) hasVerboseSink() bool { - if s.netTr == nil && s.otelSpan == nil && s.RecordingType() != RecordingVerbose { + if s.netTr == nil && s.otelSpan == nil && s.RecordingType() != tracingpb.RecordingVerbose { return false } return true diff --git a/pkg/util/tracing/span_options.go b/pkg/util/tracing/span_options.go index 997c27d83e4d..260073f325fc 100644 --- a/pkg/util/tracing/span_options.go +++ b/pkg/util/tracing/span_options.go @@ -81,7 +81,7 @@ type spanOptions struct { // that case, spanOptions.recordingType() returns recordingTypeOpt below. If // not set, recordingType() looks at the parent. recordingTypeExplicit bool - recordingTypeOpt RecordingType + recordingTypeOpt tracingpb.RecordingType } func (opts *spanOptions) parentTraceID() tracingpb.TraceID { @@ -102,12 +102,12 @@ func (opts *spanOptions) parentSpanID() tracingpb.SpanID { return 0 } -func (opts *spanOptions) recordingType() RecordingType { +func (opts *spanOptions) recordingType() tracingpb.RecordingType { if opts.recordingTypeExplicit { return opts.recordingTypeOpt } - recordingType := RecordingOff + recordingType := tracingpb.RecordingOff if !opts.Parent.empty() && !opts.Parent.IsNoop() { recordingType = opts.Parent.i.crdb.recordingType() } else if !opts.RemoteParent.Empty() { @@ -369,22 +369,22 @@ func (forceRealSpanOption) apply(opts spanOptions) spanOptions { } type recordingSpanOption struct { - recType RecordingType + recType tracingpb.RecordingType } -var structuredRecordingSingleton = SpanOption(recordingSpanOption{recType: RecordingStructured}) -var verboseRecordingSingleton = SpanOption(recordingSpanOption{recType: RecordingVerbose}) +var structuredRecordingSingleton = SpanOption(recordingSpanOption{recType: tracingpb.RecordingStructured}) +var verboseRecordingSingleton = SpanOption(recordingSpanOption{recType: tracingpb.RecordingVerbose}) // WithRecording configures the span to record in the given mode. // // The recording mode can be changed later with SetVerbose(). -func WithRecording(recType RecordingType) SpanOption { +func WithRecording(recType tracingpb.RecordingType) SpanOption { switch recType { - case RecordingStructured: + case tracingpb.RecordingStructured: return structuredRecordingSingleton - case RecordingVerbose: + case tracingpb.RecordingVerbose: return verboseRecordingSingleton - case RecordingOff: + case tracingpb.RecordingOff: panic("invalid recording option: RecordingOff") default: recCpy := recType // copy excaping to the heap @@ -451,7 +451,7 @@ func (ev eventListenersOption) apply(opts spanOptions) spanOptions { // will be respected instead. if !opts.recordingTypeExplicit { opts.recordingTypeExplicit = true - opts.recordingTypeOpt = RecordingStructured + opts.recordingTypeOpt = tracingpb.RecordingStructured } eventListeners := ([]EventListener)(ev) opts.EventListeners = eventListeners diff --git a/pkg/util/tracing/span_test.go b/pkg/util/tracing/span_test.go index d240102d8b52..c7c7876c8b8f 100644 --- a/pkg/util/tracing/span_test.go +++ b/pkg/util/tracing/span_test.go @@ -39,7 +39,7 @@ func TestStartSpan(t *testing.T) { defer sp.Finish() require.Equal(t, "noop", sp.OperationName()) - sp2 := tr.StartSpan("test", WithRecording(RecordingStructured)) + sp2 := tr.StartSpan("test", WithRecording(tracingpb.RecordingStructured)) defer sp2.Finish() require.Equal(t, "test", sp2.OperationName()) } @@ -48,7 +48,7 @@ func TestRecordingString(t *testing.T) { tr := NewTracer() tr2 := NewTracer() - root := tr.StartSpan("root", WithRecording(RecordingVerbose)) + root := tr.StartSpan("root", WithRecording(tracingpb.RecordingVerbose)) root.Record("root 1") { // Hackily fix the timing on the first log message, so that we can check it later. @@ -72,7 +72,7 @@ func TestRecordingString(t *testing.T) { root.Record("root 2") remoteChild.Record("remote child 1") - remoteRec := remoteChild.FinishAndGetRecording(RecordingVerbose) + remoteRec := remoteChild.FinishAndGetRecording(tracingpb.RecordingVerbose) root.ImportRemoteRecording(remoteRec) root.Record("root 3") @@ -84,7 +84,7 @@ func TestRecordingString(t *testing.T) { root.Record("root 5") - rec := root.FinishAndGetRecording(RecordingVerbose) + rec := root.FinishAndGetRecording(tracingpb.RecordingVerbose) // Sanity check that the recording looks like we want. Note that this is not // its String() representation; this just lists all the spans in order. require.NoError(t, CheckRecordedSpans(rec, ` @@ -156,14 +156,14 @@ func parseLine(s string) (traceLine, error) { func TestRecordingInRecording(t *testing.T) { tr := NewTracer() - root := tr.StartSpan("root", WithRecording(RecordingVerbose)) - child := tr.StartSpan("child", WithParent(root), WithRecording(RecordingVerbose)) + root := tr.StartSpan("root", WithRecording(tracingpb.RecordingVerbose)) + child := tr.StartSpan("child", WithParent(root), WithRecording(tracingpb.RecordingVerbose)) // The remote grandchild is also recording, however since it's remote the spans // have to be imported into the parent manually (this would usually happen via // code at the RPC boundaries). grandChild := tr.StartSpan("grandchild", WithParent(child), WithDetachedRecording()) - child.ImportRemoteRecording(grandChild.FinishAndGetRecording(RecordingVerbose)) - childRec := child.FinishAndGetRecording(RecordingVerbose) + child.ImportRemoteRecording(grandChild.FinishAndGetRecording(tracingpb.RecordingVerbose)) + childRec := child.FinishAndGetRecording(tracingpb.RecordingVerbose) require.NoError(t, CheckRecordedSpans(childRec, ` span: child tags: _verbose=1 @@ -171,7 +171,7 @@ func TestRecordingInRecording(t *testing.T) { tags: _verbose=1 `)) - rootRec := root.FinishAndGetRecording(RecordingVerbose) + rootRec := root.FinishAndGetRecording(tracingpb.RecordingVerbose) require.NoError(t, CheckRecordedSpans(rootRec, ` span: root tags: _verbose=1 @@ -195,18 +195,18 @@ func TestImportRemoteRecording(t *testing.T) { tr := NewTracerWithOpt(context.Background()) var opt SpanOption if verbose { - opt = WithRecording(RecordingVerbose) + opt = WithRecording(tracingpb.RecordingVerbose) } else { - opt = WithRecording(RecordingStructured) + opt = WithRecording(tracingpb.RecordingStructured) } sp := tr.StartSpan("root", opt) ch := tr.StartSpan("child", WithParent(sp), WithDetachedRecording()) ch.RecordStructured(&types.Int32Value{Value: 4}) ch.Record("foo") - sp.ImportRemoteRecording(ch.FinishAndGetRecording(RecordingVerbose)) + sp.ImportRemoteRecording(ch.FinishAndGetRecording(tracingpb.RecordingVerbose)) if verbose { - require.NoError(t, CheckRecording(sp.FinishAndGetRecording(RecordingVerbose), ` + require.NoError(t, CheckRecording(sp.FinishAndGetRecording(tracingpb.RecordingVerbose), ` === operation:root _verbose:1 === operation:child _verbose:1 event:&Int32Value{Value:4,XXX_unrecognized:[],} @@ -214,7 +214,7 @@ func TestImportRemoteRecording(t *testing.T) { structured:{"@type":"type.googleapis.com/google.protobuf.Int32Value","value":4} `)) } else { - require.NoError(t, CheckRecording(sp.FinishAndGetRecording(RecordingStructured), ` + require.NoError(t, CheckRecording(sp.FinishAndGetRecording(tracingpb.RecordingStructured), ` === operation:root structured:{"@type":"type.googleapis.com/google.protobuf.Int32Value","value":4} `)) @@ -226,12 +226,12 @@ func TestImportRemoteRecording(t *testing.T) { func TestImportRemoteRecordingMaintainsRightByteSize(t *testing.T) { tr1 := NewTracer() - child := tr1.StartSpan("child", WithRecording(RecordingStructured)) + child := tr1.StartSpan("child", WithRecording(tracingpb.RecordingStructured)) child.RecordStructured(&types.Int32Value{Value: 42}) child.RecordStructured(&types.StringValue{Value: "test"}) - root := tr1.StartSpan("root", WithRecording(RecordingStructured)) - root.ImportRemoteRecording(child.GetRecording(RecordingStructured)) + root := tr1.StartSpan("root", WithRecording(tracingpb.RecordingStructured)) + root.ImportRemoteRecording(child.GetRecording(tracingpb.RecordingStructured)) c := root.i.crdb c.mu.Lock() buf := c.mu.recording.structured @@ -246,11 +246,11 @@ func TestImportRemoteRecordingMaintainsRightByteSize(t *testing.T) { func TestSpanRecordStructured(t *testing.T) { tr := NewTracer() - sp := tr.StartSpan("root", WithRecording(RecordingStructured)) + sp := tr.StartSpan("root", WithRecording(tracingpb.RecordingStructured)) defer sp.Finish() sp.RecordStructured(&types.Int32Value{Value: 4}) - rec := sp.GetRecording(RecordingStructured) + rec := sp.GetRecording(tracingpb.RecordingStructured) require.Len(t, rec, 1) require.Len(t, rec[0].StructuredRecords, 1) item := rec[0].StructuredRecords[0] @@ -274,7 +274,7 @@ func TestSpanRecordStructuredLimit(t *testing.T) { clock := timeutil.NewManualTime(now) tr := NewTracerWithOpt(context.Background(), WithTestingKnobs(TracerTestingKnobs{Clock: clock})) - sp := tr.StartSpan("root", WithRecording(RecordingStructured)) + sp := tr.StartSpan("root", WithRecording(tracingpb.RecordingStructured)) defer sp.Finish() pad := func(i int) string { return fmt.Sprintf("%06d", i) } @@ -292,8 +292,8 @@ func TestSpanRecordStructuredLimit(t *testing.T) { sp.RecordStructured(payload(i)) } - sp.SetRecordingType(RecordingVerbose) - rec := sp.GetRecording(RecordingVerbose) + sp.SetRecordingType(tracingpb.RecordingVerbose) + rec := sp.GetRecording(tracingpb.RecordingVerbose) require.Len(t, rec, 1) require.Len(t, rec[0].StructuredRecords, numStructuredRecordings) require.Equal(t, "1", rec[0].Tags["_dropped"]) @@ -327,13 +327,13 @@ func TestSpanRecordLimit(t *testing.T) { // Determine the size of a log record by actually recording once. logSize := func() int { - sp := tr.StartSpan("dummy", WithRecording(RecordingVerbose)) + sp := tr.StartSpan("dummy", WithRecording(tracingpb.RecordingVerbose)) defer sp.Finish() sp.Recordf("%s", msg(42)) - return sp.GetRecording(RecordingVerbose)[0].Logs[0].MemorySize() + return sp.GetRecording(tracingpb.RecordingVerbose)[0].Logs[0].MemorySize() }() - sp := tr.StartSpan("root", WithRecording(RecordingVerbose)) + sp := tr.StartSpan("root", WithRecording(tracingpb.RecordingVerbose)) defer sp.Finish() numLogs := maxLogBytesPerSpan / logSize @@ -342,7 +342,7 @@ func TestSpanRecordLimit(t *testing.T) { sp.Recordf("%s", msg(i)) } - rec := sp.GetRecording(RecordingVerbose) + rec := sp.GetRecording(tracingpb.RecordingVerbose) require.Len(t, rec, 1) require.Len(t, rec[0].Logs, numLogs) require.Equal(t, rec[0].Tags["_dropped"], "1") @@ -356,7 +356,7 @@ func TestSpanRecordLimit(t *testing.T) { func TestChildSpanRegisteredWithRecordingParent(t *testing.T) { tr := NewTracer() - sp := tr.StartSpan("root", WithRecording(RecordingStructured)) + sp := tr.StartSpan("root", WithRecording(tracingpb.RecordingStructured)) defer sp.Finish() ch := tr.StartSpan("child", WithParent(sp)) defer ch.Finish() @@ -365,7 +365,7 @@ func TestChildSpanRegisteredWithRecordingParent(t *testing.T) { require.Equal(t, ch.i.crdb, children[0].spanRef.i.crdb) ch.RecordStructured(&types.Int32Value{Value: 5}) // Check that the child's structured event is in the recording. - rec := sp.GetRecording(RecordingStructured) + rec := sp.GetRecording(tracingpb.RecordingStructured) require.Len(t, rec, 1) require.Len(t, rec[0].StructuredRecords, 1) } @@ -373,7 +373,7 @@ func TestChildSpanRegisteredWithRecordingParent(t *testing.T) { // TestRecordingMaxSpans verifies that recordings don't grow over the limit. func TestRecordingMaxSpans(t *testing.T) { tr := NewTracer() - sp := tr.StartSpan("root", WithRecording(RecordingVerbose)) + sp := tr.StartSpan("root", WithRecording(tracingpb.RecordingVerbose)) defer sp.Finish() extraChildren := 10 numChildren := maxRecordedSpansPerTrace + extraChildren @@ -393,9 +393,9 @@ func TestRecordingMaxSpans(t *testing.T) { child.RecordStructured(&types.Int32Value{Value: int32(i)}) } child.Finish() - require.Len(t, sp.GetRecording(RecordingVerbose), exp) + require.Len(t, sp.GetRecording(tracingpb.RecordingVerbose), exp) } - rec := sp.GetRecording(RecordingVerbose) + rec := sp.GetRecording(tracingpb.RecordingVerbose) root := rec[0] require.Len(t, root.StructuredRecords, extraChildren) } @@ -489,7 +489,7 @@ func TestSpan_UseAfterFinish(t *testing.T) { sp.Record("boom") }) require.Panics(t, func() { - sp.GetRecording(RecordingStructured) + sp.GetRecording(tracingpb.RecordingStructured) }) require.Panics(t, func() { sp.Finish() @@ -533,7 +533,7 @@ func (t testStringerLazyTag) String() string { func TestSpanTags(t *testing.T) { tr := NewTracer() - sp := tr.StartSpan("root", WithRecording(RecordingVerbose)) + sp := tr.StartSpan("root", WithRecording(tracingpb.RecordingVerbose)) defer sp.Finish() sp.SetTag("tag", attribute.IntValue(42)) sp.SetLazyTag("lazy expanding tag", testExpandingTag{}) @@ -543,7 +543,7 @@ func TestSpanTags(t *testing.T) { require.True(t, ok) require.IsType(t, testExpandingTag{}, tag) - rec := sp.GetRecording(RecordingVerbose) + rec := sp.GetRecording(tracingpb.RecordingVerbose) tags := rec[0].Tags require.Contains(t, tags, "tag") require.Contains(t, tags, "lazy tag") @@ -570,13 +570,13 @@ func TestSpanTagsInRecordings(t *testing.T) { require.False(t, sp.IsVerbose()) sp.SetTag("foo2", attribute.StringValue("bar2")) sp.Record("dummy recording") - rec := sp.GetRecording(RecordingStructured) + rec := sp.GetRecording(tracingpb.RecordingStructured) require.Nil(t, rec) // We didn't stringify the log tag. require.Zero(t, int(counter)) - sp.SetRecordingType(RecordingVerbose) - rec = sp.GetRecording(RecordingVerbose) + sp.SetRecordingType(tracingpb.RecordingVerbose) + rec = sp.GetRecording(tracingpb.RecordingVerbose) require.Len(t, rec, 1) require.Len(t, rec[0].Tags, 5) // _unfinished:1 _verbose:1 foo:tagbar foo1:1 foor2:bar2 _, ok := rec[0].Tags["foo"] @@ -587,7 +587,7 @@ func TestSpanTagsInRecordings(t *testing.T) { // Verify that subsequent tags are also captured. sp.SetTag("foo3", attribute.StringValue("bar3")) - rec = sp.GetRecording(RecordingVerbose) + rec = sp.GetRecording(tracingpb.RecordingVerbose) require.Len(t, rec, 1) require.Len(t, rec[0].Tags, 6) _, ok = rec[0].Tags["foo3"] @@ -602,20 +602,20 @@ func TestVerboseTag(t *testing.T) { sp := tr.StartSpan("root") defer sp.Finish() - sp.SetRecordingType(RecordingStructured) - rec := sp.GetRecording(RecordingVerbose) + sp.SetRecordingType(tracingpb.RecordingStructured) + rec := sp.GetRecording(tracingpb.RecordingVerbose) _, ok := rec[0].Tags["_verbose"] require.False(t, ok) // The tag is present while the span is recording verbosely. - sp.SetRecordingType(RecordingVerbose) - rec = sp.GetRecording(RecordingVerbose) + sp.SetRecordingType(tracingpb.RecordingVerbose) + rec = sp.GetRecording(tracingpb.RecordingVerbose) _, ok = rec[0].Tags["_verbose"] require.True(t, ok) // After we stop recording, the tag goes away. - sp.SetRecordingType(RecordingStructured) - rec = sp.GetRecording(RecordingVerbose) + sp.SetRecordingType(tracingpb.RecordingStructured) + rec = sp.GetRecording(tracingpb.RecordingVerbose) _, ok = rec[0].Tags["_verbose"] require.False(t, ok) } @@ -626,7 +626,7 @@ func TestStructureRecording(t *testing.T) { for _, finishCh2 := range []bool{true, false} { t.Run(fmt.Sprintf("finish2=%t", finishCh2), func(t *testing.T) { tr := NewTracerWithOpt(context.Background(), WithTracingMode(TracingModeActiveSpansRegistry)) - sp := tr.StartSpan("root", WithRecording(RecordingStructured)) + sp := tr.StartSpan("root", WithRecording(tracingpb.RecordingStructured)) ch1 := tr.StartSpan("child", WithParent(sp)) ch2 := tr.StartSpan("grandchild", WithParent(ch1)) for i := int32(0); i < 5; i++ { @@ -640,7 +640,7 @@ func TestStructureRecording(t *testing.T) { if finishCh1 { ch1.Finish() } - rec := sp.GetRecording(RecordingStructured) + rec := sp.GetRecording(tracingpb.RecordingStructured) require.Len(t, rec, 1) require.Len(t, rec[0].StructuredRecords, 15) @@ -662,9 +662,9 @@ func TestStructureRecording(t *testing.T) { // recording. func TestOpenChildIncludedRecording(t *testing.T) { tr := NewTracerWithOpt(context.Background()) - parent := tr.StartSpan("parent", WithRecording(RecordingVerbose)) + parent := tr.StartSpan("parent", WithRecording(tracingpb.RecordingVerbose)) child := tr.StartSpan("child", WithParent(parent)) - rec := parent.FinishAndGetRecording(RecordingVerbose) + rec := parent.FinishAndGetRecording(tracingpb.RecordingVerbose) require.NoError(t, CheckRecording(rec, ` === operation:parent _verbose:1 === operation:child _unfinished:1 _verbose:1 @@ -695,7 +695,7 @@ func TestWithRemoteParentFromTraceInfo(t *testing.T) { require.Equal(t, traceID, sp.TraceID()) require.Equal(t, parentSpanID, sp.i.crdb.parentSpanID) - require.Equal(t, RecordingStructured, sp.RecordingType()) + require.Equal(t, tracingpb.RecordingStructured, sp.RecordingType()) require.NotNil(t, sp.i.otelSpan) otelCtx := sp.i.otelSpan.SpanContext() require.Equal(t, oteltrace.TraceID(otelTraceID), otelCtx.TraceID()) @@ -718,7 +718,7 @@ var _ EventListener = &mockEventListener{} func TestEventListener(t *testing.T) { tr := NewTracer() rootEventListener := &mockEventListener{} - sp := tr.StartSpan("root", WithRecording(RecordingStructured), + sp := tr.StartSpan("root", WithRecording(tracingpb.RecordingStructured), WithEventListeners([]EventListener{rootEventListener})) // Record a few Structured events. @@ -774,7 +774,7 @@ func TestEventListener(t *testing.T) { func TestEventListenerNotifiedWithoutHoldingSpanMutex(t *testing.T) { tr := NewTracer() rootEventListener := &mockEventListener{} - sp := tr.StartSpan("root", WithRecording(RecordingStructured), + sp := tr.StartSpan("root", WithRecording(tracingpb.RecordingStructured), WithEventListeners([]EventListener{rootEventListener})) defer sp.Finish() diff --git a/pkg/util/tracing/tags_test.go b/pkg/util/tracing/tags_test.go index 1101cea04523..44a567602811 100644 --- a/pkg/util/tracing/tags_test.go +++ b/pkg/util/tracing/tags_test.go @@ -14,6 +14,7 @@ import ( "context" "testing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/logtags" "github.com/stretchr/testify/require" "go.opentelemetry.io/otel/attribute" @@ -30,8 +31,8 @@ func TestLogTags(t *testing.T) { l := logtags.SingleTagBuffer("tag1", "val1") l = l.Add("tag2", "val2") sp1 := tr.StartSpan("foo", WithLogTags(l)) - sp1.SetRecordingType(RecordingVerbose) - require.NoError(t, CheckRecordedSpans(sp1.FinishAndGetRecording(RecordingVerbose), ` + sp1.SetRecordingType(tracingpb.RecordingVerbose) + require.NoError(t, CheckRecordedSpans(sp1.FinishAndGetRecording(tracingpb.RecordingVerbose), ` span: foo tags: _verbose=1 tag1=val1 tag2=val2 `)) @@ -49,8 +50,8 @@ func TestLogTags(t *testing.T) { RegisterTagRemapping("tag2", "two") sp2 := tr.StartSpan("bar", WithLogTags(l)) - sp2.SetRecordingType(RecordingVerbose) - require.NoError(t, CheckRecordedSpans(sp2.FinishAndGetRecording(RecordingVerbose), ` + sp2.SetRecordingType(tracingpb.RecordingVerbose) + require.NoError(t, CheckRecordedSpans(sp2.FinishAndGetRecording(tracingpb.RecordingVerbose), ` span: bar tags: _verbose=1 one=val1 two=val2 `)) @@ -66,8 +67,8 @@ func TestLogTags(t *testing.T) { } sp3 := tr.StartSpan("baz", WithLogTags(l)) - sp3.SetRecordingType(RecordingVerbose) - require.NoError(t, CheckRecordedSpans(sp3.FinishAndGetRecording(RecordingVerbose), ` + sp3.SetRecordingType(tracingpb.RecordingVerbose) + require.NoError(t, CheckRecordedSpans(sp3.FinishAndGetRecording(tracingpb.RecordingVerbose), ` span: baz tags: _verbose=1 one=val1 two=val2 `)) diff --git a/pkg/util/tracing/test_utils.go b/pkg/util/tracing/test_utils.go index ae4239e7eddb..4701cc7f94fd 100644 --- a/pkg/util/tracing/test_utils.go +++ b/pkg/util/tracing/test_utils.go @@ -23,7 +23,7 @@ import ( // FindMsgInRecording returns the index of the first Span containing msg in its // logs, or -1 if no Span is found. -func FindMsgInRecording(recording Recording, msg string) int { +func FindMsgInRecording(recording tracingpb.Recording, msg string) int { for i, sp := range recording { if LogsContainMsg(sp, msg) { return i @@ -73,7 +73,7 @@ func CountLogMessages(sp tracingpb.RecordedSpan, msg string) int { // Note: this test function is in this file because it needs to be used by // both tests in the tracing package and tests outside of it, and the function // itself depends on tracing. -func CheckRecordedSpans(rec Recording, expected string) error { +func CheckRecordedSpans(rec tracingpb.Recording, expected string) error { normalize := func(rec string) string { // normalize the string form of a recording for ease of comparison. // @@ -169,7 +169,7 @@ func CheckRecordedSpans(rec Recording, expected string) error { // t.Fatal(err) // } // -func CheckRecording(rec Recording, expected string) error { +func CheckRecording(rec tracingpb.Recording, expected string) error { normalize := func(rec string) string { // normalize the string form of a recording for ease of comparison. // diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 48ebefe6c096..ff139f47e36c 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -1122,7 +1122,7 @@ child operation: %s, tracer created at: // Are we tracing everything, or have a parent, or want a real span, or were // asked for a recording? Then we create a real trace span. In all other // cases, a noop span will do. - if !(t.AlwaysTrace() || opts.parentTraceID() != 0 || opts.ForceRealSpan || opts.recordingType() != RecordingOff) { + if !(t.AlwaysTrace() || opts.parentTraceID() != 0 || opts.ForceRealSpan || opts.recordingType() != tracingpb.RecordingOff) { if !opts.Sterile { return maybeWrapCtx(ctx, t.noopSpan) } @@ -1299,7 +1299,7 @@ func (t *Tracer) InjectMetaInto(sm SpanMeta, carrier Carrier) { // we're not recording. A 21.1 node interprets a traceID as wanting structured // recording (or verbose recording if fieldNameDeprecatedVerboseTracing is also // set). - if compatMode && sm.recordingType == RecordingOff { + if compatMode && sm.recordingType == tracingpb.RecordingOff { return } @@ -1307,7 +1307,7 @@ func (t *Tracer) InjectMetaInto(sm SpanMeta, carrier Carrier) { carrier.Set(fieldNameSpanID, strconv.FormatUint(uint64(sm.spanID), 16)) carrier.Set(fieldNameRecordingType, sm.recordingType.ToCarrierValue()) - if compatMode && sm.recordingType == RecordingVerbose { + if compatMode && sm.recordingType == tracingpb.RecordingVerbose { carrier.Set(fieldNameDeprecatedVerboseTracing, "1") } } @@ -1323,7 +1323,7 @@ func (t *Tracer) ExtractMetaFrom(carrier Carrier) (SpanMeta, error) { var otelTraceID oteltrace.TraceID var otelSpanID oteltrace.SpanID var recordingTypeExplicit bool - var recordingType RecordingType + var recordingType tracingpb.RecordingType iterFn := func(k, v string) error { switch k = strings.ToLower(k); k { @@ -1355,11 +1355,11 @@ func (t *Tracer) ExtractMetaFrom(carrier Carrier) (SpanMeta, error) { } case fieldNameRecordingType: recordingTypeExplicit = true - recordingType = RecordingTypeFromCarrierValue(v) + recordingType = tracingpb.RecordingTypeFromCarrierValue(v) case fieldNameDeprecatedVerboseTracing: // Compatibility with 21.2. if !recordingTypeExplicit { - recordingType = RecordingVerbose + recordingType = tracingpb.RecordingVerbose } } return nil @@ -1384,11 +1384,11 @@ func (t *Tracer) ExtractMetaFrom(carrier Carrier) (SpanMeta, error) { return noopSpanMeta, nil } - if !recordingTypeExplicit && recordingType == RecordingOff { + if !recordingTypeExplicit && recordingType == tracingpb.RecordingOff { // A 21.1 node (or a 21.2 mode running in backwards-compatibility mode) // that passed a TraceID but not fieldNameDeprecatedVerboseTracing wants the // structured events. - recordingType = RecordingStructured + recordingType = tracingpb.RecordingStructured } var otelCtx oteltrace.SpanContext @@ -1422,15 +1422,15 @@ type RegistrySpan interface { // WithDetachedRecording option. In other situations, the recording of such // children is not included in the parent's recording but, in the case of the // span registry, we want as much information as possible to be included. - GetFullRecording(recType RecordingType) Recording + GetFullRecording(recType tracingpb.RecordingType) tracingpb.Recording // SetRecordingType sets the recording mode of the span and its children, // recursively. Setting it to RecordingOff disables further recording. // Everything recorded so far remains in memory. - SetRecordingType(to RecordingType) + SetRecordingType(to tracingpb.RecordingType) // RecordingType returns the span's current recording type. - RecordingType() RecordingType + RecordingType() tracingpb.RecordingType } var _ RegistrySpan = &crdbSpan{} @@ -1606,11 +1606,11 @@ func EnsureChildSpan( // Recording.String(). Tests can also use FindMsgInRecording(). func ContextWithRecordingSpan( ctx context.Context, tr *Tracer, opName string, -) (_ context.Context, finishAndGetRecording func() Recording) { - ctx, sp := tr.StartSpanCtx(ctx, opName, WithRecording(RecordingVerbose)) - var rec Recording +) (_ context.Context, finishAndGetRecording func() tracingpb.Recording) { + ctx, sp := tr.StartSpanCtx(ctx, opName, WithRecording(tracingpb.RecordingVerbose)) + var rec tracingpb.Recording return ctx, - func() Recording { + func() tracingpb.Recording { if rec != nil { return rec } diff --git a/pkg/util/tracing/tracer_snapshots.go b/pkg/util/tracing/tracer_snapshots.go index 143674706aac..989e6780b2cf 100644 --- a/pkg/util/tracing/tracer_snapshots.go +++ b/pkg/util/tracing/tracer_snapshots.go @@ -18,6 +18,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" ) @@ -28,7 +29,7 @@ type SpansSnapshot struct { CapturedAt time.Time // Traces contains the collected traces. Each "local route" corresponds to one // trace. "Detached recording" spans are included in the parent's trace. - Traces []Recording + Traces []tracingpb.Recording // Stacks is a map from groutine ID to the goroutine's stack trace. All // goroutines running at the time when this snapshot is produced are // represented here. A goroutine referenced by a span's GoroutineID field will @@ -144,9 +145,9 @@ func (t *Tracer) GetSnapshots() []SnapshotInfo { func (t *Tracer) generateSnapshot() SpansSnapshot { capturedAt := timeutil.Now() // Collect the traces. - traces := make([]Recording, 0, 1000) + traces := make([]tracingpb.Recording, 0, 1000) _ = t.SpanRegistry().VisitRoots(func(sp RegistrySpan) error { - rec := sp.GetFullRecording(RecordingVerbose) + rec := sp.GetFullRecording(tracingpb.RecordingVerbose) traces = append(traces, rec) return nil }) diff --git a/pkg/util/tracing/tracer_test.go b/pkg/util/tracing/tracer_test.go index e1e9c04fb6a7..be2746541e44 100644 --- a/pkg/util/tracing/tracer_test.go +++ b/pkg/util/tracing/tracer_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/iterutil" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/logtags" "github.com/gogo/protobuf/types" "github.com/stretchr/testify/require" @@ -60,7 +61,7 @@ func TestTracingOffRecording(t *testing.T) { require.True(t, noop2.IsNoop()) // Noop span returns empty recording. - require.Nil(t, noop1.GetRecording(RecordingVerbose)) + require.Nil(t, noop1.GetRecording(tracingpb.RecordingVerbose)) } func TestTracerRecording(t *testing.T) { @@ -70,13 +71,13 @@ func TestTracerRecording(t *testing.T) { // Check that a span that was not configured to record returns nil for its // recording. sNonRecording := tr.StartSpan("not recording") - require.Equal(t, RecordingOff, sNonRecording.RecordingType()) + require.Equal(t, tracingpb.RecordingOff, sNonRecording.RecordingType()) require.Nil(t, sNonRecording.GetConfiguredRecording()) - require.Nil(t, sNonRecording.GetRecording(RecordingVerbose)) - require.Nil(t, sNonRecording.GetRecording(RecordingStructured)) + require.Nil(t, sNonRecording.GetRecording(tracingpb.RecordingVerbose)) + require.Nil(t, sNonRecording.GetRecording(tracingpb.RecordingStructured)) require.Nil(t, sNonRecording.FinishAndGetConfiguredRecording()) - s1 := tr.StartSpan("a", WithRecording(RecordingStructured)) + s1 := tr.StartSpan("a", WithRecording(tracingpb.RecordingStructured)) if s1.IsNoop() { t.Error("recording Span should not be noop") } @@ -85,24 +86,24 @@ func TestTracerRecording(t *testing.T) { } // Initial recording of this fresh (real) span. - require.Nil(t, s1.GetRecording(RecordingStructured)) + require.Nil(t, s1.GetRecording(tracingpb.RecordingStructured)) s1.RecordStructured(&types.Int32Value{Value: 5}) - if err := CheckRecording(s1.GetRecording(RecordingStructured), ` + if err := CheckRecording(s1.GetRecording(tracingpb.RecordingStructured), ` === operation:a structured:{"@type":"type.googleapis.com/google.protobuf.Int32Value","value":5} `); err != nil { t.Fatal(err) } - s1.SetRecordingType(RecordingVerbose) - if err := CheckRecordedSpans(s1.GetRecording(RecordingVerbose), ` + s1.SetRecordingType(tracingpb.RecordingVerbose) + if err := CheckRecordedSpans(s1.GetRecording(tracingpb.RecordingVerbose), ` span: a tags: _unfinished=1 _verbose=1 `); err != nil { t.Fatal(err) } - s1.SetRecordingType(RecordingOff) + s1.SetRecordingType(tracingpb.RecordingOff) // Real parent --> real child. real3 := tr.StartSpan("noop3", WithRemoteParentFromSpanMeta(s1.Meta())) @@ -112,7 +113,7 @@ func TestTracerRecording(t *testing.T) { real3.Finish() s1.Recordf("x=%d", 1) - s1.SetRecordingType(RecordingVerbose) + s1.SetRecordingType(tracingpb.RecordingVerbose) s1.Recordf("x=%d", 2) s2 := tr.StartSpan("b", WithParent(s1)) if !s2.IsVerbose() { @@ -120,7 +121,7 @@ func TestTracerRecording(t *testing.T) { } s2.Recordf("x=%d", 3) - if err := CheckRecordedSpans(s1.GetRecording(RecordingVerbose), ` + if err := CheckRecordedSpans(s1.GetRecording(tracingpb.RecordingVerbose), ` span: a tags: _unfinished=1 _verbose=1 event: x=2 @@ -131,7 +132,7 @@ func TestTracerRecording(t *testing.T) { t.Fatal(err) } - if err := CheckRecordedSpans(s2.GetRecording(RecordingVerbose), ` + if err := CheckRecordedSpans(s2.GetRecording(tracingpb.RecordingVerbose), ` span: b tags: _unfinished=1 _verbose=1 event: x=3 @@ -145,7 +146,7 @@ func TestTracerRecording(t *testing.T) { s2.Finish() - if err := CheckRecordedSpans(s1.GetRecording(RecordingVerbose), ` + if err := CheckRecordedSpans(s1.GetRecording(tracingpb.RecordingVerbose), ` span: a tags: _unfinished=1 _verbose=1 event: x=2 @@ -161,7 +162,7 @@ func TestTracerRecording(t *testing.T) { // We Finish() s3, but note that the recording shows it as _unfinished. That's // because s2's recording was snapshotted at the time s2 was finished, above. s3.Finish() - if err := CheckRecordedSpans(s1.GetRecording(RecordingVerbose), ` + if err := CheckRecordedSpans(s1.GetRecording(tracingpb.RecordingVerbose), ` span: a tags: _unfinished=1 _verbose=1 event: x=2 @@ -176,20 +177,20 @@ func TestTracerRecording(t *testing.T) { } s1.Finish() - s4 := tr.StartSpan("a", WithRecording(RecordingStructured)) - s4.SetRecordingType(RecordingOff) + s4 := tr.StartSpan("a", WithRecording(tracingpb.RecordingStructured)) + s4.SetRecordingType(tracingpb.RecordingOff) s4.Recordf("x=%d", 100) - require.Nil(t, s4.GetRecording(RecordingStructured)) + require.Nil(t, s4.GetRecording(tracingpb.RecordingStructured)) s4.Finish() } func TestStartChildSpan(t *testing.T) { tr := NewTracer() - sp1 := tr.StartSpan("parent", WithRecording(RecordingVerbose)) + sp1 := tr.StartSpan("parent", WithRecording(tracingpb.RecordingVerbose)) sp2 := tr.StartSpan("child", WithParent(sp1)) sp2.Finish() - if err := CheckRecordedSpans(sp1.FinishAndGetRecording(RecordingVerbose), ` + if err := CheckRecordedSpans(sp1.FinishAndGetRecording(tracingpb.RecordingVerbose), ` span: parent tags: _verbose=1 span: child @@ -198,26 +199,26 @@ func TestStartChildSpan(t *testing.T) { t.Fatal(err) } - sp1 = tr.StartSpan("parent", WithRecording(RecordingVerbose)) + sp1 = tr.StartSpan("parent", WithRecording(tracingpb.RecordingVerbose)) sp2 = tr.StartSpan("child", WithParent(sp1), WithDetachedRecording()) - if err := CheckRecordedSpans(sp2.FinishAndGetRecording(RecordingVerbose), ` + if err := CheckRecordedSpans(sp2.FinishAndGetRecording(tracingpb.RecordingVerbose), ` span: child tags: _verbose=1 `); err != nil { t.Fatal(err) } - if err := CheckRecordedSpans(sp1.FinishAndGetRecording(RecordingVerbose), ` + if err := CheckRecordedSpans(sp1.FinishAndGetRecording(tracingpb.RecordingVerbose), ` span: parent tags: _verbose=1 `); err != nil { t.Fatal(err) } - sp1 = tr.StartSpan("parent", WithRecording(RecordingVerbose)) + sp1 = tr.StartSpan("parent", WithRecording(tracingpb.RecordingVerbose)) sp2 = tr.StartSpan("child", WithParent(sp1), WithLogTags(logtags.SingleTagBuffer("key", "val"))) sp2.Finish() - if err := CheckRecordedSpans(sp1.FinishAndGetRecording(RecordingVerbose), ` + if err := CheckRecordedSpans(sp1.FinishAndGetRecording(tracingpb.RecordingVerbose), ` span: parent tags: _verbose=1 span: child @@ -233,7 +234,7 @@ func TestSterileSpan(t *testing.T) { // Check that a children of sterile spans are roots. // Make the span verbose so that we can use its recording below to assert that // there were no children. - sp1 := tr.StartSpan("parent", WithSterile(), WithRecording(RecordingVerbose)) + sp1 := tr.StartSpan("parent", WithSterile(), WithRecording(tracingpb.RecordingVerbose)) defer sp1.Finish() sp2 := tr.StartSpan("child", WithParent(sp1)) require.Zero(t, sp2.i.crdb.parentSpanID) @@ -245,7 +246,7 @@ func TestSterileSpan(t *testing.T) { sp2.Finish() sp3.Finish() - require.NoError(t, CheckRecordedSpans(sp1.GetRecording(RecordingVerbose), ` + require.NoError(t, CheckRecordedSpans(sp1.GetRecording(tracingpb.RecordingVerbose), ` span: parent tags: _unfinished=1 _verbose=1 `)) @@ -294,7 +295,7 @@ func TestTracerInjectExtract(t *testing.T) { // Verify that verbose tracing is propagated and triggers verbosity on the // remote side. - s1 := tr.StartSpan("a", WithRecording(RecordingVerbose)) + s1 := tr.StartSpan("a", WithRecording(tracingpb.RecordingVerbose)) carrier := metadataCarrier{metadata.MD{}} tr.InjectMetaInto(s1.Meta(), carrier) @@ -314,7 +315,7 @@ func TestTracerInjectExtract(t *testing.T) { s2.Recordf("x=%d", 1) // Verify that recording was started automatically. - rec := s2.FinishAndGetRecording(RecordingVerbose) + rec := s2.FinishAndGetRecording(tracingpb.RecordingVerbose) if err := CheckRecordedSpans(rec, ` span: remote op tags: _verbose=1 @@ -323,7 +324,7 @@ func TestTracerInjectExtract(t *testing.T) { t.Fatal(err) } - if err := CheckRecordedSpans(s1.GetRecording(RecordingVerbose), ` + if err := CheckRecordedSpans(s1.GetRecording(tracingpb.RecordingVerbose), ` span: a tags: _unfinished=1 _verbose=1 `); err != nil { @@ -331,7 +332,7 @@ func TestTracerInjectExtract(t *testing.T) { } s1.ImportRemoteRecording(rec) - if err := CheckRecordedSpans(s1.FinishAndGetRecording(RecordingVerbose), ` + if err := CheckRecordedSpans(s1.FinishAndGetRecording(tracingpb.RecordingVerbose), ` span: a tags: _verbose=1 span: remote op @@ -452,7 +453,7 @@ func getSpanOpsWithFinished(t *testing.T, tr *Tracer) map[string]bool { spanOpsWithFinished := make(map[string]bool) require.NoError(t, tr.VisitSpans(func(sp RegistrySpan) error { - for _, rec := range sp.GetFullRecording(RecordingVerbose) { + for _, rec := range sp.GetFullRecording(tracingpb.RecordingVerbose) { spanOpsWithFinished[rec.Operation] = rec.Finished } return nil @@ -469,7 +470,7 @@ func getSortedSpanOps(t *testing.T, tr *Tracer) []string { var spanOps []string require.NoError(t, tr.VisitSpans(func(sp RegistrySpan) error { - for _, rec := range sp.GetFullRecording(RecordingVerbose) { + for _, rec := range sp.GetFullRecording(tracingpb.RecordingVerbose) { spanOps = append(spanOps, rec.Operation) } return nil @@ -485,14 +486,14 @@ func TestTracer_VisitSpans(t *testing.T) { tr1 := NewTracer() tr2 := NewTracer() - root := tr1.StartSpan("root", WithRecording(RecordingVerbose)) + root := tr1.StartSpan("root", WithRecording(tracingpb.RecordingVerbose)) child := tr1.StartSpan("root.child", WithParent(root)) require.Len(t, tr1.activeSpansRegistry.mu.m, 1) childChild := tr2.StartSpan("root.child.remotechild", WithRemoteParentFromSpanMeta(child.Meta())) childChildFinished := tr2.StartSpan("root.child.remotechilddone", WithRemoteParentFromSpanMeta(child.Meta())) require.Len(t, tr2.activeSpansRegistry.mu.m, 2) - child.ImportRemoteRecording(childChildFinished.FinishAndGetRecording(RecordingVerbose)) + child.ImportRemoteRecording(childChildFinished.FinishAndGetRecording(tracingpb.RecordingVerbose)) require.Len(t, tr2.activeSpansRegistry.mu.m, 1) // All spans are part of the recording (root.child.remotechilddone was @@ -516,7 +517,7 @@ func TestTracer_VisitSpans(t *testing.T) { // in-flight trace have recordings indicating that they have, in fact, finished. func TestSpanRecordingFinished(t *testing.T) { tr1 := NewTracer() - root := tr1.StartSpan("root", WithRecording(RecordingVerbose)) + root := tr1.StartSpan("root", WithRecording(tracingpb.RecordingVerbose)) child := tr1.StartSpan("root.child", WithParent(root)) childChild := tr1.StartSpan("root.child.child", WithParent(child)) @@ -524,7 +525,7 @@ func TestSpanRecordingFinished(t *testing.T) { tr2 := NewTracer() childTraceInfo := child.Meta().ToProto() remoteChildChild := tr2.StartSpan("root.child.remotechild", WithRemoteParentFromTraceInfo(&childTraceInfo)) - child.ImportRemoteRecording(remoteChildChild.GetRecording(RecordingVerbose)) + child.ImportRemoteRecording(remoteChildChild.GetRecording(tracingpb.RecordingVerbose)) remoteChildChild.Finish() // All spans are un-finished. @@ -599,7 +600,7 @@ func TestSpanWithNoopParentIsInActiveSpans(t *testing.T) { func TestConcurrentChildAndRecording(t *testing.T) { tr := NewTracer() - rootSp := tr.StartSpan("root", WithRecording(RecordingVerbose)) + rootSp := tr.StartSpan("root", WithRecording(tracingpb.RecordingVerbose)) defer rootSp.Finish() var wg sync.WaitGroup const n = 1000 @@ -615,7 +616,7 @@ func TestConcurrentChildAndRecording(t *testing.T) { }() go func() { defer wg.Done() - _ = rootSp.GetRecording(RecordingVerbose) + _ = rootSp.GetRecording(tracingpb.RecordingVerbose) }() } wg.Wait() @@ -623,13 +624,13 @@ func TestConcurrentChildAndRecording(t *testing.T) { func TestFinishedSpanInRecording(t *testing.T) { tr := NewTracer() - s1 := tr.StartSpan("a", WithRecording(RecordingVerbose)) + s1 := tr.StartSpan("a", WithRecording(tracingpb.RecordingVerbose)) s2 := tr.StartSpan("b", WithParent(s1)) s3 := tr.StartSpan("c", WithParent(s2)) // Check that s2 is included in the recording both before and after it's // finished. - require.NoError(t, CheckRecordedSpans(s1.GetRecording(RecordingVerbose), ` + require.NoError(t, CheckRecordedSpans(s1.GetRecording(tracingpb.RecordingVerbose), ` span: a tags: _unfinished=1 _verbose=1 span: b @@ -638,7 +639,7 @@ span: a tags: _unfinished=1 _verbose=1 `)) s3.Finish() - require.NoError(t, CheckRecordedSpans(s1.GetRecording(RecordingVerbose), ` + require.NoError(t, CheckRecordedSpans(s1.GetRecording(tracingpb.RecordingVerbose), ` span: a tags: _unfinished=1 _verbose=1 span: b @@ -647,7 +648,7 @@ span: a tags: _verbose=1 `)) s2.Finish() - require.NoError(t, CheckRecordedSpans(s1.GetRecording(RecordingVerbose), ` + require.NoError(t, CheckRecordedSpans(s1.GetRecording(tracingpb.RecordingVerbose), ` span: a tags: _unfinished=1 _verbose=1 span: b @@ -658,12 +659,12 @@ span: a s1.Finish() // Now the same thing, but finish s2 first. - s1 = tr.StartSpan("a", WithRecording(RecordingVerbose)) + s1 = tr.StartSpan("a", WithRecording(tracingpb.RecordingVerbose)) s2 = tr.StartSpan("b", WithParent(s1)) s3 = tr.StartSpan("c", WithParent(s2)) s2.Finish() - require.NoError(t, CheckRecordedSpans(s1.FinishAndGetRecording(RecordingVerbose), ` + require.NoError(t, CheckRecordedSpans(s1.FinishAndGetRecording(tracingpb.RecordingVerbose), ` span: a tags: _verbose=1 span: b @@ -681,7 +682,7 @@ func TestRegistryOrphanSpansBecomeRoots(t *testing.T) { tr := NewTracerWithOpt(ctx, WithTracingMode(TracingModeActiveSpansRegistry)) // s1 must be recording because, otherwise, the child spans are not linked to // it. - s1 := tr.StartSpan("parent", WithRecording(RecordingStructured)) + s1 := tr.StartSpan("parent", WithRecording(tracingpb.RecordingStructured)) s2 := tr.StartSpan("child1", WithParent(s1)) s3 := tr.StartSpan("child2", WithParent(s1)) require.Equal(t, []*crdbSpan{s1.i.crdb}, tr.activeSpansRegistry.testingAll()) @@ -800,12 +801,12 @@ func TestSpanFinishRaces(t *testing.T) { if i > 0 { opt = WithParent(sps[i-1]) } - sps[i] = tr.StartSpan(fmt.Sprint(i), WithRecording(RecordingVerbose), opt) + sps[i] = tr.StartSpan(fmt.Sprint(i), WithRecording(tracingpb.RecordingVerbose), opt) sps[i].Recordf("msg %d", i) } finishOrder := rand.Perm(numSpans) - var rec Recording + var rec tracingpb.Recording g := sync.WaitGroup{} g.Add(len(finishOrder)) for _, idx := range finishOrder { @@ -813,7 +814,7 @@ func TestSpanFinishRaces(t *testing.T) { if idx != 0 { sps[idx].Finish() } else { - rec = sps[0].FinishAndGetRecording(RecordingVerbose) + rec = sps[0].FinishAndGetRecording(tracingpb.RecordingVerbose) } g.Done() }(idx) diff --git a/pkg/util/tracing/tracingpb/BUILD.bazel b/pkg/util/tracing/tracingpb/BUILD.bazel index 7c2ce2142ad3..9e73a6eb3ef4 100644 --- a/pkg/util/tracing/tracingpb/BUILD.bazel +++ b/pkg/util/tracing/tracingpb/BUILD.bazel @@ -6,14 +6,20 @@ go_library( name = "tracingpb", srcs = [ "recorded_span.go", + "recording.go", "tracing.go", ], embed = [":tracingpb_go_proto"], importpath = "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb", visibility = ["//visibility:public"], deps = [ + "//pkg/util/protoutil", + "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", + "@com_github_gogo_protobuf//jsonpb", + "@com_github_gogo_protobuf//proto", "@com_github_gogo_protobuf//types", + "@com_github_jaegertracing_jaeger//model/json", ], ) diff --git a/pkg/util/tracing/recording.go b/pkg/util/tracing/tracingpb/recording.go similarity index 89% rename from pkg/util/tracing/recording.go rename to pkg/util/tracing/tracingpb/recording.go index b72cb7eb872c..c721c9425781 100644 --- a/pkg/util/tracing/recording.go +++ b/pkg/util/tracing/tracingpb/recording.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package tracing +package tracingpb import ( "encoding/json" @@ -19,8 +19,11 @@ import ( "strings" "time" - "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" + "github.com/gogo/protobuf/jsonpb" + "github.com/gogo/protobuf/proto" "github.com/gogo/protobuf/types" jaegerjson "github.com/jaegertracing/jaeger/model/json" ) @@ -57,27 +60,27 @@ func (t RecordingType) ToCarrierValue() string { } // ToProto converts t to the corresponding proto enum. -func (t RecordingType) ToProto() tracingpb.RecordingMode { +func (t RecordingType) ToProto() RecordingMode { switch t { case RecordingOff: - return tracingpb.RecordingMode_OFF + return RecordingMode_OFF case RecordingStructured: - return tracingpb.RecordingMode_STRUCTURED + return RecordingMode_STRUCTURED case RecordingVerbose: - return tracingpb.RecordingMode_VERBOSE + return RecordingMode_VERBOSE default: panic(fmt.Sprintf("invalid RecordingType: %d", t)) } } // RecordingTypeFromProto converts from the proto values to the corresponding enum. -func RecordingTypeFromProto(val tracingpb.RecordingMode) RecordingType { +func RecordingTypeFromProto(val RecordingMode) RecordingType { switch val { - case tracingpb.RecordingMode_OFF: + case RecordingMode_OFF: return RecordingOff - case tracingpb.RecordingMode_STRUCTURED: + case RecordingMode_STRUCTURED: return RecordingStructured - case tracingpb.RecordingMode_VERBOSE: + case RecordingMode_VERBOSE: return RecordingVerbose default: panic(fmt.Sprintf("invalid RecordingType: %d", val)) @@ -174,13 +177,13 @@ func (r Recording) String() string { } // OrphanSpans returns the spans with parents missing from the recording. -func (r Recording) OrphanSpans() []tracingpb.RecordedSpan { - spanIDs := make(map[tracingpb.SpanID]struct{}) +func (r Recording) OrphanSpans() []RecordedSpan { + spanIDs := make(map[SpanID]struct{}) for _, sp := range r { spanIDs[sp.SpanID] = struct{}{} } - var orphans []tracingpb.RecordedSpan + var orphans []RecordedSpan for i, sp := range r { if i == 0 { // The first Span can be a root Span. Note that any other root Span will @@ -214,20 +217,20 @@ func (r Recording) FindLogMessage(pattern string) (string, bool) { // FindSpan returns the Span with the given operation. The bool retval is false // if the Span is not found. -func (r Recording) FindSpan(operation string) (tracingpb.RecordedSpan, bool) { +func (r Recording) FindSpan(operation string) (RecordedSpan, bool) { for _, sp := range r { if sp.Operation == operation { return sp, true } } - return tracingpb.RecordedSpan{}, false + return RecordedSpan{}, false } // visitSpan returns the log messages for sp, and all of sp's children. // // All messages from a Span are kept together. Sibling spans are ordered within // the parent in their start order. -func (r Recording) visitSpan(sp tracingpb.RecordedSpan, depth int) []traceLogData { +func (r Recording) visitSpan(sp RecordedSpan, depth int) []traceLogData { ownLogs := make([]traceLogData, 0, len(sp.Logs)+1) conv := func(msg redact.RedactableString, timestamp time.Time, ref time.Time) traceLogData { @@ -354,7 +357,7 @@ func (r Recording) ToJaegerJSON(stmt, comment, nodeStr string) (string, error) { tagsCopy["statement"] = stmt r[0].Tags = tagsCopy - toJaegerSpanID := func(spanID tracingpb.SpanID) jaegerjson.SpanID { + toJaegerSpanID := func(spanID SpanID) jaegerjson.SpanID { return jaegerjson.SpanID(strconv.FormatUint(uint64(spanID), 10)) } @@ -365,7 +368,7 @@ func (r Recording) ToJaegerJSON(stmt, comment, nodeStr string) (string, error) { // getProcessID figures out what "process" a Span belongs to. It looks for an // "node: " tag. The processes map is populated with an entry for every // node present in the trace. - getProcessID := func(sp tracingpb.RecordedSpan) jaegerjson.ProcessID { + getProcessID := func(sp RecordedSpan) jaegerjson.ProcessID { node := "unknown node" for k, v := range sp.Tags { if k == "node" { @@ -435,7 +438,7 @@ func (r Recording) ToJaegerJSON(stmt, comment, nodeStr string) (string, error) { // or not at the time each event was recorded, so we make a guess based on // whether the span was verbose at the moment when the Recording was // produced. - if !(sp.Verbose || sp.RecordingMode == tracingpb.RecordingMode_VERBOSE) { + if !(sp.Verbose || sp.RecordingMode == RecordingMode_VERBOSE) { sp.Structured(func(sr *types.Any, t time.Time) { jl := jaegerjson.Log{Timestamp: uint64(t.UnixNano() / 1000)} jsonStr, err := MessageToJSONString(sr, true /* emitDefaults */) @@ -475,3 +478,20 @@ type TraceCollection struct { Comment string `json:"_comment"` Data []jaegerjson.Trace `json:"data"` } + +// MessageToJSONString converts a protocol message into a JSON string. The +// emitDefaults flag dictates whether fields with zero values are rendered or +// not. +// +// TODO(andrei): It'd be nice if this function dealt with redactable vs safe +// fields, like EventPayload.AppendJSONFields does. +func MessageToJSONString(msg protoutil.Message, emitDefaults bool) (string, error) { + // Convert to json. + jsonEncoder := jsonpb.Marshaler{EmitDefaults: emitDefaults} + msgJSON, err := jsonEncoder.MarshalToString(msg) + if err != nil { + return "", errors.Newf("error when converting %s to JSON string", proto.MessageName(msg)) + } + + return msgJSON, nil +} diff --git a/pkg/util/tracing/tracingui/span_registry_ui.go b/pkg/util/tracing/tracingui/span_registry_ui.go index 0b6de713940c..c4deea36035f 100644 --- a/pkg/util/tracing/tracingui/span_registry_ui.go +++ b/pkg/util/tracing/tracingui/span_registry_ui.go @@ -27,7 +27,7 @@ func ProcessSnapshot( snapshot tracing.SpansSnapshot, registry *tracing.SpanRegistry, ) *ProcessedSnapshot { // Build a map of current spans. - currentSpans := make(map[tracingpb.SpanID]tracing.RecordingType, 1000) + currentSpans := make(map[tracingpb.SpanID]tracingpb.RecordingType, 1000) registry.VisitSpans(func(sp tracing.RegistrySpan) { currentSpans[sp.SpanID()] = sp.RecordingType() }) @@ -118,7 +118,7 @@ type processedSpan struct { Current bool // CurrentRecordingMode indicates the spans's current recording mode. The // field is not set if Current == false. - CurrentRecordingMode tracing.RecordingType + CurrentRecordingMode tracingpb.RecordingType } // ProcessedTag is a span tag that was processed and expanded by processTag. diff --git a/pkg/util/tracing/utils.go b/pkg/util/tracing/utils.go index d8c323e4e4b2..0eaced03770d 100644 --- a/pkg/util/tracing/utils.go +++ b/pkg/util/tracing/utils.go @@ -11,19 +11,16 @@ package tracing import ( - "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" - "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" "github.com/gogo/protobuf/jsonpb" - "github.com/gogo/protobuf/proto" ) // TraceToJSON returns the string representation of the trace in JSON format. // // TraceToJSON assumes that the first span in the recording contains all the // other spans. -func TraceToJSON(trace Recording) (string, error) { +func TraceToJSON(trace tracingpb.Recording) (string, error) { root := normalizeSpan(trace[0], trace) marshaller := jsonpb.Marshaler{ Indent: "\t", @@ -35,7 +32,7 @@ func TraceToJSON(trace Recording) (string, error) { return str, nil } -func normalizeSpan(s tracingpb.RecordedSpan, trace Recording) tracingpb.NormalizedSpan { +func normalizeSpan(s tracingpb.RecordedSpan, trace tracingpb.Recording) tracingpb.NormalizedSpan { var n tracingpb.NormalizedSpan n.Operation = s.Operation n.StartTime = s.StartTime @@ -53,23 +50,6 @@ func normalizeSpan(s tracingpb.RecordedSpan, trace Recording) tracingpb.Normaliz return n } -// MessageToJSONString converts a protocol message into a JSON string. The -// emitDefaults flag dictates whether fields with zero values are rendered or -// not. -// -// TODO(andrei): It'd be nice if this function dealt with redactable vs safe -// fields, like EventPayload.AppendJSONFields does. -func MessageToJSONString(msg protoutil.Message, emitDefaults bool) (string, error) { - // Convert to json. - jsonEncoder := jsonpb.Marshaler{EmitDefaults: emitDefaults} - msgJSON, err := jsonEncoder.MarshalToString(msg) - if err != nil { - return "", errors.Newf("error when converting %s to JSON string", proto.MessageName(msg)) - } - - return msgJSON, nil -} - // RedactAndTruncateError redacts the error and truncates the string // representation of the error to a fixed length. func RedactAndTruncateError(err error) string { diff --git a/pkg/util/tracing/zipper/BUILD.bazel b/pkg/util/tracing/zipper/BUILD.bazel index f330b06c2061..50f692afc36a 100644 --- a/pkg/util/tracing/zipper/BUILD.bazel +++ b/pkg/util/tracing/zipper/BUILD.bazel @@ -10,7 +10,7 @@ go_library( "//pkg/sql/sqlutil", "//pkg/util/log", "//pkg/util/memzipper", - "//pkg/util/tracing", + "//pkg/util/tracing/tracingpb", "@com_github_cockroachdb_errors//:errors", ], ) diff --git a/pkg/util/tracing/zipper/zipper.go b/pkg/util/tracing/zipper/zipper.go index 7a3d08f18e3d..1be2d098adfd 100644 --- a/pkg/util/tracing/zipper/zipper.go +++ b/pkg/util/tracing/zipper/zipper.go @@ -22,7 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/memzipper" - "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" ) @@ -40,7 +40,7 @@ type inflightTraceRow struct { // InflightTraceZipper provides a method to generate a trace zip containing // per-node traces for all inflight trace spans of a particular traceID. type InflightTraceZipper interface { - getNodeTraceCollection() *tracing.TraceCollection + getNodeTraceCollection() *tracingpb.TraceCollection getTraceStrBuffer() *bytes.Buffer getZipper() *memzipper.Zipper reset() @@ -52,12 +52,12 @@ type InflightTraceZipper interface { // SQL connection to collect cluster wide traces. type InternalInflightTraceZipper struct { traceStrBuf *bytes.Buffer - nodeTraceCollection *tracing.TraceCollection + nodeTraceCollection *tracingpb.TraceCollection ie sqlutil.InternalExecutor z *memzipper.Zipper } -func (i *InternalInflightTraceZipper) getNodeTraceCollection() *tracing.TraceCollection { +func (i *InternalInflightTraceZipper) getNodeTraceCollection() *tracingpb.TraceCollection { return i.nodeTraceCollection } @@ -106,18 +106,18 @@ func (i *InternalInflightTraceZipper) Zip( flushAndReset(ctx, prevNodeID, i) } - // If we are reading another row (tracing.Recording) from the same node as + // If we are reading another row (tracingpb.Recording) from the same node as // prevNodeID then we want to stitch the JaegerJSON into the existing // JaegerJSON object for this node. This allows us to output a per node // Jaeger file that can easily be imported into JaegerUI. // - // It is safe to do this since the tracing.Recording returned as rows are + // It is safe to do this since the tracingpb.Recording returned as rows are // sorted by the StartTime of the root span, and so appending to the // existing JaegerJSON will maintain the chronological order of the traces. // // In practice, it is more useful to view all the Jaeger tracing.Recordings // on a node for a given TraceID in a single view, rather than having to - // generate different Jaeger files for each tracing.Recording, and going + // generate different Jaeger files for each tracingpb.Recording, and going // through the hassle of importing each one and toggling through the tabs. if i.nodeTraceCollection, err = stitchJaegerJSON(i.nodeTraceCollection, traceRow.jaegerJSON); err != nil { return nil, err @@ -209,12 +209,12 @@ type queryI interface { // backed SQL connection to collect cluster wide traces. type SQLConnInflightTraceZipper struct { traceStrBuf *bytes.Buffer - nodeTraceCollection *tracing.TraceCollection + nodeTraceCollection *tracingpb.TraceCollection z *memzipper.Zipper sqlConn queryI } -func (s *SQLConnInflightTraceZipper) getNodeTraceCollection() *tracing.TraceCollection { +func (s *SQLConnInflightTraceZipper) getNodeTraceCollection() *tracingpb.TraceCollection { return s.nodeTraceCollection } @@ -273,18 +273,18 @@ func (s *SQLConnInflightTraceZipper) Zip(ctx context.Context, traceID int64) ([] flushAndReset(ctx, prevNodeID, s) } - // If we are reading another row (tracing.Recording) from the same node as + // If we are reading another row (tracingpb.Recording) from the same node as // prevNodeID then we want to stitch the JaegerJSON into the existing // JaegerJSON object for this node. This allows us to output a per node // Jaeger file that can easily be imported into JaegerUI. // - // It is safe to do this since the tracing.Recording returned as rows are + // It is safe to do this since the tracingpb.Recording returned as rows are // sorted by the StartTime of the root span, and so appending to the // existing JaegerJSON will maintain the chronological order of the traces. // // In practice, it is more useful to view all the Jaeger tracing.Recordings // on a node for a given TraceID in a single view, rather than having to - // generate different Jaeger files for each tracing.Recording, and going + // generate different Jaeger files for each tracingpb.Recording, and going // through the hassle of importing each one and toggling through the tabs. if s.nodeTraceCollection, err = stitchJaegerJSON(s.nodeTraceCollection, row.jaegerJSON); err != nil { return nil, err @@ -384,12 +384,12 @@ func flushAndReset(ctx context.Context, nodeID int64, t InflightTraceZipper) { // nodeTraceCollection object, and returns a new cumulative // tracing.TraceCollection object. func stitchJaegerJSON( - nodeTraceCollection *tracing.TraceCollection, jaegerJSON string, -) (*tracing.TraceCollection, error) { - var cumulativeTraceCollection *tracing.TraceCollection + nodeTraceCollection *tracingpb.TraceCollection, jaegerJSON string, +) (*tracingpb.TraceCollection, error) { + var cumulativeTraceCollection *tracingpb.TraceCollection // Unmarshal the jaegerJSON string to a TraceCollection. - var curTraceCollection tracing.TraceCollection + var curTraceCollection tracingpb.TraceCollection if err := json.Unmarshal([]byte(jaegerJSON), &curTraceCollection); err != nil { return cumulativeTraceCollection, err }