From c9454929daa9ff3a52956d6c43b7105a4bf89138 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Fri, 24 Jul 2020 13:08:17 -0400 Subject: [PATCH] Revert "[dbnode] Validate individual index entries on decode instead of entire file on open (#2468)" This reverts commit bba274f90b122e69cb2a52c894ef790fe18f8af2. --- src/dbnode/persist/fs/msgpack/decoder.go | 32 +-- src/dbnode/persist/fs/msgpack/decoder_test.go | 16 -- .../persist/fs/msgpack/stream_with_digest.go | 146 ------------- .../fs/msgpack/stream_with_digest_test.go | 192 ------------------ src/dbnode/persist/fs/seek.go | 17 +- src/dbnode/persist/fs/seek_test.go | 67 +----- src/dbnode/persist/schema/types.go | 2 +- src/dbnode/persist/schema/version_checker.go | 43 ---- .../persist/schema/version_checker_test.go | 46 ----- 9 files changed, 11 insertions(+), 550 deletions(-) delete mode 100644 src/dbnode/persist/fs/msgpack/stream_with_digest.go delete mode 100644 src/dbnode/persist/fs/msgpack/stream_with_digest_test.go delete mode 100644 src/dbnode/persist/schema/version_checker.go delete mode 100644 src/dbnode/persist/schema/version_checker_test.go diff --git a/src/dbnode/persist/fs/msgpack/decoder.go b/src/dbnode/persist/fs/msgpack/decoder.go index 3cad323a20..0d9ada5112 100644 --- a/src/dbnode/persist/fs/msgpack/decoder.go +++ b/src/dbnode/persist/fs/msgpack/decoder.go @@ -46,7 +46,6 @@ var ( errorUnableToDetermineNumFieldsToSkip = errors.New("unable to determine num fields to skip") errorCalledDecodeBytesWithoutByteStreamDecoder = errors.New("called decodeBytes with out byte stream decoder") - errorIndexEntryChecksumMismatch = errors.New("decode index entry encountered checksum mismatch") ) // Decoder decodes persisted msgpack-encoded data @@ -54,10 +53,7 @@ type Decoder struct { reader DecoderStream // Will only be set if the Decoder is Reset() with a DecoderStream // that also implements ByteStream. - byteReader ByteStream - // Wraps original reader with reader that can calculate digest. Digest calculation must be enabled, - // otherwise it defaults to off. - readerWithDigest *decoderStreamWithDigest + byteReader ByteStream dec *msgpack.Decoder err error allocDecodedBytes bool @@ -80,7 +76,6 @@ func newDecoder(legacy legacyEncodingOptions, opts DecodingOptions) *Decoder { reader: reader, dec: msgpack.NewDecoder(reader), legacy: legacy, - readerWithDigest: newDecoderStreamWithDigest(nil), } } @@ -96,8 +91,7 @@ func (dec *Decoder) Reset(stream DecoderStream) { dec.byteReader = nil } - dec.readerWithDigest.reset(dec.reader) - dec.dec.Reset(dec.readerWithDigest) + dec.dec.Reset(dec.reader) dec.err = nil } @@ -121,10 +115,8 @@ func (dec *Decoder) DecodeIndexEntry(bytesPool pool.BytesPool) (schema.IndexEntr if dec.err != nil { return emptyIndexEntry, dec.err } - dec.readerWithDigest.setDigestReaderEnabled(true) _, numFieldsToSkip := dec.decodeRootObject(indexEntryVersion, indexEntryType) indexEntry := dec.decodeIndexEntry(bytesPool) - dec.readerWithDigest.setDigestReaderEnabled(false) dec.skip(numFieldsToSkip) if dec.err != nil { return emptyIndexEntry, dec.err @@ -414,22 +406,13 @@ func (dec *Decoder) decodeIndexEntry(bytesPool pool.BytesPool) schema.IndexEntry return indexEntry } - // NB(nate): Any new fields should be parsed here. - // Intentionally skip any extra fields here as we've stipulated that from V3 onward, IndexEntryChecksum will be the // final field on index entries dec.skip(numFieldsToSkip) - // Retrieve actual checksum value here. Attempting to retrieve after decoding the upcoming expected checksum field - // would include value in actual checksum calculation which would cause a mismatch - actualChecksum := dec.readerWithDigest.digest().Sum32() - // Decode checksum field originally added in V3 - expectedChecksum := uint32(dec.decodeVarint()) - - if expectedChecksum != actualChecksum { - dec.err = errorIndexEntryChecksumMismatch - } + // TODO(nate): actually use the checksum value for index entry validation - #2629 + _ = dec.decodeVarint() return indexEntry } @@ -680,11 +663,6 @@ func (dec *Decoder) decodeBytes() ([]byte, int, int) { return nil, -1, -1 } value = backingBytes[currPos:targetPos] - if err := dec.readerWithDigest.capture(value); err != nil { - dec.err = err - return nil, -1, -1 - } - return value, currPos, bytesLen } @@ -702,7 +680,7 @@ func (dec *Decoder) decodeBytesWithPool(bytesPool pool.BytesPool) []byte { } bytes := bytesPool.Get(bytesLen)[:bytesLen] - n, err := io.ReadFull(dec.readerWithDigest, bytes) + n, err := io.ReadFull(dec.reader, bytes) if err != nil { dec.err = err bytesPool.Put(bytes) diff --git a/src/dbnode/persist/fs/msgpack/decoder_test.go b/src/dbnode/persist/fs/msgpack/decoder_test.go index d36f0b120b..de52019f1b 100644 --- a/src/dbnode/persist/fs/msgpack/decoder_test.go +++ b/src/dbnode/persist/fs/msgpack/decoder_test.go @@ -262,19 +262,3 @@ func TestDecodeBytesAllocNew(t *testing.T) { } require.Equal(t, []byte("testIndexEntry"), res.ID) } - -func TestDecodeIndexEntryInvalidChecksum(t *testing.T) { - var ( - enc = NewEncoder() - dec = NewDecoder(nil) - ) - require.NoError(t, enc.EncodeIndexEntry(testIndexEntry)) - - // Update to invalid checksum - enc.buf.Truncate(len(enc.Bytes()) - 5) // 5 bytes = 1 byte for integer code + 4 bytes for checksum - require.NoError(t, enc.enc.EncodeInt64(1234)) - - dec.Reset(NewByteDecoderStream(enc.Bytes())) - _, err := dec.DecodeIndexEntry(nil) - require.Error(t, err) -} diff --git a/src/dbnode/persist/fs/msgpack/stream_with_digest.go b/src/dbnode/persist/fs/msgpack/stream_with_digest.go deleted file mode 100644 index ff03e2c45d..0000000000 --- a/src/dbnode/persist/fs/msgpack/stream_with_digest.go +++ /dev/null @@ -1,146 +0,0 @@ -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package msgpack - -import ( - "errors" - "hash" - "hash/adler32" -) - -var ( - // errChecksumMismatch returned when the calculated checksum doesn't match the stored checksum - errChecksumMismatch = errors.New("calculated checksum doesn't match stored checksum") -) - -var _ DecoderStream = &decoderStreamWithDigest{} - -// decoderStreamWithDigest calculates the digest as it processes a decoder stream. -type decoderStreamWithDigest struct { - reader DecoderStream - readerDigest hash.Hash32 - unreadByte bool - enabled bool - singleByteBuf []byte -} - -// newDecoderStreamWithDigest returns a new decoderStreamWithDigest -func newDecoderStreamWithDigest(reader DecoderStream) *decoderStreamWithDigest { - return &decoderStreamWithDigest{ - reader: reader, - readerDigest: adler32.New(), - singleByteBuf: make([]byte, 1), - } -} - -func (d *decoderStreamWithDigest) Read(p []byte) (n int, err error) { - n, err = d.reader.Read(p) - if err != nil { - return n, err - } - if n <= 0 { - return n, nil - } - - start := 0 - if d.unreadByte { - d.unreadByte = false - start++ - } - if d.enabled { - if _, err := d.readerDigest.Write(p[start:n]); err != nil { - return 0, err - } - } - return n, err -} - -func (d *decoderStreamWithDigest) ReadByte() (byte, error) { - b, err := d.reader.ReadByte() - if err != nil { - return 0, err - } - - if d.unreadByte { - d.unreadByte = false - } else if d.enabled { - d.singleByteBuf[0] = b - if _, err := d.readerDigest.Write(d.singleByteBuf); err != nil { - return b, err - } - } - return b, err -} - -func (d *decoderStreamWithDigest) UnreadByte() error { - err := d.reader.UnreadByte() - if err == nil { - d.unreadByte = true - } - return err -} - -// reset resets the reader for use with a new reader. -func (d *decoderStreamWithDigest) reset(stream DecoderStream) { - d.reader = stream - d.readerDigest.Reset() -} - -// digest returns the digest -func (d *decoderStreamWithDigest) digest() hash.Hash32 { - return d.readerDigest -} - -// validate compares the current digest against the expected digest and returns -// an error if they don't match. -func (d *decoderStreamWithDigest) validate(expectedDigest uint32) error { - if d.readerDigest.Sum32() != expectedDigest { - return errChecksumMismatch - } - return nil -} - -// capture provides a mechanism for manually capturing bytes to add to digest when reader is manipulated -// through atypical means (e.g. reading directly from the backing byte slice of a ByteReader) -func (d *decoderStreamWithDigest) capture(bytes []byte) error { - // No-op if not actually capturing at the moment - if d.enabled { - if _, err := d.readerDigest.Write(bytes); err != nil { - return err - } - } - return nil -} - -// setDigestReaderEnabled enables calculating of digest for bytes read. If this is false, behaves like a regular reader. -func (d *decoderStreamWithDigest) setDigestReaderEnabled(enabled bool) { - if !d.enabled && enabled { - d.enabled = true - d.readerDigest.Reset() - } else if d.enabled && !enabled { - d.enabled = false - } -} - -// Returns the decoder stream wrapped by this object -func (d *decoderStreamWithDigest) wrappedStream() DecoderStream { - return d.reader -} diff --git a/src/dbnode/persist/fs/msgpack/stream_with_digest_test.go b/src/dbnode/persist/fs/msgpack/stream_with_digest_test.go deleted file mode 100644 index 87041a73fb..0000000000 --- a/src/dbnode/persist/fs/msgpack/stream_with_digest_test.go +++ /dev/null @@ -1,192 +0,0 @@ -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package msgpack - -import ( - "bufio" - "bytes" - "hash/adler32" - "testing" - - "github.com/stretchr/testify/require" -) - -const srcString = "foo bar baz qux quux corge grault" - -func TestDecoderStreamWithDigestRead(t *testing.T) { - stream := newTestDecoderStream() - - // Read srcString in chunkLen size chunks - chunkLen := 3 - buf := make([]byte, len(srcString)) - for start := 0; start < len(srcString); start = start + chunkLen { - end := start + chunkLen - if end > len(srcString) { - end = len(srcString) - } - - n, err := stream.Read(buf[start:end]) - require.NoError(t, err) - require.Equal(t, chunkLen, n) - require.Equal(t, adler32.Checksum(buf[:end]), stream.digest().Sum32()) - } -} - -func TestDecoderStreamWithDigestReadByte(t *testing.T) { - stream := newTestDecoderStream() - - buf := make([]byte, len(srcString)) - for i := 1; i < len(srcString); i++ { - n, err := stream.Read(buf[i-1 : i]) - require.NoError(t, err) - require.Equal(t, 1, n) - require.Equal(t, adler32.Checksum(buf[:i]), stream.digest().Sum32()) - } -} - -func TestDecoderStreamWithDigestUnreadByte(t *testing.T) { - stream := decoderStreamWithDigest{ - reader: bufio.NewReader(bytes.NewReader([]byte(srcString))), - readerDigest: adler32.New(), - } - - b, err := stream.ReadByte() - require.NoError(t, err) - require.Equal(t, srcString[0], b) - require.False(t, stream.unreadByte) - - err = stream.UnreadByte() - require.NoError(t, err) - require.True(t, stream.unreadByte) -} - -func TestDecoderStreamWithDigestReset(t *testing.T) { - stream := newTestDecoderStream() - - b, err := stream.ReadByte() - require.NoError(t, err) - require.Equal(t, srcString[0], b) - - b, err = stream.ReadByte() - require.NoError(t, err) - require.Equal(t, srcString[1], b) - - stream.reset(bufio.NewReader(bytes.NewReader([]byte(srcString)))) - - b, err = stream.ReadByte() - require.NoError(t, err) - require.Equal(t, srcString[0], b) -} - -func TestDecoderStreamWithDigestValidate(t *testing.T) { - stream := newTestDecoderStream() - buf := make([]byte, 5) - - n, err := stream.Read(buf) - require.NoError(t, err) - require.Equal(t, 5, n) - - require.NoError(t, stream.validate(adler32.Checksum(buf))) - require.Error(t, stream.validate(adler32.Checksum([]byte("asdf")))) -} - -func TestDecoderStreamWithDigestCapture(t *testing.T) { - stream := newTestDecoderStream() - - require.NoError(t, stream.validate(1)) - - bytes := []byte("manual capture") - require.NoError(t, stream.capture(bytes)) - - require.Equal(t, adler32.Checksum(bytes), stream.digest().Sum32()) -} - -func TestDecoderStreamWithDigestReadUnreadRead(t *testing.T) { - stream := newTestDecoderStream() - - buf := make([]byte, len(srcString)) - end := 0 - - b1, err := stream.ReadByte() - require.NoError(t, err) - buf[0] = b1 - end++ - require.Equal(t, adler32.Checksum(buf[:end]), stream.digest().Sum32()) - - err = stream.UnreadByte() - end-- - require.NoError(t, err) - - b2, err := stream.ReadByte() - require.NoError(t, err) - end++ - require.Equal(t, b1, b2) - require.Equal(t, adler32.Checksum(buf[:end]), stream.digest().Sum32()) - - n, err := stream.Read(buf[end : end+4]) - require.NoError(t, err) - require.Equal(t, 4, n) - end += n - require.Equal(t, adler32.Checksum(buf[:end]), stream.digest().Sum32()) - - err = stream.UnreadByte() - end-- - require.NoError(t, err) - - n, err = stream.Read(buf[end : end+4]) - require.NoError(t, err) - require.Equal(t, 4, n) - end += n - require.Equal(t, adler32.Checksum(buf[:end]), stream.digest().Sum32()) -} - -func TestDecoderStreamWithDigestSetEnabled(t *testing.T) { - stream := newTestDecoderStream() - - // Disable digest calculation - stream.setDigestReaderEnabled(false) - - buf := make([]byte, 5) - _, err := stream.Read(buf) - require.NoError(t, err) - require.Equal(t, stream.digest().Sum32(), uint32(1)) - - _, err = stream.ReadByte() - require.NoError(t, err) - require.Equal(t, stream.digest().Sum32(), uint32(1)) - - // Enable digest calculation - stream.setDigestReaderEnabled(true) - - _, err = stream.Read(buf) - require.NoError(t, err) - require.Equal(t, stream.digest().Sum32(), adler32.Checksum([]byte(srcString[6:11]))) - - _, err = stream.ReadByte() - require.NoError(t, err) - require.Equal(t, stream.digest().Sum32(), adler32.Checksum([]byte(srcString[6:12]))) -} - -func newTestDecoderStream() *decoderStreamWithDigest { - d := newDecoderStreamWithDigest(bufio.NewReader(bytes.NewReader([]byte(srcString)))) - d.setDigestReaderEnabled(true) - return d -} diff --git a/src/dbnode/persist/fs/seek.go b/src/dbnode/persist/fs/seek.go index 24e2e5f62c..fa6faa3e4f 100644 --- a/src/dbnode/persist/fs/seek.go +++ b/src/dbnode/persist/fs/seek.go @@ -65,9 +65,8 @@ type seeker struct { // Data read from the indexInfo file. Note that we use xtime.UnixNano // instead of time.Time to avoid keeping an extra pointer around. - start xtime.UnixNano - blockSize time.Duration - versionChecker schema.VersionChecker + start xtime.UnixNano + blockSize time.Duration dataFd *os.File indexFd *os.File @@ -225,7 +224,6 @@ func (s *seeker) Open( } s.start = xtime.UnixNano(info.BlockStart) s.blockSize = time.Duration(info.BlockSize) - s.versionChecker = schema.NewVersionChecker(int(info.MajorVersion), int(info.MinorVersion)) err = s.validateIndexFileDigest( indexFdWithDigest, expectedDigests.indexDigest) @@ -405,7 +403,8 @@ func (s *seeker) SeekIndexEntry( // this is a tight loop (scanning linearly through the index file) we want to use a // very cheap pool until we find what we're looking for, and then we can perform a single // copy into checked.Bytes from the more expensive pool. - entry, err := resources.xmsgpackDecoder.DecodeIndexEntry(resources.decodeIndexEntryBytesPool) + entry, err := resources.xmsgpackDecoder.DecodeIndexEntry( + resources.decodeIndexEntryBytesPool) if err == io.EOF { // We reached the end of the file without finding it. return IndexEntry{}, errSeekIDNotFound @@ -511,8 +510,6 @@ func (s *seeker) ConcurrentClone() (ConcurrentDataFileSetSeeker, error) { // they are concurrency safe and can be shared among clones. indexFd: s.indexFd, dataFd: s.dataFd, - - versionChecker: s.versionChecker, } return seeker, nil @@ -522,12 +519,6 @@ func (s *seeker) validateIndexFileDigest( indexFdWithDigest digest.FdWithDigestReader, expectedDigest uint32, ) error { - // If piecemeal checksumming validation enabled for index entries, do not attempt to validate the - // checksum of the entire file - if s.versionChecker.IndexEntryValidationEnabled() { - return nil - } - buf := make([]byte, s.opts.dataBufferSize) for { n, err := indexFdWithDigest.Read(buf) diff --git a/src/dbnode/persist/fs/seek_test.go b/src/dbnode/persist/fs/seek_test.go index f74dadadae..6b9fa9db62 100644 --- a/src/dbnode/persist/fs/seek_test.go +++ b/src/dbnode/persist/fs/seek_test.go @@ -30,8 +30,8 @@ import ( "github.com/m3db/m3/src/dbnode/digest" "github.com/m3db/m3/src/dbnode/persist" - "github.com/m3db/m3/src/dbnode/persist/schema" "github.com/m3db/m3/src/x/ident" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -436,71 +436,6 @@ func TestCloneSeeker(t *testing.T) { assert.Equal(t, []byte{1, 2, 1}, data.Bytes()) } -func TestSeekValidateIndexEntriesFile(t *testing.T) { - dir, err := ioutil.TempDir("", "testdb") - if err != nil { - t.Fatal(err) - } - filePathPrefix := filepath.Join(dir, "") - defer os.RemoveAll(dir) - - w := newTestWriter(t, filePathPrefix) - writerOpts := DataWriterOpenOptions{ - BlockSize: testBlockSize, - Identifier: FileSetFileIdentifier{ - Namespace: testNs1ID, - Shard: 0, - BlockStart: testWriterStart, - }, - } - err = w.Open(writerOpts) - assert.NoError(t, err) - - // Write data - assert.NoError(t, w.Write( - persist.NewMetadataFromIDAndTags( - ident.StringID("foo"), - ident.Tags{}, - persist.MetadataOptions{}), - bytesRefd([]byte{1, 2, 3}), - digest.Checksum([]byte{1, 2, 3}))) - assert.NoError(t, w.Close()) - - shardDir := ShardDataDirPath(filePathPrefix, testNs1ID, 0) - - // With full file validation disabled - s := seeker{opts: seekerOpts{ - filePathPrefix: filePathPrefix, - dataBufferSize: testReaderBufferSize, - infoBufferSize: testReaderBufferSize, - bytesPool: testBytesPool, - keepUnreadBuf: false, - opts: testDefaultOpts, - }} - s.versionChecker = schema.NewVersionChecker(1, 1) - - indexFilePath := dataFilesetPathFromTimeAndIndex(shardDir, testWriterStart, 0, indexFileSuffix, false) - indexFd, err := os.Open(indexFilePath) - assert.NoError(t, err) - indexReader := digest.NewFdWithDigestReader(defaultInfoReaderBufferSize) - indexReader.Reset(indexFd) - - assert.NoError(t, s.validateIndexFileDigest(indexReader, 0)) - - // With full file validation enabled - s.versionChecker = schema.NewVersionChecker(1, 0) - _, err = indexFd.Seek(0, 0) - assert.NoError(t, err) - indexReader.Reset(indexFd) - - assert.Error(t, s.validateIndexFileDigest(indexReader, 0)) - - // Sanity check -- call seeker#Open and ensure VersionChecker is set correctly - err = s.Open(testNs1ID, 0, testWriterStart, 0, newTestReusableSeekerResources()) - assert.NoError(t, err) - assert.True(t, s.versionChecker.IndexEntryValidationEnabled()) -} - func newTestReusableSeekerResources() ReusableSeekerResources { return NewReusableSeekerResources(testDefaultOpts) } diff --git a/src/dbnode/persist/schema/types.go b/src/dbnode/persist/schema/types.go index 1579fe0e78..71046044f8 100644 --- a/src/dbnode/persist/schema/types.go +++ b/src/dbnode/persist/schema/types.go @@ -32,7 +32,7 @@ const MajorVersion = 1 // MinorVersion is the minor schema version for a set of fileset files. // This is only incremented when *non-breaking* changes are introduced that // we want to have some level of control around how they're rolled out. -const MinorVersion = 1 +const MinorVersion = 0 // IndexInfo stores metadata information about block filesets type IndexInfo struct { diff --git a/src/dbnode/persist/schema/version_checker.go b/src/dbnode/persist/schema/version_checker.go deleted file mode 100644 index ac14edec07..0000000000 --- a/src/dbnode/persist/schema/version_checker.go +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package schema - -// VersionChecker centralizes logic for checking if a major, minor version combo supports -// specific functionality -type VersionChecker struct { - majorVersion int - minorVersion int -} - -// NewVersionChecker creates a new VersionChecker -func NewVersionChecker(majorVersion int, minorVersion int) VersionChecker { - return VersionChecker{ - majorVersion: majorVersion, - minorVersion: minorVersion, - } -} - -// IndexEntryValidationEnabled checks the version to determine if -// fileset files of the specified version allow for doing checksum validation -// on individual index entries -func (v *VersionChecker) IndexEntryValidationEnabled() bool { - return v.majorVersion >= 2 || v.majorVersion == 1 && v.minorVersion >= 1 -} diff --git a/src/dbnode/persist/schema/version_checker_test.go b/src/dbnode/persist/schema/version_checker_test.go deleted file mode 100644 index 217c38e1d5..0000000000 --- a/src/dbnode/persist/schema/version_checker_test.go +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package schema - -import ( - "testing" - - "github.com/stretchr/testify/require" -) - -func TestIndexEntryValidationEnabled(t *testing.T) { - checker := NewVersionChecker(1, 1) - require.True(t, checker.IndexEntryValidationEnabled()) - - checker = NewVersionChecker(1, 2) - require.True(t, checker.IndexEntryValidationEnabled()) - - checker = NewVersionChecker(2, 1) - require.True(t, checker.IndexEntryValidationEnabled()) - - checker = NewVersionChecker(2, 0) - require.True(t, checker.IndexEntryValidationEnabled()) -} - -func TestIndexEntryValidationDisabled(t *testing.T) { - checker := NewVersionChecker(1, 0) - require.False(t, checker.IndexEntryValidationEnabled()) -}