Skip to content

Commit

Permalink
Merge #138369
Browse files Browse the repository at this point in the history
138369: pkg/cli: add some node specific table dumps to zip upload r=aa-joshi a=arjunmahishi

This commits adds all the plain text, node specific table dumps to the zip upload process.

Part of: CC-28886
Epic: CC-28996
Release note: None

Co-authored-by: Arjun Mahishi <[email protected]>
  • Loading branch information
craig[bot] and arjunmahishi committed Jan 9, 2025
2 parents bfa7cf0 + 1eb653a commit 58e75b8
Show file tree
Hide file tree
Showing 7 changed files with 156 additions and 18 deletions.
1 change: 1 addition & 0 deletions pkg/cli/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -134,6 +134,7 @@ go_library(
"//pkg/kv/kvserver/loqrecovery/loqrecoverypb",
"//pkg/kv/kvserver/rditer",
"//pkg/kv/kvserver/stateloader",
"//pkg/multitenant/mtinfopb",
"//pkg/raft/raftpb",
"//pkg/roachpb",
"//pkg/rpc",
Expand Down
25 changes: 25 additions & 0 deletions pkg/cli/testdata/table_dump_column_parsing
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
# format:
#
# <table dump name>
# <column>\t<encoded value>
# ...
# ----
# <decoded value>
# ...
# claim_session_id NULL
# claim_session_id \x010180b3c8d31b42174175aeca685a7b0ac5e5

crdb_internal.system_jobs.txt
progress \x10bdb9a5929b9f8a03aa0200
progress \x10f5d5f4949b9f8a03b2010c0a0a08c098af8786ec838818
progress \x10effdc3929b9f8a030d0000803fa8019984a0c1e09cd9e3486a0d1204000000002a010032003a00
----
{"Progress":null,"modified_micros":1733902763383997,"Details":{"table_metadata_cache":{}},"trace_id":0}
{"Progress":null,"modified_micros":1733902768876277,"Details":{"AutoSpanConfigReconciliation":{"checkpoint":{"wall_time":1733902763154984000}}},"trace_id":0}
{"Progress":{"fraction_completed":1},"modified_micros":1733902763884271,"Details":{"import":{"read_progress":[0],"span_progress":null,"resume_pos":[0],"sequence_details":[{}],"summary":{}}},"trace_id":5244271230238327321}


system.tenants.txt
info \x080110001a0020002a004200
----
{"deprecated_id":1,"deprecated_data_state":0,"dropped_name":"","physical_replication_consumer_job_id":0,"capabilities":{},"last_revert_tenant_timestamp":{}}
Original file line number Diff line number Diff line change
@@ -0,0 +1,7 @@
store_id name value
NULL gossip.connections.incoming 0
NULL jobs.row_level_ttl.select_duration-p99.9 0
NULL sql.restart_savepoint.rollback.count.internal 0
1 leases.transfers.success 0
1 storage.iterator.category-pebble-ingest.block-load.latency-sum 0
1 range.snapshots.upreplication.sent-bytes 0
6 changes: 6 additions & 0 deletions pkg/cli/testdata/table_dumps/nodes/2/crdb_internal.leases.txt
Original file line number Diff line number Diff line change
@@ -0,0 +1,6 @@
node_id table_id name parent_id expiration deleted
1 3 descriptor 1 2024-12-11 07:44:44.824583 f
1 53 external_connections 1 2024-12-11 07:44:49.537097 f
1 36 statement_diagnostics 1 2024-12-11 07:44:24.205086 f
1 6 settings 1 2024-12-11 07:44:31.289861 f
1 108 rides 104 2024-12-11 07:44:27.222137 f
15 changes: 15 additions & 0 deletions pkg/cli/testdata/upload/tables
Original file line number Diff line number Diff line change
Expand Up @@ -9,14 +9,29 @@ Logs API Hook: claim_instance_id=NULL claim_session_id=NULL created=2024-12-11 0
Logs API Hook: claim_instance_id=NULL claim_session_id=NULL created=2024-12-11 07:39:23.366318 created_by_id=3 created_by_type=node ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:crdb_internal.system_jobs id=101 last_run=NULL num_runs=NULL payload=redacted progress=map[Details:map[pollJobsStats:map[]] Progress:<nil> modified_micros:1.733902763366318e+15 trace_id:0] status=running
Logs API Hook: claim_instance_id=NULL claim_session_id=NULL created=2024-12-11 07:39:23.374069 created_by_id=NULL created_by_type=NULL ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:crdb_internal.system_jobs id=103 last_run=NULL num_runs=NULL payload=redacted progress=map[Details:map[update_sql_activity:map[]] Progress:<nil> modified_micros:1.733902763374069e+15 trace_id:0] status=running
Logs API Hook: claim_instance_id=NULL claim_session_id=NULL created=2024-12-11 07:39:23.378479 created_by_id=NULL created_by_type=NULL ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:crdb_internal.system_jobs id=104 last_run=NULL num_runs=NULL payload=redacted progress=map[Details:map[mvcc_statistics_progress:map[]] Progress:<nil> modified_micros:1.733902763378479e+15 trace_id:0] status=running
Logs API Hook: ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:nodes/1/crdb_internal.node_metrics,node_id:1 name=gossip.connections.incoming store_id=NULL value=0
Logs API Hook: ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:nodes/1/crdb_internal.node_metrics,node_id:1 name=jobs.row_level_ttl.select_duration-p99.9 store_id=NULL value=0
Logs API Hook: ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:nodes/1/crdb_internal.node_metrics,node_id:1 name=leases.transfers.success store_id=1 value=0
Logs API Hook: ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:nodes/1/crdb_internal.node_metrics,node_id:1 name=range.snapshots.upreplication.sent-bytes store_id=1 value=0
Logs API Hook: ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:nodes/1/crdb_internal.node_metrics,node_id:1 name=sql.restart_savepoint.rollback.count.internal store_id=NULL value=0
Logs API Hook: ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:nodes/1/crdb_internal.node_metrics,node_id:1 name=storage.iterator.category-pebble-ingest.block-load.latency-sum store_id=1 value=0
Logs API Hook: ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:nodes/2/crdb_internal.leases,node_id:2 deleted=f expiration=2024-12-11 07:44:24.205086 name=statement_diagnostics node_id=1 parent_id=1 table_id=36
Logs API Hook: ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:nodes/2/crdb_internal.leases,node_id:2 deleted=f expiration=2024-12-11 07:44:27.222137 name=rides node_id=1 parent_id=104 table_id=108
Logs API Hook: ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:nodes/2/crdb_internal.leases,node_id:2 deleted=f expiration=2024-12-11 07:44:31.289861 name=settings node_id=1 parent_id=1 table_id=6
Logs API Hook: ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:nodes/2/crdb_internal.leases,node_id:2 deleted=f expiration=2024-12-11 07:44:44.824583 name=descriptor node_id=1 parent_id=1 table_id=3
Logs API Hook: ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:nodes/2/crdb_internal.leases,node_id:2 deleted=f expiration=2024-12-11 07:44:49.537097 name=external_connections node_id=1 parent_id=1 table_id=53
Logs API Hook: ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:system.namespace id=1 name=system parentID=0 parentSchemaID=0
Logs API Hook: ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:system.namespace id=100 name=defaultdb parentID=0 parentSchemaID=0
Logs API Hook: ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:system.namespace id=102 name=postgres parentID=0 parentSchemaID=0
Logs API Hook: ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:system.namespace id=104 name=movr parentID=0 parentSchemaID=0
Logs API Hook: ddtags=env:debug,source:debug-zip,upload_id:abc-20241114000000,cluster:ABC,table:system.namespace id=29 name=public parentID=1 parentSchemaID=0
Logs API Hook: https://http-intake.logs.us5.datadoghq.com/api/v2/logs
Logs API Hook: https://http-intake.logs.us5.datadoghq.com/api/v2/logs
Logs API Hook: https://http-intake.logs.us5.datadoghq.com/api/v2/logs
Logs API Hook: https://http-intake.logs.us5.datadoghq.com/api/v2/logs
Upload ID: abc-20241114000000
debug zip upload debugDir --dd-api-key=dd-api-key --dd-app-key=dd-app-key --cluster=ABC --include=tables
uploaded crdb_internal.system_jobs.txt
uploaded nodes/1/crdb_internal.node_metrics.txt
uploaded nodes/2/crdb_internal.leases.txt
uploaded system.namespace.txt
87 changes: 74 additions & 13 deletions pkg/cli/zip_upload.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"cloud.google.com/go/storage"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb"
"github.com/cockroachdb/cockroach/pkg/util/httputil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/logpb"
Expand Down Expand Up @@ -70,8 +71,9 @@ const (
datadogAppKeyHeader = "DD-APPLICATION-KEY"

// the path pattern to search for specific artifacts in the debug zip directory
zippedProfilePattern = "nodes/*/*.pprof"
zippedLogsPattern = "nodes/*/logs/*"
zippedProfilePattern = "nodes/*/*.pprof"
zippedLogsPattern = "nodes/*/logs/*"
zippedNodeTableDumpsPattern = "nodes/*/*.txt"

// this is not the pprof version, but the version of the profile
// upload format supported by datadog
Expand Down Expand Up @@ -663,22 +665,70 @@ var clusterWideTableDumps = map[string]columnParserMap{
"crdb_internal.cluster_contention_events.txt": {},
"crdb_internal.cluster_queries.txt": {},
"crdb_internal.jobs.txt": {},
"crdb_internal.regions.txt": {},
"system.table_statistics.txt": {},

// table dumps with columns that need to be interpreted as protos
"crdb_internal.system_jobs.txt": {
"progress": makeProtoColumnParser[*jobspb.Progress](),
},
"system.tenants.txt": {
"info": makeProtoColumnParser[*mtinfopb.ProtoInfo](),
},
}

var nodeSpecificTableDumps = map[string]columnParserMap{
"crdb_internal.node_metrics.txt": {},
"crdb_internal.node_txn_stats.txt": {},
"crdb_internal.node_contention_events.txt": {},
"crdb_internal.gossip_liveness.txt": {},
"crdb_internal.gossip_nodes.txt": {},
"crdb_internal.node_runtime_info.txt": {},
"crdb_internal.node_transaction_statistics.txt": {},
"crdb_internal.node_tenant_capabilities_cache.txt": {},
"crdb_internal.node_sessions.txt": {},
"crdb_internal.node_statement_statistics.txt": {},
"crdb_internal.leases.txt": {},
"crdb_internal.node_build_info.txt": {},
"crdb_internal.node_memory_monitors.txt": {},
"crdb_internal.active_range_feeds.txt": {},
"crdb_internal.gossip_alerts.txt": {},
"crdb_internal.node_transactions.txt": {},
"crdb_internal.feature_usage.txt": {},
"crdb_internal.node_queries.txt": {},
}

func getNodeSpecificTableDumps(debugDirPath string) ([]string, error) {
allTxtFiles, err := expandPatterns([]string{path.Join(debugDirPath, zippedNodeTableDumpsPattern)})
if err != nil {
return nil, err
}

filteredTxtFiles := []string{}
for _, txtFile := range allTxtFiles {
if _, ok := nodeSpecificTableDumps[filepath.Base(txtFile)]; ok {
filteredTxtFiles = append(filteredTxtFiles, strings.TrimPrefix(txtFile, debugDirPath+"/"))
}
}

return filteredTxtFiles, nil
}

// uploadZipTables uploads the table dumps to datadog. The concurrency model
// here is much simpler than the logs upload. We just fan-out work to a limited
// set of workers and fan-in the errors if any. The workers read the file,
// parse the columns and uploads the data to datadog.
func uploadZipTables(ctx context.Context, uploadID string, debugDirPath string) error {
nodeTableDumps, err := getNodeSpecificTableDumps(debugDirPath)
if err != nil {
return err
}

var (
noOfWorkers = min(debugZipUploadOpts.maxConcurrentUploads, len(clusterWideTableDumps))
workChan = make(chan string, len(clusterWideTableDumps))
errChan = make(chan error, len(clusterWideTableDumps))
totalJobs = len(clusterWideTableDumps) + len(nodeTableDumps)
noOfWorkers = min(debugZipUploadOpts.maxConcurrentUploads, totalJobs)
workChan = make(chan string, totalJobs)
errChan = make(chan error, totalJobs)

errTables []string
)
Expand All @@ -702,7 +752,11 @@ func uploadZipTables(ctx context.Context, uploadID string, debugDirPath string)
workChan <- fileName
}

for range clusterWideTableDumps {
for _, fileName := range nodeTableDumps {
workChan <- fileName
}

for i := 0; i < totalJobs; i++ {
if err := <-errChan; err != nil {
errTables = append(errTables, err.Error())
}
Expand Down Expand Up @@ -730,9 +784,19 @@ func processTableDump(
}
defer f.Close()

defaultTags := []string{"env:debug", "source:debug-zip"}
tableName := strings.TrimSuffix(fileName, filepath.Ext(fileName))
lines := [][]byte{}
var (
lines = [][]byte{}
tableName = strings.TrimSuffix(fileName, filepath.Ext(fileName))
tags = append(
[]string{"env:debug", "source:debug-zip"}, makeDDTag(uploadIDTag, uploadID),
makeDDTag(clusterTag, debugZipUploadOpts.clusterName), makeDDTag(tableTag, tableName),
)
)

if strings.HasPrefix(fileName, "nodes/") {
tags = append(tags, makeDDTag(nodeIDTag, strings.Split(fileName, "/")[1]))
}

header, iter := makeTableIterator(f)
if err := iter(func(row string) error {
cols := strings.Split(row, "\t")
Expand All @@ -741,10 +805,7 @@ func processTableDump(
}

headerColumnMapping := map[string]any{
ddTagsTag: strings.Join(append(
defaultTags, makeDDTag(uploadIDTag, uploadID), makeDDTag(clusterTag, debugZipUploadOpts.clusterName),
makeDDTag(tableTag, tableName),
), ","),
ddTagsTag: strings.Join(tags, ","),
}
for i, h := range header {
if parser, ok := parsers[h]; ok {
Expand Down
33 changes: 28 additions & 5 deletions pkg/cli/zip_upload_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import (
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -135,10 +134,8 @@ func TestUploadZipEndToEnd(t *testing.T) {
// those two in this list to avoid unnecessary errors
origTableDumps := clusterWideTableDumps
clusterWideTableDumps = map[string]columnParserMap{
"system.namespace.txt": {},
"crdb_internal.system_jobs.txt": {
"progress": makeProtoColumnParser[*jobspb.Progress](),
},
"system.namespace.txt": {},
"crdb_internal.system_jobs.txt": origTableDumps["crdb_internal.system_jobs.txt"],
}
defer func() {
clusterWideTableDumps = origTableDumps
Expand Down Expand Up @@ -570,6 +567,32 @@ func TestLogUploadSigSplit(t *testing.T) {
}
}

func TestTableDumpColumnParsing(t *testing.T) {
defer leaktest.AfterTest(t)()

datadriven.RunTest(t, "testdata/table_dump_column_parsing", func(t *testing.T, d *datadriven.TestData) string {
table, ok := clusterWideTableDumps[d.Cmd]
require.True(t, ok, "table dump not found: %s", d.Cmd)

var buf bytes.Buffer
for _, line := range strings.Split(strings.TrimSpace(d.Input), "\n") {
cols := strings.Fields(strings.TrimSpace(line))
fn, ok := table[cols[0]]
require.True(t, ok, "column not found: %s", cols[0])

decoded, err := fn(strings.TrimSpace(cols[1]))
require.NoError(t, err)

raw, err := json.Marshal(decoded)
require.NoError(t, err)

buf.Write(append(raw, '\n'))
}

return buf.String()
})
}

func copyZipFiles(t *testing.T, src, dest string) {
t.Helper()

Expand Down

0 comments on commit 58e75b8

Please sign in to comment.