Skip to content

Commit

Permalink
Merge #132594
Browse files Browse the repository at this point in the history
132594: roachtest: remove CRDB log usages r=DarrylWong a=herkolategan

This PR removes all references to CRDB log usage in roachtest. There are still a few transitive dependencies that end up calling CRDB log. These are small enough and get redirected to a separate file if they need to be inspected. Ultimately, we want to ensure that the appropriate loggers get used in roachtests, that are supplied by the test framework. After this PR, a linter can be introduced to ban the top-level import of CRDB log from roachtest. The only remaining direct usage in roachtest is to configure CRDB log to use a file sink, but this will be updated by another PR #132586 that moves the redirect functionality alongside the roachprod logger implementation.

Informs: #131412

Epic: None
Release note: None

Co-authored-by: Herko Lategan <[email protected]>
  • Loading branch information
craig[bot] and herkolategan committed Oct 24, 2024
2 parents 345acc4 + 2059cac commit 88ba5a1
Show file tree
Hide file tree
Showing 17 changed files with 81 additions and 45 deletions.
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ import (
"fmt"
"io"
"io/fs"
"log"
"math/rand"
"net"
"net/url"
Expand Down Expand Up @@ -47,7 +48,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachprod/vm"
"github.com/cockroachdb/cockroach/pkg/roachprod/vm/gce"
"github.com/cockroachdb/cockroach/pkg/util/httputil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -974,7 +974,7 @@ func (f *clusterFactory) newCluster(
logPath := filepath.Join(f.artifactsDir, runnerLogsDir, "cluster-create", genName+retryStr+".log")
l, err := logger.RootLogger(logPath, teeOpt)
if err != nil {
log.Fatalf(ctx, "%v", err)
log.Fatalf("%v", err)
}

c := &clusterImpl{
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/roachtestutil/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,9 @@ go_library(
"//pkg/roachprod/install",
"//pkg/roachprod/logger",
"//pkg/testutils/sqlutils",
"//pkg/util",
"//pkg/util/httputil",
"//pkg/util/humanizeutil",
"//pkg/util/log",
"//pkg/util/protoutil",
"//pkg/util/retry",
"//pkg/util/syncutil",
Expand Down
22 changes: 22 additions & 0 deletions pkg/cmd/roachtest/roachtestutil/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,9 +6,13 @@
package roachtestutil

import (
"time"

"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
"github.com/cockroachdb/cockroach/pkg/roachprod/config"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
)

// SystemInterfaceSystemdUnitName is a convenience function that
Expand All @@ -32,3 +36,21 @@ func SetDefaultAdminUIPort(c cluster.Cluster, opts *install.StartOpts) {
opts.AdminUIPort = config.DefaultAdminUIPort
}
}

// EveryN provides a way to rate limit noisy log messages. It tracks how
// recently a given log message has been emitted so that it can determine
// whether it's worth logging again.
type EveryN struct {
util.EveryN
}

// Every is a convenience constructor for an EveryN object that allows a log
// message every n duration.
func Every(n time.Duration) EveryN {
return EveryN{EveryN: util.Every(n)}
}

// ShouldLog returns whether it's been more than N time since the last event.
func (e *EveryN) ShouldLog() bool {
return e.ShouldProcess(timeutil.Now())
}
3 changes: 1 addition & 2 deletions pkg/cmd/roachtest/roachtestutil/validation_check.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -63,7 +62,7 @@ FROM crdb_internal.check_consistency(false, '', '') as t;`)
}
defer rows.Close()

logEvery := log.Every(time.Minute)
logEvery := Every(time.Minute)
logEvery.ShouldLog() // don't immediately log

var ranges int
Expand Down
9 changes: 4 additions & 5 deletions pkg/cmd/roachtest/test_runner.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachprod/vm"
"github.com/cockroachdb/cockroach/pkg/util/allstacks"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/quotapool"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/stop"
Expand Down Expand Up @@ -581,7 +580,7 @@ func (r *testRunner) runWorker(
) error {
stdout := lopt.stdout

wStatus := r.addWorker(ctx, name)
wStatus := r.addWorker(ctx, l, name)
defer func() {
r.removeWorker(ctx, name)
}()
Expand Down Expand Up @@ -650,7 +649,7 @@ func (r *testRunner) runWorker(
testToRun := testToRunRes{noWork: true}
if c != nil {
// Try to reuse cluster.
testToRun = work.selectTestForCluster(ctx, c.spec, r.cr, roachtestflags.Cloud)
testToRun = work.selectTestForCluster(ctx, l, c.spec, r.cr, roachtestflags.Cloud)
if !testToRun.noWork {
// We found a test to run on this cluster. Wipe the cluster.
if err := c.WipeForReuse(ctx, l, testToRun.spec.Cluster); err != nil {
Expand Down Expand Up @@ -1696,12 +1695,12 @@ func (r *testRunner) generateReport() string {
}

// addWorker updates the bookkeeping for one more worker.
func (r *testRunner) addWorker(ctx context.Context, name string) *workerStatus {
func (r *testRunner) addWorker(ctx context.Context, l *logger.Logger, name string) *workerStatus {
r.workersMu.Lock()
defer r.workersMu.Unlock()
w := &workerStatus{name: name}
if _, ok := r.workersMu.workers[name]; ok {
log.Fatalf(ctx, "worker %q already exists", name)
l.FatalfCtx(ctx, "worker %q already exists", name)
}
r.workersMu.workers[name] = w
return w
Expand Down
1 change: 0 additions & 1 deletion pkg/cmd/roachtest/tests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -273,7 +273,6 @@ go_library(
"//pkg/util/humanizeutil",
"//pkg/util/intsets",
"//pkg/util/ioctx",
"//pkg/util/log",
"//pkg/util/parquet",
"//pkg/util/protoutil",
"//pkg/util/randutil",
Expand Down
5 changes: 2 additions & 3 deletions pkg/cmd/roachtest/tests/backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/jobutils"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
Expand Down Expand Up @@ -354,10 +353,10 @@ func registerBackup(r registry.Registry) {
// runner copies it into an appropriate directory path.
dest := filepath.Join(t.PerfArtifactsDir(), "stats.json")
if err := c.RunE(ctx, option.WithNodes(c.Node(1)), "mkdir -p "+filepath.Dir(dest)); err != nil {
log.Errorf(ctx, "failed to create perf dir: %+v", err)
t.L().ErrorfCtx(ctx, "failed to create perf dir: %+v", err)
}
if err := c.PutString(ctx, perfBuf.String(), dest, 0755, c.Node(1)); err != nil {
log.Errorf(ctx, "failed to upload perf artifacts to node: %s", err.Error())
t.L().ErrorfCtx(ctx, "failed to upload perf artifacts to node: %s", err.Error())
}
return nil
})
Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/tests/cdc.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/grafana"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
Expand All @@ -61,7 +62,6 @@ import (
roachprodaws "github.com/cockroachdb/cockroach/pkg/roachprod/vm/aws"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/workload/debug"
Expand Down Expand Up @@ -3348,7 +3348,7 @@ func (k kafkaManager) startTopicConsumers(
return err
}
defer topicConsumer.close()
everyN := log.Every(30 * time.Second)
everyN := roachtestutil.Every(30 * time.Second)
for {
select {
case <-stopper:
Expand Down
3 changes: 1 addition & 2 deletions pkg/cmd/roachtest/tests/disk_stall.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -107,7 +106,7 @@ func runDiskStalledWALFailover(
nextStallAt := workloadStartAt.Add(pauseBetweenStalls)
defer ticker.Stop()

progressEvery := log.Every(time.Minute)
progressEvery := roachtestutil.Every(time.Minute)
for timeutil.Since(workloadStartAt) < time.Hour+5*time.Minute {
select {
case <-ctx.Done():
Expand Down
9 changes: 4 additions & 5 deletions pkg/cmd/roachtest/tests/import.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -161,10 +160,10 @@ func registerImportTPCC(r registry.Registry) {
// runner copies it into an appropriate directory path.
dest := filepath.Join(t.PerfArtifactsDir(), "stats.json")
if err := c.RunE(ctx, option.WithNodes(c.Node(1)), "mkdir -p "+filepath.Dir(dest)); err != nil {
log.Errorf(ctx, "failed to create perf dir: %+v", err)
t.L().ErrorfCtx(ctx, "failed to create perf dir: %+v", err)
}
if err := c.PutString(ctx, perfBuf.String(), dest, 0755, c.Node(1)); err != nil {
log.Errorf(ctx, "failed to upload perf artifacts to node: %s", err.Error())
t.L().ErrorfCtx(ctx, "failed to upload perf artifacts to node: %s", err.Error())
}
return nil
})
Expand Down Expand Up @@ -328,10 +327,10 @@ func registerImportTPCH(r registry.Registry) {
// runner copies it into an appropriate directory path.
dest := filepath.Join(t.PerfArtifactsDir(), "stats.json")
if err := c.RunE(ctx, option.WithNodes(c.Node(1)), "mkdir -p "+filepath.Dir(dest)); err != nil {
log.Errorf(ctx, "failed to create perf dir: %+v", err)
t.L().ErrorfCtx(ctx, "failed to create perf dir: %+v", err)
}
if err := c.PutString(ctx, perfBuf.String(), dest, 0755, c.Node(1)); err != nil {
log.Errorf(ctx, "failed to upload perf artifacts to node: %s", err.Error())
t.L().ErrorfCtx(ctx, "failed to upload perf artifacts to node: %s", err.Error())
}
return nil
})
Expand Down
10 changes: 5 additions & 5 deletions pkg/cmd/roachtest/tests/latency_verifier.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,9 +11,9 @@ import (
"fmt"
"time"

"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/codahale/hdrhistogram"
Expand Down Expand Up @@ -41,10 +41,10 @@ type latencyVerifier struct {
initialScanHighwater time.Time
initialScanLatency time.Duration

catchupScanEveryN log.EveryN
catchupScanEveryN roachtestutil.EveryN

maxSeenSteadyLatency time.Duration
maxSeenSteadyEveryN log.EveryN
maxSeenSteadyEveryN roachtestutil.EveryN
latencyBecameSteady bool

latencyHist *hdrhistogram.Histogram
Expand Down Expand Up @@ -74,8 +74,8 @@ func makeLatencyVerifier(
setTestStatus: setTestStatus,
latencyHist: hist,
tolerateErrors: tolerateErrors,
maxSeenSteadyEveryN: log.Every(10 * time.Second),
catchupScanEveryN: log.Every(2 * time.Second),
maxSeenSteadyEveryN: roachtestutil.Every(10 * time.Second),
catchupScanEveryN: roachtestutil.Every(2 * time.Second),
}
}

Expand Down
3 changes: 1 addition & 2 deletions pkg/cmd/roachtest/tests/mixed_version_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/jobutils"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/retry"
Expand Down Expand Up @@ -1792,7 +1791,7 @@ func (d *BackupRestoreTestDriver) computeTableContents(
}

if err := eg.Wait(); err != nil {
log.Errorf(ctx, "Error loading system table content %s", err)
l.ErrorfCtx(ctx, "Error loading system table content %s", err)
return nil, err
}

Expand Down
3 changes: 1 addition & 2 deletions pkg/cmd/roachtest/tests/mixed_version_cdc.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/cockroach/pkg/roachprod/vm"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -241,7 +240,7 @@ func (cmvt *cdcMixedVersionTester) setupValidator(
func (cmvt *cdcMixedVersionTester) runKafkaConsumer(
ctx context.Context, l *logger.Logger, r *rand.Rand, h *mixedversion.Helper,
) error {
everyN := log.Every(30 * time.Second)
everyN := roachtestutil.Every(30 * time.Second)

// This runs until the test finishes, which will be signaled via
// context cancellation. We rely on consumer.Next() to check
Expand Down
5 changes: 2 additions & 3 deletions pkg/cmd/roachtest/tests/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/ts/tspb"
"github.com/cockroachdb/cockroach/pkg/util/httputil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/workload/histogram"
Expand Down Expand Up @@ -1145,10 +1144,10 @@ func exportToRoachperf(
// runner copies it into an appropriate directory path.
dest := filepath.Join(t.PerfArtifactsDir(), "stats.json")
if err := c.RunE(ctx, option.WithNodes(c.Node(1)), "mkdir -p "+filepath.Dir(dest)); err != nil {
log.Errorf(ctx, "failed to create perf dir: %+v", err)
t.L().ErrorfCtx(ctx, "failed to create perf dir: %+v", err)
}
if err := c.PutString(ctx, bytesBuf.String(), dest, 0755, c.Node(1)); err != nil {
log.Errorf(ctx, "failed to upload perf artifacts to node: %s", err.Error())
t.L().ErrorfCtx(ctx, "failed to upload perf artifacts to node: %s", err.Error())
}
}

Expand Down
Loading

0 comments on commit 88ba5a1

Please sign in to comment.