Skip to content

Commit

Permalink
Merge pull request #19843 from r-vasquez/bundle-logs-yesterday
Browse files Browse the repository at this point in the history
rpk: more debug bundle improvements
  • Loading branch information
twmb authored Jun 18, 2024
2 parents 627697c + 1b88856 commit 66229b9
Show file tree
Hide file tree
Showing 5 changed files with 39 additions and 26 deletions.
16 changes: 9 additions & 7 deletions src/go/rpk/pkg/cli/debug/bundle/bundle.go
Original file line number Diff line number Diff line change
Expand Up @@ -84,8 +84,6 @@ func NewCommand(fs afero.Fs, p *config.Params) *cobra.Command {
if cpuProfilerWait < 15*time.Second {
out.Die("--cpu-profiler-wait must be higher than 15 seconds")
}
path, err := determineFilepath(fs, outFile, cmd.Flags().Changed(outputFlag))
out.MaybeDie(err, "unable to determine filepath %q: %v", outFile, err)

cfg, err := p.Load(fs)
out.MaybeDie(err, "rpk unable to load config: %v", err)
Expand All @@ -101,6 +99,9 @@ func NewCommand(fs afero.Fs, p *config.Params) *cobra.Command {
yActual = y
}

path, err := determineFilepath(fs, yActual, outFile, cmd.Flags().Changed(outputFlag))
out.MaybeDie(err, "unable to determine filepath %q: %v", outFile, err)

partitions, err := parsePartitionFlag(partitionFlag)
out.MaybeDie(err, "unable to parse partition flag %v: %v", partitionFlag, err)

Expand Down Expand Up @@ -160,8 +161,8 @@ func NewCommand(fs afero.Fs, p *config.Params) *cobra.Command {
f.StringVarP(&outFile, outputFlag, "o", "", "The file path where the debug file will be written (default ./<timestamp>-bundle.zip)")
f.DurationVar(&timeout, "timeout", 31*time.Second, "How long to wait for child commands to execute (e.g. 30s, 1.5m)")
f.DurationVar(&metricsInterval, "metrics-interval", 10*time.Second, "Interval between metrics snapshots (e.g. 30s, 1.5m)")
f.StringVar(&logsSince, "logs-since", "", "Include log entries on or newer than the specified date (journalctl date format, e.g. YYYY-MM-DD")
f.StringVar(&logsUntil, "logs-until", "", "Include log entries on or older than the specified date (journalctl date format, e.g. YYYY-MM-DD")
f.StringVar(&logsSince, "logs-since", "yesterday", "Include logs dated from specified date onward; (journalctl date format: YYYY-MM-DD, 'yesterday', or 'today'). Refer to journalctl documentation for more options")
f.StringVar(&logsUntil, "logs-until", "", "Include logs older than the specified date; (journalctl date format: YYYY-MM-DD, 'yesterday', or 'today'). Refer to journalctl documentation for more options")
f.StringVar(&logsSizeLimit, "logs-size-limit", "100MiB", "Read the logs until the given size is reached (e.g. 3MB, 1GiB)")
f.StringVar(&controllerLogsSizeLimit, "controller-logs-size-limit", "132MB", "The size limit of the controller logs that can be stored in the bundle (e.g. 3MB, 1GiB)")
f.StringVar(&uploadURL, "upload-url", "", "If provided, where to upload the bundle in addition to creating a copy on disk")
Expand Down Expand Up @@ -268,9 +269,10 @@ BARE-METAL
- Disk usage: The disk usage for the data directory, as output by 'du'.
- redpanda logs: The node's redpanda logs written to journald. If --logs-since
or --logs-until are passed, then only the logs within the resulting time frame
will be included.
- Redpanda logs: The node's Redpanda logs written to journald since yesterday
(00:00:00 of the previous day based on systemd.time). If '--logs-since' or
'--logs-until' are passed, then only the logs within the resulting time frame
are included.
- Socket info: The active sockets data output by 'ss'.
Expand Down
3 changes: 2 additions & 1 deletion src/go/rpk/pkg/cli/debug/bundle/bundle_all.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"context"
"errors"

"github.com/redpanda-data/redpanda/src/go/rpk/pkg/config"
"github.com/spf13/afero"
)

Expand All @@ -26,6 +27,6 @@ func executeK8SBundle(context.Context, bundleParams) error {
return errors.New("rpk debug bundle is unsupported on your operating system")
}

func determineFilepath(afero.Fs, string, bool) (string, error) {
func determineFilepath(afero.Fs, *config.RedpandaYaml, string, bool) (string, error) {
return "", errors.New("rpk debug bundle is not supported on your operating system")
}
24 changes: 14 additions & 10 deletions src/go/rpk/pkg/cli/debug/bundle/bundle_linux.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,11 +56,15 @@ const linuxUtilsRoot = "utils"
// - File Extension: if no extension is provided we default to .zip
// - File Location: we check for write permissions in the pwd (for backcompat);
// if permission is denied we default to $HOME unless isFlag is true.
func determineFilepath(fs afero.Fs, path string, isFlag bool) (finalPath string, err error) {
func determineFilepath(fs afero.Fs, rp *config.RedpandaYaml, path string, isFlag bool) (finalPath string, err error) {
// if it's empty, use ./<timestamp>-bundle.zip
if path == "" {
timestamp := time.Now().Unix()
path = fmt.Sprintf("%d-bundle.zip", timestamp)
if rp.Redpanda.AdvertisedRPCAPI != nil {
path = fmt.Sprintf("%v-%d-bundle.zip", sanitizeName(rp.Redpanda.AdvertisedRPCAPI.Address), timestamp)
} else {
path = fmt.Sprintf("%d-bundle.zip", timestamp)
}
} else if isDir, _ := afero.IsDir(fs, path); isDir {
return "", fmt.Errorf("output file path is a directory, please specify the name of the file")
}
Expand Down Expand Up @@ -312,12 +316,7 @@ func writeCommandOutputToZipLimit(
if !strings.Contains(err.Error(), "broken pipe") {
return fmt.Errorf("couldn't save '%s': %w; %[1]v contains the full error message", filename, err)
}
zap.L().Sugar().Debugf(
"Got '%v' while running '%s'. This is probably due to the"+
" command's output exceeding its limit in bytes.",
err,
cmd,
)
zap.L().Sugar().Warnf("%v: got '%v' while running '%s'. This is probably due to the command's output exceeding its limit in bytes.", filename, err, cmd)
}
return nil
}
Expand Down Expand Up @@ -955,8 +954,13 @@ func saveControllerLogDir(ps *stepParams, y *config.RedpandaYaml, logLimitBytes
return fmt.Errorf("unable to save controller logs: %v", err)
}

// Our decoding tools look for the base of the data directory, and it
// searches for the expected directory: redpanda/controller/0_0. If we
// use this folder structure, we will make the life easier to the users
// who wish to decode the controller logs using our tools.
baseDestDir := filepath.Join("controller-logs", "redpanda", "controller", "0_0")
if int(size) < logLimitBytes {
return writeDirToZip(ps, controllerDir, "controller", exclude)
return writeDirToZip(ps, controllerDir, baseDestDir, exclude)
}

fmt.Printf("WARNING: controller logs directory size is too big (%v). Saving a slice of the logs; you can adjust the limit by changing --controller-logs-size-limit flag\n", units.HumanSize(float64(size)))
Expand All @@ -974,7 +978,7 @@ func saveControllerLogDir(ps *stepParams, y *config.RedpandaYaml, logLimitBytes
if err != nil {
return fmt.Errorf("unable to save controller logs: %v", err)
}
err = writeFileToZip(ps, filepath.Join("controller", filepath.Base(cLog.path)), file)
err = writeFileToZip(ps, filepath.Join(baseDestDir, filepath.Base(cLog.path)), file)
if err != nil {
return fmt.Errorf("unable to save controller logs: %v", err)
}
Expand Down
20 changes: 13 additions & 7 deletions src/go/rpk/pkg/cli/debug/bundle/bundle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"testing"
"time"

"github.com/redpanda-data/redpanda/src/go/rpk/pkg/config"
"github.com/spf13/afero"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -87,22 +88,27 @@ func TestDetermineFilepath(t *testing.T) {
for _, test := range []struct {
name string
filepath string
rpcAddr *config.SocketAddress
exp string
expErr bool
}{
{"empty filepath", "", "-bundle.zip", false},
{"correct filepath", "/tmp/customDebugName.zip", "/tmp/customDebugName.zip", false},
{"filepath with no extension", "/tmp/file", "/tmp/file.zip", false},
{"filepath is a directory", "/tmp", "", true},
{"unsupported extension", "customDebugName.tar.gz", "", true},
{"empty filepath", "", nil, ".*-bundle.zip", false},
{"correct filepath", "/tmp/customDebugName.zip", nil, "/tmp/customDebugName.zip", false},
{"filepath with no extension", "/tmp/file", nil, "/tmp/file.zip", false},
{"rpc + filepath", "/tmp/file", &config.SocketAddress{Address: "127.0.0.1"}, "/tmp/file.zip", false}, // It should NOT use the rpc
{"rpc + empty filepath", "", &config.SocketAddress{Address: "127.0.0.1"}, "(127.0.0.1-).*(-bundle.zip)", false},
{"sanitized rpc + empty filepath", "", &config.SocketAddress{Address: "my.awesome.address:1994"}, "(my.awesome.address-1994-).*(-bundle.zip)", false},
{"filepath is a directory", "/tmp", nil, "", true},
{"unsupported extension", "customDebugName.tar.gz", nil, "", true},
} {
t.Run(test.name, func(t *testing.T) {
fs := afero.NewMemMapFs()
// create /tmp folder for the test cases.
err := fs.Mkdir("/tmp", 0o755)
require.NoError(t, err)

filepath, err := determineFilepath(fs, test.filepath, false)
rp := &config.RedpandaYaml{Redpanda: config.RedpandaNodeConfig{AdvertisedRPCAPI: test.rpcAddr}}
filepath, err := determineFilepath(fs, rp, test.filepath, false)
if test.expErr {
require.Error(t, err)
return
Expand All @@ -112,7 +118,7 @@ func TestDetermineFilepath(t *testing.T) {
require.Equal(t, test.exp, filepath)
return
}
require.Contains(t, filepath, test.exp)
require.Regexp(t, test.exp, filepath)
})
}
}
Expand Down
2 changes: 1 addition & 1 deletion tests/rptest/tests/rpk_cluster_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -110,7 +110,7 @@ def test_debug_bundle(self):
assert f'{root_name}/redpanda.log' in files

# At least the first controller log is being saved:
assert f'{root_name}/controller/0-1-v1.log' in files
assert f'{root_name}/controller-logs/redpanda/controller/0_0/0-1-v1.log' in files

# Cluster admin API calls:
assert f'{root_name}/admin/brokers.json' in files
Expand Down

0 comments on commit 66229b9

Please sign in to comment.