From 0f0dd9e55c7822cbd6396cd17af4ba6c17c2647b Mon Sep 17 00:00:00 2001 From: Paul Bardea Date: Tue, 2 Feb 2021 15:42:14 -0500 Subject: [PATCH 1/2] importccl: do not try to rollback data if timestamp was not chosen If a timestamp for the import was not chosen, there should be no ingested data to rollback. However, we do want to continue with the IMPORT cleanup to bring any OFFLINE tables back to PUBLIC. Release note (bug fix): Fixes a bug where some import failures would cause tables to stay OFFLINE, when they should have been brough back to PUBLIC. --- pkg/ccl/importccl/import_stmt.go | 29 ++++++++++++++++------------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_stmt.go index 15ff10424af3..7b213a169042 100644 --- a/pkg/ccl/importccl/import_stmt.go +++ b/pkg/ccl/importccl/import_stmt.go @@ -1658,19 +1658,22 @@ func (r *importResumer) dropTables( } } - // NB: if a revert fails it will abort the rest of this failure txn, which is - // also what brings tables back online. We _could_ change the error handling - // or just move the revert into Resume()'s error return path, however it isn't - // clear that just bringing a table back online with partially imported data - // that may or may not be partially reverted is actually a good idea. It seems - // better to do the revert here so that the table comes back if and only if, - // it was rolled back to its pre-IMPORT state, and instead provide a manual - // admin knob (e.g. ALTER TABLE REVERT TO SYSTEM TIME) if anything goes wrong. - if len(revert) > 0 { - // Sanity check Walltime so it doesn't become a TRUNCATE if there's a bug. - if details.Walltime == 0 { - return errors.Errorf("invalid pre-IMPORT time to rollback") - } + // The walltime can be 0 if there is a failure between publishing the tables + // as OFFLINE and then choosing a ingestion timestamp. This might happen + // while waiting for the descriptor version to propagate across the cluster + // for example. + // + // In this case, we don't want to rollback the data since data ingestion has + // not yet begun (since we have not chosen a timestamp at which to ingest.) + if details.Walltime != 0 && len(revert) > 0 { + // NB: if a revert fails it will abort the rest of this failure txn, which is + // also what brings tables back online. We _could_ change the error handling + // or just move the revert into Resume()'s error return path, however it isn't + // clear that just bringing a table back online with partially imported data + // that may or may not be partially reverted is actually a good idea. It seems + // better to do the revert here so that the table comes back if and only if, + // it was rolled back to its pre-IMPORT state, and instead provide a manual + // admin knob (e.g. ALTER TABLE REVERT TO SYSTEM TIME) if anything goes wrong. ts := hlc.Timestamp{WallTime: details.Walltime}.Prev() if err := sql.RevertTables(ctx, txn.DB(), execCfg, revert, ts, sql.RevertTableDefaultBatchSize); err != nil { return errors.Wrap(err, "rolling back partially completed IMPORT") From 308dc984f3b1bf7def7767822042146d38fcdd98 Mon Sep 17 00:00:00 2001 From: angelapwen Date: Tue, 2 Feb 2021 11:45:20 -0800 Subject: [PATCH 2/2] util/tracing, sql: expose span's goroutine IDs This commit adds a goroutine ID to each span's recording, as well as as a column in the `crdb_internal.node_inflight_trace_spans` virtual table. The goroutine ID is retrieved upon span creation. Release note (sql change): adds `goroutine_id` column to the `crdb_internal.node_inflight_trace_spans` virtual table that represents the goroutine ID associated with a particular span. --- pkg/sql/crdb_internal.go | 3 + .../testdata/logic_test/crdb_internal | 4 +- .../testdata/logic_test/crdb_internal_tenant | 4 +- pkg/util/tracing/BUILD.bazel | 1 + pkg/util/tracing/crdbspan.go | 2 + pkg/util/tracing/tracer.go | 3 + .../tracing/tracingpb/recorded_span.pb.go | 129 +++++++++++------- .../tracing/tracingpb/recorded_span.proto | 3 + 8 files changed, 96 insertions(+), 53 deletions(-) diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 36d724e56aa4..ae67e778e67b 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1142,6 +1142,7 @@ CREATE TABLE crdb_internal.node_inflight_trace_spans ( trace_id INT NOT NULL, -- The trace's ID. parent_span_id INT NOT NULL, -- The span's parent ID. span_id INT NOT NULL, -- The span's ID. + goroutine_id INT NOT NULL, -- The ID of the goroutine on which the span was created. start_time TIMESTAMPTZ, -- The span's start time. duration INTERVAL, -- The span's duration, measured by time of -- collection - start time for all in-flight spans. @@ -1153,6 +1154,7 @@ CREATE TABLE crdb_internal.node_inflight_trace_spans ( traceID := rec.TraceID parentSpanID := rec.ParentSpanID spanID := rec.SpanID + goroutineID := rec.GoroutineID startTime, err := tree.MakeDTimestampTZ(rec.StartTime, time.Microsecond) if err != nil { @@ -1166,6 +1168,7 @@ CREATE TABLE crdb_internal.node_inflight_trace_spans ( tree.NewDInt(tree.DInt(traceID)), tree.NewDInt(tree.DInt(parentSpanID)), tree.NewDInt(tree.DInt(spanID)), + tree.NewDInt(tree.DInt(goroutineID)), startTime, tree.NewDInterval( duration.MakeDuration(spanDuration.Nanoseconds(), 0, 0), diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 160d0e6d39b5..7c86e3bfd4ef 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -250,10 +250,10 @@ SELECT * FROM crdb_internal.zones WHERE false zone_id subzone_id target range_name database_name table_name index_name partition_name raw_config_yaml raw_config_sql raw_config_protobuf full_config_yaml full_config_sql -query IIITTT colnames +query IIIITTT colnames SELECT * FROM crdb_internal.node_inflight_trace_spans WHERE span_id < 0 ---- -trace_id parent_span_id span_id start_time duration operation +trace_id parent_span_id span_id goroutine_id start_time duration operation query ITTTTTTTTTTTTI colnames SELECT * FROM crdb_internal.ranges WHERE range_id < 0 diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant b/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant index 0bbafa3f0879..830c65f05885 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant @@ -261,10 +261,10 @@ SELECT * FROM crdb_internal.zones WHERE false zone_id subzone_id target range_name database_name table_name index_name partition_name raw_config_yaml raw_config_sql raw_config_protobuf full_config_yaml full_config_sql -query IIITTT colnames +query IIIITTT colnames SELECT * FROM crdb_internal.node_inflight_trace_spans WHERE span_id < 0 ---- -trace_id parent_span_id span_id start_time duration operation +trace_id parent_span_id span_id goroutine_id start_time duration operation statement error not fully contained in tenant keyspace SELECT * FROM crdb_internal.ranges WHERE range_id < 0 diff --git a/pkg/util/tracing/BUILD.bazel b/pkg/util/tracing/BUILD.bazel index 3b0b57d53305..51607afe703b 100644 --- a/pkg/util/tracing/BUILD.bazel +++ b/pkg/util/tracing/BUILD.bazel @@ -37,6 +37,7 @@ go_library( "@com_github_opentracing_opentracing_go//ext", "@com_github_opentracing_opentracing_go//log", "@com_github_openzipkin_contrib_zipkin_go_opentracing//:zipkin-go-opentracing", + "@com_github_petermattis_goid//:goid", "@org_golang_google_grpc//:go_default_library", "@org_golang_google_grpc//codes", "@org_golang_google_grpc//metadata", diff --git a/pkg/util/tracing/crdbspan.go b/pkg/util/tracing/crdbspan.go index 439d2d9cd183..c89b4cea11e8 100644 --- a/pkg/util/tracing/crdbspan.go +++ b/pkg/util/tracing/crdbspan.go @@ -32,6 +32,7 @@ type crdbSpan struct { traceID uint64 // probabilistically unique. spanID uint64 // probabilistically unique. parentSpanID uint64 + goroutineID uint64 operation string startTime time.Time @@ -241,6 +242,7 @@ func (s *crdbSpan) getRecordingLocked(m mode) tracingpb.RecordedSpan { TraceID: s.traceID, SpanID: s.spanID, ParentSpanID: s.parentSpanID, + GoroutineID: s.goroutineID, Operation: s.operation, StartTime: s.startTime, Duration: s.mu.duration, diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 1a90dc487e05..f4a512f57a12 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/logtags" opentracing "github.com/opentracing/opentracing-go" + "github.com/petermattis/goid" "golang.org/x/net/trace" ) @@ -364,6 +365,7 @@ func (t *Tracer) startSpanGeneric( traceID = uint64(rand.Int63()) } spanID := uint64(rand.Int63()) + goroutineID := uint64(goid.Get()) // Now allocate the main *Span and contained crdbSpan. // Allocate these together to save on individual allocs. @@ -380,6 +382,7 @@ func (t *Tracer) startSpanGeneric( helper.crdbSpan = crdbSpan{ traceID: traceID, spanID: spanID, + goroutineID: goroutineID, operation: opName, startTime: startTime, parentSpanID: opts.parentSpanID(), diff --git a/pkg/util/tracing/tracingpb/recorded_span.pb.go b/pkg/util/tracing/tracingpb/recorded_span.pb.go index 67476c23f356..0eade366ea78 100644 --- a/pkg/util/tracing/tracingpb/recorded_span.pb.go +++ b/pkg/util/tracing/tracingpb/recorded_span.pb.go @@ -39,7 +39,7 @@ func (m *LogRecord) Reset() { *m = LogRecord{} } func (m *LogRecord) String() string { return proto.CompactTextString(m) } func (*LogRecord) ProtoMessage() {} func (*LogRecord) Descriptor() ([]byte, []int) { - return fileDescriptor_recorded_span_91eb39344bb7b1f1, []int{0} + return fileDescriptor_recorded_span_dcd6691cfee4521a, []int{0} } func (m *LogRecord) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -73,7 +73,7 @@ func (m *LogRecord_Field) Reset() { *m = LogRecord_Field{} } func (m *LogRecord_Field) String() string { return proto.CompactTextString(m) } func (*LogRecord_Field) ProtoMessage() {} func (*LogRecord_Field) Descriptor() ([]byte, []int) { - return fileDescriptor_recorded_span_91eb39344bb7b1f1, []int{0, 0} + return fileDescriptor_recorded_span_dcd6691cfee4521a, []int{0, 0} } func (m *LogRecord_Field) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -137,12 +137,14 @@ type RecordedSpan struct { // // TODO(tbg): rename once DeprecatedStats is removed. InternalStructured []*types.Any `protobuf:"bytes,11,rep,name=internal_structured,json=internalStructured,proto3" json:"internal_structured,omitempty"` + // The ID of the goroutine on which the span was created. + GoroutineID uint64 `protobuf:"varint,12,opt,name=goroutine_id,json=goroutineId,proto3" json:"goroutine_id,omitempty"` } func (m *RecordedSpan) Reset() { *m = RecordedSpan{} } func (*RecordedSpan) ProtoMessage() {} func (*RecordedSpan) Descriptor() ([]byte, []int) { - return fileDescriptor_recorded_span_91eb39344bb7b1f1, []int{1} + return fileDescriptor_recorded_span_dcd6691cfee4521a, []int{1} } func (m *RecordedSpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -185,7 +187,7 @@ func (m *NormalizedSpan) Reset() { *m = NormalizedSpan{} } func (m *NormalizedSpan) String() string { return proto.CompactTextString(m) } func (*NormalizedSpan) ProtoMessage() {} func (*NormalizedSpan) Descriptor() ([]byte, []int) { - return fileDescriptor_recorded_span_91eb39344bb7b1f1, []int{2} + return fileDescriptor_recorded_span_dcd6691cfee4521a, []int{2} } func (m *NormalizedSpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -417,6 +419,11 @@ func (m *RecordedSpan) MarshalTo(dAtA []byte) (int, error) { i += n } } + if m.GoroutineID != 0 { + dAtA[i] = 0x60 + i++ + i = encodeVarintRecordedSpan(dAtA, i, uint64(m.GoroutineID)) + } return i, nil } @@ -604,6 +611,9 @@ func (m *RecordedSpan) Size() (n int) { n += 1 + l + sovRecordedSpan(uint64(l)) } } + if m.GoroutineID != 0 { + n += 1 + sovRecordedSpan(uint64(m.GoroutineID)) + } return n } @@ -1382,6 +1392,25 @@ func (m *RecordedSpan) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 12: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field GoroutineID", wireType) + } + m.GoroutineID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRecordedSpan + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.GoroutineID |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipRecordedSpan(dAtA[iNdEx:]) @@ -1828,51 +1857,53 @@ var ( ) func init() { - proto.RegisterFile("util/tracing/tracingpb/recorded_span.proto", fileDescriptor_recorded_span_91eb39344bb7b1f1) + proto.RegisterFile("util/tracing/tracingpb/recorded_span.proto", fileDescriptor_recorded_span_dcd6691cfee4521a) } -var fileDescriptor_recorded_span_91eb39344bb7b1f1 = []byte{ - // 668 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x93, 0xcd, 0x6e, 0xd3, 0x4e, - 0x14, 0xc5, 0xe3, 0xc4, 0xf9, 0xba, 0x89, 0xfa, 0xaf, 0xe6, 0xdf, 0x85, 0x1b, 0x21, 0xbb, 0x2a, - 0x12, 0xaa, 0xa8, 0xe4, 0x40, 0x91, 0xa0, 0x0a, 0x8b, 0x8a, 0xd0, 0x22, 0x45, 0xaa, 0x0a, 0x72, - 0xcb, 0x86, 0x4d, 0x34, 0xb1, 0xa7, 0xae, 0x55, 0xd7, 0x63, 0x8d, 0x27, 0x48, 0xe1, 0x29, 0xba, - 0xec, 0x92, 0x15, 0xaf, 0x42, 0x97, 0x5d, 0x76, 0x15, 0xc0, 0x7d, 0x11, 0x34, 0xe3, 0xb1, 0xfb, - 0x85, 0x14, 0x42, 0x57, 0xc9, 0xcc, 0x3d, 0xe7, 0xe7, 0x3b, 0x77, 0xce, 0xc0, 0xd3, 0x31, 0x0f, - 0xc2, 0x2e, 0x67, 0xd8, 0x0d, 0x22, 0x3f, 0xff, 0x8d, 0x47, 0x5d, 0x46, 0x5c, 0xca, 0x3c, 0xe2, - 0x0d, 0x93, 0x18, 0x47, 0x76, 0xcc, 0x28, 0xa7, 0x68, 0xc5, 0xa5, 0xee, 0x31, 0xa3, 0xd8, 0x3d, - 0xb2, 0x85, 0xcb, 0x56, 0x6a, 0xbb, 0x70, 0x75, 0x96, 0x7c, 0xea, 0x53, 0x29, 0xee, 0x8a, 0x7f, - 0x99, 0xaf, 0xb3, 0xec, 0x53, 0xea, 0x87, 0xa4, 0x2b, 0x57, 0xa3, 0xf1, 0x61, 0x17, 0x47, 0x13, - 0x55, 0xb2, 0xee, 0x96, 0x78, 0x70, 0x42, 0x12, 0x8e, 0x4f, 0x62, 0x25, 0x30, 0xef, 0x0a, 0xbc, - 0x31, 0xc3, 0x3c, 0xa0, 0xaa, 0xa7, 0xd5, 0xef, 0x1a, 0x34, 0x77, 0xa9, 0xef, 0xc8, 0x76, 0xd1, - 0x26, 0xe8, 0x02, 0x60, 0x68, 0x2b, 0xda, 0x5a, 0x6b, 0xa3, 0x63, 0x67, 0x66, 0x3b, 0x37, 0xdb, - 0x07, 0x39, 0xbd, 0xdf, 0x38, 0x9f, 0x5a, 0xa5, 0xd3, 0x1f, 0x96, 0xe6, 0x48, 0x07, 0x7a, 0x0f, - 0xb5, 0xc3, 0x80, 0x84, 0x5e, 0x62, 0x94, 0x57, 0x2a, 0x6b, 0xad, 0x8d, 0xe7, 0xf6, 0xac, 0xc3, - 0xda, 0xc5, 0x67, 0xed, 0x77, 0xc2, 0xd9, 0xd7, 0x05, 0xd2, 0x51, 0x98, 0x4e, 0x17, 0xaa, 0x72, - 0x1b, 0x2d, 0x42, 0xe5, 0x98, 0x4c, 0x64, 0x4b, 0x4d, 0x47, 0xfc, 0x45, 0x4b, 0x50, 0xfd, 0x8c, - 0xc3, 0x31, 0x31, 0xca, 0x72, 0x2f, 0x5b, 0xac, 0x7e, 0xab, 0x41, 0xdb, 0x51, 0x53, 0xdf, 0x8f, - 0x71, 0x84, 0x9e, 0x40, 0x43, 0x7c, 0x8c, 0x0c, 0x03, 0x4f, 0xba, 0xf5, 0x7e, 0x2b, 0x9d, 0x5a, - 0xf5, 0x03, 0xb1, 0x37, 0xd8, 0x76, 0xea, 0xb2, 0x38, 0xf0, 0xd0, 0x63, 0xa8, 0x8b, 0x4b, 0x12, - 0xb2, 0xb2, 0x94, 0x41, 0x3a, 0xb5, 0x6a, 0x02, 0x31, 0xd8, 0x76, 0x6a, 0xa2, 0x34, 0xf0, 0xd0, - 0x4b, 0x58, 0x88, 0x31, 0x23, 0x11, 0x1f, 0xe6, 0xda, 0x8a, 0xd4, 0x2e, 0xa6, 0x53, 0xab, 0xfd, - 0x41, 0x56, 0x94, 0xa3, 0x1d, 0x5f, 0xaf, 0x3c, 0xf4, 0x08, 0x9a, 0x34, 0x26, 0xd9, 0xc8, 0x0d, - 0x5d, 0xf6, 0x7b, 0xbd, 0x81, 0x3e, 0x42, 0x7d, 0x84, 0x7d, 0x1f, 0xfb, 0xc4, 0xa8, 0xca, 0xb1, - 0xbd, 0x9e, 0x3d, 0xb6, 0x9b, 0x67, 0xb4, 0xfb, 0x99, 0x7b, 0x27, 0xe2, 0x6c, 0xe2, 0xe4, 0x2c, - 0xb4, 0x0b, 0x3a, 0xc7, 0x7e, 0x62, 0xd4, 0x24, 0x73, 0x73, 0x4e, 0xe6, 0x01, 0xf6, 0x93, 0x0c, - 0x28, 0x29, 0xe8, 0x2d, 0x40, 0xc2, 0x31, 0xe3, 0x43, 0x19, 0x8d, 0xfa, 0x1c, 0xd1, 0x68, 0x4a, - 0x9f, 0xa8, 0xa0, 0x2d, 0x68, 0xe4, 0xc9, 0x33, 0x1a, 0x12, 0xb1, 0x7c, 0x0f, 0xb1, 0xad, 0x04, - 0x19, 0xe1, 0x4c, 0x10, 0x0a, 0x13, 0xda, 0x01, 0x3d, 0xa4, 0x7e, 0x62, 0x34, 0xe5, 0x99, 0xd6, - 0xe7, 0x88, 0x97, 0x0a, 0x96, 0xb4, 0xa3, 0x2d, 0x58, 0xf4, 0x48, 0xcc, 0x88, 0x8b, 0xb9, 0x78, - 0x9c, 0x1c, 0xf3, 0xc4, 0x00, 0xd9, 0xcf, 0xd2, 0xbd, 0x7e, 0xde, 0x44, 0x13, 0xe7, 0xbf, 0x6b, - 0xf5, 0xbe, 0x10, 0xa3, 0x1d, 0xf8, 0x3f, 0x88, 0x38, 0x61, 0x11, 0x0e, 0x87, 0x09, 0x67, 0x63, - 0x97, 0x8f, 0x19, 0xf1, 0x8c, 0x96, 0x6c, 0xeb, 0xcf, 0x0c, 0x94, 0x1b, 0xf6, 0x0b, 0x7d, 0xa7, - 0x07, 0xed, 0x9b, 0x77, 0xf7, 0xb7, 0x29, 0xef, 0x95, 0x37, 0xb5, 0xce, 0x2b, 0x68, 0x16, 0x77, - 0x34, 0x8f, 0xb1, 0xa7, 0x9f, 0x7d, 0xb5, 0x4a, 0xab, 0x97, 0x15, 0x58, 0xd8, 0xa3, 0xec, 0x04, - 0x87, 0xc1, 0x17, 0xf5, 0x54, 0x6e, 0xa5, 0x54, 0xbb, 0x9b, 0xd2, 0x3d, 0x15, 0xa7, 0xec, 0x65, - 0xf7, 0x66, 0x8f, 0xfe, 0x36, 0x7d, 0x46, 0xa0, 0x2a, 0x0f, 0x0f, 0x94, 0xfe, 0x90, 0x40, 0x55, - 0x1f, 0x16, 0x28, 0x07, 0x1a, 0xee, 0x51, 0x10, 0x7a, 0x8c, 0x44, 0xea, 0xbd, 0x3d, 0x9b, 0x77, - 0x40, 0x8a, 0x57, 0x70, 0xfe, 0xf9, 0x82, 0xfb, 0xeb, 0xe7, 0xbf, 0xcc, 0xd2, 0x79, 0x6a, 0x6a, - 0x17, 0xa9, 0xa9, 0x5d, 0xa6, 0xa6, 0xf6, 0x33, 0x35, 0xb5, 0xd3, 0x2b, 0xb3, 0x74, 0x71, 0x65, - 0x96, 0x2e, 0xaf, 0xcc, 0xd2, 0xa7, 0x66, 0xd1, 0xc4, 0xa8, 0x26, 0xe7, 0xf4, 0xe2, 0x77, 0x00, - 0x00, 0x00, 0xff, 0xff, 0xb9, 0xb4, 0xb3, 0xcd, 0xc3, 0x06, 0x00, 0x00, +var fileDescriptor_recorded_span_dcd6691cfee4521a = []byte{ + // 695 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x54, 0x4d, 0x6f, 0xd3, 0x4c, + 0x10, 0x8e, 0x13, 0xe7, 0x6b, 0x12, 0xb5, 0xd5, 0xbe, 0x3d, 0xb8, 0xd1, 0x2b, 0xbb, 0x2a, 0x12, + 0xaa, 0xa8, 0xe4, 0x40, 0x91, 0xa0, 0x0a, 0x87, 0x8a, 0x90, 0x82, 0x22, 0x55, 0x05, 0xb9, 0xe5, + 0xc2, 0x25, 0xda, 0xd8, 0x5b, 0xd7, 0xaa, 0xeb, 0xb5, 0xd6, 0x6b, 0xa4, 0xf0, 0x2b, 0x7a, 0xec, + 0x91, 0x7f, 0x43, 0x8f, 0x3d, 0x56, 0x1c, 0x02, 0xb8, 0x7f, 0x04, 0xed, 0xfa, 0xa3, 0x5f, 0x48, + 0x21, 0xf4, 0x14, 0xef, 0xcc, 0xf3, 0x3c, 0x3b, 0x3b, 0xf3, 0x4c, 0xe0, 0x49, 0xcc, 0x3d, 0xbf, + 0xcb, 0x19, 0xb6, 0xbd, 0xc0, 0xcd, 0x7f, 0xc3, 0x71, 0x97, 0x11, 0x9b, 0x32, 0x87, 0x38, 0xa3, + 0x28, 0xc4, 0x81, 0x19, 0x32, 0xca, 0x29, 0x5a, 0xb5, 0xa9, 0x7d, 0xcc, 0x28, 0xb6, 0x8f, 0x4c, + 0xc1, 0x32, 0x33, 0xb4, 0x59, 0xb0, 0x3a, 0xcb, 0x2e, 0x75, 0xa9, 0x04, 0x77, 0xc5, 0x57, 0xca, + 0xeb, 0xac, 0xb8, 0x94, 0xba, 0x3e, 0xe9, 0xca, 0xd3, 0x38, 0x3e, 0xec, 0xe2, 0x60, 0x92, 0xa5, + 0x8c, 0xbb, 0x29, 0xee, 0x9d, 0x90, 0x88, 0xe3, 0x93, 0x30, 0x03, 0xe8, 0x77, 0x01, 0x4e, 0xcc, + 0x30, 0xf7, 0x68, 0x56, 0xd3, 0xda, 0x37, 0x05, 0x9a, 0xbb, 0xd4, 0xb5, 0x64, 0xb9, 0x68, 0x0b, + 0x54, 0x21, 0xa0, 0x29, 0xab, 0xca, 0x7a, 0x6b, 0xb3, 0x63, 0xa6, 0x64, 0x33, 0x27, 0x9b, 0x07, + 0xb9, 0x7a, 0xbf, 0x71, 0x3e, 0x35, 0x4a, 0xa7, 0x3f, 0x0c, 0xc5, 0x92, 0x0c, 0xf4, 0x1e, 0x6a, + 0x87, 0x1e, 0xf1, 0x9d, 0x48, 0x2b, 0xaf, 0x56, 0xd6, 0x5b, 0x9b, 0xcf, 0xcc, 0x59, 0x8f, 0x35, + 0x8b, 0x6b, 0xcd, 0xb7, 0x82, 0xd9, 0x57, 0x85, 0xa4, 0x95, 0xc9, 0x74, 0xba, 0x50, 0x95, 0x61, + 0xb4, 0x04, 0x95, 0x63, 0x32, 0x91, 0x25, 0x35, 0x2d, 0xf1, 0x89, 0x96, 0xa1, 0xfa, 0x19, 0xfb, + 0x31, 0xd1, 0xca, 0x32, 0x96, 0x1e, 0xd6, 0xbe, 0xd7, 0xa0, 0x6d, 0x65, 0x5d, 0xdf, 0x0f, 0x71, + 0x80, 0x1e, 0x43, 0x43, 0x5c, 0x46, 0x46, 0x9e, 0x23, 0xd9, 0x6a, 0xbf, 0x95, 0x4c, 0x8d, 0xfa, + 0x81, 0x88, 0x0d, 0x07, 0x56, 0x5d, 0x26, 0x87, 0x0e, 0x7a, 0x04, 0x75, 0x31, 0x24, 0x01, 0x2b, + 0x4b, 0x18, 0x24, 0x53, 0xa3, 0x26, 0x24, 0x86, 0x03, 0xab, 0x26, 0x52, 0x43, 0x07, 0xbd, 0x80, + 0x85, 0x10, 0x33, 0x12, 0xf0, 0x51, 0x8e, 0xad, 0x48, 0xec, 0x52, 0x32, 0x35, 0xda, 0x1f, 0x64, + 0x26, 0x63, 0xb4, 0xc3, 0xeb, 0x93, 0x83, 0xfe, 0x87, 0x26, 0x0d, 0x49, 0xda, 0x72, 0x4d, 0x95, + 0xf5, 0x5e, 0x07, 0xd0, 0x47, 0xa8, 0x8f, 0xb1, 0xeb, 0x62, 0x97, 0x68, 0x55, 0xd9, 0xb6, 0x57, + 0xb3, 0xdb, 0x76, 0xf3, 0x8d, 0x66, 0x3f, 0x65, 0xef, 0x04, 0x9c, 0x4d, 0xac, 0x5c, 0x0b, 0xed, + 0x82, 0xca, 0xb1, 0x1b, 0x69, 0x35, 0xa9, 0xb9, 0x35, 0xa7, 0xe6, 0x01, 0x76, 0xa3, 0x54, 0x50, + 0xaa, 0xa0, 0x37, 0x00, 0x11, 0xc7, 0x8c, 0x8f, 0xa4, 0x35, 0xea, 0x73, 0x58, 0xa3, 0x29, 0x79, + 0x22, 0x83, 0xb6, 0xa1, 0x91, 0x3b, 0x4f, 0x6b, 0x48, 0x89, 0x95, 0x7b, 0x12, 0x83, 0x0c, 0x90, + 0x2a, 0x9c, 0x09, 0x85, 0x82, 0x84, 0x76, 0x40, 0xf5, 0xa9, 0x1b, 0x69, 0x4d, 0xf9, 0xa6, 0x8d, + 0x39, 0xec, 0x95, 0x19, 0x4b, 0xd2, 0xd1, 0x36, 0x2c, 0x39, 0x24, 0x64, 0xc4, 0xc6, 0x5c, 0x2c, + 0x27, 0xc7, 0x3c, 0xd2, 0x40, 0xd6, 0xb3, 0x7c, 0xaf, 0x9e, 0xd7, 0xc1, 0xc4, 0x5a, 0xbc, 0x46, + 0xef, 0x0b, 0x30, 0xda, 0x81, 0xff, 0xbc, 0x80, 0x13, 0x16, 0x60, 0x7f, 0x14, 0x71, 0x16, 0xdb, + 0x3c, 0x66, 0xc4, 0xd1, 0x5a, 0xb2, 0xac, 0x3f, 0x6b, 0xa0, 0x9c, 0xb0, 0x5f, 0xe0, 0xd1, 0x26, + 0xb4, 0x5d, 0xca, 0x68, 0xcc, 0xbd, 0x40, 0x1a, 0xb4, 0x2d, 0xdd, 0xb4, 0x98, 0x4c, 0x8d, 0xd6, + 0xbb, 0x3c, 0x3e, 0x1c, 0x58, 0xad, 0x02, 0x34, 0x74, 0x3a, 0x3d, 0x68, 0xdf, 0x9c, 0xf7, 0xdf, + 0x6e, 0x46, 0xaf, 0xbc, 0xa5, 0x74, 0x5e, 0x42, 0xb3, 0x98, 0xeb, 0x3c, 0xc4, 0x9e, 0x7a, 0xf6, + 0xd5, 0x28, 0xad, 0x5d, 0x56, 0x60, 0x61, 0x8f, 0xb2, 0x13, 0xec, 0x7b, 0x5f, 0xb2, 0xf5, 0xba, + 0xe5, 0x6c, 0xe5, 0xae, 0xb3, 0xf7, 0x32, 0x0b, 0xa6, 0xff, 0x06, 0xbd, 0xd9, 0xe3, 0xba, 0xad, + 0x3e, 0xc3, 0x84, 0x95, 0x87, 0x9b, 0x50, 0x7d, 0x88, 0x09, 0xab, 0x0f, 0x33, 0xa1, 0x05, 0x0d, + 0xfb, 0xc8, 0xf3, 0x1d, 0x46, 0x82, 0x6c, 0x47, 0x9f, 0xce, 0xdb, 0xa0, 0x4c, 0xaf, 0xd0, 0xf9, + 0xe7, 0x01, 0xf7, 0x37, 0xce, 0x7f, 0xe9, 0xa5, 0xf3, 0x44, 0x57, 0x2e, 0x12, 0x5d, 0xb9, 0x4c, + 0x74, 0xe5, 0x67, 0xa2, 0x2b, 0xa7, 0x57, 0x7a, 0xe9, 0xe2, 0x4a, 0x2f, 0x5d, 0x5e, 0xe9, 0xa5, + 0x4f, 0xcd, 0xa2, 0x88, 0x71, 0x4d, 0xf6, 0xe9, 0xf9, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0x8a, + 0xe6, 0x42, 0x34, 0xf7, 0x06, 0x00, 0x00, } diff --git a/pkg/util/tracing/tracingpb/recorded_span.proto b/pkg/util/tracing/tracingpb/recorded_span.proto index ca357e4ef0a8..1ac204ceeae7 100644 --- a/pkg/util/tracing/tracingpb/recorded_span.proto +++ b/pkg/util/tracing/tracingpb/recorded_span.proto @@ -76,6 +76,9 @@ message RecordedSpan { // // TODO(tbg): rename once DeprecatedStats is removed. repeated google.protobuf.Any internal_structured = 11; + + // The ID of the goroutine on which the span was created. + uint64 goroutine_id = 12 [(gogoproto.customname) = "GoroutineID"]; } // NormalizedSpan is a representation of a RecordedSpan from a trace with all