Skip to content

Commit

Permalink
Merge #63767 #64071
Browse files Browse the repository at this point in the history
63767: bazel: tweak logic for staging test artifacts in bazci r=rail a=rickystewart

Because Bazel aggressively caches build/test artifacts, if we're not
careful, bazci can copy OLD artifacts from previous build/test runs into
the artifacts directory. This is particularly an issue because TeamCity
watches that directory for `test.xml` reports, and if an old `test.xml`
reports a test failure, TeamCity will notice that and report the failure
in the UI -- and importantly, even if we replace that `test.xml` with a
completely different one that reports that the test succeeed, TC will
not amend what's displayed in the UI accordingly. So this can manifest
as reported test failures from unrelated PR's showing up in TC in an
apparently unpredictable (though uncommmon) manner.

We fix this by making bazci a little smarter about when we choose to
stage artifacts:

1. The first time the watcher loops over all the test artifacts, never
   stage anything (the artifacts are probably cached -- not enough time
   has passed for any legitimate artifacts to appear, probably).
2. Only stage artifacts incrementally if their stats have changed since
   the initial round of caching.
3. During the final loop, stage ALL artifacts (if they haven't been
   staged yet), just to make sure we don't miss anything.

Also add lots of comments to make these design decisions and their
motivations a little clearer.

Resolves #63740.

Release note: None

64071: cli/debug: new sub-command `list-files` r=stevendanna a=knz

Prerequisite to #64094.

This is meant to assist the human operator select appropriate
flags for the `debug zip` command.

Release note (cli change): The new command `debug list-files`
show the list of files that can be retrieved via the `debug zip`
command. It supports the `--nodes` and `--exclude-nodes` parameters
in the same way as `debug zip`.

Co-authored-by: Ricky Stewart <ricky@cockroachlabs.com>
Co-authored-by: Raphael 'kena' Poss <knz@thaumogen.net>
  • Loading branch information
3 people committed Apr 23, 2021
3 parents 9e821aa + 6c64f0e + 84e9db4 commit fe96523
Show file tree
Hide file tree
Showing 6 changed files with 424 additions and 48 deletions.
2 changes: 2 additions & 0 deletions pkg/cli/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ go_library(
"cpuprofile.go",
"debug.go",
"debug_check_store.go",
"debug_list_files.go",
"debug_logconfig.go",
"debug_merge_logs.go",
"debug_reset_quorum.go",
Expand Down Expand Up @@ -264,6 +265,7 @@ go_test(
"cli_debug_test.go",
"cli_test.go",
"debug_check_store_test.go",
"debug_list_files_test.go",
"debug_merge_logs_test.go",
"debug_test.go",
"decode_test.go",
Expand Down
1 change: 1 addition & 0 deletions pkg/cli/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -1230,6 +1230,7 @@ var debugCmds = append(DebugCmdsForRocksDB,
debugEnvCmd,
debugZipCmd,
debugMergeLogsCommand,
debugListFilesCmd,
debugResetQuorumCmd,
)

Expand Down
180 changes: 180 additions & 0 deletions pkg/cli/debug_list_files.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,180 @@
// Copyright 2021 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 cli

import (
"context"
"fmt"
"os"
"regexp"
"sort"
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/server/status/statuspb"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/logpb"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/spf13/cobra"
)

var debugListFilesCmd = &cobra.Command{
Use: "list-files",
Short: "list files available for retrieval via 'debug zip'",
RunE: MaybeDecorateGRPCError(runDebugListFiles),
}

func runDebugListFiles(cmd *cobra.Command, _ []string) error {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

// Connect to the node pointed to in the command line.
conn, _, finish, err := getClientGRPCConn(ctx, serverCfg)
if err != nil {
return err
}
defer finish()

status := serverpb.NewStatusClient(conn)

// Retrieve the details for the head node.
firstNodeDetails, err := status.Details(ctx, &serverpb.DetailsRequest{NodeId: "local"})
if err != nil {
return err
}

// Retrieve the list of all nodes.
nodes, err := status.Nodes(ctx, &serverpb.NodesRequest{})
if err != nil {
log.Warningf(ctx, "cannot retrieve node list: %v", err)
}
// In case nodes came up back empty (the Nodes() RPC failed), we
// still want to inspect the per-node endpoints on the head
// node. As per the above, we were able to connect at least to
// that.
inputNodeList := []statuspb.NodeStatus{{Desc: roachpb.NodeDescriptor{
NodeID: firstNodeDetails.NodeID,
Address: firstNodeDetails.Address,
SQLAddress: firstNodeDetails.SQLAddress,
}}}
if nodes != nil {
// If the nodes were found, use that instead.
inputNodeList = nodes.Nodes
}

// Filter the list of nodes according to configuration.
var nodeList []roachpb.NodeID
for _, n := range inputNodeList {
if zipCtx.nodes.isIncluded(n.Desc.NodeID) {
nodeList = append(nodeList, n.Desc.NodeID)
}
}

// Determine the list of non-log file types.
fileTypes := make([]int, 0, len(serverpb.FileType_value))
for _, v := range serverpb.FileType_value {
fileTypes = append(fileTypes, int(v))
}
sort.Ints(fileTypes)

// The log files for each node.
logFiles := make(map[roachpb.NodeID][]logpb.FileInfo)
// The non-log files on each node. The int32 is the file type.
otherFiles := make(map[roachpb.NodeID]map[int32][]*serverpb.File)

// Retrieve the files.
for _, nodeID := range nodeList {
nodeIDs := fmt.Sprintf("%d", nodeID)
nodeLogs, err := status.LogFilesList(ctx, &serverpb.LogFilesListRequest{NodeId: nodeIDs})
if err != nil {
log.Warningf(ctx, "cannot retrieve log file list from node %d: %v", nodeID, err)
} else {
logFiles[nodeID] = nodeLogs.Files
}

otherFiles[nodeID] = make(map[int32][]*serverpb.File)
for _, fileTypeI := range fileTypes {
fileType := int32(fileTypeI)
nodeFiles, err := status.GetFiles(ctx, &serverpb.GetFilesRequest{
NodeId: nodeIDs,
ListOnly: true,
Type: serverpb.FileType(fileType),
Patterns: []string{"*"},
})
if err != nil {
log.Warningf(ctx, "cannot retrieve %s file list from node %d: %v", serverpb.FileType_name[fileType], nodeID, err)
} else {
otherFiles[nodeID][fileType] = nodeFiles.Files
}
}
}

// Format the entries and compute the total size.
fileTypeNames := map[int32]string{}
for t, n := range serverpb.FileType_name {
fileTypeNames[t] = strings.ToLower(n)
}
var totalSize int64
fileTableHeaders := []string{"node_id", "type", "file_name", "ctime_utc", "mtime_utc", "size"}
alignment := "lllr"
var rows [][]string
for _, nodeID := range nodeList {
nodeIDs := fmt.Sprintf("%d", nodeID)
for _, logFile := range logFiles[nodeID] {
totalSize += logFile.SizeBytes
ctime := formatTimeSimple(extractTimeFromFileName(logFile.Name))
mtime := formatTimeSimple(timeutil.Unix(0, logFile.ModTimeNanos))
rows = append(rows, []string{nodeIDs, "log", logFile.Name, ctime, mtime, fmt.Sprintf("%d", logFile.SizeBytes)})
}
for _, ft := range fileTypes {
fileType := int32(ft)
for _, other := range otherFiles[nodeID][fileType] {
totalSize += other.FileSize
ctime := formatTimeSimple(extractTimeFromFileName(other.Name))
mtime := ctime
rows = append(rows, []string{nodeIDs, fileTypeNames[fileType], other.Name, ctime, mtime, fmt.Sprintf("%d", other.FileSize)})
}
}
}
// Append the total size.
rows = append(rows, []string{"", "total", fmt.Sprintf("(%s)", humanizeutil.IBytes(totalSize)), "", "", fmt.Sprintf("%d", totalSize)})

// Display the file listing.
return PrintQueryOutput(os.Stdout, fileTableHeaders, NewRowSliceIter(rows, alignment))
}

var tzRe = regexp.MustCompile(`\d\d\d\d-\d\d-\d\dT\d\d_\d\d_\d\d`)

// formatTimeSimple formats a timestamp for use in file lists.
// It simplifies the display to just a date and time.
func formatTimeSimple(t time.Time) string {
return t.Format("2006-01-02 15:04")
}

// extractTimeFromFileName extracts a timestamp from the name of one of the
// artifacts produced server-side. We use the knowledge that the server
// always embeds the creation timestamp with format YYYY-MM-DDTHH_MM_SS.
func extractTimeFromFileName(f string) time.Time {
ts := tzRe.FindString(f)
if ts == "" {
// No match.
return time.Time{}
}
tm, err := time.ParseInLocation("2006-01-02T15_04_05", ts, time.UTC)
if err != nil {
// No match.
return time.Time{}
}
return tm
}
53 changes: 53 additions & 0 deletions pkg/cli/debug_list_files_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
// Copyright 2021 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 cli

import (
"testing"

"github.com/cockroachdb/cockroach/pkg/util/leaktest"
)

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

testCases := []struct {
filename string
expected string
}{
// v1.0 log file names.
{"cockroach.kenax.kena.2017-08-16T13_31_00+02_00.028222.log", "2017-08-16 13:31"},
// v1.1 and later log files.
{"cockroach.kenax.kena.2021-04-23T09_18_23Z.026682.log", "2021-04-23 09:18"},
// v19.1 goroutine dumps.
{"goroutine_dump.double_since_last_dump.2021-04-23T09_18_23.1231", "2021-04-23 09:18"},
// v20.x and later goroutine dumps.
{"goroutine_dump.2021-03-11T08_13_57.498.double_since_last_dump.000001137.txt", "2021-03-11 08:13"},
// v20.1 and later memstats.
{"memstats.2021-04-22T18_31_54.413.371441664.txt", "2021-04-22 18:31"},
// v1.0 heap profile names.
{"memprof.2021-04-22T18_31_54.413", "2021-04-22 18:31"},
// Transient profile names at some variant of v19.x or v20.x.
{"memprof.fraction_system_memory.000000019331059712_2020-03-04T16_58_39.54.pprof", "2020-03-04 16:58"},
{"memprof.000000000030536024_2020-06-15T13_19_19.543", "2020-06-15 13:19"},
// v20.1 transition format.
{"memprof.2020-06-15T13_19_19.54.123456", "2020-06-15 13:19"},
// v20.2 format and later.
{"memprof.2020-06-15T13_19_19.123.123132.pprof", "2020-06-15 13:19"},
}

for _, tc := range testCases {
tm := extractTimeFromFileName(tc.filename)
if s := formatTimeSimple(tm); s != tc.expected {
t.Errorf("%s: expected %q, got %q", tc.filename, tc.expected, s)
}
}
}
17 changes: 14 additions & 3 deletions pkg/cli/flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -555,6 +555,7 @@ func init() {
debugGossipValuesCmd,
debugTimeSeriesDumpCmd,
debugZipCmd,
debugListFilesCmd,
doctorExamineClusterCmd,
doctorExamineFallbackClusterCmd,
doctorRecreateClusterCmd,
Expand Down Expand Up @@ -621,12 +622,16 @@ func init() {
// Zip command.
{
f := debugZipCmd.Flags()
varFlag(f, &zipCtx.nodes.inclusive, cliflags.ZipNodes)
varFlag(f, &zipCtx.nodes.exclusive, cliflags.ZipExcludeNodes)
boolFlag(f, &zipCtx.redactLogs, cliflags.ZipRedactLogs)
durationFlag(f, &zipCtx.cpuProfDuration, cliflags.ZipCPUProfileDuration)
intFlag(f, &zipCtx.concurrency, cliflags.ZipConcurrency)
}
// List-nodes + Zip commands.
for _, cmd := range []*cobra.Command{debugZipCmd, debugListFilesCmd} {
f := cmd.Flags()
varFlag(f, &zipCtx.nodes.inclusive, cliflags.ZipNodes)
varFlag(f, &zipCtx.nodes.exclusive, cliflags.ZipExcludeNodes)
}

// Decommission command.
varFlag(decommissionNodeCmd.Flags(), &nodeCtx.nodeDecommissionWait, cliflags.Wait)
Expand Down Expand Up @@ -716,7 +721,13 @@ func init() {

// Commands that print tables.
tableOutputCommands := append(
[]*cobra.Command{sqlShellCmd, genSettingsListCmd, demoCmd, debugTimeSeriesDumpCmd},
[]*cobra.Command{
sqlShellCmd,
genSettingsListCmd,
demoCmd,
debugListFilesCmd,
debugTimeSeriesDumpCmd,
},
demoCmd.Commands()...)
tableOutputCommands = append(tableOutputCommands, nodeCmds...)
tableOutputCommands = append(tableOutputCommands, authCmds...)
Expand Down
Loading

0 comments on commit fe96523

Please sign in to comment.