Skip to content

Commit

Permalink
storage: add file storing min version needed for backwards compatibility
Browse files Browse the repository at this point in the history
A new STORAGE_MIN_VERSION file containing the minimum version that the
storage engine needs to maintain backwards compatibility with will be
added to the directory for each storage engine. This will be used in
the encryption-at-rest registry migration.

Release note: None
  • Loading branch information
Andy Yang committed Aug 12, 2021
1 parent 764a1a9 commit 3f65e23
Show file tree
Hide file tree
Showing 3 changed files with 187 additions and 0 deletions.
3 changes: 3 additions & 0 deletions pkg/storage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ go_library(
"in_mem.go",
"intent_interleaving_iter.go",
"intent_reader_writer.go",
"min_version.go",
"multi_iterator.go",
"mvcc.go",
"mvcc_incremental_iterator.go",
Expand Down Expand Up @@ -91,6 +92,7 @@ go_test(
"intent_interleaving_iter_test.go",
"intent_reader_writer_test.go",
"main_test.go",
"min_version_test.go",
"multi_iterator_test.go",
"mvcc_history_test.go",
"mvcc_incremental_iterator_test.go",
Expand Down Expand Up @@ -141,6 +143,7 @@ go_test(
"@com_github_cockroachdb_errors//oserror",
"@com_github_cockroachdb_pebble//:pebble",
"@com_github_cockroachdb_pebble//vfs",
"@com_github_gogo_protobuf//proto",
"@com_github_kr_pretty//:pretty",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
Expand Down
92 changes: 92 additions & 0 deletions pkg/storage/min_version.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
// 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 storage

import (
"io/ioutil"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/errors/oserror"
"github.com/cockroachdb/pebble/vfs"
)

// MinVersionFilename is the name of the file containing a marshaled
// roachpb.Version that can be updated during storage-related migrations
// and checked on startup to determine if we can safely use a
// backwards-incompatible feature.
const MinVersionFilename = "STORAGE_MIN_VERSION"

// WriteMinVersionFile writes the provided version to disk. The caller must
// guarantee that the version will never be downgraded below the given version.
func WriteMinVersionFile(fs vfs.FS, dir string, version *roachpb.Version) error {
if version == nil {
return errors.New("min version should not be nil")
}
ok, err := MinVersionIsAtLeastTargetVersion(fs, dir, version)
if err != nil {
return err
}
if ok {
return nil
}
b, err := protoutil.Marshal(version)
if err != nil {
return err
}
filename := fs.PathJoin(dir, MinVersionFilename)
if err := SafeWriteToFile(fs, dir, filename, b); err != nil {
return err
}
return nil
}

// MinVersionIsAtLeastTargetVersion returns whether the min version recorded
// on disk is at least the target version.
func MinVersionIsAtLeastTargetVersion(
fs vfs.FS, dir string, target *roachpb.Version,
) (bool, error) {
if target == nil {
return false, errors.New("target version should not be nil")
}
minVersion, err := GetMinVersion(fs, dir)
if err != nil {
return false, err
}
if minVersion == nil {
return false, nil
}
return !minVersion.Less(*target), nil
}

// GetMinVersion returns the min version recorded on disk if the min version
// file exists and nil otherwise.
func GetMinVersion(fs vfs.FS, dir string) (*roachpb.Version, error) {
filename := fs.PathJoin(dir, MinVersionFilename)
f, err := fs.Open(filename)
if oserror.IsNotExist(err) {
return nil, nil
}
if err != nil {
return nil, err
}
defer f.Close()
b, err := ioutil.ReadAll(f)
if err != nil {
return nil, err
}
version := &roachpb.Version{}
if err := protoutil.Unmarshal(b, version); err != nil {
return nil, err
}
return version, nil
}
92 changes: 92 additions & 0 deletions pkg/storage/min_version_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
// 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 storage

import (
"os"
"testing"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/pebble/vfs"
"github.com/gogo/protobuf/proto"
"github.com/stretchr/testify/require"
)

func TestMinVersion(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

version1 := &roachpb.Version{Major: 21, Minor: 1, Patch: 0, Internal: 122}
version2 := &roachpb.Version{Major: 21, Minor: 1, Patch: 0, Internal: 126}

mem := vfs.NewMem()
dir := "/foo"
require.NoError(t, mem.MkdirAll(dir, os.ModeDir))

// Expect nil version when min version file doesn't exist.
v, err := GetMinVersion(mem, dir)
require.NoError(t, err)
require.Nil(t, v)

// Expect min version to not be at least any target version.
ok, err := MinVersionIsAtLeastTargetVersion(mem, dir, version1)
require.NoError(t, err)
require.False(t, ok)
ok, err = MinVersionIsAtLeastTargetVersion(mem, dir, version2)
require.NoError(t, err)
require.False(t, ok)

// Expect no error when updating min version if no file currently exists.
v = &roachpb.Version{}
proto.Merge(v, version1)
require.NoError(t, WriteMinVersionFile(mem, dir, v))

// Expect min version to be version1.
v, err = GetMinVersion(mem, dir)
require.NoError(t, err)
require.True(t, version1.Equal(v))

// Expect min version to be at least version1 but not version2.
ok, err = MinVersionIsAtLeastTargetVersion(mem, dir, version1)
require.NoError(t, err)
require.True(t, ok)
ok, err = MinVersionIsAtLeastTargetVersion(mem, dir, version2)
require.NoError(t, err)
require.False(t, ok)

// Expect no error when updating min version to a higher version.
v = &roachpb.Version{}
proto.Merge(v, version2)
require.NoError(t, WriteMinVersionFile(mem, dir, v))

// Expect min version to be at least version1 and version2.
ok, err = MinVersionIsAtLeastTargetVersion(mem, dir, version1)
require.NoError(t, err)
require.True(t, ok)
ok, err = MinVersionIsAtLeastTargetVersion(mem, dir, version2)
require.NoError(t, err)
require.True(t, ok)

// Expect min version to be version2.
v, err = GetMinVersion(mem, dir)
require.NoError(t, err)
require.True(t, version2.Equal(v))

// Expect no-op when trying to update min version to a lower version.
v = &roachpb.Version{}
proto.Merge(v, version1)
require.NoError(t, WriteMinVersionFile(mem, dir, v))
v, err = GetMinVersion(mem, dir)
require.NoError(t, err)
require.True(t, version2.Equal(v))
}

0 comments on commit 3f65e23

Please sign in to comment.