diff --git a/pkg/ccl/migrationccl/migrationsccl/BUILD.bazel b/pkg/ccl/migrationccl/migrationsccl/BUILD.bazel index 6ee91ba7301c..103055c0cf96 100644 --- a/pkg/ccl/migrationccl/migrationsccl/BUILD.bazel +++ b/pkg/ccl/migrationccl/migrationsccl/BUILD.bazel @@ -4,12 +4,10 @@ go_test( name = "migrationsccl_test", srcs = [ "main_test.go", - "records_based_registry_external_test.go", "seed_tenant_span_configs_external_test.go", ], deps = [ "//pkg/base", - "//pkg/ccl/baseccl", "//pkg/ccl/kvccl/kvtenantccl", "//pkg/clusterversion", "//pkg/keys", @@ -23,7 +21,6 @@ go_test( "//pkg/testutils/testcluster", "//pkg/util/leaktest", "//pkg/util/log", - "//pkg/util/protoutil", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/ccl/migrationccl/migrationsccl/records_based_registry_external_test.go b/pkg/ccl/migrationccl/migrationsccl/records_based_registry_external_test.go deleted file mode 100644 index 708e1a1ed769..000000000000 --- a/pkg/ccl/migrationccl/migrationsccl/records_based_registry_external_test.go +++ /dev/null @@ -1,145 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Licensed as a CockroachDB Enterprise file under the Cockroach Community -// License (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt - -package migrationsccl_test - -import ( - "context" - "testing" - - "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/ccl/baseccl" - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/server" - "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" - "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/protoutil" - "github.com/stretchr/testify/require" -) - -func TestRecordsBasedRegistryMigration(t *testing.T) { - defer leaktest.AfterTest(t)() - - const ( - stickyEngineId = "engine1" - key128 = "111111111111111111111111111111111234567890123456" - keyFile128 = "a.key" - ) - - type encryptionTiming bool - const ( - beforeMigration encryptionTiming = false - afterMigration encryptionTiming = true - ) - - testCases := []struct { - name string - encryptionStartTime encryptionTiming - }{ - { - name: "enable-encryption-before-migration", - encryptionStartTime: beforeMigration, - }, - { - name: "enable-encryption-after-migration", - encryptionStartTime: afterMigration, - }, - } - - for _, c := range testCases { - ctx := context.Background() - registry := server.NewStickyInMemEnginesRegistry() - - storeSpec := base.DefaultTestStoreSpec - storeSpec.StickyInMemoryEngineID = stickyEngineId - - clusterArgs := base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - Knobs: base.TestingKnobs{ - Server: &server.TestingKnobs{ - DisableAutomaticVersionUpgrade: 1, - BinaryVersionOverride: clusterversion.ByKey(clusterversion.RecordsBasedRegistry - 1), - StickyEngineRegistry: registry, - }, - }, - StoreSpecs: []base.StoreSpec{storeSpec}, - }, - } - - // Start and stop the cluster in order to create a sticky engine. - tc := testcluster.StartTestCluster(t, 1 /* nodes */, clusterArgs) - if c.encryptionStartTime == afterMigration { - tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - tdb.Exec(t, `SET CLUSTER SETTING version = $1`, - clusterversion.ByKey(clusterversion.RecordsBasedRegistry).String()) - } - tc.Stopper().Stop(ctx) - - // Write the keyfile to the sticky engine's underlying FS. - memFS, err := registry.GetUnderlyingFS(storeSpec) - require.NoError(t, err) - f, err := memFS.Create(keyFile128) - require.NoError(t, err) - _, err = f.Write([]byte(key128)) - require.NoError(t, err) - - // Add the keyfile details to the store spec. - encOpts := &baseccl.EncryptionOptions{ - KeySource: baseccl.EncryptionKeySource_KeyFiles, - KeyFiles: &baseccl.EncryptionKeyFiles{ - OldKey: "plain", - CurrentKey: keyFile128, - }, - DataKeyRotationPeriod: 1000, - } - b, err := protoutil.Marshal(encOpts) - require.NoError(t, err) - storeSpec.EncryptionOptions = b - storeSpec.UseFileRegistry = true - - // Restart the cluster with encryption-at-rest turned on. - clusterArgs.ServerArgs.StoreSpecs = []base.StoreSpec{storeSpec} - tc = testcluster.StartTestCluster(t, 1 /* nodes */, clusterArgs) - - // Create a SQL table, write some rows, and start migration. - tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - tdb.Exec(t, `CREATE TABLE test (id int, abc string)`) - tdb.Exec(t, `INSERT INTO test ( VALUES (1, 'hi'), (2, 'bye'))`) - if c.encryptionStartTime == beforeMigration { - tdb.Exec(t, `SET CLUSTER SETTING version = $1`, - clusterversion.ByKey(clusterversion.RecordsBasedRegistry).String()) - } - - // Check that all engines are using the new records based registry and verify - // that we can still retrieve the rows from the table. - svr := tc.Server(0) - for _, eng := range svr.Engines() { - target := clusterversion.ByKey(clusterversion.RecordsBasedRegistry) - ok, err := eng.MinVersionIsAtLeastTargetVersion(target) - require.NoError(t, err) - require.True(t, ok) - ok, err = eng.UsingRecordsEncryptionRegistry() - require.NoError(t, err) - require.True(t, ok) - } - tdb.Exec(t, `SELECT * FROM test`) - tc.Stopper().Stop(ctx) - - // Restart the cluster to ensure there are no problems with loading the new - // records based registries. - tc = testcluster.StartTestCluster(t, 1 /* nodes */, clusterArgs) - tdb = sqlutils.MakeSQLRunner(tc.ServerConn(0)) - tdb.Exec(t, `SELECT * FROM test`) - tc.Stopper().Stop(ctx) - - // NB: We cannot defer this without leaking resources since a new sticky - // engine registry is created in each loop iteration. - registry.CloseAllStickyInMemEngines() - } -} diff --git a/pkg/ccl/storageccl/engineccl/BUILD.bazel b/pkg/ccl/storageccl/engineccl/BUILD.bazel index 2e5df92d868f..d68cb8ed0b6c 100644 --- a/pkg/ccl/storageccl/engineccl/BUILD.bazel +++ b/pkg/ccl/storageccl/engineccl/BUILD.bazel @@ -15,7 +15,6 @@ go_library( "//pkg/ccl/storageccl/engineccl/enginepbccl:enginepbccl_go_proto", "//pkg/storage", "//pkg/storage/enginepb", - "//pkg/storage/fs", "//pkg/util/log", "//pkg/util/protoutil", "//pkg/util/syncutil", @@ -60,6 +59,7 @@ go_test( "@com_github_cockroachdb_errors//oserror", "@com_github_cockroachdb_pebble//:pebble", "@com_github_cockroachdb_pebble//vfs", + "@com_github_cockroachdb_pebble//vfs/atomicfs", "@com_github_gogo_protobuf//proto", "@com_github_kr_pretty//:pretty", "@com_github_stretchr_testify//require", diff --git a/pkg/ccl/storageccl/engineccl/encrypted_fs.go b/pkg/ccl/storageccl/engineccl/encrypted_fs.go index c90977a5e676..dfd4a0f91558 100644 --- a/pkg/ccl/storageccl/engineccl/encrypted_fs.go +++ b/pkg/ccl/storageccl/engineccl/encrypted_fs.go @@ -364,7 +364,6 @@ func newEncryptedEnv( storeKM: storeKeyManager, dataKM: dataKeyManager, }, - UpgradeVersion: dataKeyManager.UseMarker, }, nil } diff --git a/pkg/ccl/storageccl/engineccl/pebble_key_manager.go b/pkg/ccl/storageccl/engineccl/pebble_key_manager.go index 5340f09e20c5..efedf16fcced 100644 --- a/pkg/ccl/storageccl/engineccl/pebble_key_manager.go +++ b/pkg/ccl/storageccl/engineccl/pebble_key_manager.go @@ -17,7 +17,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/ccl/storageccl/engineccl/enginepbccl" - "github.com/cockroachdb/cockroach/pkg/storage/fs" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -178,12 +177,6 @@ type DataKeyManager struct { // Transitions to true when SetActiveStoreKeyInfo() is called for the // first time. rotationEnabled bool - // useMarker indicates whether or not the data key registry - // should write new files and use an atomic marker to mark the - // active file. This is set to true once the 21.2 version is - // finalized. - // TODO(jackson): Remove after v21.2. - useMarker bool // marker is an atomic file marker used to denote which of the // data keys registry files is the current one. When we rotate // files, the marker is atomically moved to the new file. It's @@ -214,23 +207,10 @@ func (m *DataKeyManager) Load(ctx context.Context) error { if err != nil { return err } - useMarker := filename != "" - - // If the marker doesn't exist, filename is the empty string. In - // this case, we fall back to looking for the file at the fixed - // `DATA_KEYS_REGISTRY` path. - if filename == "" { - filename = keyRegistryFilename - _, err = m.fs.Stat(m.fs.PathJoin(m.dbDir, filename)) - if err != nil && !oserror.IsNotExist(err) { - return err - } - } m.mu.Lock() defer m.mu.Unlock() m.mu.marker = marker - m.mu.useMarker = useMarker if oserror.IsNotExist(err) { // First run. m.mu.keyRegistry = makeRegistryProto() @@ -239,21 +219,23 @@ func (m *DataKeyManager) Load(ctx context.Context) error { // Load the existing state from the file named by `filename`. m.mu.filename = filename - f, err := m.fs.Open(m.fs.PathJoin(m.dbDir, filename)) - if err != nil { - return err - } - defer f.Close() - b, err := ioutil.ReadAll(f) - if err != nil { - return err - } m.mu.keyRegistry = makeRegistryProto() - if err = protoutil.Unmarshal(b, m.mu.keyRegistry); err != nil { - return err - } - if err = validateRegistry(m.mu.keyRegistry); err != nil { - return err + if filename != "" { + f, err := m.fs.Open(m.fs.PathJoin(m.dbDir, filename)) + if err != nil { + return err + } + defer f.Close() + b, err := ioutil.ReadAll(f) + if err != nil { + return err + } + if err = protoutil.Unmarshal(b, m.mu.keyRegistry); err != nil { + return err + } + if err = validateRegistry(m.mu.keyRegistry); err != nil { + return err + } } if m.mu.keyRegistry.ActiveDataKeyId != "" { key, found := m.mu.keyRegistry.DataKeys[m.mu.keyRegistry.ActiveDataKeyId] @@ -269,31 +251,6 @@ func (m *DataKeyManager) Load(ctx context.Context) error { return nil } -// UseMarker informs the data key manager that it should begin using the -// marker file to denote which file is active. -// -// TODO(jackson): Remove this in 22.1. In 22.1 we can unconditionally -// use the marker file. -func (m *DataKeyManager) UseMarker() error { - if m.readOnly { - return nil - } - - m.mu.Lock() - defer m.mu.Unlock() - m.mu.useMarker = true - - // If there is no filename, the data keys registry has never been - // written. There's no file to mark yet. The first rotation will set - // the marker. - if m.mu.filename == "" { - return nil - } - // NB: This may move the marker to mark the file with the previous - // static filename "COCKROACHDB_DATA_KEYS". - return m.mu.marker.Move(m.mu.filename) -} - // ActiveKey implements PebbleKeyManager.ActiveKey. // // TODO(sbhola): do rotation via a background activity instead of in this function so that we don't @@ -473,26 +430,6 @@ func (m *DataKeyManager) rotateDataKeyAndWrite( return err } - if !m.mu.useMarker { - // If the v21.2 version hasn't been finalized yet, write the - // registry to the static filename `COCKROACHDB_DATA_KEYS`. If - // there's a crash mid-Rename, it's possible that we'll be left - // with a corrupt data key registry. - // TODO(jackson): Remove this for 22.1. - path := m.fs.PathJoin(m.dbDir, keyRegistryFilename) - if err = fs.SafeWriteToFile(m.fs, m.dbDir, path, bytes); err != nil { - return - } - m.mu.filename = keyRegistryFilename - m.mu.keyRegistry = keyRegistry - m.mu.activeKey = newKey - return err - } - - // The v21.2 version has been finalized. Write a new file - // containing the updated state, and move the atomic marker to - // point to the new file. - // Write the current registry state to a new file and sync it. // The new file's filename incorporates the marker's iteration // number to ensure we're not overwriting the existing registry. diff --git a/pkg/ccl/storageccl/engineccl/pebble_key_manager_test.go b/pkg/ccl/storageccl/engineccl/pebble_key_manager_test.go index d710b0a1e26a..7b4a6a5f741b 100644 --- a/pkg/ccl/storageccl/engineccl/pebble_key_manager_test.go +++ b/pkg/ccl/storageccl/engineccl/pebble_key_manager_test.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/datadriven" "github.com/cockroachdb/pebble/vfs" + "github.com/cockroachdb/pebble/vfs/atomicfs" "github.com/gogo/protobuf/proto" "github.com/stretchr/testify/require" ) @@ -290,6 +291,10 @@ func TestDataKeyManager(t *testing.T) { return err.Error() } writeToFile(t, memFS, memFS.PathJoin(data[0], keyRegistryFilename), b) + marker, _, err := atomicfs.LocateMarker(memFS, data[0], keysRegistryMarkerName) + require.NoError(t, err) + require.NoError(t, marker.Move(keyRegistryFilename)) + require.NoError(t, marker.Close()) } return "" case "load": @@ -446,9 +451,6 @@ func TestDataKeyManagerIO(t *testing.T) { d.ScanArgs(t, "id", &id) fmt.Fprintf(&buf, "%s", setActiveStoreKey(dkm, id, enginepbccl.EncryptionType_AES128_CTR)) return buf.String() - case "use-marker": - appendError(dkm.UseMarker()) - return buf.String() default: return fmt.Sprintf("unknown command: %s\n", d.Cmd) } diff --git a/pkg/ccl/storageccl/engineccl/testdata/data_key_manager_io b/pkg/ccl/storageccl/engineccl/testdata/data_key_manager_io index acc2bdc7f657..9e64db9b77e5 100644 --- a/pkg/ccl/storageccl/engineccl/testdata/data_key_manager_io +++ b/pkg/ccl/storageccl/engineccl/testdata/data_key_manager_io @@ -10,19 +10,10 @@ OK load dir=data-dir ---- open-dir("data-dir") -stat("data-dir/COCKROACHDB_DATA_KEYS") -OK - -# Since there's no data keys registry file yet, use-marker should be a -# noop. - -use-marker ----- OK # Setting the active store key should trigger initializing the registry -# on disk. Since use-marker has already been called, the registry should -# be initialized with a marker file. +# on disk. set-active-store-key id=foo ---- @@ -98,32 +89,17 @@ OK load dir=data-dir ---- open-dir("data-dir") -stat("data-dir/COCKROACHDB_DATA_KEYS") OK -# Since use-marker has not yet been called, the registry shouldn't use -# it. Instead, it should replace the static COCKROACHDB_DATA_KEYS file. - set-active-store-key id=foo ---- -create("data-dir/COCKROACHDB_DATA_KEYS.crdbtmp") -write("data-dir/COCKROACHDB_DATA_KEYS.crdbtmp", <...280 bytes...>) -sync("data-dir/COCKROACHDB_DATA_KEYS.crdbtmp") -close("data-dir/COCKROACHDB_DATA_KEYS.crdbtmp") -rename("data-dir/COCKROACHDB_DATA_KEYS.crdbtmp", "data-dir/COCKROACHDB_DATA_KEYS") -open-dir("data-dir") -sync("data-dir") -close("data-dir") - -# Calling use-marker should lay down a marker pointing to the existing -# file. - -use-marker ----- -create("data-dir/marker.datakeys.000001.COCKROACHDB_DATA_KEYS") -close("data-dir/marker.datakeys.000001.COCKROACHDB_DATA_KEYS") +create("data-dir/COCKROACHDB_DATA_KEYS_000001_monolith") +write("data-dir/COCKROACHDB_DATA_KEYS_000001_monolith", <...280 bytes...>) +sync("data-dir/COCKROACHDB_DATA_KEYS_000001_monolith") +create("data-dir/marker.datakeys.000001.COCKROACHDB_DATA_KEYS_000001_monolith") +close("data-dir/marker.datakeys.000001.COCKROACHDB_DATA_KEYS_000001_monolith") sync("data-dir") -OK +close("data-dir/COCKROACHDB_DATA_KEYS_000001_monolith") set-active-store-key id=bar ---- @@ -132,9 +108,9 @@ write("data-dir/COCKROACHDB_DATA_KEYS_000002_monolith", <...489 bytes...>) sync("data-dir/COCKROACHDB_DATA_KEYS_000002_monolith") create("data-dir/marker.datakeys.000002.COCKROACHDB_DATA_KEYS_000002_monolith") close("data-dir/marker.datakeys.000002.COCKROACHDB_DATA_KEYS_000002_monolith") -remove("data-dir/marker.datakeys.000001.COCKROACHDB_DATA_KEYS") +remove("data-dir/marker.datakeys.000001.COCKROACHDB_DATA_KEYS_000001_monolith") sync("data-dir") -remove("data-dir/COCKROACHDB_DATA_KEYS") +remove("data-dir/COCKROACHDB_DATA_KEYS_000001_monolith") close("data-dir/COCKROACHDB_DATA_KEYS_000002_monolith") close diff --git a/pkg/migration/migrations/BUILD.bazel b/pkg/migration/migrations/BUILD.bazel index 936dc7d74121..4c9c27ca098c 100644 --- a/pkg/migration/migrations/BUILD.bazel +++ b/pkg/migration/migrations/BUILD.bazel @@ -13,7 +13,6 @@ go_library( "insert_missing_public_schema_namespace_entry.go", "join_tokens.go", "migrations.go", - "records_based_registry.go", "retry_jobs_with_exponential_backoff.go", "schema_changes.go", "seed_tenant_span_configs.go", @@ -39,7 +38,6 @@ go_library( "//pkg/migration", "//pkg/roachpb:with-mocks", "//pkg/security", - "//pkg/server/serverpb", "//pkg/sql/catalog", "//pkg/sql/catalog/bootstrap", "//pkg/sql/catalog/catalogkeys", diff --git a/pkg/migration/migrations/migrations.go b/pkg/migration/migrations/migrations.go index c294903ee2b2..c5e62e385f5f 100644 --- a/pkg/migration/migrations/migrations.go +++ b/pkg/migration/migrations/migrations.go @@ -40,11 +40,6 @@ func NoPrecondition(context.Context, clusterversion.ClusterVersion, migration.Te var registry = make(map[clusterversion.ClusterVersion]migration.Migration) var migrations = []migration.Migration{ - migration.NewSystemMigration( - "stop using monolithic encryption-at-rest registry for all stores", - toCV(clusterversion.RecordsBasedRegistry), - recordsBasedRegistryMigration, - ), migration.NewTenantMigration( "add the systems.join_tokens table", toCV(clusterversion.JoinTokensTable), diff --git a/pkg/migration/migrations/records_based_registry.go b/pkg/migration/migrations/records_based_registry.go deleted file mode 100644 index 08c593669ede..000000000000 --- a/pkg/migration/migrations/records_based_registry.go +++ /dev/null @@ -1,30 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package migrations - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/jobs" - "github.com/cockroachdb/cockroach/pkg/migration" - "github.com/cockroachdb/cockroach/pkg/server/serverpb" -) - -func recordsBasedRegistryMigration( - ctx context.Context, cv clusterversion.ClusterVersion, deps migration.SystemDeps, _ *jobs.Job, -) error { - return deps.Cluster.ForEveryNode(ctx, "deprecate-base-encryption-registry", func(ctx context.Context, client serverpb.MigrationClient) error { - req := &serverpb.DeprecateBaseEncryptionRegistryRequest{Version: &cv.Version} - _, err := client.DeprecateBaseEncryptionRegistry(ctx, req) - return err - }) -} diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 6bd25981a687..5198a553ab36 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -765,10 +765,6 @@ type Engine interface { // version that it must maintain compatibility with. SetMinVersion(version roachpb.Version) error - // UsingRecordsEncryptionRegistry returns whether the engine is using the - // Records version incremental encryption-at-rest registry. - UsingRecordsEncryptionRegistry() (bool, error) - // MinVersionIsAtLeastTargetVersion returns whether the engine's recorded // storage min version is at least the target version. MinVersionIsAtLeastTargetVersion(target roachpb.Version) (bool, error) diff --git a/pkg/storage/enginepb/file_registry.go b/pkg/storage/enginepb/file_registry.go index e371cc5158f0..53a075a491e0 100644 --- a/pkg/storage/enginepb/file_registry.go +++ b/pkg/storage/enginepb/file_registry.go @@ -10,34 +10,6 @@ package enginepb -// ProcessBatch processes a batch of updates to the file registry. -func (r *FileRegistry) ProcessBatch(batch *RegistryUpdateBatch) { - for _, update := range batch.Updates { - r.ProcessUpdate(update) - } -} - -// ProcessUpdate processes a single update to the file registry. -func (r *FileRegistry) ProcessUpdate(update *RegistryUpdate) { - if update.Entry == nil { - delete(r.Files, update.Filename) - } else { - if r.Files == nil { - r.Files = make(map[string]*FileEntry) - } - r.Files[update.Filename] = update.Entry - } -} - -// SetVersion updates the version of the file registry. This function will -// panic if the provided version is lower than the current version. -func (r *FileRegistry) SetVersion(version RegistryVersion) { - if version < r.Version { - panic("illegal downgrade of file registry version") - } - r.Version = version -} - // Empty returns whether a batch is empty. func (b *RegistryUpdateBatch) Empty() bool { return len(b.Updates) == 0 diff --git a/pkg/storage/min_version_test.go b/pkg/storage/min_version_test.go index a559a7b80464..23d26e0af840 100644 --- a/pkg/storage/min_version_test.go +++ b/pkg/storage/min_version_test.go @@ -176,9 +176,8 @@ func fauxNewEncryptedEnvFunc( fs vfs.FS, fr *PebbleFileRegistry, dbDir string, readOnly bool, optionBytes []byte, ) (*EncryptionEnv, error) { return &EncryptionEnv{ - Closer: nopCloser{}, - FS: fauxEncryptedFS{FS: fs}, - UpgradeVersion: func() error { return nil }, + Closer: nopCloser{}, + FS: fauxEncryptedFS{FS: fs}, }, nil } diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index fc7f1b598a64..0ca52e07e21e 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -611,13 +611,6 @@ type EncryptionEnv struct { FS vfs.FS // StatsHandler exposes encryption-at-rest state for observability. StatsHandler EncryptionStatsHandler - // UpgradeVersion is a temporary field that allows Pebble to inform - // low-level encryption-at-rest machinery that the CockroachDB 21.2 - // version has been finalized, and it's okay to begin writing in a - // backwards in-compatible format. - // - // TODO(jackson): Remove this in 22.1. - UpgradeVersion func() error } var _ Engine = &Pebble{} @@ -1575,34 +1568,9 @@ func (p *Pebble) SetMinVersion(version roachpb.Version) error { return errors.Wrap(err, "ratcheting format major version") } } - - if p.fileRegistry != nil { - recordsRegistryCV := clusterversion.ByKey(clusterversion.RecordsBasedRegistry) - if !version.Less(recordsRegistryCV) { - if err := p.fileRegistry.StopUsingOldRegistry(); err != nil { - return err - } - } - } - if p.encryption != nil { - markerDataKeysRegistryCV := clusterversion.ByKey(clusterversion.MarkerDataKeysRegistry) - if !version.Less(markerDataKeysRegistryCV) { - if err := p.encryption.UpgradeVersion(); err != nil { - return err - } - } - } return nil } -// UsingRecordsEncryptionRegistry implements the Engine interface. -func (p *Pebble) UsingRecordsEncryptionRegistry() (bool, error) { - if p.fileRegistry != nil { - return p.fileRegistry.UpgradedToRecordsVersion(), nil - } - return true, nil -} - // MinVersionIsAtLeastTargetVersion implements the Engine interface. func (p *Pebble) MinVersionIsAtLeastTargetVersion(target roachpb.Version) (bool, error) { return MinVersionIsAtLeastTargetVersion(p.unencryptedFS, p.path, target) diff --git a/pkg/storage/pebble_file_registry.go b/pkg/storage/pebble_file_registry.go index 82fcf98071fa..239168279a0d 100644 --- a/pkg/storage/pebble_file_registry.go +++ b/pkg/storage/pebble_file_registry.go @@ -19,9 +19,7 @@ import ( "sort" "strings" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/storage/fs" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" @@ -29,7 +27,6 @@ import ( "github.com/cockroachdb/pebble/record" "github.com/cockroachdb/pebble/vfs" "github.com/cockroachdb/pebble/vfs/atomicfs" - "github.com/gogo/protobuf/proto" ) // CanRegistryElideFunc is a function that returns true for entries that can be @@ -66,15 +63,11 @@ type PebbleFileRegistry struct { ReadOnly bool // Implementation. - // TODO(ayang): remove oldRegistryPath when we deprecate the old registry - oldRegistryPath string mu struct { syncutil.Mutex - // currProto stores the current state of the file registry. - // TODO(ayang): convert enginepb.FileRegistry to a regular struct - // when we deprecate the old registry and rename currProto - currProto *enginepb.FileRegistry + // entries stores the current state of the file registry. + entries map[string]*enginepb.FileEntry // registryFile is the opened file for the records-based registry. registryFile vfs.File // registryWriter is a record.Writer for registryFile. @@ -92,37 +85,13 @@ type PebbleFileRegistry struct { } const ( - // TODO(ayang): mark COCKROACHDB_REGISTRY as deprecated so it isn't reused - registryFilename = "COCKROACHDB_REGISTRY" - registryMarkerName = "registry" + registryFilenameBase = "COCKROACHDB_REGISTRY" + registryMarkerName = "registry" ) // CheckNoRegistryFile checks that no registry file currently exists. // CheckNoRegistryFile should be called if the file registry will not be used. func (r *PebbleFileRegistry) CheckNoRegistryFile() error { - if err := r.checkNoBaseRegistry(); err != nil { - return err - } - if err := r.checkNoRecordsRegistry(); err != nil { - return err - } - return nil -} - -func (r *PebbleFileRegistry) checkNoBaseRegistry() error { - // NB: We do not assign r.oldRegistryPath if the registry will not be used. - oldRegistryPath := r.FS.PathJoin(r.DBDir, registryFilename) - _, err := r.FS.Stat(oldRegistryPath) - if err == nil { - return os.ErrExist - } - if !oserror.IsNotExist(err) { - return err - } - return nil -} - -func (r *PebbleFileRegistry) checkNoRecordsRegistry() error { filename, err := atomicfs.ReadMarker(r.FS, r.DBDir, registryMarkerName) if oserror.IsNotExist(err) { // ReadMarker may return oserror.IsNotExist if the data @@ -145,8 +114,7 @@ func (r *PebbleFileRegistry) Load() error { defer r.mu.Unlock() // Initialize private fields needed when the file registry will be used. - r.oldRegistryPath = r.FS.PathJoin(r.DBDir, registryFilename) - r.mu.currProto = &enginepb.FileRegistry{} + r.mu.entries = make(map[string]*enginepb.FileEntry) if err := r.loadRegistryFromFile(); err != nil { return err @@ -182,61 +150,13 @@ func (r *PebbleFileRegistry) loadRegistryFromFile() error { } } - // We treat the old registry file as the source of truth until the version - // is finalized. At that point, we upgrade to the new records-based registry - // file and delete the old registry file. - ok, err := r.maybeLoadOldBaseRegistry() - if err != nil { - return err - } - if ok { - return nil - } - ok, err = r.maybeLoadNewRecordsRegistry() - if err != nil { + if _, err := r.maybeLoadExistingRegistry(); err != nil { return err } - if ok { - return nil - } - // If encryption-at-rest was not previously enabled, we check the storage min - // version to determine whether we still need to create an old base registry. - target := clusterversion.ByKey(clusterversion.RecordsBasedRegistry) - ok, err = MinVersionIsAtLeastTargetVersion(r.FS, r.DBDir, target) - if err != nil { - return err - } - if ok { - r.mu.currProto.SetVersion(enginepb.RegistryVersion_Records) - } return nil } -func (r *PebbleFileRegistry) maybeLoadOldBaseRegistry() (bool, error) { - f, err := r.FS.Open(r.oldRegistryPath) - if oserror.IsNotExist(err) { - return false, nil - } - if err != nil { - return false, err - } - var b []byte - if b, err = ioutil.ReadAll(f); err != nil { - return false, errors.CombineErrors(err, f.Close()) - } - if err := f.Close(); err != nil { - return false, err - } - if err := protoutil.Unmarshal(b, r.mu.currProto); err != nil { - return false, err - } - if r.mu.currProto.Version == enginepb.RegistryVersion_Records { - return false, errors.New("old encryption registry with version Records should not exist") - } - return true, nil -} - -func (r *PebbleFileRegistry) maybeLoadNewRecordsRegistry() (bool, error) { +func (r *PebbleFileRegistry) maybeLoadExistingRegistry() (bool, error) { if r.mu.registryFilename == "" { return false, nil } @@ -261,12 +181,11 @@ func (r *PebbleFileRegistry) maybeLoadNewRecordsRegistry() (bool, error) { if err := protoutil.Unmarshal(registryHeaderBytes, registryHeader); err != nil { return false, err } - // Since we only load the new registry if the old registry does not exist, - // we should never load a new registry that has version Base. + // All registries of the base version should've been removed in 21.2 before + // upgrade finalization. if registryHeader.Version == enginepb.RegistryVersion_Base { return false, errors.New("new encryption registry with version Base should not exist") } - r.mu.currProto.SetVersion(registryHeader.Version) for { rdr, err := rr.Next() if err == io.EOF { @@ -283,7 +202,7 @@ func (r *PebbleFileRegistry) maybeLoadNewRecordsRegistry() (bool, error) { if err := protoutil.Unmarshal(b, batch); err != nil { return false, err } - r.mu.currProto.ProcessBatch(batch) + r.applyBatch(batch) } if err := records.Close(); err != nil { return false, err @@ -291,23 +210,6 @@ func (r *PebbleFileRegistry) maybeLoadNewRecordsRegistry() (bool, error) { return true, nil } -// StopUsingOldRegistry is called to signal that the old file registry -// is no longer needed and can be safely deleted. -// TODO(ayang): delete this function when we deprecate the old registry -func (r *PebbleFileRegistry) StopUsingOldRegistry() error { - r.mu.Lock() - defer r.mu.Unlock() - return r.upgradeToRecordsVersion() -} - -// UpgradedToRecordsVersion returns whether the file registry has completed -// its upgrade to the Records version of the file registry. -func (r *PebbleFileRegistry) UpgradedToRecordsVersion() bool { - r.mu.Lock() - defer r.mu.Unlock() - return r.mu.currProto.Version == enginepb.RegistryVersion_Records -} - func (r *PebbleFileRegistry) maybeElideEntries() error { if r.ReadOnly { return nil @@ -321,15 +223,15 @@ func (r *PebbleFileRegistry) maybeElideEntries() error { // recursively List each directory and walk two lists of sorted // filenames. We should test a store with many files to see how much // the current approach slows node start. - filenames := make([]string, 0, len(r.mu.currProto.Files)) - for filename := range r.mu.currProto.Files { + filenames := make([]string, 0, len(r.mu.entries)) + for filename := range r.mu.entries { filenames = append(filenames, filename) } sort.Strings(filenames) batch := &enginepb.RegistryUpdateBatch{} for _, filename := range filenames { - entry := r.mu.currProto.Files[filename] + entry := r.mu.entries[filename] // Some entries may be elided. This is used within // ccl/storageccl/engineccl to elide plaintext file entries. @@ -360,7 +262,7 @@ func (r *PebbleFileRegistry) GetFileEntry(filename string) *enginepb.FileEntry { filename = r.tryMakeRelativePath(filename) r.mu.Lock() defer r.mu.Unlock() - return r.mu.currProto.Files[filename] + return r.mu.entries[filename] } // SetFileEntry sets filename => entry in the registry map and persists the registry. @@ -387,7 +289,7 @@ func (r *PebbleFileRegistry) MaybeDeleteEntry(filename string) error { r.mu.Lock() defer r.mu.Unlock() - if r.mu.currProto.Files[filename] == nil { + if r.mu.entries[filename] == nil { return nil } batch := &enginepb.RegistryUpdateBatch{} @@ -405,8 +307,8 @@ func (r *PebbleFileRegistry) MaybeCopyEntry(src, dst string) error { r.mu.Lock() defer r.mu.Unlock() - srcEntry := r.mu.currProto.Files[src] - if srcEntry == nil && r.mu.currProto.Files[dst] == nil { + srcEntry := r.mu.entries[src] + if srcEntry == nil && r.mu.entries[dst] == nil { return nil } batch := &enginepb.RegistryUpdateBatch{} @@ -426,14 +328,14 @@ func (r *PebbleFileRegistry) MaybeLinkEntry(src, dst string) error { r.mu.Lock() defer r.mu.Unlock() - if r.mu.currProto.Files[src] == nil && r.mu.currProto.Files[dst] == nil { + if r.mu.entries[src] == nil && r.mu.entries[dst] == nil { return nil } batch := &enginepb.RegistryUpdateBatch{} - if r.mu.currProto.Files[src] == nil { + if r.mu.entries[src] == nil { batch.DeleteEntry(dst) } else { - batch.PutEntry(dst, r.mu.currProto.Files[src]) + batch.PutEntry(dst, r.mu.entries[src]) } return r.processBatchLocked(batch) } @@ -468,36 +370,6 @@ func (r *PebbleFileRegistry) tryMakeRelativePath(filename string) string { return filename } -// TODO(ayang): delete this function when we deprecate the old registry -func (r *PebbleFileRegistry) upgradeToRecordsVersion() error { - if r.mu.currProto.Version == enginepb.RegistryVersion_Records { - return nil - } - - dir, err := r.FS.OpenDir(r.DBDir) - if err != nil { - return err - } - defer dir.Close() - - // Create a new registry file to record the upgraded version. - r.mu.currProto.SetVersion(enginepb.RegistryVersion_Records) - if err := r.createNewRegistryFile(); err != nil { - return err - } - if err := r.FS.Remove(r.oldRegistryPath); err != nil && !oserror.IsNotExist(err) { - return err - } - // We need to sync the removal of the old registry because the - // presence of the old registry will cause future starts to read - // from the old registry. - if err := dir.Sync(); err != nil { - // Fsync errors must be fatal. - panic(errors.Wrap(err, "syncing database directory")) - } - return nil -} - func (r *PebbleFileRegistry) processBatchLocked(batch *enginepb.RegistryUpdateBatch) error { if r.ReadOnly { return errors.New("cannot write file registry since db is read-only") @@ -505,33 +377,25 @@ func (r *PebbleFileRegistry) processBatchLocked(batch *enginepb.RegistryUpdateBa if batch.Empty() { return nil } - // For durability reasons, we persist the changes to disk first before we - // update the in-memory registry. Any error during persisting is fatal. - if r.mu.currProto.Version == enginepb.RegistryVersion_Base { - newProto := &enginepb.FileRegistry{} - proto.Merge(newProto, r.mu.currProto) - newProto.ProcessBatch(batch) - if err := r.rewriteOldRegistry(newProto); err != nil { - panic(err) - } - } if err := r.writeToRegistryFile(batch); err != nil { panic(err) } - r.mu.currProto.ProcessBatch(batch) + r.applyBatch(batch) return nil } -// TODO(ayang): delete this function when we deprecate the old registry -func (r *PebbleFileRegistry) rewriteOldRegistry(newProto *enginepb.FileRegistry) error { - b, err := protoutil.Marshal(newProto) - if err != nil { - return err - } - if err := fs.SafeWriteToFile(r.FS, r.DBDir, r.oldRegistryPath, b); err != nil { - return err +// processBatch processes a batch of updates to the file registry. +func (r *PebbleFileRegistry) applyBatch(batch *enginepb.RegistryUpdateBatch) { + for _, update := range batch.Updates { + if update.Entry == nil { + delete(r.mu.entries, update.Filename) + } else { + if r.mu.entries == nil { + r.mu.entries = make(map[string]*enginepb.FileEntry) + } + r.mu.entries[update.Filename] = update.Entry + } } - return nil } func (r *PebbleFileRegistry) writeToRegistryFile(batch *enginepb.RegistryUpdateBatch) error { @@ -569,7 +433,7 @@ func (r *PebbleFileRegistry) writeToRegistryFile(batch *enginepb.RegistryUpdateB } func makeRegistryFilename(iter uint64) string { - return fmt.Sprintf("%s_%06d", registryFilename, iter) + return fmt.Sprintf("%s_%06d", registryFilenameBase, iter) } func (r *PebbleFileRegistry) createNewRegistryFile() error { @@ -595,7 +459,7 @@ func (r *PebbleFileRegistry) createNewRegistryFile() error { // Write the registry header as the first record in the registry file. registryHeader := &enginepb.RegistryHeader{ - Version: r.mu.currProto.Version, + Version: enginepb.RegistryVersion_Records, } b, err := protoutil.Marshal(registryHeader) if err != nil { @@ -607,7 +471,7 @@ func (r *PebbleFileRegistry) createNewRegistryFile() error { // Write a RegistryUpdateBatch containing the current state of the registry. batch := &enginepb.RegistryUpdateBatch{} - for filename, entry := range r.mu.currProto.Files { + for filename, entry := range r.mu.entries { batch.PutEntry(filename, entry) } b, err = protoutil.Marshal(batch) @@ -651,15 +515,21 @@ func (r *PebbleFileRegistry) createNewRegistryFile() error { r.mu.registryFile = f r.mu.registryWriter = records r.mu.registryFilename = filename - return err } func (r *PebbleFileRegistry) getRegistryCopy() *enginepb.FileRegistry { r.mu.Lock() defer r.mu.Unlock() - rv := &enginepb.FileRegistry{} - proto.Merge(rv, r.mu.currProto) + rv := &enginepb.FileRegistry{ + Version: enginepb.RegistryVersion_Records, + Files: make(map[string]*enginepb.FileEntry, len(r.mu.entries)), + } + for filename, entry := range r.mu.entries { + ev := &enginepb.FileEntry{} + *ev = *entry + rv.Files[filename] = ev + } return rv } diff --git a/pkg/storage/pebble_file_registry_test.go b/pkg/storage/pebble_file_registry_test.go index c4f58fb6f6d4..99ed6315886e 100644 --- a/pkg/storage/pebble_file_registry_test.go +++ b/pkg/storage/pebble_file_registry_test.go @@ -66,17 +66,6 @@ func TestFileRegistryRelativePaths(t *testing.T) { } } -func TestFileRegistry_UpgradeEmpty(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - mem := vfs.NewMem() - registry := &PebbleFileRegistry{FS: mem, DBDir: ""} - require.NoError(t, registry.Load()) - require.NoError(t, registry.StopUsingOldRegistry()) - require.NoError(t, registry.Close()) -} - func TestFileRegistryOps(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -106,9 +95,9 @@ func TestFileRegistryOps(t *testing.T) { require.NoError(t, registry.Load()) registry.mu.Lock() defer registry.mu.Unlock() - if diff := pretty.Diff(registry.mu.currProto.Files, expected); diff != nil { + if diff := pretty.Diff(registry.mu.entries, expected); diff != nil { t.Log(string(debug.Stack())) - t.Fatalf("%s\n%v", strings.Join(diff, "\n"), registry.mu.currProto.Files) + t.Fatalf("%s\n%v", strings.Join(diff, "\n"), registry.mu.entries) } } @@ -226,19 +215,22 @@ func TestFileRegistryElideUnencrypted(t *testing.T) { registry := &PebbleFileRegistry{FS: mem} require.NoError(t, registry.Load()) - newProto := &enginepb.FileRegistry{} - newProto.Files = make(map[string]*enginepb.FileEntry) - newProto.Files["test1"] = &enginepb.FileEntry{EnvType: enginepb.EnvType_Data, EncryptionSettings: []byte(nil)} - newProto.Files["test2"] = &enginepb.FileEntry{EnvType: enginepb.EnvType_Store, EncryptionSettings: []byte("foo")} - require.NoError(t, registry.rewriteOldRegistry(newProto)) + require.NoError(t, registry.writeToRegistryFile(&enginepb.RegistryUpdateBatch{ + Updates: []*enginepb.RegistryUpdate{ + {Filename: "test1", Entry: &enginepb.FileEntry{EnvType: enginepb.EnvType_Data, EncryptionSettings: []byte(nil)}}, + {Filename: "test2", Entry: &enginepb.FileEntry{EnvType: enginepb.EnvType_Store, EncryptionSettings: []byte("foo")}}, + }, + })) + require.NoError(t, registry.Close()) // Create another pebble file registry to verify that the unencrypted file is elided on startup. registry2 := &PebbleFileRegistry{FS: mem} require.NoError(t, registry2.Load()) - require.NotContains(t, registry2.mu.currProto.Files, "test1") - entry := registry2.mu.currProto.Files["test2"] + require.NotContains(t, registry2.mu.entries, "test1") + entry := registry2.mu.entries["test2"] require.NotNil(t, entry) require.Equal(t, entry.EncryptionSettings, []byte("foo")) + require.NoError(t, registry2.Close()) } func TestFileRegistryElideNonexistent(t *testing.T) { @@ -252,12 +244,13 @@ func TestFileRegistryElideNonexistent(t *testing.T) { { registry := &PebbleFileRegistry{FS: mem} require.NoError(t, registry.Load()) - require.NoError(t, registry.rewriteOldRegistry(&enginepb.FileRegistry{ - Files: map[string]*enginepb.FileEntry{ - "foo": {EnvType: enginepb.EnvType_Data, EncryptionSettings: []byte("foo")}, - "bar": {EnvType: enginepb.EnvType_Data, EncryptionSettings: []byte("bar")}, + require.NoError(t, registry.writeToRegistryFile(&enginepb.RegistryUpdateBatch{ + Updates: []*enginepb.RegistryUpdate{ + {Filename: "foo", Entry: &enginepb.FileEntry{EnvType: enginepb.EnvType_Data, EncryptionSettings: []byte("foo")}}, + {Filename: "bar", Entry: &enginepb.FileEntry{EnvType: enginepb.EnvType_Data, EncryptionSettings: []byte("bar")}}, }, })) + require.NoError(t, registry.Close()) } // Create another registry and verify that the nonexistent `foo` file @@ -265,10 +258,11 @@ func TestFileRegistryElideNonexistent(t *testing.T) { { registry := &PebbleFileRegistry{FS: mem} require.NoError(t, registry.Load()) - require.NotContains(t, registry.mu.currProto.Files, "foo") - require.Contains(t, registry.mu.currProto.Files, "bar") - require.NotNil(t, registry.mu.currProto.Files["bar"]) - require.Equal(t, []byte("bar"), registry.mu.currProto.Files["bar"].EncryptionSettings) + require.NotContains(t, registry.mu.entries, "foo") + require.Contains(t, registry.mu.entries, "bar") + require.NotNil(t, registry.mu.entries["bar"]) + require.Equal(t, []byte("bar"), registry.mu.entries["bar"].EncryptionSettings) + require.NoError(t, registry.Close()) } } @@ -307,10 +301,6 @@ func TestFileRegistryRecordsReadAndWrite(t *testing.T) { for filename, entry := range files { require.Equal(t, entry, registry2.GetFileEntry(filename)) } - - // Signal that we no longer need the monolithic one. - require.NoError(t, registry2.StopUsingOldRegistry()) - require.NoError(t, registry2.checkNoBaseRegistry()) require.NoError(t, registry2.Close()) registry3 := &PebbleFileRegistry{FS: mem} @@ -387,9 +377,6 @@ func TestFileRegistry(t *testing.T) { require.NoError(t, f.Close()) } return buf.String() - case "upgrade-to-records": - require.NoError(t, registry.StopUsingOldRegistry()) - return buf.String() default: panic("unrecognized command " + d.Cmd) } diff --git a/pkg/storage/testdata/file_registry b/pkg/storage/testdata/file_registry index 42a406fd7cc6..71938ac1587a 100644 --- a/pkg/storage/testdata/file_registry +++ b/pkg/storage/testdata/file_registry @@ -1,6 +1,5 @@ check-no-registry-file ---- -stat("COCKROACHDB_REGISTRY") OK # Open and close a registry on an empty store. Nothing should be written @@ -9,8 +8,6 @@ OK load ---- open-dir("") -open("COCKROACHDB_REGISTRY") -open("STORAGE_MIN_VERSION") close ---- @@ -18,31 +15,25 @@ close("") check-no-registry-file ---- -stat("COCKROACHDB_REGISTRY") OK -# Open an empty store and immediately upgrade to the records-based -# registry. The new registry file should be written, and then a marker -# file should be written alongside it. +# Open an empty store and write to the registry. A new registry file should be +# written, and a marker file should be written alongside it. load ---- open-dir("") -open("COCKROACHDB_REGISTRY") -open("STORAGE_MIN_VERSION") -upgrade-to-records +set filename=foo settings=bar ---- -open-dir("") create("COCKROACHDB_REGISTRY_000001") write("COCKROACHDB_REGISTRY_000001", <...16 bytes...>) sync("COCKROACHDB_REGISTRY_000001") create("marker.registry.000001.COCKROACHDB_REGISTRY_000001") close("marker.registry.000001.COCKROACHDB_REGISTRY_000001") sync("") -remove("COCKROACHDB_REGISTRY") -sync("") -close("") +write("COCKROACHDB_REGISTRY_000001", <...23 bytes...>) +sync("COCKROACHDB_REGISTRY_000001") close ---- @@ -52,24 +43,32 @@ close("") check-no-registry-file ---- -stat("COCKROACHDB_REGISTRY") Error: file already exists -# Re-opening the store should check for the old-style monolithic -# registry, see that it doesn't exist and open the incremental, -# records-based registry indicated by the marker file. -# -# No registry rotation should occur because nothing is written. +# Re-opening the store should observe the registry file. It should recognize +# that the file foo does not exist and rotate the registry to elide it. load ---- open-dir("") -open("COCKROACHDB_REGISTRY") open("COCKROACHDB_REGISTRY_000001") close("COCKROACHDB_REGISTRY_000001") +stat("foo") +create("COCKROACHDB_REGISTRY_000002") +write("COCKROACHDB_REGISTRY_000002", <...32 bytes...>) +sync("COCKROACHDB_REGISTRY_000002") +create("marker.registry.000002.COCKROACHDB_REGISTRY_000002") +close("marker.registry.000002.COCKROACHDB_REGISTRY_000002") +remove("marker.registry.000001.COCKROACHDB_REGISTRY_000001") +sync("") +remove("COCKROACHDB_REGISTRY_000001") +write("COCKROACHDB_REGISTRY_000002", <...14 bytes...>) +sync("COCKROACHDB_REGISTRY_000002") close ---- +write("COCKROACHDB_REGISTRY_000002", <...0 bytes...>) +close("COCKROACHDB_REGISTRY_000002") close("") # Re-opening the store and writing something to the file registry should @@ -83,22 +82,21 @@ close("") load ---- open-dir("") -open("COCKROACHDB_REGISTRY") -open("COCKROACHDB_REGISTRY_000001") -close("COCKROACHDB_REGISTRY_000001") +open("COCKROACHDB_REGISTRY_000002") +close("COCKROACHDB_REGISTRY_000002") set filename=foo settings=bar ---- -create("COCKROACHDB_REGISTRY_000002") -write("COCKROACHDB_REGISTRY_000002", <...16 bytes...>) -sync("COCKROACHDB_REGISTRY_000002") -create("marker.registry.000002.COCKROACHDB_REGISTRY_000002") -close("marker.registry.000002.COCKROACHDB_REGISTRY_000002") -remove("marker.registry.000001.COCKROACHDB_REGISTRY_000001") +create("COCKROACHDB_REGISTRY_000003") +write("COCKROACHDB_REGISTRY_000003", <...16 bytes...>) +sync("COCKROACHDB_REGISTRY_000003") +create("marker.registry.000003.COCKROACHDB_REGISTRY_000003") +close("marker.registry.000003.COCKROACHDB_REGISTRY_000003") +remove("marker.registry.000002.COCKROACHDB_REGISTRY_000002") sync("") -remove("COCKROACHDB_REGISTRY_000001") -write("COCKROACHDB_REGISTRY_000002", <...23 bytes...>) -sync("COCKROACHDB_REGISTRY_000002") +remove("COCKROACHDB_REGISTRY_000002") +write("COCKROACHDB_REGISTRY_000003", <...23 bytes...>) +sync("COCKROACHDB_REGISTRY_000003") get filename=foo ---- @@ -106,8 +104,8 @@ bar set filename=foo settings=helloworld ---- -write("COCKROACHDB_REGISTRY_000002", <...30 bytes...>) -sync("COCKROACHDB_REGISTRY_000002") +write("COCKROACHDB_REGISTRY_000003", <...30 bytes...>) +sync("COCKROACHDB_REGISTRY_000003") get filename=foo ---- @@ -115,8 +113,8 @@ helloworld close ---- -write("COCKROACHDB_REGISTRY_000002", <...0 bytes...>) -close("COCKROACHDB_REGISTRY_000002") +write("COCKROACHDB_REGISTRY_000003", <...0 bytes...>) +close("COCKROACHDB_REGISTRY_000003") close("") # Re-opening the store again, while there are entries for files (`foo`) @@ -126,59 +124,46 @@ close("") load ---- open-dir("") -open("COCKROACHDB_REGISTRY") -open("COCKROACHDB_REGISTRY_000002") -close("COCKROACHDB_REGISTRY_000002") +open("COCKROACHDB_REGISTRY_000003") +close("COCKROACHDB_REGISTRY_000003") stat("foo") -create("COCKROACHDB_REGISTRY_000003") -write("COCKROACHDB_REGISTRY_000003", <...39 bytes...>) -sync("COCKROACHDB_REGISTRY_000003") -create("marker.registry.000003.COCKROACHDB_REGISTRY_000003") -close("marker.registry.000003.COCKROACHDB_REGISTRY_000003") -remove("marker.registry.000002.COCKROACHDB_REGISTRY_000002") +create("COCKROACHDB_REGISTRY_000004") +write("COCKROACHDB_REGISTRY_000004", <...39 bytes...>) +sync("COCKROACHDB_REGISTRY_000004") +create("marker.registry.000004.COCKROACHDB_REGISTRY_000004") +close("marker.registry.000004.COCKROACHDB_REGISTRY_000004") +remove("marker.registry.000003.COCKROACHDB_REGISTRY_000003") sync("") -remove("COCKROACHDB_REGISTRY_000002") -write("COCKROACHDB_REGISTRY_000003", <...14 bytes...>) -sync("COCKROACHDB_REGISTRY_000003") +remove("COCKROACHDB_REGISTRY_000003") +write("COCKROACHDB_REGISTRY_000004", <...14 bytes...>) +sync("COCKROACHDB_REGISTRY_000004") get filename=foo ---- close ---- -write("COCKROACHDB_REGISTRY_000003", <...0 bytes...>) -close("COCKROACHDB_REGISTRY_000003") +write("COCKROACHDB_REGISTRY_000004", <...0 bytes...>) +close("COCKROACHDB_REGISTRY_000004") close("") -# Reset the filesystem and start again. We should default to using both -# the monolithic `COCKROACHDB_REGISTRY` file and the incremental file. +# Reset the filesystem and start again. reset ---- check-no-registry-file ---- -stat("COCKROACHDB_REGISTRY") OK load ---- open-dir("") -open("COCKROACHDB_REGISTRY") -open("STORAGE_MIN_VERSION") set filename=foo settings=helloworld ---- -create("COCKROACHDB_REGISTRY.crdbtmp") -write("COCKROACHDB_REGISTRY.crdbtmp", <...23 bytes...>) -sync("COCKROACHDB_REGISTRY.crdbtmp") -close("COCKROACHDB_REGISTRY.crdbtmp") -rename("COCKROACHDB_REGISTRY.crdbtmp", "COCKROACHDB_REGISTRY") -open-dir("") -sync("") -close("") create("COCKROACHDB_REGISTRY_000001") -write("COCKROACHDB_REGISTRY_000001", <...14 bytes...>) +write("COCKROACHDB_REGISTRY_000001", <...16 bytes...>) sync("COCKROACHDB_REGISTRY_000001") create("marker.registry.000001.COCKROACHDB_REGISTRY_000001") close("marker.registry.000001.COCKROACHDB_REGISTRY_000001") @@ -186,20 +171,11 @@ sync("") write("COCKROACHDB_REGISTRY_000001", <...30 bytes...>) sync("COCKROACHDB_REGISTRY_000001") -# A second write to the registry should again completely rewrite -# `COCKROACHDB_REGISTRY`, but only perform a single write and sync on +# A second write to the registry should only perform a single write and sync on # the incremental registry. set filename=bar settings=hi ---- -create("COCKROACHDB_REGISTRY.crdbtmp") -write("COCKROACHDB_REGISTRY.crdbtmp", <...38 bytes...>) -sync("COCKROACHDB_REGISTRY.crdbtmp") -close("COCKROACHDB_REGISTRY.crdbtmp") -rename("COCKROACHDB_REGISTRY.crdbtmp", "COCKROACHDB_REGISTRY") -open-dir("") -sync("") -close("") write("COCKROACHDB_REGISTRY_000001", <...22 bytes...>) sync("COCKROACHDB_REGISTRY_000001") @@ -211,7 +187,6 @@ close("") check-no-registry-file ---- -stat("COCKROACHDB_REGISTRY") Error: file already exists # Re-opening the registry should read the state from the monolithic @@ -229,8 +204,8 @@ close("bar") load ---- open-dir("") -open("COCKROACHDB_REGISTRY") -close("COCKROACHDB_REGISTRY") +open("COCKROACHDB_REGISTRY_000001") +close("COCKROACHDB_REGISTRY_000001") stat("bar") stat("foo") @@ -240,16 +215,8 @@ hi set filename=bax settings=hello ---- -create("COCKROACHDB_REGISTRY.crdbtmp") -write("COCKROACHDB_REGISTRY.crdbtmp", <...56 bytes...>) -sync("COCKROACHDB_REGISTRY.crdbtmp") -close("COCKROACHDB_REGISTRY.crdbtmp") -rename("COCKROACHDB_REGISTRY.crdbtmp", "COCKROACHDB_REGISTRY") -open-dir("") -sync("") -close("") create("COCKROACHDB_REGISTRY_000002") -write("COCKROACHDB_REGISTRY_000002", <...52 bytes...>) +write("COCKROACHDB_REGISTRY_000002", <...54 bytes...>) sync("COCKROACHDB_REGISTRY_000002") create("marker.registry.000002.COCKROACHDB_REGISTRY_000002") close("marker.registry.000002.COCKROACHDB_REGISTRY_000002")