Skip to content

Commit

Permalink
kvserver, storage: Move SeparatedIntentsEnabled to a testing knob
Browse files Browse the repository at this point in the history
Since we want to remove the ability to disable the writing
of separated intents once a version upgrade is finalized,
this change moves the SeparatedIntentsEnabled setting to
a testing knob. The bulk of changes here are just propagating
a flag from the new storage.TestingKnobs down to the engine.

Release note: None.
  • Loading branch information
itsbilal committed Aug 11, 2021
1 parent 1e5c1ef commit 4eb9d28
Show file tree
Hide file tree
Showing 27 changed files with 145 additions and 126 deletions.
3 changes: 3 additions & 0 deletions pkg/base/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -515,6 +515,9 @@ type StorageConfig struct {
// UseFileRegistry is true if the file registry is needed (eg: encryption-at-rest).
// This may force the store version to versionFileRegistry if currently lower.
UseFileRegistry bool
// DisableSeparatedIntents is true if separated intents should not be written
// by intent writers. Only true for tests.
DisableSeparatedIntents bool
// EncryptionOptions is a serialized protobuf set by Go CCL code and passed
// through to C CCL code to set up encryption-at-rest. Must be set if and
// only if encryption is enabled, otherwise left empty.
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_revert_range_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ func getStats(t *testing.T, reader storage.Reader) enginepb.MVCCStats {
// createTestPebbleEngine returns a new in-memory Pebble storage engine.
func createTestPebbleEngine(ctx context.Context) (storage.Engine, error) {
return storage.Open(ctx, storage.InMemory(),
storage.MaxSize(1<<20), storage.SettingsForTesting())
storage.MaxSize(1<<20), storage.ForTesting)
}

var engineImpls = []struct {
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -238,7 +238,7 @@ func (tc *testContext) StartWithStoreConfigAndVersion(
storage.InMemory(),
storage.Attributes(roachpb.Attributes{Attrs: []string{"dc1", "mem"}}),
storage.MaxSize(1<<20),
storage.SettingsForTesting())
storage.ForTesting)
if err != nil {
t.Fatal(err)
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/tenantrate"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
)
Expand All @@ -35,6 +36,7 @@ type StoreTestingKnobs struct {
TxnWaitKnobs txnwait.TestingKnobs
ConsistencyTestingKnobs ConsistencyTestingKnobs
TenantRateKnobs tenantrate.TestingKnobs
StorageKnobs storage.TestingKnobs

// TestingRequestFilter is called before evaluating each request on a
// replica. The filter is run before the request acquires latches, so
Expand Down
20 changes: 12 additions & 8 deletions pkg/migration/migrations/separated_intents_external_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,8 @@ package migrations_test

import (
"context"
"strconv"
"fmt"
"path"
"testing"
"time"

Expand Down Expand Up @@ -44,26 +45,28 @@ func TestSeparatedIntentsMigration(t *testing.T) {
clusterversion.ByKey(clusterversion.SeparatedIntentsMigration-1),
false, /* initializeVersion */
)
storage.SeparatedIntentsEnabled.Override(ctx, &settings.SV, false)
stickyEngineRegistry := server.NewStickyInMemEnginesRegistry()
defer stickyEngineRegistry.CloseAllStickyInMemEngines()
const numServers int = 3
stickyServerArgs := make(map[int]base.TestServerArgs)
storeKnobs := &kvserver.StoreTestingKnobs{
StorageKnobs: storage.TestingKnobs{DisableSeparatedIntents: true},
}
tempDir, cleanup := testutils.TempDir(t)
defer cleanup()
for i := 0; i < numServers; i++ {
stickyServerArgs[i] = base.TestServerArgs{
Settings: settings,
StoreSpecs: []base.StoreSpec{
{
InMemory: true,
StickyInMemoryEngineID: strconv.FormatInt(int64(i), 10),
InMemory: false,
Path: path.Join(tempDir, fmt.Sprintf("engine-%d", i)),
},
},
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
DisableAutomaticVersionUpgrade: 1,
BinaryVersionOverride: clusterversion.ByKey(clusterversion.SeparatedIntentsMigration - 1),
StickyEngineRegistry: stickyEngineRegistry,
},
Store: storeKnobs,
},
}
}
Expand Down Expand Up @@ -143,7 +146,8 @@ func TestSeparatedIntentsMigration(t *testing.T) {
}
require.True(t, interleavedIntentFound)

storage.SeparatedIntentsEnabled.Override(ctx, &settings.SV, true)
// Start writing separated intents.
storeKnobs.StorageKnobs.DisableSeparatedIntents = false
require.NoError(t, tc.Restart())
time.Sleep(10 * time.Second)
tdb = tc.ServerConn(0)
Expand Down
15 changes: 9 additions & 6 deletions pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -479,6 +479,8 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) {

skipSizeCheck := cfg.TestingKnobs.Store != nil &&
cfg.TestingKnobs.Store.(*kvserver.StoreTestingKnobs).SkipMinSizeCheck
disableSeparatedIntents := cfg.TestingKnobs.Store != nil &&
cfg.TestingKnobs.Store.(*kvserver.StoreTestingKnobs).StorageKnobs.DisableSeparatedIntents
for i, spec := range cfg.Stores.Specs {
log.Eventf(ctx, "initializing %+v", spec)
var sizeInBytes = spec.Size.InBytes
Expand Down Expand Up @@ -542,12 +544,13 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) {
i, humanizeutil.IBytes(sizeInBytes), openFileLimitPerStore))

storageConfig := base.StorageConfig{
Attrs: spec.Attributes,
Dir: spec.Path,
MaxSize: sizeInBytes,
Settings: cfg.Settings,
UseFileRegistry: spec.UseFileRegistry,
EncryptionOptions: spec.EncryptionOptions,
Attrs: spec.Attributes,
Dir: spec.Path,
MaxSize: sizeInBytes,
Settings: cfg.Settings,
UseFileRegistry: spec.UseFileRegistry,
DisableSeparatedIntents: disableSeparatedIntents,
EncryptionOptions: spec.EncryptionOptions,
}
pebbleConfig := storage.PebbleConfig{
StorageConfig: storageConfig,
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/settings_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ func TestCachedSettingsStoreAndLoad(t *testing.T) {
ctx := context.Background()
engine, err := storage.Open(ctx, storage.InMemory(),
storage.MaxSize(512<<20 /* 512 MiB */),
storage.SettingsForTesting())
storage.ForTesting)
require.NoError(t, err)
defer engine.Close()

Expand Down
1 change: 0 additions & 1 deletion pkg/server/settingswatcher/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,6 @@ go_test(
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/sql",
"//pkg/storage",
"//pkg/testutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
Expand Down
4 changes: 0 additions & 4 deletions pkg/server/settingswatcher/settings_watcher_external_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
Expand Down Expand Up @@ -93,9 +92,6 @@ func TestSettingWatcher(t *testing.T) {
s0.ExecutorConfig().(sql.ExecutorConfig).RangeFeedFactory,
tc.Stopper())
require.NoError(t, sw.Start(ctx))
// TestCluster randomizes the value of SeparatedIntentsEnabled, so set it to
// the same as in fakeSettings for the subsequent equality check.
storage.SeparatedIntentsEnabled.Override(ctx, &s0.ClusterSettings().SV, storage.SeparatedIntentsEnabled.Get(&fakeSettings.SV))
require.NoError(t, checkSettingsValuesMatch(s0.ClusterSettings(), fakeSettings))
for k, v := range toSet {
tdb.Exec(t, "SET CLUSTER SETTING "+k+" = $1", v[1])
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/sticky_engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ func (registry *stickyInMemEnginesRegistryImpl) GetOrCreateStickyInMemEngine(
storage.Attributes(spec.Attributes),
storage.CacheSize(cfg.CacheSize),
storage.MaxSize(spec.Size.InBytes),
storage.SettingsForTesting())
storage.ForTesting)

engineEntry := &stickyInMemEngine{
id: spec.StickyInMemoryEngineID,
Expand Down
6 changes: 0 additions & 6 deletions pkg/server/testserver.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (
"context"
"encoding/base64"
"fmt"
"math/rand"
"net/http"
"net/http/cookiejar"
"net/url"
Expand Down Expand Up @@ -56,7 +55,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/ts"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/retry"
Expand Down Expand Up @@ -129,10 +127,6 @@ func makeTestConfigFromParams(params base.TestServerArgs) Config {
st := params.Settings
if params.Settings == nil {
st = cluster.MakeClusterSettings()
enabledSeparated := rand.Intn(2) == 0
log.Infof(context.Background(),
"test Config is randomly setting enabledSeparated: %t", enabledSeparated)
storage.SeparatedIntentsEnabled.Override(context.Background(), &st.SV, enabledSeparated)
}
st.ExternalIODir = params.ExternalIODir
cfg := makeTestConfig(st)
Expand Down
1 change: 1 addition & 0 deletions pkg/storage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ go_library(
"sst_writer.go",
"temp_dir.go",
"temp_engine.go",
"testing_knobs.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/storage",
visibility = ["//visibility:public"],
Expand Down
11 changes: 6 additions & 5 deletions pkg/storage/bench_pebble_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,9 +31,7 @@ func setupMVCCPebble(b testing.TB, dir string) Engine {
peb, err := Open(
context.Background(),
Filesystem(dir),
CacheSize(testCacheSize),
Settings(makeSettingsForSeparatedIntents(
false /* oldClusterVersion */, true /* enabled */)))
CacheSize(testCacheSize))
if err != nil {
b.Fatalf("could not create new pebble instance at %s: %+v", dir, err)
}
Expand All @@ -42,14 +40,17 @@ func setupMVCCPebble(b testing.TB, dir string) Engine {

func setupMVCCInMemPebble(b testing.TB, loc string) Engine {
return setupMVCCInMemPebbleWithSettings(b, makeSettingsForSeparatedIntents(
false /* oldClusterVersion */, true /* enabled */))
false /* oldClusterVersion */), false /* disabledSeparatedIntents */)
}

func setupMVCCInMemPebbleWithSettings(b testing.TB, settings *cluster.Settings) Engine {
func setupMVCCInMemPebbleWithSettings(
b testing.TB, settings *cluster.Settings, disableSeparatedIntents bool,
) Engine {
peb, err := Open(
context.Background(),
InMemory(),
Settings(settings),
SetSeparatedIntents(false, disableSeparatedIntents),
CacheSize(testCacheSize))
if err != nil {
b.Fatalf("could not create new in-mem pebble instance: %+v", err)
Expand Down
10 changes: 5 additions & 5 deletions pkg/storage/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -213,7 +213,7 @@ func BenchmarkIntentScan(b *testing.B) {
for _, percentFlushed := range []int{0, 50, 80, 90, 100} {
b.Run(fmt.Sprintf("percent-flushed=%d", percentFlushed), func(b *testing.B) {
eng := setupMVCCInMemPebbleWithSettings(
b, makeSettingsForSeparatedIntents(false, sep))
b, makeSettingsForSeparatedIntents(false), sep)
numFlushedVersions := (percentFlushed * numVersions) / 100
setupKeysWithIntent(b, eng, numVersions, numFlushedVersions, false /* resolveAll */)
lower := makeKey(nil, 0)
Expand Down Expand Up @@ -277,7 +277,7 @@ func BenchmarkScanAllIntentsResolved(b *testing.B) {
for _, percentFlushed := range []int{0, 50, 90, 100} {
b.Run(fmt.Sprintf("percent-flushed=%d", percentFlushed), func(b *testing.B) {
eng := setupMVCCInMemPebbleWithSettings(
b, makeSettingsForSeparatedIntents(false, sep))
b, makeSettingsForSeparatedIntents(false), sep)
numFlushedVersions := (percentFlushed * numVersions) / 100
setupKeysWithIntent(b, eng, numVersions, numFlushedVersions, true /* resolveAll */)
lower := makeKey(nil, 0)
Expand Down Expand Up @@ -348,7 +348,7 @@ func BenchmarkScanOneAllIntentsResolved(b *testing.B) {
for _, percentFlushed := range []int{0, 50, 90, 100} {
b.Run(fmt.Sprintf("percent-flushed=%d", percentFlushed), func(b *testing.B) {
eng := setupMVCCInMemPebbleWithSettings(
b, makeSettingsForSeparatedIntents(false, sep))
b, makeSettingsForSeparatedIntents(false), sep)
numFlushedVersions := (percentFlushed * numVersions) / 100
setupKeysWithIntent(b, eng, numVersions, numFlushedVersions, true /* resolveAll */)
lower := makeKey(nil, 0)
Expand Down Expand Up @@ -400,7 +400,7 @@ func BenchmarkIntentResolution(b *testing.B) {
for _, percentFlushed := range []int{0, 50, 80, 90, 100} {
b.Run(fmt.Sprintf("percent-flushed=%d", percentFlushed), func(b *testing.B) {
eng := setupMVCCInMemPebbleWithSettings(
b, makeSettingsForSeparatedIntents(false, sep))
b, makeSettingsForSeparatedIntents(false), sep)
numFlushedVersions := (percentFlushed * numVersions) / 100
lockUpdate := setupKeysWithIntent(b, eng, numVersions, numFlushedVersions, false /* resolveAll */)
keys := make([]roachpb.Key, numIntentKeys)
Expand Down Expand Up @@ -444,7 +444,7 @@ func BenchmarkIntentRangeResolution(b *testing.B) {
for _, percentFlushed := range []int{0, 50, 80, 90, 100} {
b.Run(fmt.Sprintf("percent-flushed=%d", percentFlushed), func(b *testing.B) {
eng := setupMVCCInMemPebbleWithSettings(
b, makeSettingsForSeparatedIntents(false, sep))
b, makeSettingsForSeparatedIntents(false), sep)
numFlushedVersions := (percentFlushed * numVersions) / 100
lockUpdate := setupKeysWithIntent(b, eng, numVersions, numFlushedVersions, false /* resolveAll */)
keys := make([]roachpb.Key, numIntentKeys+1)
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/engine_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1639,7 +1639,7 @@ func TestScanSeparatedIntents(t *testing.T) {
for name, enableSeparatedIntents := range map[string]bool{"interleaved": false, "separated": true} {
t.Run(name, func(t *testing.T) {
eng, err := Open(ctx, InMemory(), CacheSize(1<<20 /* 1 MiB */),
Settings(makeSettingsForSeparatedIntents(false, enableSeparatedIntents)))
SetSeparatedIntents(false, !enableSeparatedIntents))
require.NoError(t, err)
defer eng.Close()

Expand Down
19 changes: 2 additions & 17 deletions pkg/storage/in_mem.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,11 +12,9 @@ package storage

import (
"context"
"math/rand"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/pebble/vfs"
)

Expand Down Expand Up @@ -50,31 +48,18 @@ func InMemFromFS(ctx context.Context, fs vfs.FS, dir string, opts ...ConfigOptio
// the default configuration. The caller must call the engine's Close method
// when the engine is no longer needed.
func NewDefaultInMemForTesting() Engine {
eng, err := Open(context.Background(), InMemory(), SettingsForTesting(), MaxSize(1<<20))
eng, err := Open(context.Background(), InMemory(), ForTesting, MaxSize(1<<20))
if err != nil {
panic(err)
}
return eng
}

// makeRandomSettingsForSeparatedIntents makes settings for which it randomly
// picks whether the cluster understands separated intents, and if yes,
// whether to write separated intents. Once made, these setting do not change.
func makeRandomSettingsForSeparatedIntents() *cluster.Settings {
oldClusterVersion := rand.Intn(2) == 0
enabledSeparated := rand.Intn(2) == 0
log.Infof(context.Background(),
"engine creation is randomly setting oldClusterVersion: %t, enabledSeparated: %t",
oldClusterVersion, enabledSeparated)
return makeSettingsForSeparatedIntents(oldClusterVersion, enabledSeparated)
}

func makeSettingsForSeparatedIntents(oldClusterVersion bool, enabled bool) *cluster.Settings {
func makeSettingsForSeparatedIntents(oldClusterVersion bool) *cluster.Settings {
version := clusterversion.ByKey(clusterversion.SeparatedIntents)
if oldClusterVersion {
version = clusterversion.ByKey(clusterversion.V20_2)
}
settings := cluster.MakeTestingClusterSettingsWithVersions(version, version, true)
SeparatedIntentsEnabled.Override(context.TODO(), &settings.SV, enabled)
return settings
}
Loading

0 comments on commit 4eb9d28

Please sign in to comment.