Skip to content

Commit

Permalink
Merge #123432
Browse files Browse the repository at this point in the history
123432: util/metamorphic: introduce `metamorphic` package r=srosenberg,RaduBerinde a=rickystewart

This separates the metamorphic functionality out into its own package from `pkg/util`. This is useful for some future functionality that I would like to implement:

* Simplify/make explicit metamorphic constant logging
* Remove the `metamorphic_disable` build tag (see #123319)

This is a re-implementation of #102474.

Epic: none
Release note: None

Co-authored-by: Ricky Stewart <[email protected]>
  • Loading branch information
craig[bot] and rickystewart committed May 7, 2024
2 parents 1fb89af + 73a2a79 commit 2f498d0
Show file tree
Hide file tree
Showing 105 changed files with 228 additions and 173 deletions.
3 changes: 3 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -695,6 +695,7 @@ ALL_TESTS = [
"//pkg/util/log/logmetrics:logmetrics_test",
"//pkg/util/log/testshout:testshout_test",
"//pkg/util/log:log_test",
"//pkg/util/metamorphic:metamorphic_test",
"//pkg/util/metric/aggmetric:aggmetric_test",
"//pkg/util/metric:metric_test",
"//pkg/util/mon:mon_test",
Expand Down Expand Up @@ -2455,6 +2456,8 @@ GO_TARGETS = [
"//pkg/util/log:log",
"//pkg/util/log:log_test",
"//pkg/util/memzipper:memzipper",
"//pkg/util/metamorphic:metamorphic",
"//pkg/util/metamorphic:metamorphic_test",
"//pkg/util/metric/aggmetric:aggmetric",
"//pkg/util/metric/aggmetric:aggmetric_test",
"//pkg/util/metric/tick:tick",
Expand Down
2 changes: 1 addition & 1 deletion pkg/bench/rttanalysis/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@ go_library(
"//pkg/testutils/datapathutils",
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
"//pkg/util",
"//pkg/util/log",
"//pkg/util/metamorphic",
"//pkg/util/quotapool",
"//pkg/util/syncutil",
"//pkg/util/system",
Expand Down
6 changes: 3 additions & 3 deletions pkg/bench/rttanalysis/validate_benchmark_data.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,8 @@ import (
"testing"

"github.com/cockroachdb/cockroach/pkg/testutils/datapathutils"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metamorphic"
"github.com/cockroachdb/cockroach/pkg/util/quotapool"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/system"
Expand Down Expand Up @@ -62,7 +62,7 @@ var (
// RunBenchmarkExpectationTests runs tests to validate or rewrite the contents
// of the benchmark expectations file.
func runBenchmarkExpectationTests(t *testing.T, r *Registry) {
if util.IsMetamorphicBuild() {
if metamorphic.IsMetamorphicBuild() {
execTestSubprocess(t)
return
}
Expand Down Expand Up @@ -185,7 +185,7 @@ func execTestSubprocess(t *testing.T) {
cmd.Stdout = os.Stdout
cmd.Stderr = os.Stderr
cmd.Env = os.Environ()
cmd.Env = append(cmd.Env, util.DisableMetamorphicEnvVar+"=t")
cmd.Env = append(cmd.Env, metamorphic.DisableMetamorphicEnvVar+"=t")
t.Log(cmd.Args)
if err := cmd.Run(); err != nil {
t.FailNow()
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,6 @@ go_library(
"//pkg/sql/types",
"//pkg/storage",
"//pkg/storage/enginepb",
"//pkg/util",
"//pkg/util/admission",
"//pkg/util/admission/admissionpb",
"//pkg/util/bulk",
Expand All @@ -141,6 +140,7 @@ go_library(
"//pkg/util/log",
"//pkg/util/log/eventpb",
"//pkg/util/log/logutil",
"//pkg/util/metamorphic",
"//pkg/util/metric",
"//pkg/util/mon",
"//pkg/util/pprofutil",
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/backup_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,14 +60,14 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/util"
bulkutil "github.com/cockroachdb/cockroach/pkg/util/bulk"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/interval"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
"github.com/cockroachdb/cockroach/pkg/util/log/logutil"
"github.com/cockroachdb/cockroach/pkg/util/metamorphic"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/retry"
Expand All @@ -88,7 +88,7 @@ var BackupCheckpointInterval = settings.RegisterDurationSetting(
"the minimum time between writing progress checkpoints during a backup",
time.Minute)

var forceReadBackupManifest = util.ConstantWithMetamorphicTestBool("backup-read-manifest", false)
var forceReadBackupManifest = metamorphic.ConstantWithMetamorphicTestBool("backup-read-manifest", false)

var useBulkOracle = settings.RegisterBoolSetting(
settings.ApplicationLevel,
Expand Down
6 changes: 3 additions & 3 deletions pkg/ccl/backupccl/backup_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,14 +29,14 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/rowexec"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/admission"
"github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb"
"github.com/cockroachdb/cockroach/pkg/util/bulk"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metamorphic"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
Expand Down Expand Up @@ -91,14 +91,14 @@ var (
settings.ApplicationLevel,
"bulkio.backup.presplit_request_spans.enabled",
"split the spans that will be requests before requesting them",
util.ConstantWithMetamorphicTestBool("backup-presplit-spans", true),
metamorphic.ConstantWithMetamorphicTestBool("backup-presplit-spans", true),
)

sendExportRequestWithVerboseTracing = settings.RegisterBoolSetting(
settings.ApplicationLevel,
"bulkio.backup.export_request_verbose_tracing",
"send each export request with a verbose tracing span",
util.ConstantWithMetamorphicTestBool("export_request_verbose_tracing", false),
metamorphic.ConstantWithMetamorphicTestBool("export_request_verbose_tracing", false),
settings.WithName("bulkio.backup.verbose_tracing.enabled"),
)

Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backupbase/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,6 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/settings",
"//pkg/util",
"//pkg/util/metamorphic",
],
)
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/backupbase/settings.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,11 +10,11 @@ package backupbase

import (
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/metamorphic"
)

var (
defaultSmallFileBuffer = util.ConstantWithMetamorphicTestRange(
defaultSmallFileBuffer = metamorphic.ConstantWithMetamorphicTestRange(
"backup-merge-file-buffer-size",
128<<20, /* defaultValue */
1<<20, /* metamorphic min */
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backupinfo/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -38,14 +38,14 @@ go_library(
"//pkg/sql/sem/tree",
"//pkg/sql/stats",
"//pkg/storage",
"//pkg/util",
"//pkg/util/bulk",
"//pkg/util/ctxgroup",
"//pkg/util/encoding",
"//pkg/util/hlc",
"//pkg/util/ioctx",
"//pkg/util/json",
"//pkg/util/log",
"//pkg/util/metamorphic",
"//pkg/util/mon",
"//pkg/util/protoutil",
"//pkg/util/syncutil",
Expand Down
6 changes: 3 additions & 3 deletions pkg/ccl/backupccl/backupinfo/manifest_handling.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,13 +44,13 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/bulk"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/ioctx"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metamorphic"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
Expand Down Expand Up @@ -94,7 +94,7 @@ var WriteMetadataSST = settings.RegisterBoolSetting(
settings.ApplicationLevel,
"kv.bulkio.write_metadata_sst.enabled",
"write experimental new format BACKUP metadata file",
util.ConstantWithMetamorphicTestBool("write-metadata-sst", false),
metamorphic.ConstantWithMetamorphicTestBool("write-metadata-sst", false),
)

// WriteMetadataWithExternalSSTsEnabled controls if we write a `BACKUP_METADATA`
Expand All @@ -105,7 +105,7 @@ var WriteMetadataWithExternalSSTsEnabled = settings.RegisterBoolSetting(
settings.ApplicationLevel,
"backup.write_metadata_with_external_ssts.enabled",
"write BACKUP metadata along with supporting SST files",
util.ConstantWithMetamorphicTestBool("backup.write_metadata_with_external_ssts.enabled", true),
metamorphic.ConstantWithMetamorphicTestBool("backup.write_metadata_with_external_ssts.enabled", true),
)

// IsGZipped detects whether the given bytes represent GZipped data. This check
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/backuptestutils/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ go_library(
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util",
"//pkg/util/metamorphic",
"//pkg/workload/bank",
"//pkg/workload/workloadsql",
"@com_github_cockroachdb_errors//:errors",
Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/backupccl/backuptestutils/testutils.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/metamorphic"
"github.com/cockroachdb/cockroach/pkg/workload/bank"
"github.com/cockroachdb/cockroach/pkg/workload/workloadsql"
"github.com/cockroachdb/errors"
Expand All @@ -43,7 +44,7 @@ const (
// smallEngineBlocks configures Pebble with a block size of 1 byte, to provoke
// bugs in time-bound iterators. We disable this in race builds, which can
// be too slow.
var smallEngineBlocks = !util.RaceEnabled && util.ConstantWithMetamorphicTestBool("small-engine-blocks", false)
var smallEngineBlocks = !util.RaceEnabled && metamorphic.ConstantWithMetamorphicTestBool("small-engine-blocks", false)

// InitManualReplication calls tc.ToggleReplicateQueues(false).
//
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/restore_data_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,11 +32,11 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util"
bulkutil "github.com/cockroachdb/cockroach/pkg/util/bulk"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metamorphic"
"github.com/cockroachdb/cockroach/pkg/util/pprofutil"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
Expand Down Expand Up @@ -106,7 +106,7 @@ const maxConcurrentRestoreWorkers = 32
// file.
const minWorkerMemReservation = 15 << 20

var defaultNumWorkers = util.ConstantWithMetamorphicTestRange(
var defaultNumWorkers = metamorphic.ConstantWithMetamorphicTestRange(
"restore-worker-concurrency",
func() int {
// On low-CPU instances, a default value may still allow concurrent restore
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,6 @@ go_library(
"//pkg/sql/sessiondatapb",
"//pkg/sql/syntheticprivilege",
"//pkg/sql/types",
"//pkg/util",
"//pkg/util/admission",
"//pkg/util/admission/admissionpb",
"//pkg/util/bitarray",
Expand All @@ -140,6 +139,7 @@ go_library(
"//pkg/util/log",
"//pkg/util/log/eventpb",
"//pkg/util/log/logcrash",
"//pkg/util/metamorphic",
"//pkg/util/metric",
"//pkg/util/metric/aggmetric",
"//pkg/util/mon",
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/changefeed_dist.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,12 +37,12 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metamorphic"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -351,7 +351,7 @@ var RangeDistributionStrategy = settings.RegisterEnumSetting(
"configures how work is distributed among nodes for a given changefeed. "+
"for the most balanced distribution, use `balanced_simple`. changing this setting "+
"will not override locality restrictions",
util.ConstantWithMetamorphicTestChoice("default_range_distribution_strategy",
metamorphic.ConstantWithMetamorphicTestChoice("default_range_distribution_strategy",
"default", "balanced_simple").(string),
map[int64]string{
int64(defaultDistribution): "default",
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/changefeedbase/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,8 @@ go_library(
"//pkg/sql/catalog/descpb",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/util",
"//pkg/util/iterutil",
"//pkg/util/metamorphic",
"@com_github_cockroachdb_errors//:errors",
],
)
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/changefeedbase/settings.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/metamorphic"
)

// TableDescriptorPollInterval controls how fast table descriptors are polled. A
Expand Down Expand Up @@ -238,7 +238,7 @@ var EventConsumerWorkerQueueSize = settings.RegisterIntSetting(
"changefeed.event_consumer_worker_queue_size",
"if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of events "+
"which a worker can buffer",
int64(util.ConstantWithMetamorphicTestRange("changefeed.event_consumer_worker_queue_size", 16, 0, 16)),
int64(metamorphic.ConstantWithMetamorphicTestRange("changefeed.event_consumer_worker_queue_size", 16, 0, 16)),
settings.NonNegativeInt,
settings.WithPublic)

Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/compression.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@ import (
"strings"

"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/metamorphic"
"github.com/cockroachdb/errors"
"github.com/klauspost/compress/zstd"
"github.com/klauspost/pgzip"
Expand All @@ -24,7 +24,7 @@ var useFastGzip = settings.RegisterBoolSetting(
settings.ApplicationLevel,
"changefeed.fast_gzip.enabled",
"use fast gzip implementation",
util.ConstantWithMetamorphicTestBool(
metamorphic.ConstantWithMetamorphicTestBool(
"changefeed.fast_gzip.enabled", true,
),
settings.WithPublic)
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/parallel_io.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,9 @@ import (

"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/intsets"
"github.com/cockroachdb/cockroach/pkg/util/metamorphic"
"github.com/cockroachdb/cockroach/pkg/util/quotapool"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
Expand Down Expand Up @@ -165,7 +165,7 @@ var requestQuota = settings.RegisterIntSetting(
"changefeed.parallel_io.request_quota",
"the number of requests which can be admitted into the parallelio"+
" system before blocking the producer",
int64(util.ConstantWithMetamorphicTestChoice(
int64(metamorphic.ConstantWithMetamorphicTestChoice(
"changefeed.parallel_io.request_quota",
128, 16, 32, 64, 256).(int)),
settings.IntInRange(1, 256),
Expand Down
6 changes: 3 additions & 3 deletions pkg/ccl/changefeedccl/sink.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,12 +29,12 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/admission"
"github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb"
"github.com/cockroachdb/cockroach/pkg/util/bufalloc"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metamorphic"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
Expand Down Expand Up @@ -165,7 +165,7 @@ var WebhookV2Enabled = settings.RegisterBoolSetting(
"if enabled, this setting enables a new implementation of the webhook sink"+
" that allows for a much higher throughput",
// TODO: delete the original webhook sink code
util.ConstantWithMetamorphicTestBool("changefeed.new_webhook_sink.enabled", true),
metamorphic.ConstantWithMetamorphicTestBool("changefeed.new_webhook_sink.enabled", true),
settings.WithName("changefeed.new_webhook_sink.enabled"),
)

Expand All @@ -177,7 +177,7 @@ var PubsubV2Enabled = settings.RegisterBoolSetting(
"if enabled, this setting enables a new implementation of the pubsub sink"+
" that allows for a higher throughput",
// TODO: delete the original pubsub sink code
util.ConstantWithMetamorphicTestBool("changefeed.new_pubsub_sink.enabled", true),
metamorphic.ConstantWithMetamorphicTestBool("changefeed.new_pubsub_sink.enabled", true),
settings.WithName("changefeed.new_pubsub_sink.enabled"),
)

Expand Down
2 changes: 1 addition & 1 deletion pkg/cloud/amazon/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,10 @@ go_library(
"//pkg/server/telemetry",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/util",
"//pkg/util/envutil",
"//pkg/util/ioctx",
"//pkg/util/log",
"//pkg/util/metamorphic",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"//pkg/util/tracing",
Expand Down
Loading

0 comments on commit 2f498d0

Please sign in to comment.