Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
132586: roachprod: redirect CRDB logs utility r=DarrylWong a=herkolategan

Currently, CRDB log is configured, by roachtest, to log to a file to catch any
logs written to it during a roachtest run. This is usually from a shared test
util that uses the CRDB log. The file sink on the CRDB logger will log program
arguments by default, but this can leak sensitive information.

This PR introduces a log redirect that uses the CRDB log interceptor
functionality instead of using a file sink. This way we can avoid logging the
program arguments.

Epic: None
Release note: None

133443: roachtest: improve the filename of stmtbundles r=kvoli,arulajmani a=andrewbaptist

The perturbation/* tests download statement bundles at the end of the test. Previously these only included the timestamp that the profile was taken at. However this meant searching through the profiles to find a relevant one. This commit adds both the operation type as well as the duration of the trace. It required unzipping the trace and examining the contents.

Epic: none

Release note: None

Co-authored-by: Herko Lategan <[email protected]>
Co-authored-by: Andrew Baptist <[email protected]>
  • Loading branch information
3 people committed Oct 28, 2024
3 parents 2e1be7f + 666a9cd + 74dd031 commit c3fb333
Show file tree
Hide file tree
Showing 7 changed files with 245 additions and 26 deletions.
2 changes: 2 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -306,6 +306,7 @@ ALL_TESTS = [
"//pkg/roachprod/cloud:cloud_test",
"//pkg/roachprod/config:config_test",
"//pkg/roachprod/install:install_test",
"//pkg/roachprod/logger:logger_test",
"//pkg/roachprod/opentelemetry:opentelemetry_test",
"//pkg/roachprod/prometheus:prometheus_test",
"//pkg/roachprod/promhelperclient:promhelperclient_test",
Expand Down Expand Up @@ -1611,6 +1612,7 @@ GO_TARGETS = [
"//pkg/roachprod/install:install_test",
"//pkg/roachprod/lock:lock",
"//pkg/roachprod/logger:logger",
"//pkg/roachprod/logger:logger_test",
"//pkg/roachprod/opentelemetry:opentelemetry",
"//pkg/roachprod/opentelemetry:opentelemetry_test",
"//pkg/roachprod/prometheus:prometheus",
Expand Down
3 changes: 0 additions & 3 deletions pkg/cmd/roachtest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -52,9 +52,6 @@ go_library(
"//pkg/util/ctxgroup",
"//pkg/util/httputil",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/log/logconfig",
"//pkg/util/log/logpb",
"//pkg/util/quotapool",
"//pkg/util/randutil",
"//pkg/util/stop",
Expand Down
31 changes: 13 additions & 18 deletions pkg/cmd/roachtest/run.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,9 +31,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/cockroach/pkg/util/allstacks"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/logconfig"
"github.com/cockroachdb/cockroach/pkg/util/log/logpb"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/version"
Expand Down Expand Up @@ -113,7 +110,8 @@ func runTests(register func(registry.Registry), filter *registry.TestFilter) err
if literalArtifactsDir == "" {
literalArtifactsDir = artifactsDir
}
setLogConfig(artifactsDir)
redirectLogger := redirectCRDBLogger(context.Background(), filepath.Join(artifactsDir, "roachtest.crdb.log"))
logger.InitCRDBLogConfig(redirectLogger)
runnerDir := filepath.Join(artifactsDir, runnerLogsDir)
runnerLogPath := filepath.Join(
runnerDir, fmt.Sprintf("test_runner-%d.log", timeutil.Now().Unix()))
Expand Down Expand Up @@ -221,20 +219,6 @@ func runTests(register func(registry.Registry), filter *registry.TestFilter) err
return err
}

// This diverts all the default non-fatal logging to a file in `baseDir`. This is particularly
// useful in CI, where without this, stderr/stdout are cluttered with logs from various
// packages used in roachtest like sarama and testutils.
func setLogConfig(baseDir string) {
logConf := logconfig.DefaultStderrConfig()
logConf.Sinks.Stderr.Filter = logpb.Severity_FATAL
if err := logConf.Validate(&baseDir); err != nil {
panic(err)
}
if _, err := log.ApplyConfig(logConf, nil /* fileSinkMetricsForDir */, nil /* fatalOnLogStall */); err != nil {
panic(err)
}
}

// getUser takes the value passed on the command line and comes up with the
// username to use.
func getUser(userFlag string) string {
Expand Down Expand Up @@ -395,6 +379,17 @@ func testRunnerLogger(
return l, teeOpt
}

func redirectCRDBLogger(ctx context.Context, path string) *logger.Logger {
verboseCfg := logger.Config{}
var err error
l, err := verboseCfg.NewLogger(path)
if err != nil {
panic(err)
}
shout(ctx, l, os.Stdout, "fallback runner logs in: %s", path)
return l
}

func maybeDumpSummaryMarkdown(r *testRunner) error {
if !roachtestflags.GitHubActions {
return nil
Expand Down
64 changes: 61 additions & 3 deletions pkg/cmd/roachtest/tests/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,13 +6,16 @@
package tests

import (
"archive/zip"
"bytes"
"context"
"fmt"
"io"
"net/http"
"os"
"path/filepath"
"regexp"
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
Expand Down Expand Up @@ -233,15 +236,70 @@ func downloadProfiles(
return err
}
url := urlPrefix + diagID
filename := fmt.Sprintf("%s-%s.zip", collectedAt.Format("2006-01-02T15_04_05Z07:00"), diagID)
logger.Printf("downloading profile %s", filename)
if err := client.Download(ctx, url, filepath.Join(stmtDir, filename)); err != nil {
resp, err := client.Get(context.Background(), url)
if err != nil {
return err
}
defer resp.Body.Close()
// Copy the contents of the URL to a BytesBuffer to determine the
// filename before saving it below.
var buf bytes.Buffer
_, err = io.Copy(&buf, resp.Body)
if err != nil {
return err
}
filename, err := getFilename(collectedAt, buf)
if err != nil {
return err
}
// write the buf to the filename
file, err := os.Create(filepath.Join(stmtDir, filename))
if err != nil {
return err
}
if _, err := io.Copy(file, &buf); err != nil {
return err
}
logger.Printf("downloaded profile %s", filename)
}
return nil
}

// getFilename creates a file name for the profile based on the traced operation
// and duration. An example filename is
// 2024-10-24T18_23_57Z-UPSERT-101.490ms.zip.
func getFilename(collectedAt time.Time, buf bytes.Buffer) (string, error) {
// Download the zip to a BytesBuffer.
unzip, err := zip.NewReader(bytes.NewReader(buf.Bytes()), int64(buf.Len()))
if err != nil {
return "", err
}
// NB: The format of the trace.txt file is not guaranteed to be stable. If
// this proves problematic we could parse the trace.json instead. Parsing
// the trace.txt is easier due to the nested structure of the trace.json.
r, err := unzip.Open("trace.txt")
if err != nil {
return "", err
}
bytes, err := io.ReadAll(r)
if err != nil {
return "", err
}
if err = r.Close(); err != nil {
return "", err
}
lines := strings.Split(string(bytes), "\n")
// The first line is the SQL statement. An example is `UPSERT INTO kv (k, v)
// VALUES ($1, $2)`. We only grab the operation to help differentiate
// traces. An alternative if this isn't differentiated enough is to use the
// entire fingerprint text, however that creates longs and complex
// filenames.
operation := strings.Split(strings.TrimSpace(lines[0]), " ")[0]
// Use the second to last line because the last line is empty.
duration := strings.Split(strings.TrimSpace(lines[len(lines)-2]), " ")[0]
return fmt.Sprintf("%s-%s-%s.zip", collectedAt.Format("2006-01-02T15_04_05Z07:00"), operation, duration), nil
}

type IP struct {
Query string
}
Expand Down
19 changes: 17 additions & 2 deletions pkg/roachprod/logger/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,13 +1,28 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "logger",
srcs = ["log.go"],
srcs = [
"log.go",
"log_redirect.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/roachprod/logger",
visibility = ["//visibility:public"],
deps = [
"//pkg/cli/exit",
"//pkg/util/log",
"//pkg/util/log/logconfig",
"//pkg/util/log/logpb",
"//pkg/util/syncutil",
],
)

go_test(
name = "logger_test",
srcs = ["log_redirect_test.go"],
embed = [":logger"],
deps = [
"//pkg/util/log",
"@com_github_stretchr_testify//require",
],
)
93 changes: 93 additions & 0 deletions pkg/roachprod/logger/log_redirect.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,93 @@
// Copyright 2024 The Cockroach Authors.
//
// Use of this software is governed by the CockroachDB Software License
// included in the /LICENSE file.

package logger

import (
"context"
"encoding/json"
"fmt"
"io"

"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/logconfig"
"github.com/cockroachdb/cockroach/pkg/util/log/logpb"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
)

type logRedirect struct {
syncutil.Mutex
logger *Logger
cancelIntercept func()
configured bool
}

var logRedirectInst = &logRedirect{}

// InitCRDBLogConfig sets up an interceptor for the CockroachDB log in order to
// redirect logs to a roachprod logger. This is necessary as the CockroachDB log
// is used in some test utilities shared between roachtest and CockroachDB.
// Generally, CockroachDB logs should not be used explicitly in roachtest.
func InitCRDBLogConfig(logger *Logger) {
logRedirectInst.Lock()
defer logRedirectInst.Unlock()
if logRedirectInst.configured {
panic("internal error: CRDB log interceptor already configured")
}
if logger == nil {
panic("internal error: specified logger is nil")
}
logConf := logconfig.DefaultStderrConfig()
logConf.Sinks.Stderr.Filter = logpb.Severity_FATAL
// Disable logging to a file. File sinks write the application arguments to
// the log by default (see: log_entry.go), and it is best to avoid logging
// the roachtest arguments as it may contain sensitive information.
if err := logConf.Validate(nil); err != nil {
panic(fmt.Errorf("internal error: could not validate CRDB log config: %w", err))
}
if _, err := log.ApplyConfig(logConf, nil /* fileSinkMetricsForDir */, nil /* fatalOnLogStall */); err != nil {
panic(fmt.Errorf("internal error: could not apply CRDB log config: %w", err))
}
logRedirectInst.logger = logger
logRedirectInst.cancelIntercept = log.InterceptWith(context.Background(), logRedirectInst)
logRedirectInst.configured = true
}

// TestingCRDBLogConfig is meant to be used in unit tests to reset the CRDB log,
// it's interceptor, and the redirect log config. This is necessary to avoid
// leaking state between tests, that test the logging.
func TestingCRDBLogConfig(logger *Logger) {
logRedirectInst.Lock()
defer logRedirectInst.Unlock()
if logRedirectInst.cancelIntercept != nil {
logRedirectInst.cancelIntercept()
}
log.TestingResetActive()
logRedirectInst = &logRedirect{}
InitCRDBLogConfig(logger)
}

// Intercept intercepts CockroachDB log entries and redirects it to the
// appropriate roachtest test logger or stderr.
func (i *logRedirect) Intercept(logData []byte) {
var entry logpb.Entry
if err := json.Unmarshal(logData, &entry); err != nil {
i.logger.Errorf("failed to unmarshal intercepted log entry: %v", err)
}
l := i.logger
if l != nil && !l.Closed() {
if entry.Severity == logpb.Severity_ERROR || entry.Severity == logpb.Severity_FATAL {
i.writeLog(l.Stderr, entry)
return
}
i.writeLog(l.Stdout, entry)
}
}

func (i *logRedirect) writeLog(dst io.Writer, entry logpb.Entry) {
if err := log.FormatLegacyEntry(entry, dst); err != nil {
i.logger.Errorf("could not format and write CRDB log entry: %v", err)
}
}
59 changes: 59 additions & 0 deletions pkg/roachprod/logger/log_redirect_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
// Copyright 2024 The Cockroach Authors.
//
// Use of this software is governed by the CockroachDB Software License
// included in the /LICENSE file.

package logger

import (
"context"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/stretchr/testify/require"
)

type mockLogger struct {
logger *Logger
writer *mockWriter
}

type mockWriter struct {
lines []string
}

func (w *mockWriter) Write(p []byte) (n int, err error) {
w.lines = append(w.lines, string(p))
return len(p), nil
}

func newMockLogger(t *testing.T) *mockLogger {
writer := &mockWriter{}
logConf := Config{Stdout: writer, Stderr: writer}
l, err := logConf.NewLogger("" /* path */)
require.NoError(t, err)
return &mockLogger{logger: l, writer: writer}
}

func requireLine(t *testing.T, l *mockLogger, line string) {
t.Helper()
found := false
for _, logLine := range l.writer.lines {
if strings.Contains(logLine, line) {
found = true
break
}
}
require.True(t, found, "expected line not found: %s", line)
}

func TestLogRedirect(t *testing.T) {
l := newMockLogger(t)
TestingCRDBLogConfig(l.logger)
ctx := context.Background()

log.Infof(ctx, "[simple test]")
requireLine(t, l, "[simple test]")
require.Equal(t, 1, len(l.writer.lines))
}

0 comments on commit c3fb333

Please sign in to comment.