diff --git a/pkg/cli/mt_start_sql.go b/pkg/cli/mt_start_sql.go index d33bf22340e3..664c5a292001 100644 --- a/pkg/cli/mt_start_sql.go +++ b/pkg/cli/mt_start_sql.go @@ -97,7 +97,7 @@ func runStartSQL(cmd *cobra.Command, args []string) error { tempStorageMaxSizeBytes, ) - addr, httpAddr, err := server.StartTenant( + _, addr, httpAddr, err := server.StartTenant( ctx, stopper, clusterName, diff --git a/pkg/server/diagnostics/reporter.go b/pkg/server/diagnostics/reporter.go new file mode 100644 index 000000000000..c66744179e84 --- /dev/null +++ b/pkg/server/diagnostics/reporter.go @@ -0,0 +1,24 @@ +// Copyright 2016 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package diagnostics + +import ( + "time" + + "github.com/cockroachdb/cockroach/pkg/settings" +) + +// ReportFrequency is the interval at which diagnostics data should be reported. +var ReportFrequency = settings.RegisterPublicNonNegativeDurationSetting( + "diagnostics.reporting.interval", + "interval at which diagnostics data should be reported", + time.Hour, +) diff --git a/pkg/server/diagnosticspb/diagnostics.go b/pkg/server/diagnosticspb/diagnostics.go index 34a3b5aefe16..37513d205e89 100644 --- a/pkg/server/diagnosticspb/diagnostics.go +++ b/pkg/server/diagnosticspb/diagnostics.go @@ -14,6 +14,8 @@ import ( "net/url" "strconv" + "github.com/cockroachdb/cockroach/pkg/build" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" ) @@ -60,6 +62,7 @@ type TestingKnobs struct { // ClusterInfo contains cluster information that will become part of URLs. type ClusterInfo struct { ClusterID uuid.UUID + TenantID roachpb.TenantID IsInsecure bool IsInternal bool } @@ -71,36 +74,59 @@ func BuildUpdatesURL(clusterInfo *ClusterInfo, nodeInfo *NodeInfo, knobs *Testin if knobs != nil && knobs.OverrideUpdatesURL != nil { url = *knobs.OverrideUpdatesURL } - return addInfoToURL(url, clusterInfo, nodeInfo) + report := &DiagnosticReport{Node: *nodeInfo} + return addInfoToURL(url, clusterInfo, report) } -// BuildReportingURL creates a URL to report diagnostics. +// BuildReportingURL creates a URL to report diagnostics. If this is a CRDB +// node, then nodeInfo is filled (and nodeInfo.NodeID is non-zero). Otherwise, +// this is a SQL-only tenant and sqlInfo is filled. +// // If an empty updates URL is set (via empty environment variable), returns nil. -func BuildReportingURL(clusterInfo *ClusterInfo, nodeInfo *NodeInfo, knobs *TestingKnobs) *url.URL { +func BuildReportingURL( + clusterInfo *ClusterInfo, report *DiagnosticReport, knobs *TestingKnobs, +) *url.URL { url := reportingURL if knobs != nil && knobs.OverrideReportingURL != nil { url = *knobs.OverrideReportingURL } - return addInfoToURL(url, clusterInfo, nodeInfo) + return addInfoToURL(url, clusterInfo, report) } -func addInfoToURL(url *url.URL, clusterInfo *ClusterInfo, nodeInfo *NodeInfo) *url.URL { +func addInfoToURL(url *url.URL, clusterInfo *ClusterInfo, report *DiagnosticReport) *url.URL { if url == nil { return nil } result := *url q := result.Query() - b := &nodeInfo.Build + + // If NodeID is non-zero, then maintain backwards-compatibility by using the + // NodeInfo fields. + // TODO(andyk): Update this to always use other report fields, once they're + // guaranteed to be populated by all callers. + var b build.Info + if report.Node.NodeID != 0 { + // SQLInstanceID is always set to the NodeID for CRDB nodes. + b = report.Node.Build + q.Set("nodeid", strconv.Itoa(int(report.Node.NodeID))) + q.Set("sqlid", strconv.Itoa(int(report.Node.NodeID))) + q.Set("uptime", strconv.Itoa(int(report.Node.Uptime))) + q.Set("licensetype", report.Node.LicenseType) + } else { + b = report.Env.Build + q.Set("sqlid", strconv.Itoa(int(report.SQL.SQLInstanceID))) + q.Set("uptime", strconv.Itoa(int(report.SQL.Uptime))) + q.Set("licensetype", report.Env.LicenseType) + } + q.Set("version", b.Tag) q.Set("platform", b.Platform) q.Set("uuid", clusterInfo.ClusterID.String()) - q.Set("nodeid", strconv.Itoa(int(nodeInfo.NodeID))) - q.Set("uptime", strconv.Itoa(int(nodeInfo.Uptime))) + q.Set("tenantid", clusterInfo.TenantID.String()) q.Set("insecure", strconv.FormatBool(clusterInfo.IsInsecure)) q.Set("internal", strconv.FormatBool(clusterInfo.IsInternal)) q.Set("buildchannel", b.Channel) q.Set("envchannel", b.EnvChannel) - q.Set("licensetype", nodeInfo.LicenseType) result.RawQuery = q.Encode() return &result } diff --git a/pkg/server/diagnosticspb/diagnostics.pb.go b/pkg/server/diagnosticspb/diagnostics.pb.go index c50c3b4b82cf..f9f1383d113a 100644 --- a/pkg/server/diagnosticspb/diagnostics.pb.go +++ b/pkg/server/diagnosticspb/diagnostics.pb.go @@ -12,6 +12,7 @@ import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import github_com_cockroachdb_cockroach_pkg_base "github.com/cockroachdb/cockroach/pkg/base" import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" import encoding_binary "encoding/binary" @@ -37,6 +38,8 @@ type DiagnosticReport struct { AlteredSettings map[string]string `protobuf:"bytes,6,rep,name=altered_settings,json=alteredSettings,proto3" json:"altered_settings,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` ZoneConfigs map[int64]zonepb.ZoneConfig `protobuf:"bytes,8,rep,name=zone_configs,json=zoneConfigs,proto3" json:"zone_configs" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` FeatureUsage map[string]int32 `protobuf:"bytes,9,rep,name=feature_usage,json=featureUsage,proto3" json:"feature_usage" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + SQL SQLInstanceInfo `protobuf:"bytes,10,opt,name=sql,proto3" json:"sql"` + Env Environment `protobuf:"bytes,11,opt,name=env,proto3" json:"env"` LegacyUnimplementedErrors map[string]int64 `protobuf:"bytes,5,rep,name=legacy_unimplemented_errors,json=legacyUnimplementedErrors,proto3" json:"legacy_unimplemented_errors,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` LegacyErrorCounts map[string]int64 `protobuf:"bytes,7,rep,name=legacy_error_counts,json=legacyErrorCounts,proto3" json:"legacy_error_counts,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` } @@ -45,7 +48,7 @@ func (m *DiagnosticReport) Reset() { *m = DiagnosticReport{} } func (m *DiagnosticReport) String() string { return proto.CompactTextString(m) } func (*DiagnosticReport) ProtoMessage() {} func (*DiagnosticReport) Descriptor() ([]byte, []int) { - return fileDescriptor_diagnostics_5703a37aaee3f284, []int{0} + return fileDescriptor_diagnostics_06065893ddb3d5bb, []int{0} } func (m *DiagnosticReport) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -70,25 +73,29 @@ func (m *DiagnosticReport) XXX_DiscardUnknown() { var xxx_messageInfo_DiagnosticReport proto.InternalMessageInfo +// NodeInfo describes the CRDB node which is reporting diagnostics. NodeInfo +// will be set to its empty value (e.g. NodeID = 0) when the report is generated +// by a SQL-only instance. type NodeInfo struct { - NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` - Bytes int64 `protobuf:"varint,2,opt,name=bytes,proto3" json:"bytes,omitempty"` - KeyCount int64 `protobuf:"varint,3,opt,name=key_count,json=keyCount,proto3" json:"key_count,omitempty"` - RangeCount int64 `protobuf:"varint,4,opt,name=range_count,json=rangeCount,proto3" json:"range_count,omitempty"` - Locality roachpb.Locality `protobuf:"bytes,5,opt,name=locality,proto3" json:"locality"` - Hardware HardwareInfo `protobuf:"bytes,6,opt,name=hardware,proto3" json:"hardware"` - Os OSInfo `protobuf:"bytes,7,opt,name=os,proto3" json:"os"` - Build build.Info `protobuf:"bytes,8,opt,name=build,proto3" json:"build"` - Uptime int64 `protobuf:"varint,9,opt,name=uptime,proto3" json:"uptime,omitempty"` - LicenseType string `protobuf:"bytes,10,opt,name=license_type,json=licenseType,proto3" json:"license_type,omitempty"` - Topology TopologyInfo `protobuf:"bytes,11,opt,name=topology,proto3" json:"topology"` + NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` + Bytes int64 `protobuf:"varint,2,opt,name=bytes,proto3" json:"bytes,omitempty"` + KeyCount int64 `protobuf:"varint,3,opt,name=key_count,json=keyCount,proto3" json:"key_count,omitempty"` + RangeCount int64 `protobuf:"varint,4,opt,name=range_count,json=rangeCount,proto3" json:"range_count,omitempty"` + // Uptime is the number of seconds since the CRDB node was started. + Uptime int64 `protobuf:"varint,9,opt,name=uptime,proto3" json:"uptime,omitempty"` + Locality roachpb.Locality `protobuf:"bytes,5,opt,name=locality,proto3" json:"locality"` + Hardware HardwareInfo `protobuf:"bytes,6,opt,name=hardware,proto3" json:"hardware"` + Os OSInfo `protobuf:"bytes,7,opt,name=os,proto3" json:"os"` + Build build.Info `protobuf:"bytes,8,opt,name=build,proto3" json:"build"` + LicenseType string `protobuf:"bytes,10,opt,name=license_type,json=licenseType,proto3" json:"license_type,omitempty"` + Topology TopologyInfo `protobuf:"bytes,11,opt,name=topology,proto3" json:"topology"` } func (m *NodeInfo) Reset() { *m = NodeInfo{} } func (m *NodeInfo) String() string { return proto.CompactTextString(m) } func (*NodeInfo) ProtoMessage() {} func (*NodeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_diagnostics_5703a37aaee3f284, []int{1} + return fileDescriptor_diagnostics_06065893ddb3d5bb, []int{1} } func (m *NodeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -129,7 +136,7 @@ func (m *StoreInfo) Reset() { *m = StoreInfo{} } func (m *StoreInfo) String() string { return proto.CompactTextString(m) } func (*StoreInfo) ProtoMessage() {} func (*StoreInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_diagnostics_5703a37aaee3f284, []int{2} + return fileDescriptor_diagnostics_06065893ddb3d5bb, []int{2} } func (m *StoreInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -154,6 +161,94 @@ func (m *StoreInfo) XXX_DiscardUnknown() { var xxx_messageInfo_StoreInfo proto.InternalMessageInfo +// SQLInstanceInfo describes the SQL instance which is reporting diagnostics. +// This is always set, even for full CRDB nodes running both KV and SQL. +type SQLInstanceInfo struct { + // SQLInstanceID is the ephemeral identifier for the SQL instance which is + // reporting diagnostics. This is unique across all running SQL instances in + // the cluster (physical or tenant). + SQLInstanceID github_com_cockroachdb_cockroach_pkg_base.SQLInstanceID `protobuf:"varint,1,opt,name=sql_instance_id,json=sqlInstanceId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/base.SQLInstanceID" json:"sql_instance_id,omitempty"` + // Uptime is the number of seconds since the SQL instance was started. + Uptime int64 `protobuf:"varint,2,opt,name=uptime,proto3" json:"uptime,omitempty"` +} + +func (m *SQLInstanceInfo) Reset() { *m = SQLInstanceInfo{} } +func (m *SQLInstanceInfo) String() string { return proto.CompactTextString(m) } +func (*SQLInstanceInfo) ProtoMessage() {} +func (*SQLInstanceInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_diagnostics_06065893ddb3d5bb, []int{3} +} +func (m *SQLInstanceInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SQLInstanceInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SQLInstanceInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_SQLInstanceInfo.Merge(dst, src) +} +func (m *SQLInstanceInfo) XXX_Size() int { + return m.Size() +} +func (m *SQLInstanceInfo) XXX_DiscardUnknown() { + xxx_messageInfo_SQLInstanceInfo.DiscardUnknown(m) +} + +var xxx_messageInfo_SQLInstanceInfo proto.InternalMessageInfo + +// Environment describes the context in which the CRDB node or SQL instance is +// running. +type Environment struct { + // Locality is an ordered set of key/value tiers that describe the location of + // this CRDB node or SQL instance. + Locality roachpb.Locality `protobuf:"bytes,1,opt,name=locality,proto3" json:"locality"` + // Hardware describes the physical machine, virtual machine, or Linux + // container on which CRDB is running. + Hardware HardwareInfo `protobuf:"bytes,2,opt,name=hardware,proto3" json:"hardware"` + // OS describes the operating system software that hosts CRDB. + Os OSInfo `protobuf:"bytes,3,opt,name=os,proto3" json:"os"` + // Build describes the running CockroachDB binary. + Build build.Info `protobuf:"bytes,4,opt,name=build,proto3" json:"build"` + // LicenseType describes the license that is in effect (OSS, Enterprise, etc). + LicenseType string `protobuf:"bytes,5,opt,name=license_type,json=licenseType,proto3" json:"license_type,omitempty"` + // Topology provides the cloud provider and region name that is hosting CRDB. + Topology TopologyInfo `protobuf:"bytes,6,opt,name=topology,proto3" json:"topology"` +} + +func (m *Environment) Reset() { *m = Environment{} } +func (m *Environment) String() string { return proto.CompactTextString(m) } +func (*Environment) ProtoMessage() {} +func (*Environment) Descriptor() ([]byte, []int) { + return fileDescriptor_diagnostics_06065893ddb3d5bb, []int{4} +} +func (m *Environment) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Environment) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Environment) XXX_Merge(src proto.Message) { + xxx_messageInfo_Environment.Merge(dst, src) +} +func (m *Environment) XXX_Size() int { + return m.Size() +} +func (m *Environment) XXX_DiscardUnknown() { + xxx_messageInfo_Environment.DiscardUnknown(m) +} + +var xxx_messageInfo_Environment proto.InternalMessageInfo + type CPUInfo struct { Numcpu int32 `protobuf:"varint,1,opt,name=numcpu,proto3" json:"numcpu,omitempty"` Sockets int32 `protobuf:"varint,2,opt,name=sockets,proto3" json:"sockets,omitempty"` @@ -167,7 +262,7 @@ func (m *CPUInfo) Reset() { *m = CPUInfo{} } func (m *CPUInfo) String() string { return proto.CompactTextString(m) } func (*CPUInfo) ProtoMessage() {} func (*CPUInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_diagnostics_5703a37aaee3f284, []int{3} + return fileDescriptor_diagnostics_06065893ddb3d5bb, []int{5} } func (m *CPUInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -205,7 +300,7 @@ func (m *HardwareInfo) Reset() { *m = HardwareInfo{} } func (m *HardwareInfo) String() string { return proto.CompactTextString(m) } func (*HardwareInfo) ProtoMessage() {} func (*HardwareInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_diagnostics_5703a37aaee3f284, []int{4} + return fileDescriptor_diagnostics_06065893ddb3d5bb, []int{6} } func (m *HardwareInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -240,7 +335,7 @@ func (m *OSInfo) Reset() { *m = OSInfo{} } func (m *OSInfo) String() string { return proto.CompactTextString(m) } func (*OSInfo) ProtoMessage() {} func (*OSInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_diagnostics_5703a37aaee3f284, []int{5} + return fileDescriptor_diagnostics_06065893ddb3d5bb, []int{7} } func (m *OSInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -274,7 +369,7 @@ func (m *MemInfo) Reset() { *m = MemInfo{} } func (m *MemInfo) String() string { return proto.CompactTextString(m) } func (*MemInfo) ProtoMessage() {} func (*MemInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_diagnostics_5703a37aaee3f284, []int{6} + return fileDescriptor_diagnostics_06065893ddb3d5bb, []int{8} } func (m *MemInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -308,7 +403,7 @@ func (m *TopologyInfo) Reset() { *m = TopologyInfo{} } func (m *TopologyInfo) String() string { return proto.CompactTextString(m) } func (*TopologyInfo) ProtoMessage() {} func (*TopologyInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_diagnostics_5703a37aaee3f284, []int{7} + return fileDescriptor_diagnostics_06065893ddb3d5bb, []int{9} } func (m *TopologyInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -342,6 +437,8 @@ func init() { proto.RegisterMapType((map[int64]zonepb.ZoneConfig)(nil), "cockroach.server.diagnosticspb.DiagnosticReport.ZoneConfigsEntry") proto.RegisterType((*NodeInfo)(nil), "cockroach.server.diagnosticspb.NodeInfo") proto.RegisterType((*StoreInfo)(nil), "cockroach.server.diagnosticspb.StoreInfo") + proto.RegisterType((*SQLInstanceInfo)(nil), "cockroach.server.diagnosticspb.SQLInstanceInfo") + proto.RegisterType((*Environment)(nil), "cockroach.server.diagnosticspb.Environment") proto.RegisterType((*CPUInfo)(nil), "cockroach.server.diagnosticspb.CPUInfo") proto.RegisterType((*HardwareInfo)(nil), "cockroach.server.diagnosticspb.HardwareInfo") proto.RegisterType((*OSInfo)(nil), "cockroach.server.diagnosticspb.OSInfo") @@ -522,6 +619,22 @@ func (m *DiagnosticReport) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintDiagnostics(dAtA, i, uint64(v)) } } + dAtA[i] = 0x52 + i++ + i = encodeVarintDiagnostics(dAtA, i, uint64(m.SQL.Size())) + n3, err := m.SQL.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n3 + dAtA[i] = 0x5a + i++ + i = encodeVarintDiagnostics(dAtA, i, uint64(m.Env.Size())) + n4, err := m.Env.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n4 return i, nil } @@ -563,35 +676,35 @@ func (m *NodeInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintDiagnostics(dAtA, i, uint64(m.Locality.Size())) - n3, err := m.Locality.MarshalTo(dAtA[i:]) + n5, err := m.Locality.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n3 + i += n5 dAtA[i] = 0x32 i++ i = encodeVarintDiagnostics(dAtA, i, uint64(m.Hardware.Size())) - n4, err := m.Hardware.MarshalTo(dAtA[i:]) + n6, err := m.Hardware.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n4 + i += n6 dAtA[i] = 0x3a i++ i = encodeVarintDiagnostics(dAtA, i, uint64(m.Os.Size())) - n5, err := m.Os.MarshalTo(dAtA[i:]) + n7, err := m.Os.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n5 + i += n7 dAtA[i] = 0x42 i++ i = encodeVarintDiagnostics(dAtA, i, uint64(m.Build.Size())) - n6, err := m.Build.MarshalTo(dAtA[i:]) + n8, err := m.Build.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n6 + i += n8 if m.Uptime != 0 { dAtA[i] = 0x48 i++ @@ -606,11 +719,11 @@ func (m *NodeInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintDiagnostics(dAtA, i, uint64(m.Topology.Size())) - n7, err := m.Topology.MarshalTo(dAtA[i:]) + n9, err := m.Topology.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n7 + i += n9 return i, nil } @@ -677,6 +790,98 @@ func (m *StoreInfo) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *SQLInstanceInfo) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SQLInstanceInfo) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.SQLInstanceID != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintDiagnostics(dAtA, i, uint64(m.SQLInstanceID)) + } + if m.Uptime != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintDiagnostics(dAtA, i, uint64(m.Uptime)) + } + return i, nil +} + +func (m *Environment) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Environment) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintDiagnostics(dAtA, i, uint64(m.Locality.Size())) + n10, err := m.Locality.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n10 + dAtA[i] = 0x12 + i++ + i = encodeVarintDiagnostics(dAtA, i, uint64(m.Hardware.Size())) + n11, err := m.Hardware.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n11 + dAtA[i] = 0x1a + i++ + i = encodeVarintDiagnostics(dAtA, i, uint64(m.Os.Size())) + n12, err := m.Os.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n12 + dAtA[i] = 0x22 + i++ + i = encodeVarintDiagnostics(dAtA, i, uint64(m.Build.Size())) + n13, err := m.Build.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n13 + if len(m.LicenseType) > 0 { + dAtA[i] = 0x2a + i++ + i = encodeVarintDiagnostics(dAtA, i, uint64(len(m.LicenseType))) + i += copy(dAtA[i:], m.LicenseType) + } + dAtA[i] = 0x32 + i++ + i = encodeVarintDiagnostics(dAtA, i, uint64(m.Topology.Size())) + n14, err := m.Topology.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n14 + return i, nil +} + func (m *CPUInfo) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -761,19 +966,19 @@ func (m *HardwareInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintDiagnostics(dAtA, i, uint64(m.Cpu.Size())) - n8, err := m.Cpu.MarshalTo(dAtA[i:]) + n15, err := m.Cpu.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n8 + i += n15 dAtA[i] = 0x1a i++ i = encodeVarintDiagnostics(dAtA, i, uint64(m.Mem.Size())) - n9, err := m.Mem.MarshalTo(dAtA[i:]) + n16, err := m.Mem.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n9 + i += n16 if m.Loadavg15 != 0 { dAtA[i] = 0x25 i++ @@ -965,6 +1170,10 @@ func (m *DiagnosticReport) Size() (n int) { n += mapEntrySize + 1 + sovDiagnostics(uint64(mapEntrySize)) } } + l = m.SQL.Size() + n += 1 + l + sovDiagnostics(uint64(l)) + l = m.Env.Size() + n += 1 + l + sovDiagnostics(uint64(l)) return n } @@ -1042,6 +1251,44 @@ func (m *StoreInfo) Size() (n int) { return n } +func (m *SQLInstanceInfo) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SQLInstanceID != 0 { + n += 1 + sovDiagnostics(uint64(m.SQLInstanceID)) + } + if m.Uptime != 0 { + n += 1 + sovDiagnostics(uint64(m.Uptime)) + } + return n +} + +func (m *Environment) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.Locality.Size() + n += 1 + l + sovDiagnostics(uint64(l)) + l = m.Hardware.Size() + n += 1 + l + sovDiagnostics(uint64(l)) + l = m.Os.Size() + n += 1 + l + sovDiagnostics(uint64(l)) + l = m.Build.Size() + n += 1 + l + sovDiagnostics(uint64(l)) + l = len(m.LicenseType) + if l > 0 { + n += 1 + l + sovDiagnostics(uint64(l)) + } + l = m.Topology.Size() + n += 1 + l + sovDiagnostics(uint64(l)) + return n +} + func (m *CPUInfo) Size() (n int) { if m == nil { return 0 @@ -1870,6 +2117,66 @@ func (m *DiagnosticReport) Unmarshal(dAtA []byte) error { } m.FeatureUsage[mapkey] = mapvalue iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SQL", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDiagnostics + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthDiagnostics + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.SQL.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Env", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDiagnostics + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthDiagnostics + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Env.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipDiagnostics(dAtA[iNdEx:]) @@ -2436,6 +2743,323 @@ func (m *StoreInfo) Unmarshal(dAtA []byte) error { } return nil } +func (m *SQLInstanceInfo) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDiagnostics + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SQLInstanceInfo: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SQLInstanceInfo: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SQLInstanceID", wireType) + } + m.SQLInstanceID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDiagnostics + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SQLInstanceID |= (github_com_cockroachdb_cockroach_pkg_base.SQLInstanceID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Uptime", wireType) + } + m.Uptime = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDiagnostics + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Uptime |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipDiagnostics(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthDiagnostics + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Environment) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDiagnostics + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Environment: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Environment: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Locality", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDiagnostics + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthDiagnostics + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Locality.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hardware", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDiagnostics + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthDiagnostics + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Hardware.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Os", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDiagnostics + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthDiagnostics + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Os.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Build", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDiagnostics + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthDiagnostics + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Build.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LicenseType", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDiagnostics + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDiagnostics + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LicenseType = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Topology", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDiagnostics + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthDiagnostics + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Topology.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipDiagnostics(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthDiagnostics + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *CPUInfo) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -3259,89 +3883,97 @@ var ( ) func init() { - proto.RegisterFile("server/diagnosticspb/diagnostics.proto", fileDescriptor_diagnostics_5703a37aaee3f284) -} - -var fileDescriptor_diagnostics_5703a37aaee3f284 = []byte{ - // 1271 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x56, 0xcf, 0x6e, 0x1b, 0x37, - 0x13, 0xb7, 0xb4, 0xfa, 0x4b, 0x39, 0xf9, 0x1c, 0xc6, 0x9f, 0xbb, 0x75, 0x0a, 0x39, 0x55, 0xd0, - 0x34, 0x45, 0x0b, 0x09, 0x49, 0x5a, 0xa0, 0x2d, 0x1a, 0x04, 0x96, 0x9d, 0xa6, 0x01, 0xd2, 0xa4, - 0x58, 0x27, 0x3d, 0xe4, 0x50, 0x81, 0xe2, 0x52, 0xeb, 0xad, 0xb9, 0xcb, 0x35, 0xc9, 0x55, 0x20, - 0xbf, 0x42, 0x0f, 0xed, 0xa5, 0x87, 0x02, 0x7d, 0x9d, 0x02, 0x39, 0xe6, 0x98, 0x93, 0xd1, 0x3a, - 0x6f, 0xd1, 0x53, 0xc1, 0x21, 0x25, 0xaf, 0x95, 0xc4, 0x8a, 0x0b, 0xf4, 0x20, 0x88, 0x33, 0xc3, - 0xf9, 0xcd, 0xfc, 0x86, 0xc3, 0xe1, 0xa2, 0xab, 0x8a, 0xc9, 0x31, 0x93, 0xbd, 0x30, 0x26, 0x51, - 0x2a, 0x94, 0x8e, 0xa9, 0xca, 0x86, 0x45, 0xa9, 0x9b, 0x49, 0xa1, 0x05, 0x6e, 0x53, 0x41, 0xf7, - 0xa4, 0x20, 0x74, 0xb7, 0x6b, 0x3d, 0xba, 0x27, 0x3c, 0xd6, 0x57, 0x86, 0x79, 0xcc, 0xc3, 0x5e, - 0x9c, 0x8e, 0x84, 0xf5, 0x58, 0xf7, 0xa9, 0x48, 0x47, 0x71, 0xd4, 0x3b, 0x10, 0x29, 0xcb, 0x86, - 0xf0, 0xe7, 0x2c, 0xab, 0x91, 0x88, 0x04, 0x2c, 0x7b, 0x66, 0xe5, 0xb4, 0xef, 0x00, 0x7a, 0x36, - 0xec, 0x91, 0x2c, 0x1b, 0x28, 0x4d, 0xb4, 0x0b, 0xbd, 0x7e, 0x45, 0xed, 0xf3, 0x1e, 0x25, 0x9a, - 0x70, 0x11, 0xf5, 0x42, 0xa6, 0x68, 0x36, 0xec, 0x29, 0x2d, 0x73, 0xaa, 0x73, 0xc9, 0x42, 0xb7, - 0x69, 0x6d, 0xea, 0x9d, 0x30, 0x4d, 0x42, 0xa2, 0x89, 0xd5, 0x77, 0x7e, 0x42, 0x68, 0x65, 0x7b, - 0x96, 0x69, 0xc0, 0x32, 0x21, 0x35, 0xee, 0xa3, 0x4a, 0x2a, 0x42, 0xe6, 0x97, 0x2e, 0x97, 0xae, - 0xb5, 0x6e, 0x5c, 0xeb, 0x9e, 0xce, 0xad, 0xfb, 0x40, 0x84, 0xec, 0x5e, 0x3a, 0x12, 0xfd, 0xca, - 0xb3, 0xc3, 0x8d, 0xa5, 0x00, 0x7c, 0xf1, 0x5d, 0x54, 0x53, 0x5a, 0x48, 0xa6, 0xfc, 0xf2, 0x65, - 0xef, 0x5a, 0xeb, 0xc6, 0x47, 0x8b, 0x50, 0x76, 0xcc, 0xee, 0x02, 0x8c, 0x73, 0xc7, 0xdb, 0xa8, - 0xa6, 0xe8, 0x2e, 0x4b, 0x88, 0xef, 0x01, 0xd0, 0xd5, 0x22, 0xd0, 0x3e, 0x37, 0xbf, 0x21, 0x51, - 0xac, 0xfb, 0x88, 0x0c, 0x39, 0xdb, 0x66, 0x8a, 0xca, 0x38, 0xd3, 0x42, 0xce, 0x50, 0xc0, 0x17, - 0x3f, 0x40, 0x4d, 0xb5, 0xcf, 0x6d, 0xdd, 0xfc, 0x0a, 0x00, 0x7d, 0x3c, 0x07, 0xb4, 0x25, 0x38, - 0x67, 0x54, 0xb3, 0x70, 0x47, 0x13, 0xcd, 0x12, 0x96, 0x6a, 0xb3, 0x88, 0x21, 0x3f, 0x87, 0xd6, - 0x50, 0xfb, 0xdc, 0x28, 0x15, 0xfe, 0xb9, 0x84, 0x2e, 0x71, 0x16, 0x11, 0x3a, 0x19, 0xe4, 0x69, - 0x9c, 0x64, 0x1c, 0x1c, 0x58, 0x38, 0x60, 0x52, 0x0a, 0xa9, 0xfc, 0x2a, 0x84, 0x78, 0xb8, 0x88, - 0xf4, 0x7c, 0xe9, 0xbb, 0xf7, 0x01, 0xf3, 0x71, 0x11, 0xf2, 0x0e, 0x20, 0xde, 0x49, 0xb5, 0x9c, - 0x04, 0xef, 0xf2, 0x37, 0xd9, 0x71, 0x86, 0x56, 0x08, 0xd7, 0x4c, 0xb2, 0x70, 0xa0, 0x98, 0xd6, - 0x71, 0x1a, 0x29, 0xbf, 0x06, 0x59, 0xdc, 0x39, 0x73, 0x16, 0x9b, 0x16, 0x68, 0xc7, 0xe1, 0xd8, - 0xd8, 0xff, 0x23, 0x27, 0xb5, 0xf8, 0x29, 0xba, 0xe8, 0x4a, 0x00, 0xa4, 0x07, 0x54, 0xe4, 0xa9, - 0x56, 0x7e, 0x1d, 0x82, 0xde, 0xfd, 0x97, 0xd4, 0x81, 0xcd, 0x16, 0x20, 0xd9, 0xb0, 0x17, 0xf8, - 0xbc, 0x1e, 0xff, 0x88, 0x96, 0xcd, 0x75, 0x19, 0xd8, 0x1b, 0xa4, 0xfc, 0x06, 0x44, 0xdc, 0x3c, - 0x73, 0xc4, 0x27, 0x22, 0x65, 0x5b, 0x16, 0x03, 0x62, 0xb9, 0x53, 0x6e, 0x1d, 0x1c, 0xeb, 0x71, - 0x82, 0xce, 0x8d, 0x18, 0x31, 0x57, 0x69, 0x90, 0x2b, 0x12, 0x31, 0xbf, 0x09, 0xc1, 0xfa, 0x67, - 0x0e, 0xf6, 0xb5, 0x45, 0x79, 0x6c, 0x40, 0x8a, 0xd1, 0x96, 0x47, 0x05, 0xc3, 0x7a, 0x1f, 0xad, - 0xbe, 0xae, 0xf8, 0x78, 0x05, 0x79, 0x7b, 0x6c, 0x02, 0x37, 0xb2, 0x19, 0x98, 0x25, 0x5e, 0x45, - 0xd5, 0x31, 0xe1, 0x39, 0xf3, 0xcb, 0xa0, 0xb3, 0xc2, 0x97, 0xe5, 0xcf, 0x4b, 0xeb, 0x14, 0xad, - 0xcc, 0x33, 0x2b, 0xfa, 0x7b, 0xd6, 0xff, 0x8b, 0xa2, 0x7f, 0xeb, 0xc6, 0x95, 0x02, 0x21, 0x5b, - 0xd7, 0xae, 0x9d, 0x4c, 0x85, 0x2a, 0x15, 0x83, 0xdc, 0x46, 0x17, 0x5e, 0x61, 0xb4, 0x28, 0xcb, - 0x6a, 0x11, 0xe0, 0x3e, 0x6a, 0x9f, 0xde, 0xec, 0x8b, 0xd0, 0xbc, 0x22, 0xda, 0x36, 0x5a, 0x7b, - 0x7d, 0xff, 0x9c, 0x05, 0xa5, 0xf3, 0x47, 0x05, 0x35, 0xa6, 0xd3, 0x0c, 0x3f, 0x41, 0x75, 0x33, - 0xc9, 0x06, 0x71, 0x08, 0xce, 0xd5, 0xfe, 0xe6, 0xd1, 0xe1, 0x46, 0x0d, 0xcc, 0xdb, 0x7f, 0x1f, - 0x6e, 0xdc, 0x8c, 0x62, 0xbd, 0x9b, 0x0f, 0xbb, 0x54, 0x24, 0xbd, 0x59, 0xe9, 0xc2, 0xe1, 0xf1, - 0xba, 0x97, 0xed, 0x45, 0x3d, 0x37, 0x76, 0xed, 0x8c, 0xdc, 0x0e, 0x6a, 0x06, 0xf1, 0x5e, 0x68, - 0x52, 0x18, 0x4e, 0x34, 0x0c, 0x47, 0x48, 0x01, 0x04, 0x7c, 0x09, 0x35, 0xf7, 0xd8, 0xc4, 0xde, - 0x23, 0xdf, 0x03, 0x4b, 0x63, 0x8f, 0x4d, 0x80, 0x0d, 0xde, 0x40, 0x2d, 0x49, 0xd2, 0x88, 0x39, - 0x73, 0x05, 0xcc, 0x08, 0x54, 0x76, 0xc3, 0x2d, 0xd4, 0xe0, 0x82, 0x12, 0x1e, 0xeb, 0x89, 0x5f, - 0x85, 0x33, 0xbd, 0x54, 0x38, 0xd3, 0x69, 0x22, 0xf7, 0xdd, 0x96, 0xe9, 0x44, 0x9b, 0xba, 0xe0, - 0x07, 0xa8, 0xb1, 0x4b, 0x64, 0xf8, 0x94, 0x48, 0xe6, 0xd7, 0xc0, 0xfd, 0x93, 0x45, 0x3d, 0xfe, - 0x8d, 0xdb, 0x5f, 0x98, 0xda, 0x33, 0x0c, 0xfc, 0x15, 0x2a, 0x0b, 0x33, 0x0c, 0x4a, 0xf3, 0x33, - 0xfb, 0x75, 0x48, 0x0f, 0x77, 0x0a, 0x18, 0x65, 0xa1, 0xf0, 0x75, 0x54, 0x85, 0x17, 0xd3, 0x6f, - 0x00, 0xc0, 0xff, 0x0b, 0x00, 0xa0, 0xef, 0x16, 0xf6, 0xdb, 0x9d, 0x78, 0x0d, 0xd5, 0xf2, 0x4c, - 0xc7, 0x89, 0xb9, 0xa2, 0xa6, 0x36, 0x4e, 0xc2, 0xef, 0xa3, 0x65, 0x1e, 0x53, 0x96, 0x2a, 0x36, - 0xd0, 0x93, 0x8c, 0xf9, 0x08, 0x3a, 0xa1, 0xe5, 0x74, 0x8f, 0x26, 0x19, 0x33, 0xdc, 0xb5, 0xc8, - 0x04, 0x17, 0xd1, 0xc4, 0x6f, 0xbd, 0x1d, 0xf7, 0x47, 0x6e, 0x7f, 0x91, 0xfb, 0x14, 0xa3, 0xf3, - 0xbb, 0x87, 0x9a, 0xb3, 0xf7, 0xec, 0x3f, 0x6d, 0xa4, 0x1f, 0x50, 0x03, 0xde, 0x49, 0x03, 0x0e, - 0x57, 0xac, 0xbf, 0x75, 0x74, 0xb8, 0x51, 0xb7, 0xc1, 0x0d, 0xfa, 0xa7, 0x67, 0x42, 0x77, 0x7e, - 0x41, 0x1d, 0x40, 0x8b, 0x8d, 0xea, 0xbd, 0xb1, 0x51, 0x2b, 0xa7, 0x37, 0x6a, 0xf5, 0x95, 0x46, - 0x5d, 0x47, 0x0d, 0x4a, 0x32, 0x42, 0x4d, 0xa3, 0xd6, 0xac, 0xf3, 0x54, 0xc6, 0xef, 0xa1, 0x26, - 0x19, 0x93, 0x98, 0x9b, 0xc7, 0x1c, 0x9a, 0xc7, 0x0b, 0x8e, 0x15, 0x18, 0xa3, 0x4a, 0xae, 0x98, - 0x6d, 0x0a, 0x2f, 0x80, 0x35, 0xbe, 0x8e, 0x56, 0x59, 0x4a, 0xe5, 0x24, 0xd3, 0xb1, 0x48, 0x07, - 0x84, 0x47, 0x42, 0xc6, 0x7a, 0x37, 0x71, 0x4d, 0x70, 0xf1, 0xd8, 0xb6, 0x39, 0x35, 0x75, 0x7e, - 0x2d, 0xa1, 0xfa, 0xd6, 0x77, 0x8f, 0xe1, 0x70, 0xd6, 0x50, 0x2d, 0xcd, 0x13, 0x9a, 0xe5, 0xf6, - 0x6c, 0x02, 0x27, 0x61, 0x1f, 0xd5, 0x95, 0xa0, 0x7b, 0x4c, 0x2b, 0x37, 0xba, 0xa6, 0xa2, 0x29, - 0x09, 0x85, 0x0f, 0x1b, 0xcf, 0x8e, 0x34, 0x10, 0x8c, 0x36, 0x11, 0x21, 0xe3, 0x50, 0x8e, 0x66, - 0x60, 0x05, 0x33, 0x7c, 0x92, 0xdd, 0x03, 0xa8, 0x41, 0x39, 0x30, 0x4b, 0x43, 0xde, 0x0d, 0x7c, - 0xfb, 0x3c, 0x37, 0x83, 0x99, 0xdc, 0xf9, 0xad, 0x8c, 0x96, 0x8b, 0x77, 0x0a, 0x5f, 0x45, 0xe7, - 0xc7, 0xb1, 0xd4, 0x39, 0xe1, 0xf1, 0x01, 0x31, 0x1c, 0xdc, 0x18, 0x9b, 0xd3, 0xe2, 0xdb, 0xc8, - 0x33, 0x0c, 0xec, 0x24, 0xff, 0x70, 0x51, 0xeb, 0x3a, 0xea, 0xae, 0x6b, 0x8d, 0xa7, 0x01, 0x48, - 0x58, 0x02, 0x8c, 0xde, 0x02, 0xe0, 0x5b, 0x96, 0x14, 0x01, 0x12, 0x96, 0x98, 0x73, 0xe3, 0x82, - 0x84, 0x64, 0x1c, 0x5d, 0xff, 0x0c, 0x4a, 0x50, 0x0e, 0x8e, 0x15, 0x86, 0x74, 0x26, 0xc5, 0x38, - 0x0e, 0x99, 0x84, 0x5a, 0x34, 0x83, 0x99, 0x8c, 0x3f, 0x40, 0xe7, 0xe3, 0x54, 0x69, 0x92, 0x52, - 0x36, 0xa0, 0x9c, 0x28, 0x05, 0x3d, 0xd1, 0x0c, 0xce, 0x4d, 0xb5, 0x5b, 0x46, 0xd9, 0xf9, 0x1e, - 0xd5, 0xec, 0x90, 0x30, 0x27, 0x36, 0x22, 0x49, 0xcc, 0xa7, 0x33, 0xdd, 0x49, 0x10, 0x84, 0x13, - 0x3d, 0x12, 0x32, 0x71, 0x6f, 0xe2, 0x4c, 0x36, 0xa7, 0x39, 0x66, 0x52, 0x99, 0x0a, 0x7a, 0x60, - 0x9a, 0x8a, 0x9d, 0x5b, 0xa8, 0xee, 0xe8, 0x98, 0x23, 0xd4, 0x42, 0x13, 0x0e, 0xb8, 0x95, 0xc0, - 0x0a, 0x27, 0x3b, 0xb2, 0x0c, 0x96, 0x63, 0x45, 0xa7, 0x8f, 0x96, 0x8b, 0x93, 0xe0, 0x04, 0xd3, - 0xd2, 0x1c, 0xd3, 0x35, 0x54, 0x93, 0x2c, 0x32, 0x39, 0xd8, 0xf4, 0x9c, 0xd4, 0xef, 0x3d, 0xfb, - 0xab, 0xbd, 0xf4, 0xec, 0xa8, 0x5d, 0x7a, 0x7e, 0xd4, 0x2e, 0xbd, 0x38, 0x6a, 0x97, 0xfe, 0x3c, - 0x6a, 0x97, 0x7e, 0x79, 0xd9, 0x5e, 0x7a, 0xfe, 0xb2, 0xbd, 0xf4, 0xe2, 0x65, 0x7b, 0xe9, 0xc9, - 0xb9, 0x13, 0xf5, 0x1f, 0xd6, 0xe0, 0xdb, 0xfd, 0xe6, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x04, - 0xe9, 0x8f, 0xba, 0x9d, 0x0c, 0x00, 0x00, + proto.RegisterFile("server/diagnosticspb/diagnostics.proto", fileDescriptor_diagnostics_06065893ddb3d5bb) +} + +var fileDescriptor_diagnostics_06065893ddb3d5bb = []byte{ + // 1408 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x57, 0x4f, 0x6f, 0x1b, 0x45, + 0x14, 0x8f, 0xbd, 0xfe, 0x3b, 0x4e, 0xda, 0x74, 0x1a, 0xc2, 0x92, 0x22, 0xbb, 0xb8, 0xa2, 0x14, + 0x15, 0xd9, 0x6a, 0x0b, 0xe2, 0x8f, 0xa8, 0xaa, 0xd8, 0x09, 0xa5, 0x52, 0x48, 0xe9, 0xa6, 0xe5, + 0xd0, 0x03, 0xd6, 0x78, 0x77, 0xbc, 0x59, 0x32, 0xbb, 0xb3, 0xd9, 0x19, 0xbb, 0x72, 0x4e, 0x7c, + 0x03, 0xb8, 0x70, 0x40, 0x42, 0x82, 0x2f, 0x83, 0xd4, 0x63, 0x8f, 0x3d, 0x45, 0xe0, 0x7e, 0x0b, + 0x4e, 0x68, 0xde, 0x8c, 0xed, 0xb5, 0xdb, 0xc6, 0x0d, 0x2d, 0x87, 0x28, 0xfb, 0xde, 0xcc, 0xfb, + 0xbd, 0xff, 0xef, 0x8d, 0xd1, 0x65, 0x41, 0x93, 0x01, 0x4d, 0x9a, 0x5e, 0x40, 0xfc, 0x88, 0x0b, + 0x19, 0xb8, 0x22, 0xee, 0xa6, 0xa9, 0x46, 0x9c, 0x70, 0xc9, 0x71, 0xd5, 0xe5, 0xee, 0x41, 0xc2, + 0x89, 0xbb, 0xdf, 0xd0, 0x12, 0x8d, 0x19, 0x89, 0x8d, 0xd5, 0x6e, 0x3f, 0x60, 0x5e, 0x33, 0x88, + 0x7a, 0x5c, 0x4b, 0x6c, 0xd8, 0x2e, 0x8f, 0x7a, 0x81, 0xdf, 0x3c, 0xe2, 0x11, 0x8d, 0xbb, 0xf0, + 0xcf, 0x9c, 0xac, 0xf9, 0xdc, 0xe7, 0xf0, 0xd9, 0x54, 0x5f, 0x86, 0xfb, 0x36, 0xa0, 0xc7, 0xdd, + 0x26, 0x89, 0xe3, 0x8e, 0x90, 0x44, 0x1a, 0xd5, 0x1b, 0x97, 0xc4, 0x21, 0x6b, 0xba, 0x44, 0x12, + 0xc6, 0xfd, 0xa6, 0x47, 0x85, 0x1b, 0x77, 0x9b, 0x42, 0x26, 0x7d, 0x57, 0xf6, 0x13, 0xea, 0x99, + 0x4b, 0xeb, 0x63, 0xe9, 0x90, 0x4a, 0xe2, 0x11, 0x49, 0x34, 0xbf, 0xfe, 0x47, 0x05, 0xad, 0x6e, + 0x4d, 0x2c, 0x75, 0x68, 0xcc, 0x13, 0x89, 0x5b, 0x28, 0x17, 0x71, 0x8f, 0xda, 0x99, 0x8b, 0x99, + 0x2b, 0x95, 0xeb, 0x57, 0x1a, 0x27, 0xfb, 0xd6, 0xd8, 0xe5, 0x1e, 0xbd, 0x13, 0xf5, 0x78, 0x2b, + 0xf7, 0xf8, 0xb8, 0xb6, 0xe4, 0x80, 0x2c, 0xbe, 0x8d, 0x0a, 0x42, 0xf2, 0x84, 0x0a, 0x3b, 0x7b, + 0xd1, 0xba, 0x52, 0xb9, 0xfe, 0xe1, 0x22, 0x94, 0x3d, 0x75, 0x3b, 0x05, 0x63, 0xc4, 0xf1, 0x16, + 0x2a, 0x08, 0x77, 0x9f, 0x86, 0xc4, 0xb6, 0x00, 0xe8, 0x72, 0x1a, 0xe8, 0x90, 0xa9, 0xbf, 0x2e, + 0x11, 0xb4, 0x71, 0x9f, 0x74, 0x19, 0xdd, 0xa2, 0xc2, 0x4d, 0x82, 0x58, 0xf2, 0x64, 0x82, 0x02, + 0xb2, 0x78, 0x17, 0x95, 0xc5, 0x21, 0xd3, 0x71, 0xb3, 0x73, 0x00, 0x74, 0x75, 0x0e, 0xa8, 0xcd, + 0x19, 0xa3, 0xae, 0xa4, 0xde, 0x9e, 0x24, 0x92, 0x86, 0x34, 0x92, 0xea, 0x23, 0x00, 0xfb, 0x0c, + 0x5a, 0x49, 0x1c, 0x32, 0xc5, 0x14, 0xf8, 0xa7, 0x0c, 0xba, 0xc0, 0xa8, 0x4f, 0xdc, 0x61, 0xa7, + 0x1f, 0x05, 0x61, 0xcc, 0x40, 0x80, 0x7a, 0x1d, 0x9a, 0x24, 0x3c, 0x11, 0x76, 0x1e, 0x54, 0xdc, + 0x5d, 0xe4, 0xf4, 0x7c, 0xe8, 0x1b, 0x3b, 0x80, 0xf9, 0x20, 0x0d, 0xb9, 0x0d, 0x88, 0xdb, 0x91, + 0x4c, 0x86, 0xce, 0x3b, 0xec, 0x65, 0xe7, 0x38, 0x46, 0xab, 0x84, 0x49, 0x9a, 0x50, 0xaf, 0x23, + 0xa8, 0x94, 0x41, 0xe4, 0x0b, 0xbb, 0x00, 0x56, 0x6c, 0x9f, 0xda, 0x8a, 0x4d, 0x0d, 0xb4, 0x67, + 0x70, 0xb4, 0xee, 0xb3, 0x64, 0x96, 0x8b, 0x1f, 0xa1, 0xf3, 0x26, 0x04, 0xe0, 0x74, 0xc7, 0xe5, + 0xfd, 0x48, 0x0a, 0xbb, 0x08, 0x4a, 0x6f, 0xff, 0x47, 0xd7, 0xc1, 0x9b, 0x36, 0x20, 0x69, 0xb5, + 0xe7, 0xd8, 0x3c, 0x1f, 0xff, 0x80, 0x96, 0x55, 0xbb, 0x74, 0x74, 0x07, 0x09, 0xbb, 0x04, 0x1a, + 0x37, 0x4f, 0xad, 0xf1, 0x21, 0x8f, 0x68, 0x5b, 0x63, 0x80, 0x2e, 0x93, 0xe5, 0xca, 0xd1, 0x94, + 0x8f, 0x43, 0xb4, 0xd2, 0xa3, 0x44, 0xb5, 0x52, 0xa7, 0x2f, 0x88, 0x4f, 0xed, 0x32, 0x28, 0x6b, + 0x9d, 0x5a, 0xd9, 0x57, 0x1a, 0xe5, 0x81, 0x02, 0x49, 0x6b, 0x5b, 0xee, 0xa5, 0x0e, 0xf0, 0x2e, + 0xb2, 0xc4, 0x21, 0xb3, 0x11, 0x74, 0x5e, 0x73, 0x61, 0xcf, 0xdc, 0xdb, 0xb9, 0x13, 0x09, 0x49, + 0x22, 0x57, 0x77, 0x4e, 0x45, 0x21, 0x8e, 0x8e, 0x6b, 0xd6, 0xde, 0xbd, 0x1d, 0x47, 0x01, 0xe1, + 0x36, 0xb2, 0x68, 0x34, 0xb0, 0x2b, 0x80, 0x77, 0x75, 0x11, 0xde, 0x76, 0x34, 0x08, 0x12, 0x1e, + 0xa9, 0xba, 0x32, 0xd6, 0x29, 0xe9, 0x8d, 0x16, 0x5a, 0x7b, 0x51, 0x45, 0xe0, 0x55, 0x64, 0x1d, + 0xd0, 0x21, 0x8c, 0x89, 0xb2, 0xa3, 0x3e, 0xf1, 0x1a, 0xca, 0x0f, 0x08, 0xeb, 0x53, 0x3b, 0x0b, + 0x3c, 0x4d, 0x7c, 0x91, 0xfd, 0x2c, 0xb3, 0xe1, 0xa2, 0xd5, 0xf9, 0x70, 0xa7, 0xe5, 0x2d, 0x2d, + 0xff, 0x79, 0x5a, 0xbe, 0x72, 0xfd, 0x52, 0xca, 0x60, 0x9d, 0xec, 0x86, 0x1e, 0x97, 0xa9, 0xd4, + 0xa5, 0x95, 0xdc, 0x42, 0xe7, 0x9e, 0x0b, 0xf3, 0x22, 0x2b, 0xf3, 0x69, 0x80, 0x1d, 0x54, 0x3d, + 0xb9, 0x03, 0x17, 0xa1, 0x59, 0x69, 0xb4, 0x2d, 0xb4, 0xfe, 0xe2, 0xa2, 0x3e, 0x0d, 0x4a, 0xfd, + 0xcf, 0x1c, 0x2a, 0x8d, 0x47, 0x2c, 0x7e, 0x88, 0x8a, 0x6a, 0xbc, 0x76, 0x02, 0x0f, 0x84, 0xf3, + 0xad, 0xcd, 0xd1, 0x71, 0xad, 0x00, 0xc7, 0x5b, 0xff, 0x1c, 0xd7, 0x6e, 0xf8, 0x81, 0xdc, 0xef, + 0x77, 0x1b, 0x2e, 0x0f, 0x9b, 0x93, 0xd0, 0x79, 0xdd, 0xe9, 0x77, 0x33, 0x3e, 0xf0, 0x9b, 0x66, + 0x17, 0xe8, 0xc1, 0xbd, 0xe5, 0x14, 0x14, 0xe2, 0x1d, 0x4f, 0x99, 0xd0, 0x1d, 0x4a, 0x98, 0xd8, + 0x60, 0x02, 0x10, 0xf8, 0x02, 0x2a, 0x1f, 0xd0, 0xa1, 0x6e, 0x6e, 0xdb, 0x82, 0x93, 0xd2, 0x01, + 0x1d, 0x82, 0x37, 0xb8, 0x86, 0x2a, 0x09, 0x89, 0x7c, 0x6a, 0x8e, 0x73, 0x70, 0x8c, 0x80, 0xa5, + 0x2f, 0xdc, 0x44, 0x25, 0xc6, 0x5d, 0xc2, 0x02, 0x39, 0xb4, 0xf3, 0x90, 0xd3, 0x0b, 0xa9, 0x9c, + 0x8e, 0x0d, 0xd9, 0x31, 0x57, 0xc6, 0x63, 0x76, 0x2c, 0x82, 0x77, 0x51, 0x69, 0x9f, 0x24, 0xde, + 0x23, 0x92, 0x50, 0xbb, 0x00, 0xe2, 0x1f, 0x2d, 0xaa, 0xe1, 0xaf, 0xcd, 0xfd, 0xd4, 0x2a, 0x99, + 0x60, 0xe0, 0x2f, 0x51, 0x96, 0xab, 0x09, 0x95, 0x99, 0x5f, 0x24, 0x2f, 0x42, 0xba, 0xbb, 0x97, + 0xc2, 0xc8, 0x72, 0x81, 0xaf, 0xa1, 0x3c, 0xac, 0x71, 0xbb, 0x04, 0x00, 0x6f, 0xa5, 0x00, 0x80, + 0xdf, 0x48, 0xdd, 0xd7, 0x37, 0xf1, 0x3a, 0x2a, 0xf4, 0x63, 0x19, 0x84, 0x6a, 0x6e, 0xa8, 0xd8, + 0x18, 0x0a, 0xbf, 0x87, 0x96, 0x59, 0xe0, 0xd2, 0x48, 0xd0, 0x8e, 0x1c, 0xc6, 0x14, 0x1a, 0xbe, + 0xec, 0x54, 0x0c, 0xef, 0xfe, 0x30, 0x56, 0xa3, 0xa0, 0x24, 0x79, 0xcc, 0x19, 0xf7, 0x87, 0xa6, + 0x7f, 0x17, 0xfa, 0x7e, 0xdf, 0xdc, 0x4f, 0xfb, 0x3e, 0xc6, 0xa8, 0xff, 0x66, 0xa1, 0xf2, 0x64, + 0xc9, 0xfe, 0xaf, 0x85, 0xf4, 0x3d, 0x2a, 0xc1, 0xf2, 0x56, 0xe0, 0xd0, 0x62, 0xad, 0xf6, 0xe8, + 0xb8, 0x56, 0xd4, 0xca, 0x15, 0xfa, 0xc7, 0xa7, 0x42, 0x37, 0x72, 0x4e, 0x11, 0x40, 0xd3, 0x85, + 0x6a, 0xbd, 0xb4, 0x50, 0x73, 0x27, 0x17, 0x6a, 0xfe, 0xb9, 0x42, 0xdd, 0x40, 0x25, 0x97, 0xc4, + 0xc4, 0x55, 0x85, 0x5a, 0xd0, 0xc2, 0x63, 0x1a, 0xbf, 0x8b, 0xca, 0x64, 0x40, 0x02, 0xa6, 0x5e, + 0x18, 0x50, 0x3c, 0x96, 0x33, 0x65, 0x60, 0x8c, 0x72, 0x7d, 0x41, 0x75, 0x51, 0x58, 0x0e, 0x7c, + 0xe3, 0x6b, 0x68, 0x8d, 0x46, 0x6e, 0x32, 0x8c, 0x65, 0xc0, 0xa3, 0x0e, 0x61, 0x3e, 0x4f, 0x02, + 0xb9, 0x1f, 0x9a, 0x22, 0x38, 0x3f, 0x3d, 0xdb, 0x1c, 0x1f, 0xd5, 0x7f, 0xcf, 0xa0, 0xb3, 0x73, + 0xf3, 0x1c, 0x4b, 0x74, 0x56, 0xbd, 0x5a, 0x02, 0xc3, 0x9b, 0x26, 0x6b, 0x67, 0x74, 0x5c, 0x5b, + 0x49, 0xdf, 0x56, 0x51, 0xfd, 0xf4, 0x95, 0xa2, 0x0a, 0x8f, 0xa5, 0x19, 0x51, 0x67, 0x45, 0x1c, + 0xb2, 0x09, 0x99, 0xae, 0xd9, 0x6c, 0xba, 0x66, 0xeb, 0x3f, 0x5a, 0xa8, 0x92, 0xda, 0x10, 0x33, + 0xbd, 0x9d, 0x79, 0xbd, 0xde, 0xce, 0xbe, 0xb1, 0xde, 0xb6, 0x5e, 0xb7, 0xb7, 0x73, 0xaf, 0xdc, + 0xdb, 0xf3, 0x3d, 0x9c, 0x3f, 0xb9, 0x87, 0x0b, 0x6f, 0xa0, 0x87, 0x7f, 0xc9, 0xa0, 0x62, 0xfb, + 0xdb, 0x07, 0x50, 0x1c, 0xeb, 0xa8, 0x10, 0xf5, 0x43, 0x37, 0xee, 0xeb, 0x9a, 0x70, 0x0c, 0x85, + 0x6d, 0x54, 0x14, 0xdc, 0x3d, 0xa0, 0x52, 0x98, 0xfd, 0x36, 0x26, 0x55, 0xdf, 0xb8, 0xf0, 0x24, + 0xb7, 0xf4, 0xde, 0x03, 0x42, 0x71, 0x43, 0xee, 0x51, 0x06, 0x9e, 0x97, 0x1d, 0x4d, 0xa8, 0x0d, + 0x15, 0xee, 0x1f, 0x81, 0x4f, 0x59, 0x47, 0x7d, 0xaa, 0x0e, 0x31, 0x4f, 0x15, 0xfd, 0xb0, 0x2c, + 0x3b, 0x13, 0xba, 0xfe, 0x6b, 0x16, 0x2d, 0xa7, 0x93, 0x83, 0x2f, 0xa3, 0x33, 0x83, 0x20, 0x91, + 0x7d, 0xc2, 0x82, 0x23, 0xa2, 0x0a, 0xdd, 0xec, 0xba, 0x39, 0x2e, 0xbe, 0x85, 0x2c, 0xe5, 0x81, + 0xce, 0xff, 0x07, 0x8b, 0x62, 0x63, 0x5c, 0x1f, 0xbf, 0x4d, 0x94, 0xb7, 0xb7, 0x90, 0x15, 0xd2, + 0xd0, 0xa4, 0x7d, 0x21, 0xc0, 0x37, 0x34, 0x4c, 0x03, 0x84, 0x34, 0x54, 0xcd, 0xcd, 0x38, 0xf1, + 0xc8, 0xc0, 0xbf, 0xf6, 0x09, 0x84, 0x20, 0xeb, 0x4c, 0x19, 0xca, 0xe9, 0x38, 0xe1, 0x83, 0xc0, + 0xa3, 0x89, 0xc9, 0xef, 0x84, 0xc6, 0xef, 0xa3, 0x33, 0x93, 0xce, 0x74, 0x19, 0x11, 0x02, 0x52, + 0x5c, 0x76, 0x56, 0xc6, 0xdc, 0xb6, 0x62, 0xd6, 0xbf, 0x43, 0x05, 0x5d, 0x6d, 0x2a, 0x63, 0x3d, + 0x12, 0x06, 0x6c, 0xbc, 0xf8, 0x0d, 0x05, 0x4a, 0x18, 0x91, 0x3d, 0x9e, 0x84, 0xe6, 0xe1, 0x34, + 0xa1, 0x55, 0x36, 0x07, 0x34, 0x11, 0x2a, 0x82, 0x16, 0x1c, 0x8d, 0xc9, 0xfa, 0x4d, 0x54, 0x34, + 0xee, 0xa8, 0x14, 0x4a, 0x2e, 0x09, 0x03, 0xdc, 0x9c, 0xa3, 0x89, 0xd9, 0xb1, 0x95, 0x85, 0x93, + 0x29, 0xa3, 0xde, 0x42, 0xcb, 0xe9, 0x52, 0x9b, 0xf1, 0x34, 0x33, 0xe7, 0xe9, 0x3a, 0x2a, 0x24, + 0xd4, 0x57, 0x36, 0x68, 0xf3, 0x0c, 0xd5, 0x6a, 0x3e, 0xfe, 0xbb, 0xba, 0xf4, 0x78, 0x54, 0xcd, + 0x3c, 0x19, 0x55, 0x33, 0x4f, 0x47, 0xd5, 0xcc, 0x5f, 0xa3, 0x6a, 0xe6, 0xe7, 0x67, 0xd5, 0xa5, + 0x27, 0xcf, 0xaa, 0x4b, 0x4f, 0x9f, 0x55, 0x97, 0x1e, 0xae, 0xcc, 0xc4, 0xbf, 0x5b, 0x80, 0x5f, + 0x9d, 0x37, 0xfe, 0x0d, 0x00, 0x00, 0xff, 0xff, 0x44, 0x06, 0x5a, 0x4b, 0x57, 0x0f, 0x00, 0x00, } diff --git a/pkg/server/diagnosticspb/diagnostics.proto b/pkg/server/diagnosticspb/diagnostics.proto index d101bd6c3838..b6f1e7de63d4 100644 --- a/pkg/server/diagnosticspb/diagnostics.proto +++ b/pkg/server/diagnosticspb/diagnostics.proto @@ -27,23 +27,31 @@ message DiagnosticReport { map altered_settings = 6; map zone_configs = 8 [(gogoproto.nullable) = false]; map feature_usage = 9 [(gogoproto.nullable) = false]; - + SQLInstanceInfo sql = 10 [(gogoproto.nullable) = false, (gogoproto.customname) = "SQL"]; + Environment env = 11 [(gogoproto.nullable) = false]; map legacy_unimplemented_errors = 5; map legacy_error_counts = 7; - } +} +// NodeInfo describes the CRDB node which is reporting diagnostics. NodeInfo +// will be set to its empty value (e.g. NodeID = 0) when the report is generated +// by a SQL-only instance. message NodeInfo { int32 node_id = 1 [(gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; int64 bytes = 2; int64 key_count = 3; int64 range_count = 4; - roachpb.Locality locality = 5 [(gogoproto.nullable) = false]; + // Uptime is the number of seconds since the CRDB node was started. + int64 uptime = 9; + + // The following fields are deprecated and will be removed. They are replaced + // by the separated DiagnosticsReport.Env field. + roachpb.Locality locality = 5 [(gogoproto.nullable) = false]; HardwareInfo hardware = 6 [(gogoproto.nullable) = false]; OSInfo os = 7 [(gogoproto.nullable) = false]; build.Info build = 8 [(gogoproto.nullable) = false]; - int64 uptime = 9; string license_type = 10; TopologyInfo topology = 11 [(gogoproto.nullable) = false]; } @@ -64,6 +72,36 @@ message StoreInfo { // Want: sync times, observed read/write speeds } +// SQLInstanceInfo describes the SQL instance which is reporting diagnostics. +// This is always set, even for full CRDB nodes running both KV and SQL. +message SQLInstanceInfo { + // SQLInstanceID is the ephemeral identifier for the SQL instance which is + // reporting diagnostics. This is unique across all running SQL instances in + // the cluster (physical or tenant). + int32 sql_instance_id = 1 [(gogoproto.customname) = "SQLInstanceID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/base.SQLInstanceID"]; + // Uptime is the number of seconds since the SQL instance was started. + int64 uptime = 2; +} + +// Environment describes the context in which the CRDB node or SQL instance is +// running. +message Environment { + // Locality is an ordered set of key/value tiers that describe the location of + // this CRDB node or SQL instance. + roachpb.Locality locality = 1 [(gogoproto.nullable) = false]; + // Hardware describes the physical machine, virtual machine, or Linux + // container on which CRDB is running. + HardwareInfo hardware = 2 [(gogoproto.nullable) = false]; + // OS describes the operating system software that hosts CRDB. + OSInfo os = 3 [(gogoproto.nullable) = false]; + // Build describes the running CockroachDB binary. + build.Info build = 4 [(gogoproto.nullable) = false]; + // LicenseType describes the license that is in effect (OSS, Enterprise, etc). + string license_type = 5; + // Topology provides the cloud provider and region name that is hosting CRDB. + TopologyInfo topology = 6 [(gogoproto.nullable) = false]; +} + message CPUInfo { int32 numcpu = 1; // go's reported runtime.NUMCPU() int32 sockets = 2; // number of cpus reported diff --git a/pkg/server/server.go b/pkg/server/server.go index 58e8e660b12d..53028944a451 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -151,7 +151,7 @@ type Server struct { protectedtsProvider protectedts.Provider protectedtsReconciler *ptreconcile.Reconciler - sqlServer *sqlServer + sqlServer *SQLServer // Created in NewServer but initialized (made usable) in `(*Server).Start`. externalStorageBuilder *externalStorageBuilder @@ -1875,7 +1875,7 @@ func (s *Server) startServeUI( } // TODO(tbg): move into server_sql.go. -func (s *sqlServer) startServeSQL( +func (s *SQLServer) startServeSQL( ctx context.Context, stopper *stop.Stopper, connManager netutil.Server, diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 722a5ca26972..c4c1ce555112 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -71,7 +71,14 @@ import ( "google.golang.org/grpc" ) -type sqlServer struct { +// SQLServer encapsulates the part of a CRDB server that is dedicated to SQL +// processing. All SQL commands are reduced to primitive operations on the +// lower-level KV layer. Multi-tenant installations of CRDB run zero or more +// standalone SQLServer instances per tenant (the KV layer is shared across all +// tenants). +type SQLServer struct { + stopper *stop.Stopper + sqlIDContainer *base.SQLIDContainer pgServer *pgwire.Server distSQLServer *distsql.ServerImpl execCfg *sql.ExecutorConfig @@ -196,7 +203,7 @@ type sqlServerArgs struct { sqlStatusServer serverpb.SQLStatusServer } -func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) { +func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { // NB: ValidateAddrs also fills in defaults. if err := cfg.Config.ValidateAddrs(ctx); err != nil { return nil, err @@ -629,7 +636,9 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) { sqlExecutorTestingKnobs, ) - return &sqlServer{ + return &SQLServer{ + stopper: cfg.stopper, + sqlIDContainer: cfg.nodeIDContainer, pgServer: pgServer, distSQLServer: distSQLServer, execCfg: execCfg, @@ -649,7 +658,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) { }, nil } -func (s *sqlServer) start( +func (s *SQLServer) start( ctx context.Context, stopper *stop.Stopper, knobs base.TestingKnobs, @@ -790,3 +799,10 @@ func (s *sqlServer) start( return nil } + +// SQLInstanceID returns the ephemeral ID assigned to each SQL instance. The ID +// is guaranteed to be unique across all currently running instances, but may be +// reused once an instance is stopped. +func (s *SQLServer) SQLInstanceID() base.SQLInstanceID { + return s.sqlIDContainer.SQLInstanceID() +} diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 55e5efce0b92..ebedc2cc7043 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -570,7 +570,7 @@ func makeSQLServerArgs( // StartTenant starts a SQL tenant communicating with this TestServer. func (ts *TestServer) StartTenant( params base.TestTenantArgs, -) (pgAddr string, httpAddr string, _ error) { +) (pgAddr string, httpAddr string, err error) { ctx := context.Background() if !params.Existing { @@ -595,13 +595,14 @@ func (ts *TestServer) StartTenant( if stopper == nil { stopper = ts.Stopper() } - return StartTenant( + _, pgAddr, httpAddr, err = StartTenant( ctx, stopper, ts.Cfg.ClusterName, baseCfg, sqlCfg, ) + return pgAddr, httpAddr, err } // StartTenant starts a stand-alone SQL server against a KV backend. @@ -611,14 +612,14 @@ func StartTenant( kvClusterName string, // NB: gone after https://github.com/cockroachdb/cockroach/issues/42519 baseCfg BaseConfig, sqlCfg SQLConfig, -) (pgAddr string, httpAddr string, _ error) { +) (sqlServer *SQLServer, pgAddr string, httpAddr string, _ error) { args, err := makeSQLServerArgs(stopper, kvClusterName, baseCfg, sqlCfg) if err != nil { - return "", "", err + return nil, "", "", err } s, err := newSQLServer(ctx, args) if err != nil { - return "", "", err + return nil, "", "", err } // TODO(asubiotto): remove this. Right now it is needed to initialize the @@ -635,7 +636,7 @@ func StartTenant( pgL, err := listen(ctx, &args.Config.SQLAddr, &args.Config.SQLAdvertiseAddr, "sql") if err != nil { - return "", "", err + return nil, "", "", err } args.stopper.RunWorker(ctx, func(ctx context.Context) { @@ -649,7 +650,7 @@ func StartTenant( httpL, err := listen(ctx, &args.Config.HTTPAddr, &args.Config.HTTPAdvertiseAddr, "http") if err != nil { - return "", "", err + return nil, "", "", err } args.stopper.RunWorker(ctx, func(ctx context.Context) { @@ -699,7 +700,7 @@ func StartTenant( heapProfileDirName: args.HeapProfileDirName, runtime: args.runtime, }); err != nil { - return "", "", err + return nil, "", "", err } if err := s.start(ctx, @@ -710,10 +711,10 @@ func StartTenant( socketFile, orphanedLeasesTimeThresholdNanos, ); err != nil { - return "", "", err + return nil, "", "", err } - return pgLAddr, httpLAddr, nil + return s, pgLAddr, httpLAddr, nil } // ExpectedInitialRangeCount returns the expected number of ranges that should diff --git a/pkg/server/updates.go b/pkg/server/updates.go index a27c60ac3027..d0e488babac2 100644 --- a/pkg/server/updates.go +++ b/pkg/server/updates.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/server/diagnostics" "github.com/cockroachdb/cockroach/pkg/server/diagnosticspb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" @@ -59,12 +60,6 @@ const ( updateCheckJitterSeconds = 120 ) -var diagnosticReportFrequency = settings.RegisterPublicNonNegativeDurationSetting( - "diagnostics.reporting.interval", - "interval at which diagnostics data should be reported", - time.Hour, -) - // randomly shift `d` to be up to `jitterSec` shorter or longer. func addJitter(d time.Duration, jitterSec int) time.Duration { j := time.Duration(rand.Intn(jitterSec*2)-jitterSec) * time.Second @@ -193,6 +188,7 @@ func (s *Server) checkForUpdates(ctx context.Context) bool { clusterInfo := diagnosticspb.ClusterInfo{ ClusterID: s.ClusterID(), + TenantID: s.rpcContext.TenantID, IsInsecure: s.cfg.Insecure, IsInternal: sql.ClusterIsInternal(&s.st.SV), } @@ -255,7 +251,7 @@ func (s *Server) maybeReportDiagnostics(ctx context.Context, now, scheduled time s.ReportDiagnostics(ctx) } - return scheduled.Add(diagnosticReportFrequency.Get(&s.st.SV)) + return scheduled.Add(diagnostics.ReportFrequency.Get(&s.st.SV)) } func (s *Server) collectNodeInfo(ctx context.Context) diagnosticspb.NodeInfo { @@ -423,6 +419,7 @@ func (s *Server) ReportDiagnostics(ctx context.Context) { clusterInfo := diagnosticspb.ClusterInfo{ ClusterID: s.ClusterID(), + TenantID: s.rpcContext.TenantID, IsInsecure: s.cfg.Insecure, IsInternal: sql.ClusterIsInternal(&s.st.SV), } @@ -430,7 +427,7 @@ func (s *Server) ReportDiagnostics(ctx context.Context) { if s.cfg.TestingKnobs.Server != nil { knobs = &s.cfg.TestingKnobs.Server.(*TestingKnobs).DiagnosticsTestingKnobs } - reportingURL := diagnosticspb.BuildReportingURL(&clusterInfo, &report.Node, knobs) + reportingURL := diagnosticspb.BuildReportingURL(&clusterInfo, report, knobs) if reportingURL == nil { return }