From 886793dfc7c83f8e200ff4e30fb92f5335aa0040 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Sun, 6 May 2018 17:18:46 -0400 Subject: [PATCH 01/23] Add read/write of tags to fileset to restore tags, add fs index bootstrapping --- glide.lock | 10 +- glide.yaml | 2 +- persist/fs/fs_mock.go | 38 +- persist/fs/index_lookup_prop_test.go | 6 +- persist/fs/index_read_write_test.go | 4 +- persist/fs/msgpack/decoder.go | 9 +- persist/fs/msgpack/encoder.go | 28 +- persist/fs/msgpack/encoder_test.go | 1 + persist/fs/msgpack/roundtrip_test.go | 58 ++- persist/fs/msgpack/schema.go | 2 +- persist/fs/options.go | 42 ++ persist/fs/persist_manager.go | 3 +- persist/fs/persist_manager_test.go | 26 +- persist/fs/read.go | 64 +-- persist/fs/read_test.go | 26 +- persist/fs/read_write_test.go | 123 ++++-- persist/fs/retriever.go | 23 +- persist/fs/retriever_concurrent_test.go | 8 +- persist/fs/seek.go | 14 +- persist/fs/seek_manager_test.go | 6 +- persist/fs/seek_test.go | 16 +- persist/fs/types.go | 25 +- persist/fs/write.go | 71 +++- persist/schema/types.go | 11 +- persist/types.go | 2 +- services/m3dbnode/config/bootstrap.go | 20 +- storage/block/block.go | 1 + storage/block/block_mock.go | 16 +- storage/block/types.go | 1 + storage/bootstrap/bootstrapper/fs/options.go | 23 +- storage/bootstrap/bootstrapper/fs/source.go | 365 ++++++++++++------ .../bootstrap/bootstrapper/fs/source_test.go | 53 ++- storage/bootstrap/bootstrapper/fs/types.go | 7 + .../bootstrap/bootstrapper/peers/source.go | 2 +- storage/bootstrap/result/options.go | 34 +- storage/bootstrap/result/result_index.go | 31 ++ storage/bootstrap/result/types.go | 7 + storage/index/allocator.go | 38 ++ storage/index/types.go | 5 + storage/namespace_readers.go | 6 +- storage/series/series.go | 15 +- storage/series/series_mock.go | 8 +- storage/series/series_test.go | 4 +- storage/shard.go | 20 +- storage/shard_fetch_blocks_metadata_test.go | 2 +- storage/shard_insert_queue.go | 1 + storage/shard_test.go | 10 +- 47 files changed, 946 insertions(+), 341 deletions(-) create mode 100644 storage/index/allocator.go diff --git a/glide.lock b/glide.lock index 17c45d5348..08d22d9f68 100644 --- a/glide.lock +++ b/glide.lock @@ -1,5 +1,5 @@ -hash: a2f572df5bbbecc1d3ee3d949f77aa46028123ed4bf85170730612154a375195 -updated: 2018-04-30T09:24:07.284273563-04:00 +hash: 1a35734f65b0178ec5e12d06a46808ea9a162d6de7fe89786358e71404453921 +updated: 2018-05-06T00:47:42.371751-04:00 imports: - name: github.com/apache/thrift version: 53dd39833a08ce33582e5ff31fa18bb4735d6731 @@ -207,7 +207,7 @@ imports: - search/query - search/searcher - name: github.com/m3db/m3x - version: 04f23ee000fdef43b2260cf99dd751e8b9cd3dca + version: 0ae10beff2f574adfc42a5b42ecf34e92533be50 vcs: git subpackages: - checked @@ -294,7 +294,7 @@ imports: subpackages: - msgp - name: github.com/uber-go/atomic - version: 8474b86a5a6f79c443ce4b2992817ff32cf208b8 + version: 1ea20fb1cbb1cc08cbd0d913a96dead89aa18289 - name: github.com/uber-go/tally version: 522328b48efad0c6034dba92bf39228694e9d31f subpackages: @@ -322,7 +322,7 @@ imports: - name: github.com/xiang90/probing version: 07dd2e8dfe18522e9c447ba95f2fe95262f63bb2 - name: go.uber.org/atomic - version: 8474b86a5a6f79c443ce4b2992817ff32cf208b8 + version: 1ea20fb1cbb1cc08cbd0d913a96dead89aa18289 - name: go.uber.org/multierr version: 3c4937480c32f4c13a875a1829af76c98ca3d40a - name: go.uber.org/zap diff --git a/glide.yaml b/glide.yaml index 9eb008894b..5f90a32f1b 100644 --- a/glide.yaml +++ b/glide.yaml @@ -1,7 +1,7 @@ package: github.com/m3db/m3db import: - package: github.com/m3db/m3x - version: 04f23ee000fdef43b2260cf99dd751e8b9cd3dca + version: 0ae10beff2f574adfc42a5b42ecf34e92533be50 vcs: git subpackages: - checked diff --git a/persist/fs/fs_mock.go b/persist/fs/fs_mock.go index 42e849eb8a..742953b0c5 100644 --- a/persist/fs/fs_mock.go +++ b/persist/fs/fs_mock.go @@ -83,27 +83,27 @@ func (mr *MockDataFileSetWriterMockRecorder) Open(arg0 interface{}) *gomock.Call } // Write mocks base method -func (m *MockDataFileSetWriter) Write(arg0 ident.ID, arg1 checked.Bytes, arg2 uint32) error { - ret := m.ctrl.Call(m, "Write", arg0, arg1, arg2) +func (m *MockDataFileSetWriter) Write(arg0 ident.ID, arg1 ident.Tags, arg2 checked.Bytes, arg3 uint32) error { + ret := m.ctrl.Call(m, "Write", arg0, arg1, arg2, arg3) ret0, _ := ret[0].(error) return ret0 } // Write indicates an expected call of Write -func (mr *MockDataFileSetWriterMockRecorder) Write(arg0, arg1, arg2 interface{}) *gomock.Call { - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Write", reflect.TypeOf((*MockDataFileSetWriter)(nil).Write), arg0, arg1, arg2) +func (mr *MockDataFileSetWriterMockRecorder) Write(arg0, arg1, arg2, arg3 interface{}) *gomock.Call { + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Write", reflect.TypeOf((*MockDataFileSetWriter)(nil).Write), arg0, arg1, arg2, arg3) } // WriteAll mocks base method -func (m *MockDataFileSetWriter) WriteAll(arg0 ident.ID, arg1 []checked.Bytes, arg2 uint32) error { - ret := m.ctrl.Call(m, "WriteAll", arg0, arg1, arg2) +func (m *MockDataFileSetWriter) WriteAll(arg0 ident.ID, arg1 ident.Tags, arg2 []checked.Bytes, arg3 uint32) error { + ret := m.ctrl.Call(m, "WriteAll", arg0, arg1, arg2, arg3) ret0, _ := ret[0].(error) return ret0 } // WriteAll indicates an expected call of WriteAll -func (mr *MockDataFileSetWriterMockRecorder) WriteAll(arg0, arg1, arg2 interface{}) *gomock.Call { - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WriteAll", reflect.TypeOf((*MockDataFileSetWriter)(nil).WriteAll), arg0, arg1, arg2) +func (mr *MockDataFileSetWriterMockRecorder) WriteAll(arg0, arg1, arg2, arg3 interface{}) *gomock.Call { + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WriteAll", reflect.TypeOf((*MockDataFileSetWriter)(nil).WriteAll), arg0, arg1, arg2, arg3) } // MockDataFileSetReader is a mock of DataFileSetReader interface @@ -202,13 +202,14 @@ func (mr *MockDataFileSetReaderMockRecorder) Range() *gomock.Call { } // Read mocks base method -func (m *MockDataFileSetReader) Read() (ident.ID, checked.Bytes, uint32, error) { +func (m *MockDataFileSetReader) Read() (ident.ID, ident.TagIterator, checked.Bytes, uint32, error) { ret := m.ctrl.Call(m, "Read") ret0, _ := ret[0].(ident.ID) - ret1, _ := ret[1].(checked.Bytes) - ret2, _ := ret[2].(uint32) - ret3, _ := ret[3].(error) - return ret0, ret1, ret2, ret3 + ret1, _ := ret[1].(ident.TagIterator) + ret2, _ := ret[2].(checked.Bytes) + ret3, _ := ret[3].(uint32) + ret4, _ := ret[4].(error) + return ret0, ret1, ret2, ret3, ret4 } // Read indicates an expected call of Read @@ -230,13 +231,14 @@ func (mr *MockDataFileSetReaderMockRecorder) ReadBloomFilter() *gomock.Call { } // ReadMetadata mocks base method -func (m *MockDataFileSetReader) ReadMetadata() (ident.ID, int, uint32, error) { +func (m *MockDataFileSetReader) ReadMetadata() (ident.ID, ident.TagIterator, int, uint32, error) { ret := m.ctrl.Call(m, "ReadMetadata") ret0, _ := ret[0].(ident.ID) - ret1, _ := ret[1].(int) - ret2, _ := ret[2].(uint32) - ret3, _ := ret[3].(error) - return ret0, ret1, ret2, ret3 + ret1, _ := ret[1].(ident.TagIterator) + ret2, _ := ret[2].(int) + ret3, _ := ret[3].(uint32) + ret4, _ := ret[4].(error) + return ret0, ret1, ret2, ret3, ret4 } // ReadMetadata indicates an expected call of ReadMetadata diff --git a/persist/fs/index_lookup_prop_test.go b/persist/fs/index_lookup_prop_test.go index 5e3b889e2d..13fa0a3604 100644 --- a/persist/fs/index_lookup_prop_test.go +++ b/persist/fs/index_lookup_prop_test.go @@ -65,7 +65,9 @@ func TestIndexLookupWriteRead(t *testing.T) { filePathPrefix := filepath.Join(dir, "") defer os.RemoveAll(dir) - options := NewOptions(). + // NB(r): Use testDefaultOpts to avoid allocing pools each + // time we derive options + options := testDefaultOpts. // Make sure that every index entry is also in the summaries file for the // sake of verifying behavior SetIndexSummariesPercent(1). @@ -228,7 +230,7 @@ func readIndexFileOffsets(shardDirPath string, numEntries int, start time.Time) } decoderStream := msgpack.NewDecoderStream(buf) - decoder := msgpack.NewDecoder(NewOptions().DecodingOptions()) + decoder := msgpack.NewDecoder(testDefaultOpts.DecodingOptions()) decoder.Reset(decoderStream) summariesOffsets := map[string]int64{} diff --git a/persist/fs/index_read_write_test.go b/persist/fs/index_read_write_test.go index 712c14892f..92c2e79170 100644 --- a/persist/fs/index_read_write_test.go +++ b/persist/fs/index_read_write_test.go @@ -132,7 +132,7 @@ func TestIndexSimpleReadWrite(t *testing.T) { } func newTestIndexWriter(t *testing.T, filePathPrefix string) IndexFileSetWriter { - writer, err := NewIndexWriter(NewOptions(). + writer, err := NewIndexWriter(testDefaultOpts. SetFilePathPrefix(filePathPrefix). SetWriterBufferSize(testWriterBufferSize)) require.NoError(t, err) @@ -140,7 +140,7 @@ func newTestIndexWriter(t *testing.T, filePathPrefix string) IndexFileSetWriter } func newTestIndexReader(t *testing.T, filePathPrefix string) IndexFileSetReader { - reader, err := NewIndexReader(NewOptions(). + reader, err := NewIndexReader(testDefaultOpts. SetFilePathPrefix(filePathPrefix)) require.NoError(t, err) return reader diff --git a/persist/fs/msgpack/decoder.go b/persist/fs/msgpack/decoder.go index dfe750a2b4..cc8b2b1b8b 100644 --- a/persist/fs/msgpack/decoder.go +++ b/persist/fs/msgpack/decoder.go @@ -273,16 +273,23 @@ func (dec *Decoder) decodeIndexBloomFilterInfo() schema.IndexBloomFilterInfo { } func (dec *Decoder) decodeIndexEntry() schema.IndexEntry { - numFieldsToSkip, _, ok := dec.checkNumFieldsFor(indexEntryType) + numFieldsToSkip, actual, ok := dec.checkNumFieldsFor(indexEntryType) if !ok { return emptyIndexEntry } + var indexEntry schema.IndexEntry indexEntry.Index = dec.decodeVarint() indexEntry.ID, _, _ = dec.decodeBytes() indexEntry.Size = dec.decodeVarint() indexEntry.Offset = dec.decodeVarint() indexEntry.Checksum = dec.decodeVarint() + + if actual < 6 { + return indexEntry + } + indexEntry.EncodedTags, _, _ = dec.decodeBytes() + dec.skip(numFieldsToSkip) if dec.err != nil { return emptyIndexEntry diff --git a/persist/fs/msgpack/encoder.go b/persist/fs/msgpack/encoder.go index e39d327663..6cd7c2abf5 100644 --- a/persist/fs/msgpack/encoder.go +++ b/persist/fs/msgpack/encoder.go @@ -50,18 +50,21 @@ type Encoder struct { encodeBytesFn encodeBytesFn encodeArrayLenFn encodeArrayLenFn - encodeLegacyV1IndexInfo bool + encodeLegacyV1IndexInfo bool + encodeLegacyV1IndexEntry bool } // NewEncoder creates a new encoder func NewEncoder() *Encoder { return newEncoder(newEncoderOptions{ - encodeLegacyV1IndexInfo: false, + encodeLegacyV1IndexInfo: false, + encodeLegacyV1IndexEntry: false, }) } type newEncoderOptions struct { - encodeLegacyV1IndexInfo bool + encodeLegacyV1IndexInfo bool + encodeLegacyV1IndexEntry bool } func newEncoder(opts newEncoderOptions) *Encoder { @@ -81,6 +84,7 @@ func newEncoder(opts newEncoderOptions) *Encoder { // Used primarily for testing enc.encodeLegacyV1IndexInfo = opts.encodeLegacyV1IndexInfo + enc.encodeLegacyV1IndexEntry = opts.encodeLegacyV1IndexEntry return enc } @@ -109,7 +113,7 @@ func (enc *Encoder) EncodeIndexEntry(entry schema.IndexEntry) error { return enc.err } enc.encodeRootObject(indexEntryVersion, indexEntryType) - enc.encodeIndexEntry(entry) + enc.encodeIndexEntryV2(entry) return enc.err } @@ -198,13 +202,27 @@ func (enc *Encoder) encodeIndexBloomFilterInfo(info schema.IndexBloomFilterInfo) enc.encodeVarintFn(info.NumHashesK) } -func (enc *Encoder) encodeIndexEntry(entry schema.IndexEntry) { +// We only keep this method around for the sake of testing +// backwards-compatbility +func (enc *Encoder) encodeIndexEntryV1(entry schema.IndexEntry) { + // Manually encode num fields for testing purposes + enc.encodeArrayLenFn(minNumIndexEntryFields) + enc.encodeVarintFn(entry.Index) + enc.encodeBytesFn(entry.ID) + enc.encodeVarintFn(entry.Size) + enc.encodeVarintFn(entry.Offset) + enc.encodeVarintFn(entry.Checksum) +} + +func (enc *Encoder) encodeIndexEntryV2(entry schema.IndexEntry) { + // Manually encode num fields for testing purposes enc.encodeNumObjectFieldsForFn(indexEntryType) enc.encodeVarintFn(entry.Index) enc.encodeBytesFn(entry.ID) enc.encodeVarintFn(entry.Size) enc.encodeVarintFn(entry.Offset) enc.encodeVarintFn(entry.Checksum) + enc.encodeBytesFn(entry.EncodedTags) } func (enc *Encoder) encodeIndexSummary(summary schema.IndexSummary) { diff --git a/persist/fs/msgpack/encoder_test.go b/persist/fs/msgpack/encoder_test.go index 839e0ea940..a440b0687b 100644 --- a/persist/fs/msgpack/encoder_test.go +++ b/persist/fs/msgpack/encoder_test.go @@ -96,6 +96,7 @@ func testExpectedResultForIndexEntry(t *testing.T, indexEntry schema.IndexEntry) indexEntry.Size, indexEntry.Offset, indexEntry.Checksum, + indexEntry.EncodedTags, } } diff --git a/persist/fs/msgpack/roundtrip_test.go b/persist/fs/msgpack/roundtrip_test.go index 7590f13a54..e45accd0c9 100644 --- a/persist/fs/msgpack/roundtrip_test.go +++ b/persist/fs/msgpack/roundtrip_test.go @@ -48,11 +48,12 @@ var ( } testIndexEntry = schema.IndexEntry{ - Index: 234, - ID: []byte("testIndexEntry"), - Size: 5456, - Offset: 2390423, - Checksum: 134245634534, + Index: 234, + ID: []byte("testIndexEntry"), + Size: 5456, + Offset: 2390423, + Checksum: 134245634534, + EncodedTags: []byte("testEncodedTags"), } testIndexSummary = schema.IndexSummary{ @@ -169,6 +170,53 @@ func TestIndexEntryRoundtrip(t *testing.T) { require.Equal(t, testIndexEntry, res) } +// Make sure the new decoding code can handle the old file format +func TestIndexEntryRoundTripBackwardsCompatibilityV1(t *testing.T) { + var ( + enc = newEncoder(newEncoderOptions{encodeLegacyV1IndexEntry: true}) + dec = testDecoder(t, nil) + ) + + // Set the default values on the fields that did not exist in V1 + // and then restore them at the end of the test - This is required + // because the new decoder won't try and read the new fields from + // the old file format + oldEncodedTags := testIndexEntry.EncodedTags + testIndexEntry.EncodedTags = nil + defer func() { + testIndexEntry.EncodedTags = oldEncodedTags + }() + + enc.EncodeIndexEntry(testIndexEntry) + dec.Reset(NewDecoderStream(enc.Bytes())) + res, err := dec.DecodeIndexEntry() + require.NoError(t, err) + require.Equal(t, testIndexEntry, res) +} + +// Make sure the old decoder code can handle the new file format +func TestIndexEntryRoundTripForwardsCompatibilityV2(t *testing.T) { + var ( + enc = newEncoder(newEncoderOptions{encodeLegacyV1IndexEntry: false}) + dec = testDecoder(t, nil) + ) + + // Set the default values on the fields that did not exist in V1 + // and then restore them at the end of the test - This is required + // because the old decoder won't read the new fields + oldEncodedTags := testIndexEntry.EncodedTags + testIndexEntry.EncodedTags = nil + defer func() { + testIndexEntry.EncodedTags = oldEncodedTags + }() + + enc.EncodeIndexEntry(testIndexEntry) + dec.Reset(NewDecoderStream(enc.Bytes())) + res, err := dec.DecodeIndexEntry() + require.NoError(t, err) + require.Equal(t, testIndexEntry, res) +} + func TestIndexSummaryRoundtrip(t *testing.T) { var ( enc = testEncoder(t) diff --git a/persist/fs/msgpack/schema.go b/persist/fs/msgpack/schema.go index 86c3132e28..19072b389a 100644 --- a/persist/fs/msgpack/schema.go +++ b/persist/fs/msgpack/schema.go @@ -102,7 +102,7 @@ const ( currNumIndexInfoFields = 8 currNumIndexSummariesInfoFields = 1 currNumIndexBloomFilterInfoFields = 2 - currNumIndexEntryFields = 5 + currNumIndexEntryFields = 6 currNumIndexSummaryFields = 3 currNumLogInfoFields = 3 currNumLogEntryFields = 7 diff --git a/persist/fs/options.go b/persist/fs/options.go index 41c7868fdd..866127182d 100644 --- a/persist/fs/options.go +++ b/persist/fs/options.go @@ -21,13 +21,16 @@ package fs import ( + "errors" "fmt" "os" "github.com/m3db/m3db/clock" "github.com/m3db/m3db/persist/fs/msgpack" "github.com/m3db/m3db/runtime" + "github.com/m3db/m3db/serialize" "github.com/m3db/m3x/instrument" + "github.com/m3db/m3x/pool" ) const ( @@ -60,6 +63,9 @@ var ( defaultFilePathPrefix = os.TempDir() defaultNewFileMode = os.FileMode(0666) defaultNewDirectoryMode = os.ModeDir | os.FileMode(0755) + + errTagEncoderPoolNotSet = errors.New("tag encoder pool is not set") + errTagDecoderPoolNotSet = errors.New("tag decoder pool is not set") ) type options struct { @@ -78,10 +84,18 @@ type options struct { seekReaderBufferSize int mmapEnableHugePages bool mmapHugePagesThreshold int64 + tagEncoderPool serialize.TagEncoderPool + tagDecoderPool serialize.TagDecoderPool } // NewOptions creates a new set of fs options func NewOptions() Options { + tagEncoderPool := serialize.NewTagEncoderPool( + serialize.NewTagEncoderOptions(), pool.NewObjectPoolOptions()) + tagEncoderPool.Init() + tagDecoderPool := serialize.NewTagDecoderPool( + serialize.NewTagDecoderOptions(), pool.NewObjectPoolOptions()) + tagDecoderPool.Init() return &options{ clockOpts: clock.NewOptions(), instrumentOpts: instrument.NewOptions(), @@ -98,6 +112,8 @@ func NewOptions() Options { seekReaderBufferSize: defaultSeekReaderBufferSize, mmapEnableHugePages: defaultMmapEnableHugePages, mmapHugePagesThreshold: defaultMmapHugePagesThreshold, + tagEncoderPool: tagEncoderPool, + tagDecoderPool: tagDecoderPool, } } @@ -112,6 +128,12 @@ func (o *options) Validate() error { "invalid index bloom filter false positive percent, must be >= 0 and <= 1: instead %f", o.indexBloomFilterFalsePositivePercent) } + if o.tagEncoderPool == nil { + return errTagEncoderPoolNotSet + } + if o.tagDecoderPool == nil { + return errTagDecoderPoolNotSet + } return nil } @@ -264,3 +286,23 @@ func (o *options) SetMmapHugeTLBThreshold(value int64) Options { func (o *options) MmapHugeTLBThreshold() int64 { return o.mmapHugePagesThreshold } + +func (o *options) SetTagEncoderPool(value serialize.TagEncoderPool) Options { + opts := *o + opts.tagEncoderPool = value + return &opts +} + +func (o *options) TagEncoderPool() serialize.TagEncoderPool { + return o.tagEncoderPool +} + +func (o *options) SetTagDecoderPool(value serialize.TagDecoderPool) Options { + opts := *o + opts.tagDecoderPool = value + return &opts +} + +func (o *options) TagDecoderPool() serialize.TagDecoderPool { + return o.tagDecoderPool +} diff --git a/persist/fs/persist_manager.go b/persist/fs/persist_manager.go index cf04327ce6..7eed6c11d2 100644 --- a/persist/fs/persist_manager.go +++ b/persist/fs/persist_manager.go @@ -122,6 +122,7 @@ func NewPersistManager(opts Options) (persist.Manager, error) { func (pm *persistManager) persist( id ident.ID, + tags ident.Tags, segment ts.Segment, checksum uint32, ) error { @@ -153,7 +154,7 @@ func (pm *persistManager) persist( pm.segmentHolder[0] = segment.Head pm.segmentHolder[1] = segment.Tail - err := pm.writer.WriteAll(id, pm.segmentHolder, checksum) + err := pm.writer.WriteAll(id, tags, pm.segmentHolder, checksum) pm.count++ pm.bytesWritten += int64(segment.Len()) diff --git a/persist/fs/persist_manager_test.go b/persist/fs/persist_manager_test.go index c32c4b8d31..6e738403ad 100644 --- a/persist/fs/persist_manager_test.go +++ b/persist/fs/persist_manager_test.go @@ -50,7 +50,7 @@ func testManager( ) (*persistManager, *MockDataFileSetWriter, Options) { dir := createTempDir(t) - opts := NewOptions(). + opts := testDefaultOpts. SetFilePathPrefix(dir). SetWriterBufferSize(10) @@ -159,12 +159,13 @@ func TestPersistenceManagerPrepareSuccess(t *testing.T) { var ( id = ident.StringID("foo") + tags = ident.Tags{ident.StringTag("bar", "baz")} head = checked.NewBytes([]byte{0x1, 0x2}, nil) tail = checked.NewBytes([]byte{0x3, 0x4}, nil) segment = ts.NewSegment(head, tail, ts.FinalizeNone) checksum = digest.SegmentChecksum(segment) ) - writer.EXPECT().WriteAll(id, gomock.Any(), checksum).Return(nil) + writer.EXPECT().WriteAll(id, tags, gomock.Any(), checksum).Return(nil) writer.EXPECT().Close() flush, err := pm.StartPersist() @@ -189,7 +190,7 @@ func TestPersistenceManagerPrepareSuccess(t *testing.T) { require.Nil(t, err) - require.Nil(t, prepared.Persist(id, segment, checksum)) + require.Nil(t, prepared.Persist(id, tags, segment, checksum)) require.True(t, pm.start.Equal(now)) require.Equal(t, 124, pm.count) @@ -230,6 +231,7 @@ func TestPersistenceManagerNoRateLimit(t *testing.T) { now time.Time slept time.Duration id = ident.StringID("foo") + tags = ident.Tags{ident.StringTag("bar", "baz")} head = checked.NewBytes([]byte{0x1, 0x2}, nil) tail = checked.NewBytes([]byte{0x3}, nil) segment = ts.NewSegment(head, tail, ts.FinalizeNone) @@ -239,7 +241,7 @@ func TestPersistenceManagerNoRateLimit(t *testing.T) { pm.nowFn = func() time.Time { return now } pm.sleepFn = func(d time.Duration) { slept += d } - writer.EXPECT().WriteAll(id, pm.segmentHolder, checksum).Return(nil).Times(2) + writer.EXPECT().WriteAll(id, tags, pm.segmentHolder, checksum).Return(nil).Times(2) flush, err := pm.StartPersist() require.NoError(t, err) @@ -259,11 +261,11 @@ func TestPersistenceManagerNoRateLimit(t *testing.T) { // Start persistence now = time.Now() - require.NoError(t, prepared.Persist(id, segment, checksum)) + require.NoError(t, prepared.Persist(id, tags, segment, checksum)) // Advance time and write again now = now.Add(time.Millisecond) - require.NoError(t, prepared.Persist(id, segment, checksum)) + require.NoError(t, prepared.Persist(id, tags, segment, checksum)) // Check there is no rate limiting require.Equal(t, time.Duration(0), slept) @@ -303,7 +305,7 @@ func TestPersistenceManagerWithRateLimit(t *testing.T) { BlockSize: testBlockSize, } writer.EXPECT().Open(writerOpts).Return(nil).Times(iter) - writer.EXPECT().WriteAll(id, pm.segmentHolder, checksum).Return(nil).AnyTimes() + writer.EXPECT().WriteAll(id, nil, pm.segmentHolder, checksum).Return(nil).AnyTimes() writer.EXPECT().Close().Times(iter) // Enable rate limiting @@ -342,21 +344,21 @@ func TestPersistenceManagerWithRateLimit(t *testing.T) { // Start persistence now = time.Now() - require.NoError(t, prepared.Persist(id, segment, checksum)) + require.NoError(t, prepared.Persist(id, nil, segment, checksum)) // Assert we don't rate limit if the count is not enough yet - require.NoError(t, prepared.Persist(id, segment, checksum)) + require.NoError(t, prepared.Persist(id, nil, segment, checksum)) require.Equal(t, time.Duration(0), slept) // Advance time and check we rate limit if the disk throughput exceeds the limit now = now.Add(time.Microsecond) - require.NoError(t, prepared.Persist(id, segment, checksum)) + require.NoError(t, prepared.Persist(id, nil, segment, checksum)) require.Equal(t, time.Duration(1861), slept) // Advance time and check we don't rate limit if the disk throughput is below the limit - require.NoError(t, prepared.Persist(id, segment, checksum)) + require.NoError(t, prepared.Persist(id, nil, segment, checksum)) now = now.Add(time.Second - time.Microsecond) - require.NoError(t, prepared.Persist(id, segment, checksum)) + require.NoError(t, prepared.Persist(id, nil, segment, checksum)) require.Equal(t, time.Duration(1861), slept) require.Equal(t, int64(15), pm.bytesWritten) diff --git a/persist/fs/read.go b/persist/fs/read.go index 24876bb0f3..64b03ae4ea 100644 --- a/persist/fs/read.go +++ b/persist/fs/read.go @@ -33,6 +33,7 @@ import ( "github.com/m3db/m3db/persist" "github.com/m3db/m3db/persist/fs/msgpack" "github.com/m3db/m3db/persist/schema" + "github.com/m3db/m3db/serialize" "github.com/m3db/m3db/x/mmap" "github.com/m3db/m3x/checked" xerrors "github.com/m3db/m3x/errors" @@ -81,6 +82,7 @@ type reader struct { decoder *msgpack.Decoder digestBuf digest.Buffer bytesPool pool.CheckedBytesPool + tagDecoderPool serialize.TagDecoderPool expectedInfoDigest uint32 expectedIndexDigest uint32 @@ -118,6 +120,7 @@ func NewReader( decoder: msgpack.NewDecoder(opts.DecodingOptions()), digestBuf: digest.NewBuffer(), bytesPool: bytesPool, + tagDecoderPool: opts.TagDecoderPool(), }, nil } @@ -317,18 +320,17 @@ func (r *reader) readIndexAndSortByOffsetAsc() error { return nil } -func (r *reader) Read() (ident.ID, checked.Bytes, uint32, error) { - var none ident.ID +func (r *reader) Read() (ident.ID, ident.TagIterator, checked.Bytes, uint32, error) { if r.entries > 0 && len(r.indexEntriesByOffsetAsc) < r.entries { // Have not read the index yet, this is required when reading // data as we need each index entry in order by by the offset ascending if err := r.readIndexAndSortByOffsetAsc(); err != nil { - return none, nil, 0, err + return nil, nil, nil, 0, err } } if r.entriesRead >= r.entries { - return none, nil, 0, io.EOF + return nil, nil, nil, 0, io.EOF } entry := r.indexEntriesByOffsetAsc[r.entriesRead] @@ -347,27 +349,33 @@ func (r *reader) Read() (ident.ID, checked.Bytes, uint32, error) { n, err := r.dataReader.Read(data.Bytes()) if err != nil { - return none, nil, 0, err + return nil, nil, nil, 0, err } if n != int(entry.Size) { - return none, nil, 0, errReadNotExpectedSize + return nil, nil, nil, 0, errReadNotExpectedSize } - r.entriesRead++ + id := r.entryID(entry.ID) + tags := r.entryEncodedTags(entry.EncodedTags) - return r.entryID(entry.ID), data, uint32(entry.Checksum), nil + r.entriesRead++ + return id, tags, data, uint32(entry.Checksum), nil } -func (r *reader) ReadMetadata() (id ident.ID, length int, checksum uint32, err error) { - var none ident.ID +func (r *reader) ReadMetadata() (id ident.ID, tags ident.TagIterator, length int, checksum uint32, err error) { if r.metadataRead >= r.entries { - return none, 0, 0, io.EOF + err = io.EOF + return } entry := r.indexEntriesByOffsetAsc[r.metadataRead] + id = r.entryID(entry.ID) + tags = r.entryEncodedTags(entry.EncodedTags) + length = int(entry.Size) + checksum = uint32(entry.Checksum) r.metadataRead++ - return r.entryID(entry.ID), int(entry.Size), uint32(entry.Checksum), nil + return } func (r *reader) ReadBloomFilter() (*ManagedConcurrentBloomFilter, error) { @@ -380,21 +388,31 @@ func (r *reader) ReadBloomFilter() (*ManagedConcurrentBloomFilter, error) { ) } -func (r *reader) entryID(id []byte) ident.ID { - var idClone checked.Bytes +func (r *reader) entryBytes(bytes []byte) checked.Bytes { + var bytesClone checked.Bytes if r.bytesPool != nil { - idClone = r.bytesPool.Get(len(id)) - idClone.IncRef() - defer idClone.DecRef() + bytesClone = r.bytesPool.Get(len(bytes)) } else { - idClone = checked.NewBytes(make([]byte, 0, len(id)), nil) - idClone.IncRef() - defer idClone.DecRef() + bytesClone = checked.NewBytes(make([]byte, 0, len(bytes)), nil) } + bytesClone.IncRef() + bytesClone.AppendAll(bytes) + bytesClone.DecRef() + return bytesClone +} - idClone.AppendAll(id) +func (r *reader) entryID(id []byte) ident.ID { + return ident.BinaryID(r.entryBytes(id)) +} - return ident.BinaryID(idClone) +func (r *reader) entryEncodedTags(encodedTags []byte) ident.TagIterator { + if len(encodedTags) == 0 { + // No tags set for this entry, return an empty tag iterator + return ident.EmptyTagIterator + } + decoder := r.tagDecoderPool.Get() + decoder.Reset(r.entryBytes(encodedTags)) + return decoder } // NB(xichen): Validate should be called after all data is read because @@ -469,6 +487,7 @@ func (r *reader) Close() error { decoder := r.decoder digestBuf := r.digestBuf bytesPool := r.bytesPool + tagDecoderPool := r.tagDecoderPool indexEntriesByOffsetAsc := r.indexEntriesByOffsetAsc // Reset struct @@ -486,6 +505,7 @@ func (r *reader) Close() error { r.decoder = decoder r.digestBuf = digestBuf r.bytesPool = bytesPool + r.tagDecoderPool = tagDecoderPool r.indexEntriesByOffsetAsc = indexEntriesByOffsetAsc return multiErr.FinalError() diff --git a/persist/fs/read_test.go b/persist/fs/read_test.go index 36b0b1eee1..ff8e7a99d7 100644 --- a/persist/fs/read_test.go +++ b/persist/fs/read_test.go @@ -33,6 +33,7 @@ import ( "github.com/m3db/m3db/digest" "github.com/m3db/m3db/persist/fs/msgpack" "github.com/m3db/m3db/persist/schema" + "github.com/m3db/m3db/serialize" "github.com/m3db/m3db/x/mmap" "github.com/m3db/m3x/checked" "github.com/m3db/m3x/ident" @@ -49,9 +50,11 @@ const ( ) var ( - testWriterStart = time.Now() - testBlockSize = 2 * time.Hour - testBytesPool pool.CheckedBytesPool + testWriterStart = time.Now() + testBlockSize = 2 * time.Hour + testDefaultOpts = NewOptions() // To avoid allocing pools each test exec + testBytesPool pool.CheckedBytesPool + testTagDecoderPool serialize.TagDecoderPool ) // NB(r): This is kind of brittle, but basically msgpack expects a buffered @@ -86,10 +89,13 @@ func init() { return pool.NewBytesPool(s, nil) }) testBytesPool.Init() + testTagDecoderPool = serialize.NewTagDecoderPool( + serialize.NewTagDecoderOptions(), pool.NewObjectPoolOptions()) + testTagDecoderPool.Init() } func newTestReader(t *testing.T, filePathPrefix string) DataFileSetReader { - reader, err := NewReader(testBytesPool, NewOptions(). + reader, err := NewReader(testBytesPool, testDefaultOpts. SetFilePathPrefix(filePathPrefix). SetInfoReaderBufferSize(testReaderBufferSize). SetDataReaderBufferSize(testReaderBufferSize)) @@ -135,7 +141,7 @@ func TestReadEmptyIndexUnreadData(t *testing.T) { err = r.Open(rOpenOpts) assert.NoError(t, err) - _, _, _, err = r.Read() + _, _, _, _, err = r.Read() assert.Error(t, err) assert.NoError(t, r.Close()) @@ -165,6 +171,7 @@ func TestReadDataError(t *testing.T) { require.NoError(t, err) require.NoError(t, w.Write( ident.StringID("foo"), + nil, bytesRefd([]byte{1, 2, 3}), digest.Checksum([]byte{1, 2, 3}))) require.NoError(t, w.Close()) @@ -189,7 +196,7 @@ func TestReadDataError(t *testing.T) { mockReader.EXPECT().Read(gomock.Any()).Return(0, fmt.Errorf("an error")) reader.dataReader = mockReader - _, _, _, err = r.Read() + _, _, _, _, err = r.Read() assert.Error(t, err) // Cleanly close @@ -220,6 +227,7 @@ func TestReadDataUnexpectedSize(t *testing.T) { assert.NoError(t, w.Write( ident.StringID("foo"), + nil, bytesRefd([]byte{1, 2, 3}), digest.Checksum([]byte{1, 2, 3}))) assert.NoError(t, w.Close()) @@ -238,7 +246,7 @@ func TestReadDataUnexpectedSize(t *testing.T) { err = r.Open(rOpenOpts) assert.NoError(t, err) - _, _, _, err = r.Read() + _, _, _, _, err = r.Read() assert.Error(t, err) assert.Equal(t, errReadNotExpectedSize, err) @@ -301,6 +309,7 @@ func testReadOpen(t *testing.T, fileData map[string][]byte) { assert.NoError(t, w.Write( ident.StringID("foo"), + nil, bytesRefd([]byte{0x1}), digest.Checksum([]byte{0x1}))) assert.NoError(t, w.Close()) @@ -394,6 +403,7 @@ func TestReadValidate(t *testing.T) { assert.NoError(t, w.Write( ident.StringID("foo"), + nil, bytesRefd([]byte{0x1}), digest.Checksum([]byte{0x1}))) require.NoError(t, w.Close()) @@ -407,7 +417,7 @@ func TestReadValidate(t *testing.T) { }, } require.NoError(t, r.Open(rOpenOpts)) - _, _, _, err := r.Read() + _, _, _, _, err := r.Read() require.NoError(t, err) // Mutate expected data checksum to simulate data corruption diff --git a/persist/fs/read_write_test.go b/persist/fs/read_write_test.go index f00722e31d..6f04dc9102 100644 --- a/persist/fs/read_write_test.go +++ b/persist/fs/read_write_test.go @@ -40,11 +40,27 @@ import ( type testEntry struct { id string + tags map[string]string data []byte } +func (e testEntry) ID() ident.ID { + return ident.StringID(e.id) +} + +func (e testEntry) Tags() ident.Tags { + if e.tags == nil { + return nil + } + var tags ident.Tags + for key, value := range e.tags { + tags = append(tags, ident.StringTag(key, value)) + } + return tags +} + func newTestWriter(t *testing.T, filePathPrefix string) DataFileSetWriter { - writer, err := NewWriter(NewOptions(). + writer, err := NewWriter(testDefaultOpts. SetFilePathPrefix(filePathPrefix). SetWriterBufferSize(testWriterBufferSize)) require.NoError(t, err) @@ -65,7 +81,8 @@ func writeTestData(t *testing.T, w DataFileSetWriter, shard uint32, timestamp ti for i := range entries { assert.NoError(t, w.Write( - ident.StringID(entries[i].id), + entries[i].ID(), + entries[i].Tags(), bytesRefd(entries[i].data), digest.Checksum(entries[i].data))) } @@ -121,12 +138,25 @@ func readTestData(t *testing.T, r DataFileSetReader, shard uint32, timestamp tim for i := 0; i < r.Entries(); i++ { switch underTest { case readTestTypeData: - id, data, checksum, err := r.Read() + id, tags, data, checksum, err := r.Read() require.NoError(t, err) data.IncRef() + // Assert id assert.Equal(t, entries[i].id, id.String()) + + // Assert tags + if entries[i].tags != nil { + require.NotNil(t, tags) + require.Equal(t, len(entries[i].tags), tags.Remaining()) + for tags.Next() { + curr := tags.Current() + assert.Equal(t, entries[i].tags[curr.Name.String()], curr.Value.String()) + } + assert.Equal(t, 0, tags.Remaining()) + } + assert.True(t, bytes.Equal(entries[i].data, data.Bytes())) assert.Equal(t, digest.Checksum(entries[i].data), checksum) @@ -137,13 +167,29 @@ func readTestData(t *testing.T, r DataFileSetReader, shard uint32, timestamp tim assert.True(t, bloomFilter.Test(id.Data().Bytes())) id.Finalize() + if tags != nil { + tags.Close() + } data.DecRef() data.Finalize() case readTestTypeMetadata: - id, length, checksum, err := r.ReadMetadata() + id, tags, length, checksum, err := r.ReadMetadata() require.NoError(t, err) + // Assert id assert.True(t, id.Equal(id)) + + // Assert tags + if entries[i].tags != nil { + require.NotNil(t, tags) + require.Equal(t, len(entries[i].tags), tags.Remaining()) + for tags.Next() { + curr := tags.Current() + assert.Equal(t, entries[i].tags[curr.Name.String()], curr.Value.String()) + } + assert.Equal(t, 0, tags.Remaining()) + } + assert.Equal(t, digest.Checksum(entries[i].data), checksum) assert.Equal(t, len(entries[i].data), length) @@ -154,6 +200,9 @@ func readTestData(t *testing.T, r DataFileSetReader, shard uint32, timestamp tim assert.True(t, bloomFilter.Test(id.Data().Bytes())) id.Finalize() + if tags != nil { + tags.Close() + } } } @@ -167,11 +216,14 @@ func TestSimpleReadWrite(t *testing.T) { defer os.RemoveAll(dir) entries := []testEntry{ - {"foo", []byte{1, 2, 3}}, - {"bar", []byte{4, 5, 6}}, - {"baz", make([]byte, 65536)}, - {"cat", make([]byte, 100000)}, - {"echo", []byte{7, 8, 9}}, + {"foo", nil, []byte{1, 2, 3}}, + {"bar", nil, []byte{4, 5, 6}}, + {"baz", nil, make([]byte, 65536)}, + {"cat", nil, make([]byte, 100000)}, + {"foo+bar=baz,qux=qaz", map[string]string{ + "bar": "baz", + "qux": "qaz", + }, []byte{7, 8, 9}}, } w := newTestWriter(t, filePathPrefix) @@ -187,8 +239,8 @@ func TestDuplicateWrite(t *testing.T) { defer os.RemoveAll(dir) entries := []testEntry{ - {"foo", []byte{1, 2, 3}}, - {"foo", []byte{4, 5, 6}}, + {"foo", nil, []byte{1, 2, 3}}, + {"foo", nil, []byte{4, 5, 6}}, } w := newTestWriter(t, filePathPrefix) @@ -205,7 +257,8 @@ func TestDuplicateWrite(t *testing.T) { for i := range entries { require.NoError(t, w.Write( - ident.StringID(entries[i].id), + entries[i].ID(), + entries[i].Tags(), bytesRefd(entries[i].data), digest.Checksum(entries[i].data))) } @@ -218,11 +271,14 @@ func TestReadWithReusedReader(t *testing.T) { defer os.RemoveAll(dir) entries := []testEntry{ - {"foo", []byte{1, 2, 3}}, - {"bar", []byte{4, 5, 6}}, - {"baz", make([]byte, 65536)}, - {"cat", make([]byte, 100000)}, - {"echo", []byte{7, 8, 9}}, + {"foo", nil, []byte{1, 2, 3}}, + {"bar", nil, []byte{4, 5, 6}}, + {"baz", nil, make([]byte, 65536)}, + {"cat", nil, make([]byte, 100000)}, + {"foo+bar=baz,qux=qaz", map[string]string{ + "bar": "baz", + "qux": "qaz", + }, []byte{7, 8, 9}}, } w := newTestWriter(t, filePathPrefix) @@ -240,11 +296,14 @@ func TestInfoReadWrite(t *testing.T) { defer os.RemoveAll(dir) entries := []testEntry{ - {"foo", []byte{1, 2, 3}}, - {"bar", []byte{4, 5, 6}}, - {"baz", make([]byte, 65536)}, - {"cat", make([]byte, 100000)}, - {"echo", []byte{7, 8, 9}}, + {"foo", nil, []byte{1, 2, 3}}, + {"bar", nil, []byte{4, 5, 6}}, + {"baz", nil, make([]byte, 65536)}, + {"cat", nil, make([]byte, 100000)}, + {"foo+bar=baz,qux=qaz", map[string]string{ + "bar": "baz", + "qux": "qaz", + }, []byte{7, 8, 9}}, } w := newTestWriter(t, filePathPrefix) @@ -269,11 +328,11 @@ func TestReusingReaderWriter(t *testing.T) { allEntries := [][]testEntry{ { - {"foo", []byte{1, 2, 3}}, - {"bar", []byte{4, 5, 6}}, + {"foo", nil, []byte{1, 2, 3}}, + {"bar", nil, []byte{4, 5, 6}}, }, { - {"baz", []byte{7, 8, 9}}, + {"baz", nil, []byte{7, 8, 9}}, }, {}, } @@ -294,8 +353,8 @@ func TestReusingWriterAfterWriteError(t *testing.T) { defer os.RemoveAll(dir) entries := []testEntry{ - {"foo", []byte{1, 2, 3}}, - {"bar", []byte{4, 5, 6}}, + {"foo", nil, []byte{1, 2, 3}}, + {"bar", nil, []byte{4, 5, 6}}, } w := newTestWriter(t, filePathPrefix) shard := uint32(0) @@ -309,14 +368,16 @@ func TestReusingWriterAfterWriteError(t *testing.T) { require.NoError(t, w.Open(writerOpts)) require.NoError(t, w.Write( - ident.StringID(entries[0].id), + entries[0].ID(), + entries[0].Tags(), bytesRefd(entries[0].data), digest.Checksum(entries[0].data))) // Intentionally force a writer error. w.(*writer).err = errors.New("foo") require.Equal(t, "foo", w.Write( - ident.StringID(entries[1].id), + entries[1].ID(), + entries[1].Tags(), bytesRefd(entries[1].data), digest.Checksum(entries[1].data)).Error()) w.Close() @@ -358,7 +419,7 @@ func TestWriterOnlyWritesNonNilBytes(t *testing.T) { } require.NoError(t, w.Open(writerOpts)) - w.WriteAll(ident.StringID("foo"), []checked.Bytes{ + w.WriteAll(ident.StringID("foo"), nil, []checked.Bytes{ checkedBytes([]byte{1, 2, 3}), nil, checkedBytes([]byte{4, 5, 6}), @@ -368,6 +429,6 @@ func TestWriterOnlyWritesNonNilBytes(t *testing.T) { r := newTestReader(t, filePathPrefix) readTestData(t, r, 0, testWriterStart, []testEntry{ - {"foo", []byte{1, 2, 3, 4, 5, 6}}, + {"foo", nil, []byte{1, 2, 3, 4, 5, 6}}, }) } diff --git a/persist/fs/retriever.go b/persist/fs/retriever.go index d1e74bfab6..453d71d25c 100644 --- a/persist/fs/retriever.go +++ b/persist/fs/retriever.go @@ -285,6 +285,8 @@ func (r *blockRetriever) fetchBatch( } sort.Sort(retrieveRequestByOffsetAsc(reqs)) + tagDecoderPool := r.fsOpts.TagDecoderPool() + // Seek and execute all requests for _, req := range reqs { var data checked.Bytes @@ -300,7 +302,9 @@ func (r *blockRetriever) fetchBatch( } } - var seg, onRetrieveSeg ts.Segment + var ( + seg, onRetrieveSeg ts.Segment + ) if data != nil { seg = ts.NewSegment(data, nil, ts.FinalizeHead) } @@ -316,6 +320,15 @@ func (r *blockRetriever) fetchBatch( onRetrieveSeg = ts.NewSegment(dataCopy, nil, ts.FinalizeHead) dataCopy.AppendAll(data.Bytes()) } + if tags := req.indexEntry.EncodedTags; len(tags) != 0 { + tagsCopy := r.bytesPool.Get(len(tags)) + tagsCopy.IncRef() + tagsCopy.AppendAll(req.indexEntry.EncodedTags) + tagsCopy.DecRef() + decoder := tagDecoderPool.Get() + decoder.Reset(tagsCopy) + req.tags = decoder + } } // Complete request @@ -329,7 +342,7 @@ func (r *blockRetriever) fetchBatch( go func(r *retrieveRequest) { // Call the onRetrieve callback and finalize - r.onRetrieve.OnRetrieveBlock(r.id, r.start, onRetrieveSeg) + r.onRetrieve.OnRetrieveBlock(r.id, r.tags, r.start, onRetrieveSeg) r.onCallerOrRetrieverDone() }(req) } @@ -491,6 +504,7 @@ type retrieveRequest struct { pool *reqPool id ident.ID + tags ident.TagIterator start time.Time onRetrieve block.OnRetrieveBlock @@ -523,6 +537,10 @@ func (req *retrieveRequest) onCallerOrRetrieverDone() { } req.id.Finalize() req.id = nil + if req.tags != nil { + req.tags.Close() + req.tags = nil + } req.reader.Finalize() req.reader = nil req.pool.Put(req) @@ -560,6 +578,7 @@ func (req *retrieveRequest) resetForReuse() { req.finalizes = 0 req.shard = 0 req.id = nil + req.tags = ident.EmptyTagIterator req.start = time.Time{} req.onRetrieve = nil req.indexEntry = IndexEntry{} diff --git a/persist/fs/retriever_concurrent_test.go b/persist/fs/retriever_concurrent_test.go index f81fbbf668..177c5f5bba 100644 --- a/persist/fs/retriever_concurrent_test.go +++ b/persist/fs/retriever_concurrent_test.go @@ -127,7 +127,7 @@ func testBlockRetrieverHighConcurrentSeeks(t *testing.T, shouldCacheShardIndices require.NoError(t, err) defer os.RemoveAll(dir) filePathPrefix := filepath.Join(dir, "") - fsOpts := NewOptions().SetFilePathPrefix(filePathPrefix) + fsOpts := testDefaultOpts.SetFilePathPrefix(filePathPrefix) fetchConcurrency := 4 seekConcurrency := 4 * fetchConcurrency @@ -182,7 +182,7 @@ func testBlockRetrieverHighConcurrentSeeks(t *testing.T, shouldCacheShardIndices } shardData[shard][idString][xtime.ToUnixNano(blockStart)] = data - err := w.Write(id, data, digest.Checksum(data.Bytes())) + err := w.Write(id, nil, data, digest.Checksum(data.Bytes())) require.NoError(t, err) } closer() @@ -272,7 +272,7 @@ func TestBlockRetrieverIDDoesNotExist(t *testing.T) { filePathPrefix := filepath.Join(dir, "") // Setup constants and config - fsOpts := NewOptions().SetFilePathPrefix(filePathPrefix) + fsOpts := testDefaultOpts.SetFilePathPrefix(filePathPrefix) rOpts := testNs1Metadata(t).Options().RetentionOptions() shard := uint32(0) blockStart := time.Now().Truncate(rOpts.BlockSize()) @@ -290,7 +290,7 @@ func TestBlockRetrieverIDDoesNotExist(t *testing.T) { data := checked.NewBytes([]byte("Hello world!"), nil) data.IncRef() defer data.DecRef() - err = w.Write(ident.StringID("exists"), data, digest.Checksum(data.Bytes())) + err = w.Write(ident.StringID("exists"), nil, data, digest.Checksum(data.Bytes())) assert.NoError(t, err) closer() diff --git a/persist/fs/seek.go b/persist/fs/seek.go index fb90b7e7fd..dd31eda20a 100644 --- a/persist/fs/seek.go +++ b/persist/fs/seek.go @@ -101,9 +101,10 @@ type seeker struct { // IndexEntry is an entry from the index file which can be passed to // SeekUsingIndexEntry to seek to the data for that entry type IndexEntry struct { - Size uint32 - Checksum uint32 - Offset int64 + Size uint32 + Checksum uint32 + Offset int64 + EncodedTags []byte } // NewSeeker returns a new seeker. @@ -425,9 +426,10 @@ func (s *seeker) SeekIndexEntry(id ident.ID) (IndexEntry, error) { comparison := bytes.Compare(entry.ID, idBytes) if comparison == 0 { return IndexEntry{ - Size: uint32(entry.Size), - Checksum: uint32(entry.Checksum), - Offset: entry.Offset, + Size: uint32(entry.Size), + Checksum: uint32(entry.Checksum), + Offset: entry.Offset, + EncodedTags: entry.EncodedTags, }, nil } diff --git a/persist/fs/seek_manager_test.go b/persist/fs/seek_manager_test.go index be29c3a60c..86a752a655 100644 --- a/persist/fs/seek_manager_test.go +++ b/persist/fs/seek_manager_test.go @@ -36,7 +36,7 @@ func TestSeekerManagerCacheShardIndices(t *testing.T) { defer leaktest.CheckTimeout(t, 1*time.Minute)() shards := []uint32{2, 5, 9, 478, 1023} - m := NewSeekerManager(nil, NewOptions(), NewBlockRetrieverOptions().FetchConcurrency()).(*seekerManager) + m := NewSeekerManager(nil, testDefaultOpts, NewBlockRetrieverOptions().FetchConcurrency()).(*seekerManager) var byTimes []*seekersByTime m.openAnyUnopenSeekersFn = func(byTime *seekersByTime) error { byTimes = append(byTimes, byTime) @@ -75,7 +75,7 @@ func TestSeekerManagerBorrowOpenSeekersLazy(t *testing.T) { ctrl := gomock.NewController(t) shards := []uint32{2, 5, 9, 478, 1023} - m := NewSeekerManager(nil, NewOptions(), NewBlockRetrieverOptions().FetchConcurrency()).(*seekerManager) + m := NewSeekerManager(nil, testDefaultOpts, NewBlockRetrieverOptions().FetchConcurrency()).(*seekerManager) m.newOpenSeekerFn = func( shard uint32, blockStart time.Time, @@ -118,7 +118,7 @@ func TestSeekerManagerOpenCloseLoop(t *testing.T) { ctrl := gomock.NewController(t) shards := []uint32{2, 5, 9, 478, 1023} - m := NewSeekerManager(nil, NewOptions(), NewBlockRetrieverOptions().FetchConcurrency()).(*seekerManager) + m := NewSeekerManager(nil, testDefaultOpts, NewBlockRetrieverOptions().FetchConcurrency()).(*seekerManager) clockOpts := m.opts.ClockOptions() now := clockOpts.NowFn()() startNano := xtime.ToUnixNano(now) diff --git a/persist/fs/seek_test.go b/persist/fs/seek_test.go index 996a24c264..1185341033 100644 --- a/persist/fs/seek_test.go +++ b/persist/fs/seek_test.go @@ -43,8 +43,8 @@ func newTestSeeker(filePathPrefix string) DataFileSetSeeker { return pool.NewBytesPool(s, nil) }) bytesPool.Init() - return NewSeeker( - filePathPrefix, testReaderBufferSize, testReaderBufferSize, testReaderBufferSize, bytesPool, false, nil, NewOptions()) + return NewSeeker(filePathPrefix, testReaderBufferSize, testReaderBufferSize, + testReaderBufferSize, bytesPool, false, nil, testDefaultOpts) } func TestSeekEmptyIndex(t *testing.T) { @@ -101,6 +101,7 @@ func TestSeekDataUnexpectedSize(t *testing.T) { assert.NoError(t, w.Write( ident.StringID("foo"), + ident.Tags(nil), bytesRefd([]byte{1, 2, 3}), digest.Checksum([]byte{1, 2, 3}))) assert.NoError(t, w.Close()) @@ -142,6 +143,7 @@ func TestSeekBadChecksum(t *testing.T) { // Write data with wrong checksum assert.NoError(t, w.Write( ident.StringID("foo"), + ident.Tags(nil), bytesRefd([]byte{1, 2, 3}), digest.Checksum([]byte{1, 2, 4}))) assert.NoError(t, w.Close()) @@ -180,14 +182,17 @@ func TestSeek(t *testing.T) { assert.NoError(t, err) assert.NoError(t, w.Write( ident.StringID("foo1"), + ident.Tags{ident.StringTag("num", "1")}, bytesRefd([]byte{1, 2, 1}), digest.Checksum([]byte{1, 2, 1}))) assert.NoError(t, w.Write( ident.StringID("foo2"), + ident.Tags{ident.StringTag("num", "2")}, bytesRefd([]byte{1, 2, 2}), digest.Checksum([]byte{1, 2, 2}))) assert.NoError(t, w.Write( ident.StringID("foo3"), + ident.Tags{ident.StringTag("num", "3")}, bytesRefd([]byte{1, 2, 3}), digest.Checksum([]byte{1, 2, 3}))) assert.NoError(t, w.Close()) @@ -247,14 +252,17 @@ func TestSeekIDNotExists(t *testing.T) { assert.NoError(t, err) assert.NoError(t, w.Write( ident.StringID("foo10"), + ident.Tags(nil), bytesRefd([]byte{1, 2, 1}), digest.Checksum([]byte{1, 2, 1}))) assert.NoError(t, w.Write( ident.StringID("foo20"), + ident.Tags(nil), bytesRefd([]byte{1, 2, 2}), digest.Checksum([]byte{1, 2, 2}))) assert.NoError(t, w.Write( ident.StringID("foo30"), + ident.Tags(nil), bytesRefd([]byte{1, 2, 3}), digest.Checksum([]byte{1, 2, 3}))) assert.NoError(t, w.Close()) @@ -300,6 +308,7 @@ func TestReuseSeeker(t *testing.T) { assert.NoError(t, err) assert.NoError(t, w.Write( ident.StringID("foo"), + ident.Tags(nil), bytesRefd([]byte{1, 2, 1}), digest.Checksum([]byte{1, 2, 1}))) assert.NoError(t, w.Close()) @@ -316,6 +325,7 @@ func TestReuseSeeker(t *testing.T) { assert.NoError(t, err) assert.NoError(t, w.Write( ident.StringID("foo"), + ident.Tags(nil), bytesRefd([]byte{1, 2, 3}), digest.Checksum([]byte{1, 2, 3}))) assert.NoError(t, w.Close()) @@ -364,6 +374,7 @@ func TestCloneSeeker(t *testing.T) { assert.NoError(t, err) assert.NoError(t, w.Write( ident.StringID("foo"), + ident.Tags(nil), bytesRefd([]byte{1, 2, 1}), digest.Checksum([]byte{1, 2, 1}))) assert.NoError(t, w.Close()) @@ -380,6 +391,7 @@ func TestCloneSeeker(t *testing.T) { assert.NoError(t, err) assert.NoError(t, w.Write( ident.StringID("foo"), + ident.Tags(nil), bytesRefd([]byte{1, 2, 3}), digest.Checksum([]byte{1, 2, 3}))) assert.NoError(t, w.Close()) diff --git a/persist/fs/types.go b/persist/fs/types.go index 2a807a3c75..9ea9243173 100644 --- a/persist/fs/types.go +++ b/persist/fs/types.go @@ -31,6 +31,7 @@ import ( "github.com/m3db/m3db/persist" "github.com/m3db/m3db/persist/fs/msgpack" "github.com/m3db/m3db/runtime" + "github.com/m3db/m3db/serialize" "github.com/m3db/m3db/storage/block" "github.com/m3db/m3db/storage/namespace" "github.com/m3db/m3db/x/xio" @@ -87,11 +88,11 @@ type DataFileSetWriter interface { // Write will write the id and data pair and returns an error on a write error. Callers // must not call this method with a given ID more than once. - Write(id ident.ID, data checked.Bytes, checksum uint32) error + Write(id ident.ID, tags ident.Tags, data checked.Bytes, checksum uint32) error // WriteAll will write the id and all byte slices and returns an error on a write error. // Callers must not call this method with a given ID more than once. - WriteAll(id ident.ID, data []checked.Bytes, checksum uint32) error + WriteAll(id ident.ID, tags ident.Tags, data []checked.Bytes, checksum uint32) error } // DataFileSetReaderStatus describes the status of a file set reader @@ -121,11 +122,15 @@ type DataFileSetReader interface { // Read returns the next id, data, checksum tuple or error, will return io.EOF at end of volume. // Use either Read or ReadMetadata to progress through a volume, but not both. - Read() (id ident.ID, data checked.Bytes, checksum uint32, err error) + // Note: make sure to finalize the ID, close the Tags and finalize the Data when done with + // them so they can be returned to their respective pools. + Read() (id ident.ID, tags ident.TagIterator, data checked.Bytes, checksum uint32, err error) // ReadMetadata returns the next id and metadata or error, will return io.EOF at end of volume. // Use either Read or ReadMetadata to progress through a volume, but not both. - ReadMetadata() (id ident.ID, length int, checksum uint32, err error) + // Note: make sure to finalize the ID, and close the Tags when done with them so they can + // be returned to their respective pools. + ReadMetadata() (id ident.ID, tags ident.TagIterator, length int, checksum uint32, err error) // ReadBloomFilter returns the bloom filter stored on disk in a container object that is safe // for concurrent use and has a Close() method for releasing resources when done. @@ -447,6 +452,18 @@ type Options interface { // MmapHugeTLBThreshold returns the threshold when to use mmap huge pages for mmap'd files on linux MmapHugeTLBThreshold() int64 + + // SetTagEncoderPool sets the tag encoder pool + SetTagEncoderPool(value serialize.TagEncoderPool) Options + + // TagEncoderPool returns the tag encoder pool + TagEncoderPool() serialize.TagEncoderPool + + // SetTagDecoderPool sets the tag decoder pool + SetTagDecoderPool(value serialize.TagDecoderPool) Options + + // TagDecoderPool returns the tag decoder pool + TagDecoderPool() serialize.TagDecoderPool } // BlockRetrieverOptions represents the options for block retrieval diff --git a/persist/fs/write.go b/persist/fs/write.go index 8b07e70db8..e4867d5cde 100644 --- a/persist/fs/write.go +++ b/persist/fs/write.go @@ -22,6 +22,7 @@ package fs import ( "bytes" + "errors" "fmt" "math" "os" @@ -33,11 +34,17 @@ import ( "github.com/m3db/m3db/persist" "github.com/m3db/m3db/persist/fs/msgpack" "github.com/m3db/m3db/persist/schema" + "github.com/m3db/m3db/serialize" "github.com/m3db/m3x/checked" "github.com/m3db/m3x/ident" xtime "github.com/m3db/m3x/time" ) +var ( + errWriterEncodeTagsDataNotAccessible = errors.New( + "failed to encode tags: cannot get data") +) + type writer struct { blockSize time.Duration filePathPrefix string @@ -56,18 +63,21 @@ type writer struct { checkpointFilePath string indexEntries indexEntries - start time.Time - snapshotTime time.Time - currIdx int64 - currOffset int64 - encoder *msgpack.Encoder - digestBuf digest.Buffer - err error + start time.Time + snapshotTime time.Time + currIdx int64 + currOffset int64 + encoder *msgpack.Encoder + digestBuf digest.Buffer + singleCheckedBytes []checked.Bytes + tagEncoderPool serialize.TagEncoderPool + err error } type indexEntry struct { index int64 id ident.ID + tags ident.Tags dataFileOffset int64 indexFileOffset int64 size uint32 @@ -114,6 +124,8 @@ func NewWriter(opts Options) (DataFileSetWriter, error) { digestFdWithDigestContents: digest.NewFdWithDigestContentsWriter(bufferSize), encoder: msgpack.NewEncoder(), digestBuf: digest.NewBuffer(), + singleCheckedBytes: make([]checked.Bytes, 1), + tagEncoderPool: opts.TagEncoderPool(), }, nil } @@ -224,14 +236,17 @@ func (w *writer) writeData(data []byte) error { func (w *writer) Write( id ident.ID, + tags ident.Tags, data checked.Bytes, checksum uint32, ) error { - return w.WriteAll(id, []checked.Bytes{data}, checksum) + w.singleCheckedBytes[0] = data + return w.WriteAll(id, tags, w.singleCheckedBytes, checksum) } func (w *writer) WriteAll( id ident.ID, + tags ident.Tags, data []checked.Bytes, checksum uint32, ) error { @@ -239,7 +254,7 @@ func (w *writer) WriteAll( return w.err } - if err := w.writeAll(id, data, checksum); err != nil { + if err := w.writeAll(id, tags, data, checksum); err != nil { w.err = err return err } @@ -248,6 +263,7 @@ func (w *writer) WriteAll( func (w *writer) writeAll( id ident.ID, + tags ident.Tags, data []checked.Bytes, checksum uint32, ) error { @@ -265,6 +281,7 @@ func (w *writer) writeAll( entry := indexEntry{ index: w.currIdx, id: id, + tags: tags, dataFileOffset: w.currOffset, size: uint32(size), checksum: checksum, @@ -383,6 +400,10 @@ func (w *writer) writeIndexRelatedFiles() error { return w.writeInfoFileContents(bloomFilter, summaries) } +type tagsIter struct { + tags ident.Tags +} + func (w *writer) writeIndexFileContents( bloomFilter *bloom.BloomFilter, summaryEvery int, @@ -397,9 +418,12 @@ func (w *writer) writeIndexFileContents( sort.Sort(w.indexEntries) var ( - offset int64 - prevID []byte + offset int64 + prevID []byte + tagsIter = ident.NewTagSliceIterator(nil) + tagsEncoder = w.tagEncoderPool.Get() ) + defer tagsEncoder.Finalize() for i := range w.indexEntries { id := w.indexEntries[i].id.Data().Bytes() // Need to check if i > 0 or we can never write an empty string ID @@ -408,12 +432,27 @@ func (w *writer) writeIndexFileContents( return fmt.Errorf("encountered duplicate ID: %s", id) } + var encodedTags []byte + if tags := w.indexEntries[i].tags; tags != nil { + tagsIter.Reset(tags) + tagsEncoder.Reset() + if err := tagsEncoder.Encode(tagsIter); err != nil { + return err + } + data, ok := tagsEncoder.Data() + if !ok { + return errWriterEncodeTagsDataNotAccessible + } + encodedTags = data.Bytes() + } + entry := schema.IndexEntry{ - Index: w.indexEntries[i].index, - ID: id, - Size: int64(w.indexEntries[i].size), - Offset: w.indexEntries[i].dataFileOffset, - Checksum: int64(w.indexEntries[i].checksum), + Index: w.indexEntries[i].index, + ID: id, + Size: int64(w.indexEntries[i].size), + Offset: w.indexEntries[i].dataFileOffset, + Checksum: int64(w.indexEntries[i].checksum), + EncodedTags: encodedTags, } w.encoder.Reset() diff --git a/persist/schema/types.go b/persist/schema/types.go index 67cd8bde3e..00779fdb05 100644 --- a/persist/schema/types.go +++ b/persist/schema/types.go @@ -54,11 +54,12 @@ type IndexBloomFilterInfo struct { // IndexEntry stores entry-level data indexing type IndexEntry struct { - Index int64 - ID []byte - Size int64 - Offset int64 - Checksum int64 + Index int64 + ID []byte + Size int64 + Offset int64 + Checksum int64 + EncodedTags []byte } // IndexSummary stores a summary of an index entry to lookup diff --git a/persist/types.go b/persist/types.go index 550eb5584f..1780175e45 100644 --- a/persist/types.go +++ b/persist/types.go @@ -30,7 +30,7 @@ import ( ) // Fn is a function that persists a m3db segment for a given ID. -type Fn func(id ident.ID, segment ts.Segment, checksum uint32) error +type Fn func(id ident.ID, tags ident.Tags, segment ts.Segment, checksum uint32) error // Closer is a function that performs cleanup after persisting the data // blocks for a (shard, blockStart) combination. diff --git a/services/m3dbnode/config/bootstrap.go b/services/m3dbnode/config/bootstrap.go index 2781aba950..4d267ae606 100644 --- a/services/m3dbnode/config/bootstrap.go +++ b/services/m3dbnode/config/bootstrap.go @@ -33,6 +33,7 @@ import ( "github.com/m3db/m3db/storage/bootstrap/bootstrapper/fs" "github.com/m3db/m3db/storage/bootstrap/bootstrapper/peers" "github.com/m3db/m3db/storage/bootstrap/result" + "github.com/m3db/m3db/storage/index" ) var ( @@ -89,15 +90,15 @@ func (bsc BootstrapConfiguration) New( adminClient client.AdminClient, ) (bootstrap.ProcessProvider, error) { var ( - bs bootstrap.BootstrapperProvider - err error + bs bootstrap.BootstrapperProvider + err error + iopts = opts.InstrumentOptions() + rsopts = result.NewOptions(). + SetInstrumentOptions(opts.InstrumentOptions()). + SetDatabaseBlockOptions(opts.DatabaseBlockOptions()). + SetSeriesCachePolicy(opts.SeriesCachePolicy()). + SetIndexMutableSegmentAllocator(index.NewDefaultMutableSegmentAllocator(iopts)) ) - - rsopts := result.NewOptions(). - SetInstrumentOptions(opts.InstrumentOptions()). - SetDatabaseBlockOptions(opts.DatabaseBlockOptions()). - SetSeriesCachePolicy(opts.SeriesCachePolicy()) - // Start from the end of the list because the bootstrappers are ordered by precedence in descending order. for i := len(bsc.Bootstrappers) - 1; i >= 0; i-- { switch bsc.Bootstrappers[i] { @@ -111,7 +112,8 @@ func (bsc BootstrapConfiguration) New( SetResultOptions(rsopts). SetFilesystemOptions(fsopts). SetNumProcessors(bsc.fsNumProcessors()). - SetDatabaseBlockRetrieverManager(opts.DatabaseBlockRetrieverManager()) + SetDatabaseBlockRetrieverManager(opts.DatabaseBlockRetrieverManager()). + SetIdentifierPool(opts.IdentifierPool()) bs = fs.NewFileSystemBootstrapperProvider(fsbopts, bs) case commitlog.CommitLogBootstrapperName: copts := commitlog.NewOptions(). diff --git a/storage/block/block.go b/storage/block/block.go index 2f9b995ff0..885e9e2ffd 100644 --- a/storage/block/block.go +++ b/storage/block/block.go @@ -178,6 +178,7 @@ func (b *dbBlock) Checksum() (uint32, error) { func (b *dbBlock) OnRetrieveBlock( id ident.ID, + _ ident.TagIterator, startTime time.Time, segment ts.Segment, ) { diff --git a/storage/block/block_mock.go b/storage/block/block_mock.go index 086c03c53f..091f74563c 100644 --- a/storage/block/block_mock.go +++ b/storage/block/block_mock.go @@ -254,13 +254,13 @@ func (m *MockDatabaseBlock) EXPECT() *MockDatabaseBlockMockRecorder { } // OnRetrieveBlock mocks base method -func (m *MockDatabaseBlock) OnRetrieveBlock(id ident.ID, startTime time.Time, segment ts.Segment) { - m.ctrl.Call(m, "OnRetrieveBlock", id, startTime, segment) +func (m *MockDatabaseBlock) OnRetrieveBlock(id ident.ID, tags ident.TagIterator, startTime time.Time, segment ts.Segment) { + m.ctrl.Call(m, "OnRetrieveBlock", id, tags, startTime, segment) } // OnRetrieveBlock indicates an expected call of OnRetrieveBlock -func (mr *MockDatabaseBlockMockRecorder) OnRetrieveBlock(id, startTime, segment interface{}) *gomock.Call { - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnRetrieveBlock", reflect.TypeOf((*MockDatabaseBlock)(nil).OnRetrieveBlock), id, startTime, segment) +func (mr *MockDatabaseBlockMockRecorder) OnRetrieveBlock(id, tags, startTime, segment interface{}) *gomock.Call { + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnRetrieveBlock", reflect.TypeOf((*MockDatabaseBlock)(nil).OnRetrieveBlock), id, tags, startTime, segment) } // StartTime mocks base method @@ -671,13 +671,13 @@ func (m *MockOnRetrieveBlock) EXPECT() *MockOnRetrieveBlockMockRecorder { } // OnRetrieveBlock mocks base method -func (m *MockOnRetrieveBlock) OnRetrieveBlock(id ident.ID, startTime time.Time, segment ts.Segment) { - m.ctrl.Call(m, "OnRetrieveBlock", id, startTime, segment) +func (m *MockOnRetrieveBlock) OnRetrieveBlock(id ident.ID, tags ident.TagIterator, startTime time.Time, segment ts.Segment) { + m.ctrl.Call(m, "OnRetrieveBlock", id, tags, startTime, segment) } // OnRetrieveBlock indicates an expected call of OnRetrieveBlock -func (mr *MockOnRetrieveBlockMockRecorder) OnRetrieveBlock(id, startTime, segment interface{}) *gomock.Call { - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnRetrieveBlock", reflect.TypeOf((*MockOnRetrieveBlock)(nil).OnRetrieveBlock), id, startTime, segment) +func (mr *MockOnRetrieveBlockMockRecorder) OnRetrieveBlock(id, tags, startTime, segment interface{}) *gomock.Call { + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnRetrieveBlock", reflect.TypeOf((*MockOnRetrieveBlock)(nil).OnRetrieveBlock), id, tags, startTime, segment) } // MockOnReadBlock is a mock of OnReadBlock interface diff --git a/storage/block/types.go b/storage/block/types.go index 76902f1a81..4f48bb2d88 100644 --- a/storage/block/types.go +++ b/storage/block/types.go @@ -214,6 +214,7 @@ type OnEvictedFromWiredList interface { type OnRetrieveBlock interface { OnRetrieveBlock( id ident.ID, + tags ident.TagIterator, startTime time.Time, segment ts.Segment, ) diff --git a/storage/bootstrap/bootstrapper/fs/options.go b/storage/bootstrap/bootstrapper/fs/options.go index d6cb80619d..c46aa50058 100644 --- a/storage/bootstrap/bootstrapper/fs/options.go +++ b/storage/bootstrap/bootstrapper/fs/options.go @@ -27,6 +27,8 @@ import ( "github.com/m3db/m3db/persist/fs" "github.com/m3db/m3db/storage/block" "github.com/m3db/m3db/storage/bootstrap/result" + "github.com/m3db/m3x/ident" + "github.com/m3db/m3x/pool" ) var ( @@ -38,14 +40,19 @@ type options struct { fsOpts fs.Options numProcessors int blockRetrieverManager block.DatabaseBlockRetrieverManager + identifierPool ident.Pool } // NewOptions creates new bootstrap options func NewOptions() Options { + bytesPool := pool.NewCheckedBytesPool(nil, nil, func(s []pool.Bucket) pool.BytesPool { + return pool.NewBytesPool(s, nil) + }) return &options{ - resultOpts: result.NewOptions(), - fsOpts: fs.NewOptions(), - numProcessors: defaultNumProcessors, + resultOpts: result.NewOptions(), + fsOpts: fs.NewOptions(), + numProcessors: defaultNumProcessors, + identifierPool: ident.NewPool(bytesPool, pool.NewObjectPoolOptions()), } } @@ -90,3 +97,13 @@ func (o *options) SetDatabaseBlockRetrieverManager( func (o *options) DatabaseBlockRetrieverManager() block.DatabaseBlockRetrieverManager { return o.blockRetrieverManager } + +func (o *options) SetIdentifierPool(value ident.Pool) Options { + opts := *o + opts.identifierPool = value + return &opts +} + +func (o *options) IdentifierPool() ident.Pool { + return o.identifierPool +} diff --git a/storage/bootstrap/bootstrapper/fs/source.go b/storage/bootstrap/bootstrapper/fs/source.go index c8ff58c02f..c7b8c2a127 100644 --- a/storage/bootstrap/bootstrapper/fs/source.go +++ b/storage/bootstrap/bootstrapper/fs/source.go @@ -32,6 +32,8 @@ import ( "github.com/m3db/m3db/storage/namespace" "github.com/m3db/m3db/storage/series" "github.com/m3db/m3db/ts" + "github.com/m3db/m3ninx/doc" + "github.com/m3db/m3ninx/index/segment" "github.com/m3db/m3x/checked" "github.com/m3db/m3x/ident" xlog "github.com/m3db/m3x/log" @@ -45,6 +47,25 @@ type newDataFileSetReaderFn func( opts fs.Options, ) (fs.DataFileSetReader, error) +type runType int + +const ( + bootstrapDataRunType runType = iota + bootstrapIndexRunType +) + +type runResult struct { + data result.DataBootstrapResult + index result.IndexBootstrapResult +} + +func newRunResult() *runResult { + return &runResult{ + data: result.NewDataBootstrapResult(), + index: result.NewIndexBootstrapResult(), + } +} + type fileSystemSource struct { opts Options fsopts fs.Options @@ -76,6 +97,44 @@ func (s *fileSystemSource) Can(strategy bootstrap.Strategy) bool { func (s *fileSystemSource) AvailableData( md namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, +) result.ShardTimeRanges { + return s.availability(md, shardsTimeRanges) +} + +func (s *fileSystemSource) ReadData( + md namespace.Metadata, + shardsTimeRanges result.ShardTimeRanges, + _ bootstrap.RunOptions, +) (result.DataBootstrapResult, error) { + r, err := s.read(md, shardsTimeRanges, bootstrapDataRunType) + if err != nil { + return nil, err + } + return r.data, nil +} + +func (s *fileSystemSource) AvailableIndex( + md namespace.Metadata, + shardsTimeRanges result.ShardTimeRanges, +) result.ShardTimeRanges { + return s.availability(md, shardsTimeRanges) +} + +func (s *fileSystemSource) ReadIndex( + md namespace.Metadata, + shardsTimeRanges result.ShardTimeRanges, + opts bootstrap.RunOptions, +) (result.IndexBootstrapResult, error) { + r, err := s.read(md, shardsTimeRanges, bootstrapDataRunType) + if err != nil { + return nil, err + } + return r.index, nil +} + +func (s *fileSystemSource) availability( + md namespace.Metadata, + shardsTimeRanges result.ShardTimeRanges, ) result.ShardTimeRanges { result := make(map[uint32]xtime.Ranges) for shard, ranges := range shardsTimeRanges { @@ -194,15 +253,17 @@ func (s *fileSystemSource) enqueueReaders( } func (s *fileSystemSource) bootstrapFromReaders( + ns namespace.Metadata, + run runType, readerPool *readerPool, retriever block.DatabaseBlockRetriever, readersCh <-chan shardReaders, -) result.DataBootstrapResult { +) *runResult { var ( wg sync.WaitGroup resultLock = &sync.RWMutex{} shardRetrieverMgr block.DatabaseShardBlockRetrieverManager - bootstrapResult = result.NewDataBootstrapResult() + runResult = newRunResult() bopts = s.opts.ResultOptions() ) @@ -215,20 +276,20 @@ func (s *fileSystemSource) bootstrapFromReaders( wg.Add(1) s.processors.Go(func() { defer wg.Done() - s.loadShardReadersDataIntoShardResult( - resultLock, bootstrapResult, bopts, shardRetrieverMgr, shardReaders, readerPool) + s.loadShardReadersDataIntoShardResult(ns, run, runResult, resultLock, + bopts, shardRetrieverMgr, shardReaders, readerPool) }) } wg.Wait() - shardResults := bootstrapResult.ShardResults() + shardResults := runResult.data.ShardResults() for shard, results := range shardResults { if results.NumSeries() == 0 { delete(shardResults, shard) } } - return bootstrapResult + return runResult } // handleErrorsAndUnfulfilled checks the list of times that had errors and makes @@ -237,7 +298,7 @@ func (s *fileSystemSource) bootstrapFromReaders( // as unfulfilled func (s *fileSystemSource) handleErrorsAndUnfulfilled( resultLock *sync.RWMutex, - bootstrapResult result.DataBootstrapResult, + runResult *runResult, shard uint32, remainingRanges xtime.Ranges, shardResult result.ShardResult, @@ -256,7 +317,7 @@ func (s *fileSystemSource) handleErrorsAndUnfulfilled( ).Info("deleting entries from results for times with errors") resultLock.Lock() - shardResult, ok := bootstrapResult.ShardResults()[shard] + shardResult, ok := runResult.data.ShardResults()[shard] if ok { for _, entry := range shardResult.AllSeries().Iter() { series := entry.Value() @@ -270,7 +331,7 @@ func (s *fileSystemSource) handleErrorsAndUnfulfilled( if !remainingRanges.IsEmpty() { resultLock.Lock() - unfulfilled := bootstrapResult.Unfulfilled() + unfulfilled := runResult.data.Unfulfilled() shardUnfulfilled, ok := unfulfilled[shard] if !ok { shardUnfulfilled = xtime.Ranges{}.AddRanges(remainingRanges) @@ -283,9 +344,22 @@ func (s *fileSystemSource) handleErrorsAndUnfulfilled( } } +func (s *fileSystemSource) tagsFromTagsIter( + iter ident.TagIterator, +) (ident.Tags, error) { + tags := make(ident.Tags, 0, iter.Remaining()) + for iter.Next() { + curr := iter.Current() + tags = append(tags, s.opts.IdentifierPool().CloneTag(curr)) + } + return tags, iter.Err() +} + func (s *fileSystemSource) loadShardReadersDataIntoShardResult( + ns namespace.Metadata, + run runType, + runResult *runResult, resultLock *sync.RWMutex, - bootstrapResult result.DataBootstrapResult, bopts result.Options, shardRetrieverMgr block.DatabaseShardBlockRetrieverManager, shardReaders shardReaders, @@ -294,14 +368,17 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( var ( timesWithErrors []time.Time shardResult result.ShardResult + shardSeries *result.Map shardRetriever block.DatabaseShardBlockRetriever blockPool = bopts.DatabaseBlockOptions().DatabaseBlockPool() seriesCachePolicy = bopts.SeriesCachePolicy() + indexBlockSegment segment.MutableSegment ) shard, tr, readers, err := shardReaders.shard, shardReaders.tr, shardReaders.readers, shardReaders.err if err != nil { - s.handleErrorsAndUnfulfilled(resultLock, bootstrapResult, shard, tr, shardResult, timesWithErrors) + s.handleErrorsAndUnfulfilled(resultLock, runResult, shard, tr, + shardResult, timesWithErrors) return } @@ -310,41 +387,107 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( } if seriesCachePolicy == series.CacheAllMetadata && shardRetriever == nil { s.log.Errorf("shard retriever missing for shard: %d", shard) - s.handleErrorsAndUnfulfilled(resultLock, bootstrapResult, shard, tr, shardResult, timesWithErrors) + s.handleErrorsAndUnfulfilled(resultLock, runResult, shard, tr, + shardResult, timesWithErrors) return } for _, r := range readers { - if shardResult == nil { - resultLock.RLock() - results := bootstrapResult.ShardResults() - var exists bool - shardResult, exists = results[shard] - resultLock.RUnlock() - - if !exists { - resultLock.Lock() + var ( + timeRange = r.Range() + start = timeRange.Start + hasError bool + ) + switch run { + case bootstrapDataRunType: + if shardResult == nil { + resultLock.RLock() + results := runResult.data.ShardResults() + var exists bool shardResult, exists = results[shard] + resultLock.RUnlock() + if !exists { - // NB(r): Wait until we have a reader to initialize the shard result - // to be able to somewhat estimate the size of it. - shardResult = result.NewShardResult(r.Entries(), bopts) - results[shard] = shardResult + resultLock.Lock() + shardResult, exists = results[shard] + if !exists { + // NB(r): Wait until we have a reader to initialize the shard result + // to be able to somewhat estimate the size of it. + shardResult = result.NewShardResult(r.Entries(), bopts) + results[shard] = shardResult + } + resultLock.Unlock() } - resultLock.Unlock() + + shardSeries = shardResult.AllSeries() } + case bootstrapIndexRunType: + resultLock.Lock() + results := runResult.index.IndexResults() + indexBlockSegment, err = results.GetOrAddSegment(start, + ns.Options().IndexOptions(), bopts) + resultLock.Unlock() + if err != nil { + s.log.Errorf("unable to create index segment: %v", err) + hasError = true + } + default: + // Unreachable unless an internal method calls with a run type casted from int + panic(fmt.Errorf("invalid run type: %d", run)) } - var ( - timeRange = r.Range() - start = timeRange.Start - hasError = false + var numEntries int + if !hasError { numEntries = r.Entries() - ) + } for i := 0; i < numEntries; i++ { + if run == bootstrapIndexRunType { + // If performing index run, then simply read the metadata and add to segment + id, tagsIter, _, _, err := r.ReadMetadata() + idBytes := id.Bytes() + + resultLock.RLock() + exists, err := indexBlockSegment.ContainsID(idBytes) + resultLock.RUnlock() + + if err == nil && !exists { + resultLock.Lock() + exists, err := indexBlockSegment.ContainsID(idBytes) + if err == nil && !exists { + d := doc.Document{ + ID: append([]byte(nil), idBytes...), + Fields: make(doc.Fields, 0, tagsIter.Remaining()), + } + for tagsIter.Next() { + curr := tagsIter.Current() + d.Fields = append(d.Fields, doc.Field{ + Name: append([]byte(nil), curr.Name.Bytes()...), + Value: append([]byte(nil), curr.Value.Bytes()...), + }) + } + err = tagsIter.Err() + if err == nil { + _, err = indexBlockSegment.Insert(d) + } + } + resultLock.Unlock() + } + + // Finalize the ID and tags + id.Finalize() + tagsIter.Close() + + if err != nil { + s.log.Errorf("unable to add doc to segment: %v", err) + hasError = true + } + continue + } + var ( seriesBlock = blockPool.Get() id ident.ID + tagsIter ident.TagIterator data checked.Bytes length int checksum uint32 @@ -352,9 +495,9 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( ) switch seriesCachePolicy { case series.CacheAll: - id, data, checksum, err = r.Read() + id, tagsIter, data, checksum, err = r.Read() case series.CacheAllMetadata: - id, length, checksum, err = r.ReadMetadata() + id, tagsIter, length, checksum, err = r.ReadMetadata() default: s.log.WithFields( xlog.NewField("shard", shard), @@ -375,8 +518,13 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( break } + var ( + entry result.DatabaseSeriesBlocks + tags ident.Tags + exists bool + ) resultLock.RLock() - entry, exists := shardResult.AllSeries().Get(id) + entry, exists = shardSeries.Get(id) resultLock.RUnlock() if exists { @@ -385,6 +533,17 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( // allocated ID. id.Finalize() id = entry.ID + tags = entry.Tags + } else { + tags, err = s.tagsFromTagsIter(tagsIter) + if err != nil { + s.log.Errorf("unable to decode tags: %v", err) + hasError = true + } + } + tagsIter.Close() + if hasError { + break } switch seriesCachePolicy { @@ -413,7 +572,7 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( if exists { entry.Blocks.AddBlock(seriesBlock) } else { - shardResult.AddBlock(id, nil, seriesBlock) // FOLLOWUP(prateek): retrieve tags in fs reader + shardResult.AddBlock(id, tags, seriesBlock) } resultLock.Unlock() } @@ -454,76 +613,81 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( } } - s.handleErrorsAndUnfulfilled( - resultLock, bootstrapResult, shard, tr, shardResult, timesWithErrors) + s.handleErrorsAndUnfulfilled(resultLock, runResult, shard, + tr, shardResult, timesWithErrors) } -func (s *fileSystemSource) ReadData( +func (s *fileSystemSource) read( md namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, - _ bootstrap.RunOptions, -) (result.DataBootstrapResult, error) { + run runType, +) (*runResult, error) { + var ( + nsID = md.ID() + blockRetriever block.DatabaseBlockRetriever + ) if shardsTimeRanges.IsEmpty() { - return result.NewDataBootstrapResult(), nil + return newRunResult(), nil } - var ( - nsID = md.ID() - blockRetrieverMgr = s.opts.DatabaseBlockRetrieverManager() - blockRetriever block.DatabaseBlockRetriever - ) - if blockRetrieverMgr != nil { - s.log.WithFields( - xlog.NewField("namespace", nsID.String()), - ).Infof("filesystem bootstrapper resolving block retriever") + if run == bootstrapDataRunType { + // NB(r): We can only need to cache shard indices and possibly shortcut + // the reading of filesets if we are doing a data bootstrap, otherwise + // we need to look at the filesets. + blockRetrieverMgr := s.opts.DatabaseBlockRetrieverManager() + if blockRetrieverMgr != nil { + s.log.WithFields( + xlog.NewField("namespace", nsID.String()), + ).Infof("filesystem bootstrapper resolving block retriever") - var err error - blockRetriever, err = blockRetrieverMgr.Retriever(md) - if err != nil { - return nil, err - } + var err error + blockRetriever, err = blockRetrieverMgr.Retriever(md) + if err != nil { + return nil, err + } - s.log.WithFields( - xlog.NewField("namespace", nsID.String()), - xlog.NewField("shards", len(shardsTimeRanges)), - ).Infof("filesystem bootstrapper caching block retriever shard indices") + s.log.WithFields( + xlog.NewField("namespace", nsID.String()), + xlog.NewField("shards", len(shardsTimeRanges)), + ).Infof("filesystem bootstrapper caching block retriever shard indices") - shards := make([]uint32, 0, len(shardsTimeRanges)) - for shard := range shardsTimeRanges { - shards = append(shards, shard) - } + shards := make([]uint32, 0, len(shardsTimeRanges)) + for shard := range shardsTimeRanges { + shards = append(shards, shard) + } - err = blockRetriever.CacheShardIndices(shards) - if err != nil { - return nil, err + err = blockRetriever.CacheShardIndices(shards) + if err != nil { + return nil, err + } } - } - switch s.opts.ResultOptions().SeriesCachePolicy() { - case series.CacheAll: - // No checks necessary - case series.CacheAllMetadata: - // Need to check block retriever available - if blockRetriever == nil { - return nil, fmt.Errorf( - "missing block retriever when using series cache metadata for namespace: %s", - nsID.String()) - } - default: - // Unless we're caching all series (or all series metadata) in memory, we - // return just the availability of the files we have - bootstrapResult := result.NewDataBootstrapResult() - unfulfilled := bootstrapResult.Unfulfilled() - for shard, ranges := range shardsTimeRanges { - if ranges.IsEmpty() { - continue + switch s.opts.ResultOptions().SeriesCachePolicy() { + case series.CacheAll: + // No checks necessary + case series.CacheAllMetadata: + // Need to check block retriever available + if blockRetriever == nil { + return nil, fmt.Errorf( + "missing block retriever when using series cache metadata for namespace: %s", + nsID.String()) + } + default: + // Unless we're caching all series (or all series metadata) in memory, we + // return just the availability of the files we have + runResult := newRunResult() + unfulfilled := runResult.data.Unfulfilled() + for shard, ranges := range shardsTimeRanges { + if ranges.IsEmpty() { + continue + } + availability := s.shardAvailability(md.ID(), shard, ranges) + remaining := ranges.RemoveRanges(availability) + runResult.data.Add(shard, nil, remaining) } - availability := s.shardAvailability(md.ID(), shard, ranges) - remaining := ranges.RemoveRanges(availability) - bootstrapResult.Add(shard, nil, remaining) + runResult.data.SetUnfulfilled(unfulfilled) + return runResult, nil } - bootstrapResult.SetUnfulfilled(unfulfilled) - return bootstrapResult, nil } s.log.WithFields( @@ -541,26 +705,7 @@ func (s *fileSystemSource) ReadData( }) readersCh := make(chan shardReaders) go s.enqueueReaders(nsID, shardsTimeRanges, readerPool, readersCh) - return s.bootstrapFromReaders(readerPool, blockRetriever, readersCh), nil -} - -func (s *fileSystemSource) AvailableIndex( - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, -) result.ShardTimeRanges { - // FOLLOWUP(r): implement the filesystem source returning - // index segments that are available on disk for the time range required. - return shardsTimeRanges -} - -func (s *fileSystemSource) ReadIndex( - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - opts bootstrap.RunOptions, -) (result.IndexBootstrapResult, error) { - // FOLLOWUP(r): implement the filesystem source returning - // index segments that are available on disk for the time range required. - return result.NewIndexBootstrapResult(), nil + return s.bootstrapFromReaders(md, run, readerPool, blockRetriever, readersCh), nil } type shardReaders struct { diff --git a/storage/bootstrap/bootstrapper/fs/source_test.go b/storage/bootstrap/bootstrapper/fs/source_test.go index 47432b6d5a..efc2fc9d23 100644 --- a/storage/bootstrap/bootstrapper/fs/source_test.go +++ b/storage/bootstrap/bootstrapper/fs/source_test.go @@ -125,19 +125,30 @@ func writeGoodFiles(t *testing.T, dir string, namespace ident.ID, shard uint32) inputs := []struct { start time.Time id string + tags map[string]string data []byte }{ - {testStart, "foo", []byte{1, 2, 3}}, - {testStart.Add(10 * time.Hour), "bar", []byte{4, 5, 6}}, - {testStart.Add(20 * time.Hour), "baz", []byte{7, 8, 9}}, + {testStart, "foo", map[string]string{"n": "0"}, []byte{1, 2, 3}}, + {testStart.Add(10 * time.Hour), "bar", map[string]string{"n": "1"}, []byte{4, 5, 6}}, + {testStart.Add(20 * time.Hour), "baz", nil, []byte{7, 8, 9}}, } for _, input := range inputs { - writeTSDBFiles(t, dir, namespace, shard, input.start, input.id, input.data) + writeTSDBFiles(t, dir, namespace, shard, input.start, + input.id, input.tags, input.data) } } -func writeTSDBFiles(t *testing.T, dir string, namespace ident.ID, shard uint32, start time.Time, id string, data []byte) { +func writeTSDBFiles( + t *testing.T, + dir string, + namespace ident.ID, + shard uint32, + start time.Time, + id string, + tags map[string]string, + data []byte, +) { w, err := fs.NewWriter(newTestFsOptions(dir)) require.NoError(t, err) writerOpts := fs.DataWriterOpenOptions{ @@ -150,9 +161,16 @@ func writeTSDBFiles(t *testing.T, dir string, namespace ident.ID, shard uint32, } require.NoError(t, w.Open(writerOpts)) + var seriesTags ident.Tags + for name, value := range tags { + seriesTags = append(seriesTags, ident.StringTag(name, value)) + } + bytes := checked.NewBytes(data, nil) bytes.IncRef() - require.NoError(t, w.Write(ident.StringID(id), bytes, digest.Checksum(bytes.Bytes()))) + + require.NoError(t, w.Write(ident.StringID(id), seriesTags, + bytes, digest.Checksum(bytes.Bytes()))) require.NoError(t, w.Close()) } @@ -194,7 +212,7 @@ func TestAvailableReadInfoError(t *testing.T) { defer os.RemoveAll(dir) shard := uint32(0) - writeTSDBFiles(t, dir, testNs1ID, shard, testStart, "foo", []byte{0x1}) + writeTSDBFiles(t, dir, testNs1ID, shard, testStart, "foo", nil, []byte{0x1}) // Intentionally corrupt the info file writeInfoFile(t, dir, testNs1ID, shard, testStart, []byte{0x1, 0x2}) @@ -209,7 +227,7 @@ func TestAvailableDigestOfDigestMismatch(t *testing.T) { defer os.RemoveAll(dir) shard := uint32(0) - writeTSDBFiles(t, dir, testNs1ID, shard, testStart, "foo", []byte{0x1}) + writeTSDBFiles(t, dir, testNs1ID, shard, testStart, "foo", nil, []byte{0x1}) // Intentionally corrupt the digest file writeDigestFile(t, dir, testNs1ID, shard, testStart, nil) @@ -299,7 +317,7 @@ func TestReadOpenFileError(t *testing.T) { defer os.RemoveAll(dir) shard := uint32(0) - writeTSDBFiles(t, dir, testNs1ID, shard, testStart, "foo", []byte{0x1}) + writeTSDBFiles(t, dir, testNs1ID, shard, testStart, "foo", nil, []byte{0x1}) // Intentionally truncate the info file writeInfoFile(t, dir, testNs1ID, shard, testStart, nil) @@ -322,7 +340,7 @@ func TestReadDataCorruptionError(t *testing.T) { defer os.RemoveAll(dir) shard := uint32(0) - writeTSDBFiles(t, dir, testNs1ID, shard, testStart, "foo", []byte{0x1}) + writeTSDBFiles(t, dir, testNs1ID, shard, testStart, "foo", nil, []byte{0x1}) // Intentionally corrupt the data file writeDataFile(t, dir, testNs1ID, shard, testStart, []byte{0x2}) @@ -426,7 +444,7 @@ func TestReadValidateError(t *testing.T) { shard := uint32(0) writeTSDBFiles(t, dir, testNs1ID, shard, testStart, - "foo", []byte{0x1}) + "foo", nil, []byte{0x1}) rOpenOpts := fs.ReaderOpenOptionsMatcher{ ID: fs.FileSetFileIdentifier{ Namespace: testNs1ID, @@ -480,7 +498,7 @@ func TestReadOpenError(t *testing.T) { shard := uint32(0) writeTSDBFiles(t, dir, testNs1ID, shard, testStart, - "foo", []byte{0x1}) + "foo", nil, []byte{0x1}) rOpts := fs.ReaderOpenOptionsMatcher{ ID: fs.FileSetFileIdentifier{ Namespace: testNs1ID, @@ -524,7 +542,7 @@ func TestReadDeleteOnError(t *testing.T) { shard := uint32(0) writeTSDBFiles(t, dir, testNs1ID, shard, testStart, - "foo", []byte{0x1}) + "foo", nil, []byte{0x1}) rOpts := fs.ReaderOpenOptionsMatcher{ ID: fs.FileSetFileIdentifier{ @@ -571,3 +589,12 @@ func TestReadDeleteOnError(t *testing.T) { } validateTimeRanges(t, res.Unfulfilled()[testShard], expected) } + +func TestReadTags(t *testing.T) { + // TODO(r): Add test that tests the normal bootstrap case with + // series cache policy All returns tags for series +} + +func TestBootstrapIndex(t *testing.T) { + // TODO(r): Add test to bootstrap index segments +} diff --git a/storage/bootstrap/bootstrapper/fs/types.go b/storage/bootstrap/bootstrapper/fs/types.go index caa6dc6820..30d4e0daa0 100644 --- a/storage/bootstrap/bootstrapper/fs/types.go +++ b/storage/bootstrap/bootstrapper/fs/types.go @@ -24,6 +24,7 @@ import ( "github.com/m3db/m3db/persist/fs" "github.com/m3db/m3db/storage/block" "github.com/m3db/m3db/storage/bootstrap/result" + "github.com/m3db/m3x/ident" ) // Options represents the options for bootstrapping from the filesystem. @@ -62,4 +63,10 @@ type Options interface { // use when bootstrapping retrievable blocks instead of blocks // containing data. DatabaseBlockRetrieverManager() block.DatabaseBlockRetrieverManager + + // SetIdentifierPool sets the identifier pool. + SetIdentifierPool(value ident.Pool) Options + + // IdentifierPool returns the identifier pool. + IdentifierPool() ident.Pool } diff --git a/storage/bootstrap/bootstrapper/peers/source.go b/storage/bootstrap/bootstrapper/peers/source.go index f71b139c3d..b6c4af3153 100644 --- a/storage/bootstrap/bootstrapper/peers/source.go +++ b/storage/bootstrap/bootstrapper/peers/source.go @@ -394,7 +394,7 @@ func (s *peersSource) incrementalFlush( break } - err = prepared.Persist(s.ID, segment, checksum) + err = prepared.Persist(s.ID, s.Tags, segment, checksum) tmpCtx.BlockingClose() if err != nil { blockErr = err // Need to call prepared.Close, avoid return diff --git a/storage/bootstrap/result/options.go b/storage/bootstrap/result/options.go index e8474da9a3..650a7c2cc6 100644 --- a/storage/bootstrap/result/options.go +++ b/storage/bootstrap/result/options.go @@ -23,6 +23,7 @@ package result import ( "github.com/m3db/m3db/clock" "github.com/m3db/m3db/storage/block" + "github.com/m3db/m3db/storage/index" "github.com/m3db/m3db/storage/series" "github.com/m3db/m3x/instrument" ) @@ -32,21 +33,24 @@ const ( ) type options struct { - clockOpts clock.Options - instrumentOpts instrument.Options - blockOpts block.Options - newBlocksLen int - seriesCachePolicy series.CachePolicy + clockOpts clock.Options + instrumentOpts instrument.Options + blockOpts block.Options + newBlocksLen int + seriesCachePolicy series.CachePolicy + mutableSegmentAllocator index.MutableSegmentAllocator } // NewOptions creates new bootstrap options func NewOptions() Options { + iopts := instrument.NewOptions() return &options{ - clockOpts: clock.NewOptions(), - instrumentOpts: instrument.NewOptions(), - blockOpts: block.NewOptions(), - newBlocksLen: defaultNewBlocksLen, - seriesCachePolicy: series.DefaultCachePolicy, + clockOpts: clock.NewOptions(), + instrumentOpts: iopts, + blockOpts: block.NewOptions(), + newBlocksLen: defaultNewBlocksLen, + seriesCachePolicy: series.DefaultCachePolicy, + mutableSegmentAllocator: index.NewDefaultMutableSegmentAllocator(iopts), } } @@ -99,3 +103,13 @@ func (o *options) SetSeriesCachePolicy(value series.CachePolicy) Options { func (o *options) SeriesCachePolicy() series.CachePolicy { return o.seriesCachePolicy } + +func (o *options) SetIndexMutableSegmentAllocator(value index.MutableSegmentAllocator) Options { + opts := *o + opts.mutableSegmentAllocator = value + return &opts +} + +func (o *options) IndexMutableSegmentAllocator() index.MutableSegmentAllocator { + return o.mutableSegmentAllocator +} diff --git a/storage/bootstrap/result/result_index.go b/storage/bootstrap/result/result_index.go index 88f8cf2919..657dfe11f3 100644 --- a/storage/bootstrap/result/result_index.go +++ b/storage/bootstrap/result/result_index.go @@ -23,6 +23,7 @@ package result import ( "time" + "github.com/m3db/m3db/storage/namespace" "github.com/m3db/m3ninx/index/segment" xtime "github.com/m3db/m3x/time" ) @@ -78,6 +79,36 @@ func (r IndexResults) AddResults(other IndexResults) { } } +// GetOrAddSegment get or create a new mutable segment. +func (r IndexResults) GetOrAddSegment( + t time.Time, + idxopts namespace.IndexOptions, + opts Options, +) (segment.MutableSegment, error) { + blockStart := t.Truncate(idxopts.BlockSize()) + blockStartNanos := xtime.ToUnixNano(blockStart) + block, exists := r[blockStartNanos] + if !exists { + block = NewIndexBlock(blockStart, nil) + r[blockStartNanos] = block + } + for _, seg := range block.Segments() { + if mutable, ok := seg.(segment.MutableSegment); ok { + return mutable, nil + } + } + + alloc := opts.IndexMutableSegmentAllocator() + mutable, err := alloc() + if err != nil { + return nil, err + } + + segments := []segment.Segment{mutable} + r[blockStartNanos] = block.Merged(NewIndexBlock(blockStart, segments)) + return mutable, nil +} + // MergedIndexBootstrapResult returns a merged result of two bootstrap results. // It is a mutating function that mutates the larger result by adding the // smaller result to it and then finally returns the mutated result. diff --git a/storage/bootstrap/result/types.go b/storage/bootstrap/result/types.go index 2450d57159..9fc3ddf713 100644 --- a/storage/bootstrap/result/types.go +++ b/storage/bootstrap/result/types.go @@ -25,6 +25,7 @@ import ( "github.com/m3db/m3db/clock" "github.com/m3db/m3db/storage/block" + "github.com/m3db/m3db/storage/index" "github.com/m3db/m3db/storage/series" "github.com/m3db/m3ninx/index/segment" "github.com/m3db/m3x/ident" @@ -149,4 +150,10 @@ type Options interface { // SeriesCachePolicy returns the series cache policy. SeriesCachePolicy() series.CachePolicy + + // SetIndexMutableSegmentAllocator sets the index mutable segment allocator. + SetIndexMutableSegmentAllocator(value index.MutableSegmentAllocator) Options + + // IndexMutableSegmentAllocator returns the index mutable segment allocator. + IndexMutableSegmentAllocator() index.MutableSegmentAllocator } diff --git a/storage/index/allocator.go b/storage/index/allocator.go new file mode 100644 index 0000000000..dd4d606328 --- /dev/null +++ b/storage/index/allocator.go @@ -0,0 +1,38 @@ +// Copyright (c) 2018 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 index + +import ( + "github.com/m3db/m3ninx/index/segment" + "github.com/m3db/m3ninx/index/segment/mem" + "github.com/m3db/m3x/instrument" +) + +// NewDefaultMutableSegmentAllocator returns a default mutable segment +// allocator. +func NewDefaultMutableSegmentAllocator( + iopts instrument.Options, +) MutableSegmentAllocator { + opts := mem.NewOptions().SetInstrumentOptions(iopts) + return func() (segment.MutableSegment, error) { + return mem.NewSegment(0, opts) + } +} diff --git a/storage/index/types.go b/storage/index/types.go index 38f2bcf3e8..1db52d5f63 100644 --- a/storage/index/types.go +++ b/storage/index/types.go @@ -23,6 +23,8 @@ package index import ( "time" + "github.com/m3db/m3ninx/index/segment" + "github.com/m3db/m3db/clock" "github.com/m3db/m3ninx/doc" "github.com/m3db/m3ninx/idx" @@ -109,6 +111,9 @@ type ResultsPool interface { Put(value Results) } +// MutableSegmentAllocator allocates a new MutableSegment type. +type MutableSegmentAllocator func() (segment.MutableSegment, error) + // OnIndexSeries provides a set of callback hooks to allow the reverse index // to do lifecycle management of any resources retained during indexing. type OnIndexSeries interface { diff --git a/storage/namespace_readers.go b/storage/namespace_readers.go index 37fc9a0714..83c82d9018 100644 --- a/storage/namespace_readers.go +++ b/storage/namespace_readers.go @@ -269,19 +269,21 @@ func (m *namespaceReaderManager) get( // Fast fwd through if in the middle of a volume for i := 0; i < position.dataIdx; i++ { - id, data, _, err := reader.Read() + id, tags, data, _, err := reader.Read() if err != nil { return nil, err } id.Finalize() + tags.Close() data.Finalize() } for i := 0; i < position.metadataIdx; i++ { - id, _, _, err := reader.ReadMetadata() + id, tags, _, _, err := reader.ReadMetadata() if err != nil { return nil, err } id.Finalize() + tags.Close() } return reader, nil diff --git a/storage/series/series.go b/storage/series/series.go index fe663cb765..eb355aad0e 100644 --- a/storage/series/series.go +++ b/storage/series/series.go @@ -509,6 +509,7 @@ func (s *dbSeries) Bootstrap(bootstrappedBlocks block.DatabaseSeriesBlocks) (Boo func (s *dbSeries) OnRetrieveBlock( id ident.ID, + tags ident.TagIterator, startTime time.Time, segment ts.Segment, ) { @@ -533,7 +534,7 @@ func (s *dbSeries) OnRetrieveBlock( // Also note that ResetRetrievable will mark the block as not retrieved from disk, // but OnRetrieveBlock will then properly mark it as retrieved from disk so subsequent // calls to WasRetrievedFromDisk will return true. - b.OnRetrieveBlock(s.id, startTime, segment) + b.OnRetrieveBlock(s.id, tags, startTime, segment) // NB(r): Blocks retrieved have been triggered by a read, so set the last // read time as now so caching policies are followed. @@ -603,25 +604,19 @@ func (s *dbSeries) Flush( blockStart time.Time, persistFn persist.Fn, ) (FlushOutcome, error) { - // NB(r): Do not use defer here as we need to make sure the - // call to sr.Segment() which may fetch data from disk is not - // blocking the series lock. s.RLock() + defer s.RUnlock() if s.bs != bootstrapped { - s.RUnlock() return FlushOutcomeErr, errSeriesNotBootstrapped } b, exists := s.blocks.BlockAt(blockStart) if !exists { - s.RUnlock() return FlushOutcomeBlockDoesNotExist, nil } sr, err := b.Stream(ctx) - s.RUnlock() - if err != nil { return FlushOutcomeErr, err } @@ -637,7 +632,7 @@ func (s *dbSeries) Flush( if err != nil { return FlushOutcomeErr, err } - err = persistFn(s.id, segment, checksum) + err = persistFn(s.id, s.tags, segment, checksum) if err != nil { return FlushOutcomeErr, err } @@ -672,7 +667,7 @@ func (s *dbSeries) Snapshot( return err } - return persistFn(s.id, segment, digest.SegmentChecksum(segment)) + return persistFn(s.id, s.tags, segment, digest.SegmentChecksum(segment)) } func (s *dbSeries) Close() { diff --git a/storage/series/series_mock.go b/storage/series/series_mock.go index 103fca9c32..f79d7552d2 100644 --- a/storage/series/series_mock.go +++ b/storage/series/series_mock.go @@ -183,13 +183,13 @@ func (mr *MockDatabaseSeriesMockRecorder) OnEvictedFromWiredList(arg0, arg1 inte } // OnRetrieveBlock mocks base method -func (m *MockDatabaseSeries) OnRetrieveBlock(arg0 ident.ID, arg1 time.Time, arg2 ts.Segment) { - m.ctrl.Call(m, "OnRetrieveBlock", arg0, arg1, arg2) +func (m *MockDatabaseSeries) OnRetrieveBlock(arg0 ident.ID, arg1 ident.TagIterator, arg2 time.Time, arg3 ts.Segment) { + m.ctrl.Call(m, "OnRetrieveBlock", arg0, arg1, arg2, arg3) } // OnRetrieveBlock indicates an expected call of OnRetrieveBlock -func (mr *MockDatabaseSeriesMockRecorder) OnRetrieveBlock(arg0, arg1, arg2 interface{}) *gomock.Call { - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnRetrieveBlock", reflect.TypeOf((*MockDatabaseSeries)(nil).OnRetrieveBlock), arg0, arg1, arg2) +func (mr *MockDatabaseSeriesMockRecorder) OnRetrieveBlock(arg0, arg1, arg2, arg3 interface{}) *gomock.Call { + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnRetrieveBlock", reflect.TypeOf((*MockDatabaseSeries)(nil).OnRetrieveBlock), arg0, arg1, arg2, arg3) } // ReadEncoded mocks base method diff --git a/storage/series/series_test.go b/storage/series/series_test.go index cac631ff50..3bcd9b1ed4 100644 --- a/storage/series/series_test.go +++ b/storage/series/series_test.go @@ -219,7 +219,9 @@ func TestSeriesFlush(t *testing.T) { inputs := []error{errors.New("some error"), nil} for _, input := range inputs { - persistFn := func(id ident.ID, segment ts.Segment, checksum uint32) error { return input } + persistFn := func(_ ident.ID, _ ident.Tags, _ ts.Segment, _ uint32) error { + return input + } ctx := context.NewContext() outcome, err := series.Flush(ctx, flushTime, persistFn) ctx.BlockingClose() diff --git a/storage/shard.go b/storage/shard.go index 5c6b3a96b2..df3f3dbea2 100644 --- a/storage/shard.go +++ b/storage/shard.go @@ -368,6 +368,7 @@ func (s *dbShard) IsBlockRetrievable(blockStart time.Time) bool { func (s *dbShard) OnRetrieveBlock( id ident.ID, + tags ident.TagIterator, startTime time.Time, segment ts.Segment, ) { @@ -384,13 +385,11 @@ func (s *dbShard) OnRetrieveBlock( } if entry != nil { - entry.series.OnRetrieveBlock(id, startTime, segment) + entry.series.OnRetrieveBlock(id, tags, startTime, segment) return } - // Insert batched with the retrieved block - // FOLLOWUP(prateek): Need to retrieve tags during the block retrieval path too - entry, err = s.newShardEntry(id, ident.EmptyTagIterator) + entry, err = s.newShardEntry(id, tags) if err != nil { // should never happen s.logger.WithFields( @@ -400,13 +399,18 @@ func (s *dbShard) OnRetrieveBlock( return } + // NB(r): Do not need to specify that needs to be indexed as series would + // have been already been indexed when it was written copiedID := entry.series.ID() + // TODO(r): Pool the slice iterators here. + copiedTags := ident.NewTagSliceIterator(entry.series.Tags()) s.insertQueue.Insert(dbShardInsert{ entry: entry, opts: dbShardInsertAsyncOptions{ hasPendingRetrievedBlock: true, pendingRetrievedBlock: dbShardPendingRetrievedBlock{ id: copiedID, + tags: copiedTags, start: startTime, segment: segment, }, @@ -758,7 +762,8 @@ func (s *dbShard) WriteTagged( unit xtime.Unit, annotation []byte, ) error { - return s.writeAndIndex(ctx, id, tags, timestamp, value, unit, annotation, true) + return s.writeAndIndex(ctx, id, tags, timestamp, + value, unit, annotation, true) } func (s *dbShard) Write( @@ -1203,7 +1208,7 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { if inserts[i].opts.hasPendingRetrievedBlock { releaseEntryRef = true block := inserts[i].opts.pendingRetrievedBlock - entry.series.OnRetrieveBlock(block.id, block.start, block.segment) + entry.series.OnRetrieveBlock(block.id, block.tags, block.start, block.segment) } if releaseEntryRef { @@ -1509,7 +1514,8 @@ func (s *dbShard) FetchBlocksMetadataV2( } for numResults < limit { - id, size, checksum, err := reader.ReadMetadata() + // FOLLOWUP(r): Return the tags as part of the metadata to peers. + id, _, size, checksum, err := reader.ReadMetadata() if err == io.EOF { // Clean end of volume, we can break now if err := reader.Close(); err != nil { diff --git a/storage/shard_fetch_blocks_metadata_test.go b/storage/shard_fetch_blocks_metadata_test.go index 2306617d06..05b2ea8f6d 100644 --- a/storage/shard_fetch_blocks_metadata_test.go +++ b/storage/shard_fetch_blocks_metadata_test.go @@ -238,7 +238,7 @@ func TestShardFetchBlocksMetadataV2WithSeriesCachePolicyNotCacheAll(t *testing.T bytes := checked.NewBytes(data, nil) bytes.IncRef() - err = writer.Write(id, bytes, checksum) + err = writer.Write(id, nil, bytes, checksum) require.NoError(t, err) blockMetadataResult := block.NewFetchBlockMetadataResult(at, diff --git a/storage/shard_insert_queue.go b/storage/shard_insert_queue.go index 84d376ab89..faf89ba171 100644 --- a/storage/shard_insert_queue.go +++ b/storage/shard_insert_queue.go @@ -124,6 +124,7 @@ type dbShardPendingIndex struct { type dbShardPendingRetrievedBlock struct { id ident.ID + tags ident.TagIterator start time.Time segment ts.Segment } diff --git a/storage/shard_test.go b/storage/shard_test.go index 23981b869d..d05b435d09 100644 --- a/storage/shard_test.go +++ b/storage/shard_test.go @@ -260,7 +260,7 @@ func TestShardFlushSeriesFlushError(t *testing.T) { var closed bool flush := persist.NewMockFlush(ctrl) prepared := persist.PreparedPersist{ - Persist: func(ident.ID, ts.Segment, uint32) error { return nil }, + Persist: func(ident.ID, ident.Tags, ts.Segment, uint32) error { return nil }, Close: func() error { closed = true; return nil }, } prepareOpts := persist.PrepareOptionsMatcher{ @@ -325,7 +325,7 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { var closed bool flush := persist.NewMockFlush(ctrl) prepared := persist.PreparedPersist{ - Persist: func(ident.ID, ts.Segment, uint32) error { return nil }, + Persist: func(ident.ID, ident.Tags, ts.Segment, uint32) error { return nil }, Close: func() error { closed = true; return nil }, } @@ -397,7 +397,7 @@ func TestShardSnapshotSeriesSnapshotSuccess(t *testing.T) { var closed bool flush := persist.NewMockFlush(ctrl) prepared := persist.PreparedPersist{ - Persist: func(ident.ID, ts.Segment, uint32) error { return nil }, + Persist: func(ident.ID, ident.Tags, ts.Segment, uint32) error { return nil }, Close: func() error { closed = true; return nil }, } @@ -1092,14 +1092,14 @@ func TestShardReadEncodedCachesSeriesWithRecentlyReadPolicy(t *testing.T) { Stream(ctx, shard.shard, ident.NewIDMatcher("foo"), start, shard.seriesOnRetrieveBlock). Do(func(ctx context.Context, shard uint32, id ident.ID, at time.Time, onRetrieve block.OnRetrieveBlock) { - go onRetrieve.OnRetrieveBlock(id, at, segments[0]) + go onRetrieve.OnRetrieveBlock(id, ident.EmptyTagIterator, at, segments[0]) }). Return(segReaders[0], nil) retriever.EXPECT(). Stream(ctx, shard.shard, ident.NewIDMatcher("foo"), start.Add(ropts.BlockSize()), shard.seriesOnRetrieveBlock). Do(func(ctx context.Context, shard uint32, id ident.ID, at time.Time, onRetrieve block.OnRetrieveBlock) { - go onRetrieve.OnRetrieveBlock(id, at, segments[1]) + go onRetrieve.OnRetrieveBlock(id, ident.EmptyTagIterator, at, segments[1]) }). Return(segReaders[1], nil) From cf544694cef8fd2715b5d6c16c76dd095b340379 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 7 May 2018 09:42:14 -0400 Subject: [PATCH 02/23] Fix integration test build failures --- integration/disk_flush_helpers.go | 14 +++++++++++++- integration/generate/types.go | 1 + integration/generate/writer.go | 3 ++- persist/fs/clone/cloner.go | 17 +++++++++++++++-- storage/index/types.go | 2 -- 5 files changed, 31 insertions(+), 6 deletions(-) diff --git a/integration/disk_flush_helpers.go b/integration/disk_flush_helpers.go index 11ba4ab41c..881f11ff00 100644 --- a/integration/disk_flush_helpers.go +++ b/integration/disk_flush_helpers.go @@ -140,9 +140,20 @@ func verifyForTime( } require.NoError(t, reader.Open(rOpts)) for i := 0; i < reader.Entries(); i++ { - id, data, _, err := reader.Read() + id, tagsIter, data, _, err := reader.Read() require.NoError(t, err) + var tags ident.Tags + if tagsLen := tagsIter.Remaining(); tagsLen > 0 { + tags = make(ident.Tags, 0, tagsLen) + for tagsIter.Next() { + curr := tagsIter.Current() + tags = append(tags, ident.StringTag(curr.Name.String(), curr.Value.String())) + } + require.NoError(t, tagsIter.Err()) + tagsIter.Close() + } + data.IncRef() var datapoints []ts.Datapoint @@ -157,6 +168,7 @@ func verifyForTime( actual = append(actual, generate.Series{ ID: id, + Tags: tags, Data: datapoints, }) diff --git a/integration/generate/types.go b/integration/generate/types.go index 5edfe7d132..217140aaec 100644 --- a/integration/generate/types.go +++ b/integration/generate/types.go @@ -22,6 +22,7 @@ type BlockConfig struct { // Series represents a generated series of data type Series struct { ID ident.ID + Tags ident.Tags Data []ts.Datapoint } diff --git a/integration/generate/writer.go b/integration/generate/writer.go index 871d127eae..21b1153073 100644 --- a/integration/generate/writer.go +++ b/integration/generate/writer.go @@ -139,7 +139,8 @@ func writeToDisk( data[0] = segment.Head data[1] = segment.Tail checksum := digest.SegmentChecksum(segment) - if err := writer.WriteAll(series.ID, data, checksum); err != nil { + err = writer.WriteAll(series.ID, series.Tags, data, checksum) + if err != nil { return err } } diff --git a/persist/fs/clone/cloner.go b/persist/fs/clone/cloner.go index e25ca60210..60e26a14f1 100644 --- a/persist/fs/clone/cloner.go +++ b/persist/fs/clone/cloner.go @@ -62,7 +62,7 @@ func (c *cloner) Clone(src FileSetID, dest FileSetID, destBlocksize time.Duratio } for { - id, data, checksum, err := reader.Read() + id, tagsIter, data, checksum, err := reader.Read() if err != nil { if err == io.EOF { break @@ -70,8 +70,21 @@ func (c *cloner) Clone(src FileSetID, dest FileSetID, destBlocksize time.Duratio return fmt.Errorf("unexpected error while reading data: %v", err) } + var tags ident.Tags + if tagsLen := tagsIter.Remaining(); tagsLen > 0 { + tags = make(ident.Tags, 0, tagsLen) + for tagsIter.Next() { + curr := tagsIter.Current() + tags = append(tags, ident.StringTag(curr.Name.String(), curr.Value.String())) + } + if err := tagsIter.Err(); err != nil { + return fmt.Errorf("unable to decode tags: %v", err) + } + tagsIter.Close() + } + data.IncRef() - if err := writer.Write(id, data, checksum); err != nil { + if err := writer.Write(id, tags, data, checksum); err != nil { return fmt.Errorf("unexpected error while writing data: %v", err) } data.DecRef() diff --git a/storage/index/types.go b/storage/index/types.go index 1db52d5f63..b2ba7d29bf 100644 --- a/storage/index/types.go +++ b/storage/index/types.go @@ -23,8 +23,6 @@ package index import ( "time" - "github.com/m3db/m3ninx/index/segment" - "github.com/m3db/m3db/clock" "github.com/m3db/m3ninx/doc" "github.com/m3db/m3ninx/idx" From 9014e998ff88246eb155f4ace7e75a0f5beb9ca4 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 7 May 2018 10:12:31 -0400 Subject: [PATCH 03/23] Fix unit test build errors --- persist/fs/clone/cloner.go | 20 +++++++ persist/fs/clone/cloner_test.go | 53 +++++++++++++++++-- persist/fs/clone/options.go | 20 +++++++ persist/fs/clone/types.go | 20 +++++++ .../bootstrapper/peers/source_test.go | 24 ++++----- 5 files changed, 121 insertions(+), 16 deletions(-) diff --git a/persist/fs/clone/cloner.go b/persist/fs/clone/cloner.go index 60e26a14f1..7c132d7cb8 100644 --- a/persist/fs/clone/cloner.go +++ b/persist/fs/clone/cloner.go @@ -1,3 +1,23 @@ +// Copyright (c) 2018 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 clone import ( diff --git a/persist/fs/clone/cloner_test.go b/persist/fs/clone/cloner_test.go index ec2ca92626..04269bb2e1 100644 --- a/persist/fs/clone/cloner_test.go +++ b/persist/fs/clone/cloner_test.go @@ -1,3 +1,23 @@ +// Copyright (c) 2018 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 clone import ( @@ -87,14 +107,32 @@ func TestCloner(t *testing.T) { } require.NoError(t, r2.Open(r2OpenOpts)) for { - t1, b1, c1, e1 := r1.Read() - t2, b2, c2, e2 := r2.Read() + t1, a1, b1, c1, e1 := r1.Read() + t2, a2, b2, c2, e2 := r2.Read() if e1 == e2 && e1 == io.EOF { break } + + require.NoError(t, e1) + require.NoError(t, e2) + b1.IncRef() b2.IncRef() require.Equal(t, t1.String(), t2.String()) + require.Equal(t, a1.Remaining(), a2.Remaining()) + numTags, numTagsMatched := a1.Remaining(), 0 + for a1.Next() && a2.Next() { + tag0 := a1.Current() + tag1 := a2.Current() + require.Equal(t, tag0.Name.String(), tag1.Name.String()) + require.Equal(t, tag0.Value.String(), tag1.Value.String()) + numTagsMatched++ + } + require.NoError(t, a1.Err()) + require.NoError(t, a2.Err()) + a1.Close() + a2.Close() + require.Equal(t, numTags, numTagsMatched) require.Equal(t, b1.Bytes(), b2.Bytes()) require.Equal(t, c1, c2) b1.DecRef() @@ -121,8 +159,15 @@ func writeTestData(t *testing.T, bs time.Duration, src FileSetID, opts Options) } require.NoError(t, w.Open(writerOpts)) for i := 0; i < numTestSeries; i++ { - id := ident.StringID(fmt.Sprintf("testSeries.%d", i)) - require.NoError(t, w.Write(id, testBytes, 1234)) + id := ident.StringID(fmt.Sprintf("test-series.%d", i)) + var tags ident.Tags + if i%2 == 0 { + tags = ident.Tags{ + ident.StringTag("foo", "bar"), + ident.StringTag("qux", "qaz"), + } + } + require.NoError(t, w.Write(id, tags, testBytes, 1234)) } require.NoError(t, w.Close()) } diff --git a/persist/fs/clone/options.go b/persist/fs/clone/options.go index cc8a90040c..7538fa0a29 100644 --- a/persist/fs/clone/options.go +++ b/persist/fs/clone/options.go @@ -1,3 +1,23 @@ +// Copyright (c) 2018 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 clone import ( diff --git a/persist/fs/clone/types.go b/persist/fs/clone/types.go index 7d644eff94..0c2f1fce93 100644 --- a/persist/fs/clone/types.go +++ b/persist/fs/clone/types.go @@ -1,3 +1,23 @@ +// Copyright (c) 2018 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 clone import ( diff --git a/storage/bootstrap/bootstrapper/peers/source_test.go b/storage/bootstrap/bootstrapper/peers/source_test.go index f3bc9d057d..351f261f35 100644 --- a/storage/bootstrap/bootstrapper/peers/source_test.go +++ b/storage/bootstrap/bootstrapper/peers/source_test.go @@ -248,7 +248,7 @@ func TestPeersSourceIncrementalRun(t *testing.T) { mockFlush.EXPECT(). Prepare(prepareOpts). Return(persist.PreparedPersist{ - Persist: func(id ident.ID, segment ts.Segment, checksum uint32) error { + Persist: func(id ident.ID, _ ident.Tags, segment ts.Segment, checksum uint32) error { persists["foo"]++ assert.Equal(t, "foo", id.String()) assert.Equal(t, []byte{1, 2, 3}, segment.Head.Bytes()) @@ -268,7 +268,7 @@ func TestPeersSourceIncrementalRun(t *testing.T) { mockFlush.EXPECT(). Prepare(prepareOpts). Return(persist.PreparedPersist{ - Persist: func(id ident.ID, segment ts.Segment, checksum uint32) error { + Persist: func(id ident.ID, _ ident.Tags, segment ts.Segment, checksum uint32) error { persists["bar"]++ assert.Equal(t, "bar", id.String()) assert.Equal(t, []byte{4, 5, 6}, segment.Head.Bytes()) @@ -288,7 +288,7 @@ func TestPeersSourceIncrementalRun(t *testing.T) { mockFlush.EXPECT(). Prepare(prepareOpts). Return(persist.PreparedPersist{ - Persist: func(id ident.ID, segment ts.Segment, checksum uint32) error { + Persist: func(id ident.ID, _ ident.Tags, segment ts.Segment, checksum uint32) error { persists["baz"]++ assert.Equal(t, "baz", id.String()) assert.Equal(t, []byte{7, 8, 9}, segment.Head.Bytes()) @@ -308,7 +308,7 @@ func TestPeersSourceIncrementalRun(t *testing.T) { mockFlush.EXPECT(). Prepare(prepareOpts). Return(persist.PreparedPersist{ - Persist: func(id ident.ID, segment ts.Segment, checksum uint32) error { + Persist: func(id ident.ID, _ ident.Tags, segment ts.Segment, checksum uint32) error { assert.Fail(t, "no expected shard 1 second block") return nil }, @@ -499,7 +499,7 @@ func TestPeersSourceMarksUnfulfilledOnIncrementalFlushErrors(t *testing.T) { mockFlush.EXPECT(). Prepare(prepareOpts). Return(persist.PreparedPersist{ - Persist: func(id ident.ID, segment ts.Segment, checksum uint32) error { + Persist: func(id ident.ID, _ ident.Tags, segment ts.Segment, checksum uint32) error { assert.Fail(t, "not expecting to flush shard 0 at start") return nil }, @@ -516,7 +516,7 @@ func TestPeersSourceMarksUnfulfilledOnIncrementalFlushErrors(t *testing.T) { mockFlush.EXPECT(). Prepare(prepareOpts). Return(persist.PreparedPersist{ - Persist: func(id ident.ID, segment ts.Segment, checksum uint32) error { + Persist: func(id ident.ID, _ ident.Tags, segment ts.Segment, checksum uint32) error { persists["foo"]++ return nil }, @@ -535,7 +535,7 @@ func TestPeersSourceMarksUnfulfilledOnIncrementalFlushErrors(t *testing.T) { mockFlush.EXPECT(). Prepare(prepareOpts). Return(persist.PreparedPersist{ - Persist: func(id ident.ID, segment ts.Segment, checksum uint32) error { + Persist: func(id ident.ID, _ ident.Tags, segment ts.Segment, checksum uint32) error { assert.Fail(t, "not expecting to flush shard 0 at start + block size") return nil }, @@ -552,7 +552,7 @@ func TestPeersSourceMarksUnfulfilledOnIncrementalFlushErrors(t *testing.T) { mockFlush.EXPECT(). Prepare(prepareOpts). Return(persist.PreparedPersist{ - Persist: func(id ident.ID, segment ts.Segment, checksum uint32) error { + Persist: func(id ident.ID, _ ident.Tags, segment ts.Segment, checksum uint32) error { persists["bar"]++ return nil }, @@ -571,7 +571,7 @@ func TestPeersSourceMarksUnfulfilledOnIncrementalFlushErrors(t *testing.T) { mockFlush.EXPECT(). Prepare(prepareOpts). Return(persist.PreparedPersist{ - Persist: func(id ident.ID, segment ts.Segment, checksum uint32) error { + Persist: func(id ident.ID, _ ident.Tags, segment ts.Segment, checksum uint32) error { persists["baz"]++ return fmt.Errorf("a persist error") }, @@ -588,7 +588,7 @@ func TestPeersSourceMarksUnfulfilledOnIncrementalFlushErrors(t *testing.T) { mockFlush.EXPECT(). Prepare(prepareOpts). Return(persist.PreparedPersist{ - Persist: func(id ident.ID, segment ts.Segment, checksum uint32) error { + Persist: func(id ident.ID, _ ident.Tags, segment ts.Segment, checksum uint32) error { persists["baz"]++ return nil }, @@ -607,7 +607,7 @@ func TestPeersSourceMarksUnfulfilledOnIncrementalFlushErrors(t *testing.T) { mockFlush.EXPECT(). Prepare(prepareOpts). Return(persist.PreparedPersist{ - Persist: func(id ident.ID, segment ts.Segment, checksum uint32) error { + Persist: func(id ident.ID, _ ident.Tags, segment ts.Segment, checksum uint32) error { persists["qux"]++ return nil }, @@ -624,7 +624,7 @@ func TestPeersSourceMarksUnfulfilledOnIncrementalFlushErrors(t *testing.T) { mockFlush.EXPECT(). Prepare(prepareOpts). Return(persist.PreparedPersist{ - Persist: func(id ident.ID, segment ts.Segment, checksum uint32) error { + Persist: func(id ident.ID, _ ident.Tags, segment ts.Segment, checksum uint32) error { persists["qux"]++ return nil }, From 48afa2384637573e0ceb17c4da49af564112c8af Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 7 May 2018 10:46:13 -0400 Subject: [PATCH 04/23] Fix fs identifier pool --- persist/fs/index_lookup_prop_test.go | 18 ++++++++++++++++-- storage/bootstrap/bootstrapper/fs/options.go | 4 +++- storage/bootstrap/bootstrapper/fs/source.go | 4 +++- 3 files changed, 22 insertions(+), 4 deletions(-) diff --git a/persist/fs/index_lookup_prop_test.go b/persist/fs/index_lookup_prop_test.go index 13fa0a3604..466d32749f 100644 --- a/persist/fs/index_lookup_prop_test.go +++ b/persist/fs/index_lookup_prop_test.go @@ -160,7 +160,7 @@ func calculateExpectedChecksum(t *testing.T, filePath string) uint32 { func writeTestSummariesData(w DataFileSetWriter, writes []generatedWrite) error { for _, write := range writes { - err := w.Write(write.id, write.data, write.checksum) + err := w.Write(write.id, write.tags, write.data, write.checksum) if err != nil { return err } @@ -177,6 +177,7 @@ type propTestInput struct { type generatedWrite struct { id ident.ID + tags ident.Tags data checked.Bytes checksum uint32 } @@ -208,14 +209,27 @@ func genWrite() gopter.Gen { // gopter will generate random strings, but some of them may be duplicates // (which can't normally happen for IDs and breaks this codepath), so we // filter down to unique inputs + // ID gen.AnyString(), + // Tag 1 + gen.AnyString(), + gen.AnyString(), + // Tag 2 + gen.AnyString(), + gen.AnyString(), + // Data gen.SliceOfN(100, gen.UInt8()), ).Map(func(vals []interface{}) generatedWrite { id := vals[0].(string) - data := vals[1].([]byte) + tags := []ident.Tag{ + ident.StringTag(vals[1].(string), vals[2].(string)), + ident.StringTag(vals[3].(string), vals[4].(string)), + } + data := vals[5].([]byte) return generatedWrite{ id: ident.StringID(id), + tags: tags, data: bytesRefd(data), checksum: digest.Checksum(data), } diff --git a/storage/bootstrap/bootstrapper/fs/options.go b/storage/bootstrap/bootstrapper/fs/options.go index c46aa50058..6f72d525dc 100644 --- a/storage/bootstrap/bootstrapper/fs/options.go +++ b/storage/bootstrap/bootstrapper/fs/options.go @@ -48,11 +48,13 @@ func NewOptions() Options { bytesPool := pool.NewCheckedBytesPool(nil, nil, func(s []pool.Bucket) pool.BytesPool { return pool.NewBytesPool(s, nil) }) + bytesPool.Init() + idPool := ident.NewPool(bytesPool, pool.NewObjectPoolOptions()) return &options{ resultOpts: result.NewOptions(), fsOpts: fs.NewOptions(), numProcessors: defaultNumProcessors, - identifierPool: ident.NewPool(bytesPool, pool.NewObjectPoolOptions()), + identifierPool: idPool, } } diff --git a/storage/bootstrap/bootstrapper/fs/source.go b/storage/bootstrap/bootstrapper/fs/source.go index c7b8c2a127..56c5b2ff75 100644 --- a/storage/bootstrap/bootstrapper/fs/source.go +++ b/storage/bootstrap/bootstrapper/fs/source.go @@ -70,6 +70,7 @@ type fileSystemSource struct { opts Options fsopts fs.Options log xlog.Logger + idPool ident.Pool newReaderFn newDataFileSetReaderFn processors xsync.WorkerPool } @@ -81,6 +82,7 @@ func newFileSystemSource(opts Options) bootstrap.Source { opts: opts, fsopts: opts.FilesystemOptions(), log: opts.ResultOptions().InstrumentOptions().Logger(), + idPool: opts.IdentifierPool(), newReaderFn: fs.NewReader, processors: processors, } @@ -350,7 +352,7 @@ func (s *fileSystemSource) tagsFromTagsIter( tags := make(ident.Tags, 0, iter.Remaining()) for iter.Next() { curr := iter.Current() - tags = append(tags, s.opts.IdentifierPool().CloneTag(curr)) + tags = append(tags, s.idPool.CloneTag(curr)) } return tags, iter.Err() } From 9a70a80d4fe28adbebaa9098455dcdfe34601bbd Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 7 May 2018 10:59:54 -0400 Subject: [PATCH 05/23] Fix lint issues --- storage/bootstrap/bootstrapper/fs/source.go | 6 ++++++ storage/bootstrap/bootstrapper/fs/source_test.go | 12 ++++++------ tools/read_data_files/main/main.go | 2 +- tools/read_index_ids/main/main.go | 2 +- tools/verify_index_files/main/main.go | 2 +- 5 files changed, 15 insertions(+), 9 deletions(-) diff --git a/storage/bootstrap/bootstrapper/fs/source.go b/storage/bootstrap/bootstrapper/fs/source.go index 56c5b2ff75..913f6510e9 100644 --- a/storage/bootstrap/bootstrapper/fs/source.go +++ b/storage/bootstrap/bootstrapper/fs/source.go @@ -446,6 +446,12 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( if run == bootstrapIndexRunType { // If performing index run, then simply read the metadata and add to segment id, tagsIter, _, _, err := r.ReadMetadata() + if err != nil { + s.log.Errorf("unable read metadata: %v", err) + hasError = true + continue + } + idBytes := id.Bytes() resultLock.RLock() diff --git a/storage/bootstrap/bootstrapper/fs/source_test.go b/storage/bootstrap/bootstrapper/fs/source_test.go index efc2fc9d23..a42a204b64 100644 --- a/storage/bootstrap/bootstrapper/fs/source_test.go +++ b/storage/bootstrap/bootstrapper/fs/source_test.go @@ -558,21 +558,21 @@ func TestReadDeleteOnError(t *testing.T) { Return(xtime.Range{ Start: testStart, End: testStart.Add(2 * time.Hour), - }), - reader.EXPECT().Entries().Return(2), + }).AnyTimes(), + reader.EXPECT().Entries().Return(2).AnyTimes(), reader.EXPECT(). Range(). Return(xtime.Range{ Start: testStart, End: testStart.Add(2 * time.Hour), - }), - reader.EXPECT().Entries().Return(2), + }).AnyTimes(), + reader.EXPECT().Entries().Return(2).AnyTimes(), reader.EXPECT(). Read(). - Return(ident.StringID("foo"), nil, digest.Checksum(nil), nil), + Return(ident.StringID("foo"), ident.EmptyTagIterator, nil, digest.Checksum(nil), nil), reader.EXPECT(). Read(). - Return(ident.StringID("bar"), nil, uint32(0), errors.New("foo")), + Return(ident.StringID("bar"), ident.EmptyTagIterator, nil, uint32(0), errors.New("foo")), reader.EXPECT().Close().Return(nil), ) diff --git a/tools/read_data_files/main/main.go b/tools/read_data_files/main/main.go index f23285fc2a..7c6c915e15 100644 --- a/tools/read_data_files/main/main.go +++ b/tools/read_data_files/main/main.go @@ -62,7 +62,7 @@ func main() { } for { - id, data, _, err := reader.Read() + id, _, data, _, err := reader.Read() if err == io.EOF { break } diff --git a/tools/read_index_ids/main/main.go b/tools/read_index_ids/main/main.go index 533f5078f0..34e4b2450e 100644 --- a/tools/read_index_ids/main/main.go +++ b/tools/read_index_ids/main/main.go @@ -53,7 +53,7 @@ func main() { } for { - id, _, _, err := reader.ReadMetadata() + id, _, _, _, err := reader.ReadMetadata() if err == io.EOF { break } diff --git a/tools/verify_index_files/main/main.go b/tools/verify_index_files/main/main.go index ae20ca5e3c..ecc381a5da 100644 --- a/tools/verify_index_files/main/main.go +++ b/tools/verify_index_files/main/main.go @@ -134,7 +134,7 @@ func seriesChecksumsFromReader(reader fs.DataFileSetReader, host string, shard u block: block, } for { - id, _, checksumVal, err := reader.ReadMetadata() + id, _, _, checksumVal, err := reader.ReadMetadata() if err == io.EOF { return seriesChecksums } From 2aa6dedde51958f729c4389cb1c2cf9aa9d650d1 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 7 May 2018 11:20:55 -0400 Subject: [PATCH 06/23] Fix remaining metalint issues --- persist/fs/msgpack/encoder.go | 22 ++++++++++----------- persist/fs/write.go | 4 ---- storage/bootstrap/bootstrapper/fs/source.go | 2 +- 3 files changed, 12 insertions(+), 16 deletions(-) diff --git a/persist/fs/msgpack/encoder.go b/persist/fs/msgpack/encoder.go index 6cd7c2abf5..a6fd76c9db 100644 --- a/persist/fs/msgpack/encoder.go +++ b/persist/fs/msgpack/encoder.go @@ -103,7 +103,12 @@ func (enc *Encoder) EncodeIndexInfo(info schema.IndexInfo) error { if enc.err != nil { return enc.err } - enc.encodeIndexInfo(info) + enc.encodeRootObject(indexInfoVersion, indexInfoType) + if enc.encodeLegacyV1IndexInfo { + enc.encodeIndexInfoV1(info) + } else { + enc.encodeIndexInfoV2(info) + } return enc.err } @@ -113,7 +118,11 @@ func (enc *Encoder) EncodeIndexEntry(entry schema.IndexEntry) error { return enc.err } enc.encodeRootObject(indexEntryVersion, indexEntryType) - enc.encodeIndexEntryV2(entry) + if enc.encodeLegacyV1IndexInfo { + enc.encodeIndexEntryV1(entry) + } else { + enc.encodeIndexEntryV2(entry) + } return enc.err } @@ -157,15 +166,6 @@ func (enc *Encoder) EncodeLogMetadata(entry schema.LogMetadata) error { return enc.err } -func (enc *Encoder) encodeIndexInfo(info schema.IndexInfo) { - enc.encodeRootObject(indexInfoVersion, indexInfoType) - if enc.encodeLegacyV1IndexInfo { - enc.encodeIndexInfoV1(info) - } else { - enc.encodeIndexInfoV2(info) - } -} - // We only keep this method around for the sake of testing // backwards-compatbility func (enc *Encoder) encodeIndexInfoV1(info schema.IndexInfo) { diff --git a/persist/fs/write.go b/persist/fs/write.go index e4867d5cde..c05d4d6dac 100644 --- a/persist/fs/write.go +++ b/persist/fs/write.go @@ -400,10 +400,6 @@ func (w *writer) writeIndexRelatedFiles() error { return w.writeInfoFileContents(bloomFilter, summaries) } -type tagsIter struct { - tags ident.Tags -} - func (w *writer) writeIndexFileContents( bloomFilter *bloom.BloomFilter, summaryEvery int, diff --git a/storage/bootstrap/bootstrapper/fs/source.go b/storage/bootstrap/bootstrapper/fs/source.go index 913f6510e9..07a4ef17df 100644 --- a/storage/bootstrap/bootstrapper/fs/source.go +++ b/storage/bootstrap/bootstrapper/fs/source.go @@ -460,7 +460,7 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( if err == nil && !exists { resultLock.Lock() - exists, err := indexBlockSegment.ContainsID(idBytes) + exists, err = indexBlockSegment.ContainsID(idBytes) if err == nil && !exists { d := doc.Document{ ID: append([]byte(nil), idBytes...), From fe8c7642b1fc311f42e3db08ef636e5184ed1133 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 7 May 2018 11:52:59 -0400 Subject: [PATCH 07/23] Fix encode legacy index entry flag --- persist/fs/msgpack/encoder.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/persist/fs/msgpack/encoder.go b/persist/fs/msgpack/encoder.go index a6fd76c9db..45278a0b05 100644 --- a/persist/fs/msgpack/encoder.go +++ b/persist/fs/msgpack/encoder.go @@ -118,7 +118,7 @@ func (enc *Encoder) EncodeIndexEntry(entry schema.IndexEntry) error { return enc.err } enc.encodeRootObject(indexEntryVersion, indexEntryType) - if enc.encodeLegacyV1IndexInfo { + if enc.encodeLegacyV1IndexEntry { enc.encodeIndexEntryV1(entry) } else { enc.encodeIndexEntryV2(entry) From 8095ab53b0a00fda72fd4fa0022d9d3db9bcd826 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 7 May 2018 16:28:14 -0400 Subject: [PATCH 08/23] Address feedback --- docs/architecture/storage.md | 58 +++++++++++++-------------- glide.lock | 6 +-- glide.yaml | 2 +- integration/disk_flush_helpers.go | 13 ++---- persist/fs/clone/cloner.go | 15 ++----- persist/fs/clone/cloner_test.go | 15 +------ persist/fs/msgpack/roundtrip_test.go | 2 + persist/fs/read.go | 18 ++++----- persist/fs/retriever.go | 2 +- services/m3dbnode/config/bootstrap.go | 3 +- storage/bootstrap/result/options.go | 13 +++--- storage/index/allocator.go | 6 +-- 12 files changed, 63 insertions(+), 90 deletions(-) diff --git a/docs/architecture/storage.md b/docs/architecture/storage.md index d4cff954b9..92231eff75 100644 --- a/docs/architecture/storage.md +++ b/docs/architecture/storage.md @@ -2,7 +2,7 @@ ## Overview -The primary unit of long-term storage for M3DB are fileset files which store compressed streams of time series values, one per shard block time window size. +The primary unit of long-term storage for M3DB are fileset files which store compressed streams of time series values, one per shard block time window size. They are flushed to disk after a block time window becomes unreachable, that is the end of the time window for which that block can no longer be written to. If a process is killed before it has a chance to flush the data for the current time window to disk it must be restored from the commit log (or a peer that is responsible for the same shard if replication factor is larger than 1.) @@ -12,43 +12,43 @@ A fileset has the following files: * **Info file:** Stores the block time window start and size and other important metadata about the fileset volume. * **Summaries file:** Stores a subset of the index file for purposes of keeping the contents in memory and jumping to section of the index file that within a few pages of linear scanning can find the series that is being looked up. -* **Index file:** Stores the series metadata and location of compressed stream in the data file for retrieval. +* **Index file:** Stores the series metadata, including tags if indexing is enabled, and location of compressed stream in the data file for retrieval. * **Data file:** Stores the series compressed data streams. * **Bloom filter file:** Stores a bloom filter bitset of all series contained in this fileset for quick knowledge of whether to attempt retrieving a series for this fileset volume. * **Digests file:** Stores the digest checksums of the info file, summaries file, index file, data file and bloom filter file in the fileset volume for integrity verification. * **Checkpoint file:** Stores a digest of the digests file and written at the succesful completion of a fileset volume being persisted, allows for quickly checking if a volume was completed. ``` - ┌─────────────────────┐ -┌─────────────────────┐ ┌─────────────────────┐ │ Index File │ -│ Info File │ │ Summaries File │ │ (sorted by ID) │ -├─────────────────────┤ │ (sorted by ID) │ ├─────────────────────┤ -│- Block Start │ ├─────────────────────┤ ┌─>│- Idx │ -│- Block Size │ │- Idx │ │ │- ID │ -│- Entries (Num) │ │- ID │ │ │- Size │ -│- Major Version │ │- Index Entry Offset ├──┘ │- Checksum │ + ┌─────────────────────┐ +┌─────────────────────┐ ┌─────────────────────┐ │ Index File │ +│ Info File │ │ Summaries File │ │ (sorted by ID) │ +├─────────────────────┤ │ (sorted by ID) │ ├─────────────────────┤ +│- Block Start │ ├─────────────────────┤ ┌─>│- Idx │ +│- Block Size │ │- Idx │ │ │- ID │ +│- Entries (Num) │ │- ID │ │ │- Size │ +│- Major Version │ │- Index Entry Offset ├──┘ │- Checksum │ │- Summaries (Num) │ └─────────────────────┘ │- Data Entry Offset ├──┐ -│- BloomFilter (K/M) │ └─────────────────────┘ │ -└─────────────────────┘ │ +│- BloomFilter (K/M) │ │- Encoded Tags | | +└─────────────────────┘ └─────────────────────┘ │ + │ ┌─────────────────────┐ ┌───────────────────────────┘ -┌─────────────────────┐ │ Bloom Filter File │ │ -│ Digests File │ ├─────────────────────┤ │ ┌─────────────────────┐ -├─────────────────────┤ │- Bitset │ │ │ Data File │ -│- Info file digest │ └─────────────────────┘ │ ├─────────────────────┤ -│- Summaries digest │ │ │List of: │ -│- Index digest │ └─>│ - Marker (16 bytes)│ -│- Data digest │ │ - ID │ -│- Bloom filter digest│ │ - Data (size bytes)│ -└─────────────────────┘ └─────────────────────┘ - -┌─────────────────────┐ -│ Checkpoint File │ -├─────────────────────┤ -│- Digests digest │ -└─────────────────────┘ - +┌─────────────────────┐ │ Bloom Filter File │ │ +│ Digests File │ ├─────────────────────┤ │ ┌─────────────────────┐ +├─────────────────────┤ │- Bitset │ │ │ Data File │ +│- Info file digest │ └─────────────────────┘ │ ├─────────────────────┤ +│- Summaries digest │ │ │List of: │ +│- Index digest │ └─>│ - Marker (16 bytes)│ +│- Data digest │ │ - ID │ +│- Bloom filter digest│ │ - Data (size bytes)│ +└─────────────────────┘ └─────────────────────┘ + +┌─────────────────────┐ +│ Checkpoint File │ +├─────────────────────┤ +│- Digests digest │ +└─────────────────────┘ ``` In the diagram above you can see that the data file stores compressed blocks for a given shard / block start combination. The index file (which is sorted by ID and thus can be binary searched or scanned) can be used to find the offset of a specific ID. -Fileset files will be kept for every shard / block start combination that is within the retention period. Once the files fall out of the period defined in the configurable namespace retention period they will be deleted. \ No newline at end of file +Fileset files will be kept for every shard / block start combination that is within the retention period. Once the files fall out of the period defined in the configurable namespace retention period they will be deleted. diff --git a/glide.lock b/glide.lock index 08d22d9f68..f0510d3b76 100644 --- a/glide.lock +++ b/glide.lock @@ -1,5 +1,5 @@ -hash: 1a35734f65b0178ec5e12d06a46808ea9a162d6de7fe89786358e71404453921 -updated: 2018-05-06T00:47:42.371751-04:00 +hash: 124d166053476550bef409f59deaf9cd7964494b50330e128665dbeb067bf244 +updated: 2018-05-07T16:05:15.281027-04:00 imports: - name: github.com/apache/thrift version: 53dd39833a08ce33582e5ff31fa18bb4735d6731 @@ -207,7 +207,7 @@ imports: - search/query - search/searcher - name: github.com/m3db/m3x - version: 0ae10beff2f574adfc42a5b42ecf34e92533be50 + version: f591af0fb913df0f2e8ecf6995d4aa406c4c13e0 vcs: git subpackages: - checked diff --git a/glide.yaml b/glide.yaml index 5f90a32f1b..6639eff410 100644 --- a/glide.yaml +++ b/glide.yaml @@ -1,7 +1,7 @@ package: github.com/m3db/m3db import: - package: github.com/m3db/m3x - version: 0ae10beff2f574adfc42a5b42ecf34e92533be50 + version: f591af0fb913df0f2e8ecf6995d4aa406c4c13e0 vcs: git subpackages: - checked diff --git a/integration/disk_flush_helpers.go b/integration/disk_flush_helpers.go index 881f11ff00..972cfeae9d 100644 --- a/integration/disk_flush_helpers.go +++ b/integration/disk_flush_helpers.go @@ -36,6 +36,7 @@ import ( "github.com/m3db/m3db/storage" "github.com/m3db/m3db/ts" "github.com/m3db/m3x/ident" + "github.com/m3db/m3x/ident/testutil" xtime "github.com/m3db/m3x/time" "github.com/stretchr/testify/require" @@ -143,16 +144,8 @@ func verifyForTime( id, tagsIter, data, _, err := reader.Read() require.NoError(t, err) - var tags ident.Tags - if tagsLen := tagsIter.Remaining(); tagsLen > 0 { - tags = make(ident.Tags, 0, tagsLen) - for tagsIter.Next() { - curr := tagsIter.Current() - tags = append(tags, ident.StringTag(curr.Name.String(), curr.Value.String())) - } - require.NoError(t, tagsIter.Err()) - tagsIter.Close() - } + tags, err := testutil.NewTagsFromTagIterator(tagsIter) + require.NoError(t, err) data.IncRef() diff --git a/persist/fs/clone/cloner.go b/persist/fs/clone/cloner.go index 7c132d7cb8..53739757f4 100644 --- a/persist/fs/clone/cloner.go +++ b/persist/fs/clone/cloner.go @@ -28,6 +28,7 @@ import ( "github.com/m3db/m3db/persist" "github.com/m3db/m3db/persist/fs" "github.com/m3db/m3x/ident" + "github.com/m3db/m3x/ident/testutil" ) type cloner struct { @@ -90,17 +91,9 @@ func (c *cloner) Clone(src FileSetID, dest FileSetID, destBlocksize time.Duratio return fmt.Errorf("unexpected error while reading data: %v", err) } - var tags ident.Tags - if tagsLen := tagsIter.Remaining(); tagsLen > 0 { - tags = make(ident.Tags, 0, tagsLen) - for tagsIter.Next() { - curr := tagsIter.Current() - tags = append(tags, ident.StringTag(curr.Name.String(), curr.Value.String())) - } - if err := tagsIter.Err(); err != nil { - return fmt.Errorf("unable to decode tags: %v", err) - } - tagsIter.Close() + tags, err := testutil.NewTagsFromTagIterator(tagsIter) + if err != nil { + return err } data.IncRef() diff --git a/persist/fs/clone/cloner_test.go b/persist/fs/clone/cloner_test.go index 04269bb2e1..b76869bf04 100644 --- a/persist/fs/clone/cloner_test.go +++ b/persist/fs/clone/cloner_test.go @@ -119,20 +119,7 @@ func TestCloner(t *testing.T) { b1.IncRef() b2.IncRef() require.Equal(t, t1.String(), t2.String()) - require.Equal(t, a1.Remaining(), a2.Remaining()) - numTags, numTagsMatched := a1.Remaining(), 0 - for a1.Next() && a2.Next() { - tag0 := a1.Current() - tag1 := a2.Current() - require.Equal(t, tag0.Name.String(), tag1.Name.String()) - require.Equal(t, tag0.Value.String(), tag1.Value.String()) - numTagsMatched++ - } - require.NoError(t, a1.Err()) - require.NoError(t, a2.Err()) - a1.Close() - a2.Close() - require.Equal(t, numTags, numTagsMatched) + require.True(t, ident.NewTagIterMatcher(a1).Matches(a2)) require.Equal(t, b1.Bytes(), b2.Bytes()) require.Equal(t, c1, c2) b1.DecRef() diff --git a/persist/fs/msgpack/roundtrip_test.go b/persist/fs/msgpack/roundtrip_test.go index e45accd0c9..3050af1858 100644 --- a/persist/fs/msgpack/roundtrip_test.go +++ b/persist/fs/msgpack/roundtrip_test.go @@ -214,6 +214,8 @@ func TestIndexEntryRoundTripForwardsCompatibilityV2(t *testing.T) { dec.Reset(NewDecoderStream(enc.Bytes())) res, err := dec.DecodeIndexEntry() require.NoError(t, err) + + expected require.Equal(t, testIndexEntry, res) } diff --git a/persist/fs/read.go b/persist/fs/read.go index 64b03ae4ea..91651349a5 100644 --- a/persist/fs/read.go +++ b/persist/fs/read.go @@ -355,8 +355,8 @@ func (r *reader) Read() (ident.ID, ident.TagIterator, checked.Bytes, uint32, err return nil, nil, nil, 0, errReadNotExpectedSize } - id := r.entryID(entry.ID) - tags := r.entryEncodedTags(entry.EncodedTags) + id := r.entryClonedID(entry.ID) + tags := r.entryClonedEncodedTagsTagIter(entry.EncodedTags) r.entriesRead++ return id, tags, data, uint32(entry.Checksum), nil @@ -369,8 +369,8 @@ func (r *reader) ReadMetadata() (id ident.ID, tags ident.TagIterator, length int } entry := r.indexEntriesByOffsetAsc[r.metadataRead] - id = r.entryID(entry.ID) - tags = r.entryEncodedTags(entry.EncodedTags) + id = r.entryClonedID(entry.ID) + tags = r.entryClonedEncodedTagsTagIter(entry.EncodedTags) length = int(entry.Size) checksum = uint32(entry.Checksum) @@ -388,7 +388,7 @@ func (r *reader) ReadBloomFilter() (*ManagedConcurrentBloomFilter, error) { ) } -func (r *reader) entryBytes(bytes []byte) checked.Bytes { +func (r *reader) entryClonedBytes(bytes []byte) checked.Bytes { var bytesClone checked.Bytes if r.bytesPool != nil { bytesClone = r.bytesPool.Get(len(bytes)) @@ -401,17 +401,17 @@ func (r *reader) entryBytes(bytes []byte) checked.Bytes { return bytesClone } -func (r *reader) entryID(id []byte) ident.ID { - return ident.BinaryID(r.entryBytes(id)) +func (r *reader) entryClonedID(id []byte) ident.ID { + return ident.BinaryID(r.entryClonedBytes(id)) } -func (r *reader) entryEncodedTags(encodedTags []byte) ident.TagIterator { +func (r *reader) entryClonedEncodedTagsTagIter(encodedTags []byte) ident.TagIterator { if len(encodedTags) == 0 { // No tags set for this entry, return an empty tag iterator return ident.EmptyTagIterator } decoder := r.tagDecoderPool.Get() - decoder.Reset(r.entryBytes(encodedTags)) + decoder.Reset(r.entryClonedBytes(encodedTags)) return decoder } diff --git a/persist/fs/retriever.go b/persist/fs/retriever.go index 453d71d25c..28c337afbd 100644 --- a/persist/fs/retriever.go +++ b/persist/fs/retriever.go @@ -539,7 +539,7 @@ func (req *retrieveRequest) onCallerOrRetrieverDone() { req.id = nil if req.tags != nil { req.tags.Close() - req.tags = nil + req.tags = ident.EmptyTagIterator } req.reader.Finalize() req.reader = nil diff --git a/services/m3dbnode/config/bootstrap.go b/services/m3dbnode/config/bootstrap.go index 4d267ae606..eb419de165 100644 --- a/services/m3dbnode/config/bootstrap.go +++ b/services/m3dbnode/config/bootstrap.go @@ -92,12 +92,11 @@ func (bsc BootstrapConfiguration) New( var ( bs bootstrap.BootstrapperProvider err error - iopts = opts.InstrumentOptions() rsopts = result.NewOptions(). SetInstrumentOptions(opts.InstrumentOptions()). SetDatabaseBlockOptions(opts.DatabaseBlockOptions()). SetSeriesCachePolicy(opts.SeriesCachePolicy()). - SetIndexMutableSegmentAllocator(index.NewDefaultMutableSegmentAllocator(iopts)) + SetIndexMutableSegmentAllocator(index.NewDefaultMutableSegmentAllocator(opts.IndexOptions())) ) // Start from the end of the list because the bootstrappers are ordered by precedence in descending order. for i := len(bsc.Bootstrappers) - 1; i >= 0; i-- { diff --git a/storage/bootstrap/result/options.go b/storage/bootstrap/result/options.go index 650a7c2cc6..e2c3675287 100644 --- a/storage/bootstrap/result/options.go +++ b/storage/bootstrap/result/options.go @@ -45,12 +45,13 @@ type options struct { func NewOptions() Options { iopts := instrument.NewOptions() return &options{ - clockOpts: clock.NewOptions(), - instrumentOpts: iopts, - blockOpts: block.NewOptions(), - newBlocksLen: defaultNewBlocksLen, - seriesCachePolicy: series.DefaultCachePolicy, - mutableSegmentAllocator: index.NewDefaultMutableSegmentAllocator(iopts), + clockOpts: clock.NewOptions(), + instrumentOpts: iopts, + blockOpts: block.NewOptions(), + newBlocksLen: defaultNewBlocksLen, + seriesCachePolicy: series.DefaultCachePolicy, + mutableSegmentAllocator: index.NewDefaultMutableSegmentAllocator(index.NewOptions(). + SetInstrumentOptions(iopts)), } } diff --git a/storage/index/allocator.go b/storage/index/allocator.go index dd4d606328..ade721b7ed 100644 --- a/storage/index/allocator.go +++ b/storage/index/allocator.go @@ -23,16 +23,14 @@ package index import ( "github.com/m3db/m3ninx/index/segment" "github.com/m3db/m3ninx/index/segment/mem" - "github.com/m3db/m3x/instrument" ) // NewDefaultMutableSegmentAllocator returns a default mutable segment // allocator. func NewDefaultMutableSegmentAllocator( - iopts instrument.Options, + opts Options, ) MutableSegmentAllocator { - opts := mem.NewOptions().SetInstrumentOptions(iopts) return func() (segment.MutableSegment, error) { - return mem.NewSegment(0, opts) + return mem.NewSegment(0, opts.MemSegmentOptions()) } } From ccc6305861cb6e77d4d63c0188d23e431d9703ad Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Mon, 7 May 2018 16:57:30 -0400 Subject: [PATCH 09/23] Refactor fs bootstrapper methods as per feedback --- storage/bootstrap/bootstrapper/fs/source.go | 105 +++++++++++++------- 1 file changed, 68 insertions(+), 37 deletions(-) diff --git a/storage/bootstrap/bootstrapper/fs/source.go b/storage/bootstrap/bootstrapper/fs/source.go index 07a4ef17df..396d4e1ff9 100644 --- a/storage/bootstrap/bootstrapper/fs/source.go +++ b/storage/bootstrap/bootstrapper/fs/source.go @@ -631,46 +631,36 @@ func (s *fileSystemSource) read( run runType, ) (*runResult, error) { var ( - nsID = md.ID() - blockRetriever block.DatabaseBlockRetriever + nsID = md.ID() + seriesCachePolicy = s.opts.ResultOptions().SeriesCachePolicy() + blockRetriever block.DatabaseBlockRetriever ) if shardsTimeRanges.IsEmpty() { return newRunResult(), nil } if run == bootstrapDataRunType { - // NB(r): We can only need to cache shard indices and possibly shortcut - // the reading of filesets if we are doing a data bootstrap, otherwise - // we need to look at the filesets. - blockRetrieverMgr := s.opts.DatabaseBlockRetrieverManager() - if blockRetrieverMgr != nil { - s.log.WithFields( - xlog.NewField("namespace", nsID.String()), - ).Infof("filesystem bootstrapper resolving block retriever") - - var err error - blockRetriever, err = blockRetrieverMgr.Retriever(md) - if err != nil { - return nil, err - } - - s.log.WithFields( - xlog.NewField("namespace", nsID.String()), - xlog.NewField("shards", len(shardsTimeRanges)), - ).Infof("filesystem bootstrapper caching block retriever shard indices") - + // NB(r): We only need to cache shard indices and marks blocks as + // fulfilled when bootstrapping data, because the data can be retrieved + // lazily from disk during reads. + // On the other hand, if we're bootstrapping the index then currently we + // need to rebuild it from scratch by reading all the IDs/tags until + // we can natively bootstrap persisted segments from disk and compact them + // with series metadata from other shards if topology has changed. + if mgr := s.opts.DatabaseBlockRetrieverManager(); mgr != nil { shards := make([]uint32, 0, len(shardsTimeRanges)) for shard := range shardsTimeRanges { shards = append(shards, shard) } - - err = blockRetriever.CacheShardIndices(shards) + var err error + blockRetriever, err = s.resolveBlockRetrieverAndCacheDataShardIndices(md, + mgr, shards) if err != nil { return nil, err } } - switch s.opts.ResultOptions().SeriesCachePolicy() { + switch seriesCachePolicy { case series.CacheAll: // No checks necessary case series.CacheAllMetadata: @@ -683,18 +673,8 @@ func (s *fileSystemSource) read( default: // Unless we're caching all series (or all series metadata) in memory, we // return just the availability of the files we have - runResult := newRunResult() - unfulfilled := runResult.data.Unfulfilled() - for shard, ranges := range shardsTimeRanges { - if ranges.IsEmpty() { - continue - } - availability := s.shardAvailability(md.ID(), shard, ranges) - remaining := ranges.RemoveRanges(availability) - runResult.data.Add(shard, nil, remaining) - } - runResult.data.SetUnfulfilled(unfulfilled) - return runResult, nil + result := s.bootstrapDataRunResultFromAvailability(md, shardsTimeRanges) + return result, nil } } @@ -716,6 +696,57 @@ func (s *fileSystemSource) read( return s.bootstrapFromReaders(md, run, readerPool, blockRetriever, readersCh), nil } +func (s *fileSystemSource) resolveBlockRetrieverAndCacheDataShardIndices( + md namespace.Metadata, + blockRetrieverMgr block.DatabaseBlockRetrieverManager, + shards []uint32, +) ( + block.DatabaseBlockRetriever, + error, +) { + var blockRetriever block.DatabaseBlockRetriever + + s.log.WithFields( + xlog.NewField("namespace", md.ID().String()), + ).Infof("filesystem bootstrapper resolving block retriever") + + var err error + blockRetriever, err = blockRetrieverMgr.Retriever(md) + if err != nil { + return nil, err + } + + s.log.WithFields( + xlog.NewField("namespace", md.ID().String()), + xlog.NewField("shards", len(shards)), + ).Infof("filesystem bootstrapper caching block retriever shard indices") + + err = blockRetriever.CacheShardIndices(shards) + if err != nil { + return nil, err + } + + return blockRetriever, nil +} + +func (s *fileSystemSource) bootstrapDataRunResultFromAvailability( + md namespace.Metadata, + shardsTimeRanges result.ShardTimeRanges, +) *runResult { + runResult := newRunResult() + unfulfilled := runResult.data.Unfulfilled() + for shard, ranges := range shardsTimeRanges { + if ranges.IsEmpty() { + continue + } + availability := s.shardAvailability(md.ID(), shard, ranges) + remaining := ranges.RemoveRanges(availability) + runResult.data.Add(shard, nil, remaining) + } + runResult.data.SetUnfulfilled(unfulfilled) + return runResult +} + type shardReaders struct { shard uint32 tr xtime.Ranges From dfc2e0133e856b3d4f7f73910ca20d916de0d39b Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 8 May 2018 00:24:43 -0400 Subject: [PATCH 10/23] Address further feedback --- persist/fs/msgpack/roundtrip_test.go | 2 - persist/fs/read.go | 15 +- persist/fs/read_write_test.go | 30 +--- storage/bootstrap/bootstrapper/fs/source.go | 51 +++--- .../{source_test.go => source_data_test.go} | 113 +++++++++---- .../bootstrapper/fs/source_index_test.go | 157 ++++++++++++++++++ storage/bootstrap/result/options.go | 14 +- storage/bootstrap/result/result_data_test.go | 4 +- storage/bootstrap/result/result_index.go | 1 + storage/bootstrap/result/result_index_test.go | 70 ++++++++ 10 files changed, 365 insertions(+), 92 deletions(-) rename storage/bootstrap/bootstrapper/fs/{source_test.go => source_data_test.go} (87%) create mode 100644 storage/bootstrap/bootstrapper/fs/source_index_test.go diff --git a/persist/fs/msgpack/roundtrip_test.go b/persist/fs/msgpack/roundtrip_test.go index 3050af1858..e45accd0c9 100644 --- a/persist/fs/msgpack/roundtrip_test.go +++ b/persist/fs/msgpack/roundtrip_test.go @@ -214,8 +214,6 @@ func TestIndexEntryRoundTripForwardsCompatibilityV2(t *testing.T) { dec.Reset(NewDecoderStream(enc.Bytes())) res, err := dec.DecodeIndexEntry() require.NoError(t, err) - - expected require.Equal(t, testIndexEntry, res) } diff --git a/persist/fs/read.go b/persist/fs/read.go index 91651349a5..0df5b7e90d 100644 --- a/persist/fs/read.go +++ b/persist/fs/read.go @@ -362,20 +362,19 @@ func (r *reader) Read() (ident.ID, ident.TagIterator, checked.Bytes, uint32, err return id, tags, data, uint32(entry.Checksum), nil } -func (r *reader) ReadMetadata() (id ident.ID, tags ident.TagIterator, length int, checksum uint32, err error) { +func (r *reader) ReadMetadata() (ident.ID, ident.TagIterator, int, uint32, error) { if r.metadataRead >= r.entries { - err = io.EOF - return + return nil, nil, 0, 0, io.EOF } entry := r.indexEntriesByOffsetAsc[r.metadataRead] - id = r.entryClonedID(entry.ID) - tags = r.entryClonedEncodedTagsTagIter(entry.EncodedTags) - length = int(entry.Size) - checksum = uint32(entry.Checksum) + id := r.entryClonedID(entry.ID) + tags := r.entryClonedEncodedTagsTagIter(entry.EncodedTags) + length := int(entry.Size) + checksum := uint32(entry.Checksum) r.metadataRead++ - return + return id, tags, length, checksum, nil } func (r *reader) ReadBloomFilter() (*ManagedConcurrentBloomFilter, error) { diff --git a/persist/fs/read_write_test.go b/persist/fs/read_write_test.go index 6f04dc9102..8d6e35c277 100644 --- a/persist/fs/read_write_test.go +++ b/persist/fs/read_write_test.go @@ -147,15 +147,8 @@ func readTestData(t *testing.T, r DataFileSetReader, shard uint32, timestamp tim assert.Equal(t, entries[i].id, id.String()) // Assert tags - if entries[i].tags != nil { - require.NotNil(t, tags) - require.Equal(t, len(entries[i].tags), tags.Remaining()) - for tags.Next() { - curr := tags.Current() - assert.Equal(t, entries[i].tags[curr.Name.String()], curr.Value.String()) - } - assert.Equal(t, 0, tags.Remaining()) - } + tagMatcher := ident.NewTagIterMatcher(ident.NewTagSliceIterator(entries[i].Tags())) + assert.True(t, tagMatcher.Matches(tags)) assert.True(t, bytes.Equal(entries[i].data, data.Bytes())) assert.Equal(t, digest.Checksum(entries[i].data), checksum) @@ -167,9 +160,7 @@ func readTestData(t *testing.T, r DataFileSetReader, shard uint32, timestamp tim assert.True(t, bloomFilter.Test(id.Data().Bytes())) id.Finalize() - if tags != nil { - tags.Close() - } + tags.Close() data.DecRef() data.Finalize() case readTestTypeMetadata: @@ -180,15 +171,8 @@ func readTestData(t *testing.T, r DataFileSetReader, shard uint32, timestamp tim assert.True(t, id.Equal(id)) // Assert tags - if entries[i].tags != nil { - require.NotNil(t, tags) - require.Equal(t, len(entries[i].tags), tags.Remaining()) - for tags.Next() { - curr := tags.Current() - assert.Equal(t, entries[i].tags[curr.Name.String()], curr.Value.String()) - } - assert.Equal(t, 0, tags.Remaining()) - } + tagMatcher := ident.NewTagIterMatcher(ident.NewTagSliceIterator(entries[i].Tags())) + assert.True(t, tagMatcher.Matches(tags)) assert.Equal(t, digest.Checksum(entries[i].data), checksum) assert.Equal(t, len(entries[i].data), length) @@ -200,9 +184,7 @@ func readTestData(t *testing.T, r DataFileSetReader, shard uint32, timestamp tim assert.True(t, bloomFilter.Test(id.Data().Bytes())) id.Finalize() - if tags != nil { - tags.Close() - } + tags.Close() } } diff --git a/storage/bootstrap/bootstrapper/fs/source.go b/storage/bootstrap/bootstrapper/fs/source.go index 396d4e1ff9..0065ff558c 100644 --- a/storage/bootstrap/bootstrapper/fs/source.go +++ b/storage/bootstrap/bootstrapper/fs/source.go @@ -127,7 +127,7 @@ func (s *fileSystemSource) ReadIndex( shardsTimeRanges result.ShardTimeRanges, opts bootstrap.RunOptions, ) (result.IndexBootstrapResult, error) { - r, err := s.read(md, shardsTimeRanges, bootstrapDataRunType) + r, err := s.read(md, shardsTimeRanges, bootstrapIndexRunType) if err != nil { return nil, err } @@ -266,7 +266,7 @@ func (s *fileSystemSource) bootstrapFromReaders( resultLock = &sync.RWMutex{} shardRetrieverMgr block.DatabaseShardBlockRetrieverManager runResult = newRunResult() - bopts = s.opts.ResultOptions() + resultOpts = s.opts.ResultOptions() ) if retriever != nil { @@ -277,9 +277,9 @@ func (s *fileSystemSource) bootstrapFromReaders( shardReaders := shardReaders wg.Add(1) s.processors.Go(func() { - defer wg.Done() s.loadShardReadersDataIntoShardResult(ns, run, runResult, resultLock, - bopts, shardRetrieverMgr, shardReaders, readerPool) + resultOpts, shardRetrieverMgr, shardReaders, readerPool) + wg.Done() }) } wg.Wait() @@ -303,7 +303,6 @@ func (s *fileSystemSource) handleErrorsAndUnfulfilled( runResult *runResult, shard uint32, remainingRanges xtime.Ranges, - shardResult result.ShardResult, timesWithErrors []time.Time, ) { // NB(xichen): this is the exceptional case where we encountered errors due to files @@ -319,6 +318,7 @@ func (s *fileSystemSource) handleErrorsAndUnfulfilled( ).Info("deleting entries from results for times with errors") resultLock.Lock() + // Delete all affected times from the data results. shardResult, ok := runResult.data.ShardResults()[shard] if ok { for _, entry := range shardResult.AllSeries().Iter() { @@ -328,20 +328,31 @@ func (s *fileSystemSource) handleErrorsAndUnfulfilled( } } } + // NB(r): We explicitly do not remove entries from the index results + // as they are additive and get merged together with results from other + // bootstrappers by just appending the result (unlike data bootstrap + // results that when merged replace the block with the current block). + // It would also be difficult to remove only series that was added to the + // index block as results from data files can be subsets of the index block + // and there's no way to definitively delete the entry we added as a result + // of just this data file failing. resultLock.Unlock() } if !remainingRanges.IsEmpty() { resultLock.Lock() - unfulfilled := runResult.data.Unfulfilled() - shardUnfulfilled, ok := unfulfilled[shard] - if !ok { - shardUnfulfilled = xtime.Ranges{}.AddRanges(remainingRanges) - } else { - shardUnfulfilled = shardUnfulfilled.AddRanges(remainingRanges) + for _, unfulfilled := range []result.ShardTimeRanges{ + runResult.data.Unfulfilled(), + runResult.index.Unfulfilled(), + } { + shardUnfulfilled, ok := unfulfilled[shard] + if !ok { + shardUnfulfilled = xtime.Ranges{}.AddRanges(remainingRanges) + } else { + shardUnfulfilled = shardUnfulfilled.AddRanges(remainingRanges) + } + unfulfilled[shard] = shardUnfulfilled } - - unfulfilled[shard] = shardUnfulfilled resultLock.Unlock() } } @@ -379,8 +390,8 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( shard, tr, readers, err := shardReaders.shard, shardReaders.tr, shardReaders.readers, shardReaders.err if err != nil { - s.handleErrorsAndUnfulfilled(resultLock, runResult, shard, tr, - shardResult, timesWithErrors) + s.handleErrorsAndUnfulfilled(resultLock, runResult, shard, + tr, timesWithErrors) return } @@ -389,8 +400,8 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( } if seriesCachePolicy == series.CacheAllMetadata && shardRetriever == nil { s.log.Errorf("shard retriever missing for shard: %d", shard) - s.handleErrorsAndUnfulfilled(resultLock, runResult, shard, tr, - shardResult, timesWithErrors) + s.handleErrorsAndUnfulfilled(resultLock, runResult, shard, + tr, timesWithErrors) return } @@ -449,7 +460,7 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( if err != nil { s.log.Errorf("unable read metadata: %v", err) hasError = true - continue + break } idBytes := id.Bytes() @@ -585,7 +596,7 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( resultLock.Unlock() } - if !hasError { + if run == bootstrapDataRunType && !hasError { var validateErr error switch seriesCachePolicy { case series.CacheAll: @@ -622,7 +633,7 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( } s.handleErrorsAndUnfulfilled(resultLock, runResult, shard, - tr, shardResult, timesWithErrors) + tr, timesWithErrors) } func (s *fileSystemSource) read( diff --git a/storage/bootstrap/bootstrapper/fs/source_test.go b/storage/bootstrap/bootstrapper/fs/source_data_test.go similarity index 87% rename from storage/bootstrap/bootstrapper/fs/source_test.go rename to storage/bootstrap/bootstrapper/fs/source_data_test.go index a42a204b64..a580997d8f 100644 --- a/storage/bootstrap/bootstrapper/fs/source_test.go +++ b/storage/bootstrap/bootstrapper/fs/source_data_test.go @@ -26,6 +26,7 @@ import ( "io/ioutil" "os" "path" + "sort" "testing" "time" @@ -51,6 +52,7 @@ var ( testNs1ID = ident.StringID("testNs") testStart = time.Now() testBlockSize = 2 * time.Hour + testIndexBlockSize = 4 * time.Hour testFileMode = os.FileMode(0666) testDirMode = os.ModeDir | os.FileMode(0755) testWriterBufferSize = 10 @@ -74,7 +76,11 @@ func newTestFsOptions(filePathPrefix string) fs.Options { func testNsMetadata(t *testing.T) namespace.Metadata { ropts := retention.NewOptions().SetBlockSize(testBlockSize) - md, err := namespace.NewMetadata(testNs1ID, namespace.NewOptions().SetRetentionOptions(ropts)) + md, err := namespace.NewMetadata(testNs1ID, namespace.NewOptions(). + SetRetentionOptions(ropts). + SetIndexOptions(namespace.NewIndexOptions(). + SetEnabled(true). + SetBlockSize(testIndexBlockSize))) require.NoError(t, err) return md } @@ -135,19 +141,23 @@ func writeGoodFiles(t *testing.T, dir string, namespace ident.ID, shard uint32) for _, input := range inputs { writeTSDBFiles(t, dir, namespace, shard, input.start, - input.id, input.tags, input.data) + []testSeries{{input.id, input.tags, input.data}}) } } +type testSeries struct { + id string + tags map[string]string + data []byte +} + func writeTSDBFiles( t *testing.T, dir string, namespace ident.ID, shard uint32, start time.Time, - id string, - tags map[string]string, - data []byte, + series []testSeries, ) { w, err := fs.NewWriter(newTestFsOptions(dir)) require.NoError(t, err) @@ -161,19 +171,32 @@ func writeTSDBFiles( } require.NoError(t, w.Open(writerOpts)) - var seriesTags ident.Tags - for name, value := range tags { - seriesTags = append(seriesTags, ident.StringTag(name, value)) + for _, v := range series { + bytes := checked.NewBytes(v.data, nil) + bytes.IncRef() + require.NoError(t, w.Write(ident.StringID(v.id), + sortedTagsFromTagsMap(v.tags), bytes, digest.Checksum(bytes.Bytes()))) + bytes.DecRef() } - bytes := checked.NewBytes(data, nil) - bytes.IncRef() - - require.NoError(t, w.Write(ident.StringID(id), seriesTags, - bytes, digest.Checksum(bytes.Bytes()))) require.NoError(t, w.Close()) } +func sortedTagsFromTagsMap(tags map[string]string) ident.Tags { + var ( + seriesTags ident.Tags + tagNames []string + ) + for name := range tags { + tagNames = append(tagNames, name) + } + sort.Strings(tagNames) + for _, name := range tagNames { + seriesTags = append(seriesTags, ident.StringTag(name, tags[name])) + } + return seriesTags +} + func rangesArray(ranges xtime.Ranges) []xtime.Range { var array []xtime.Range iter := ranges.Iter() @@ -212,7 +235,9 @@ func TestAvailableReadInfoError(t *testing.T) { defer os.RemoveAll(dir) shard := uint32(0) - writeTSDBFiles(t, dir, testNs1ID, shard, testStart, "foo", nil, []byte{0x1}) + writeTSDBFiles(t, dir, testNs1ID, shard, testStart, []testSeries{ + {"foo", nil, []byte{0x1}}, + }) // Intentionally corrupt the info file writeInfoFile(t, dir, testNs1ID, shard, testStart, []byte{0x1, 0x2}) @@ -227,7 +252,9 @@ func TestAvailableDigestOfDigestMismatch(t *testing.T) { defer os.RemoveAll(dir) shard := uint32(0) - writeTSDBFiles(t, dir, testNs1ID, shard, testStart, "foo", nil, []byte{0x1}) + writeTSDBFiles(t, dir, testNs1ID, shard, testStart, []testSeries{ + {"foo", nil, []byte{0x1}}, + }) // Intentionally corrupt the digest file writeDigestFile(t, dir, testNs1ID, shard, testStart, nil) @@ -317,7 +344,9 @@ func TestReadOpenFileError(t *testing.T) { defer os.RemoveAll(dir) shard := uint32(0) - writeTSDBFiles(t, dir, testNs1ID, shard, testStart, "foo", nil, []byte{0x1}) + writeTSDBFiles(t, dir, testNs1ID, shard, testStart, []testSeries{ + {"foo", nil, []byte{0x1}}, + }) // Intentionally truncate the info file writeInfoFile(t, dir, testNs1ID, shard, testStart, nil) @@ -340,7 +369,9 @@ func TestReadDataCorruptionError(t *testing.T) { defer os.RemoveAll(dir) shard := uint32(0) - writeTSDBFiles(t, dir, testNs1ID, shard, testStart, "foo", nil, []byte{0x1}) + writeTSDBFiles(t, dir, testNs1ID, shard, testStart, []testSeries{ + {"foo", nil, []byte{0x1}}, + }) // Intentionally corrupt the data file writeDataFile(t, dir, testNs1ID, shard, testStart, []byte{0x2}) @@ -443,8 +474,9 @@ func TestReadValidateError(t *testing.T) { } shard := uint32(0) - writeTSDBFiles(t, dir, testNs1ID, shard, testStart, - "foo", nil, []byte{0x1}) + writeTSDBFiles(t, dir, testNs1ID, shard, testStart, []testSeries{ + {"foo", nil, []byte{0x1}}, + }) rOpenOpts := fs.ReaderOpenOptionsMatcher{ ID: fs.FileSetFileIdentifier{ Namespace: testNs1ID, @@ -497,8 +529,9 @@ func TestReadOpenError(t *testing.T) { } shard := uint32(0) - writeTSDBFiles(t, dir, testNs1ID, shard, testStart, - "foo", nil, []byte{0x1}) + writeTSDBFiles(t, dir, testNs1ID, shard, testStart, []testSeries{ + {"foo", nil, []byte{0x1}}, + }) rOpts := fs.ReaderOpenOptionsMatcher{ ID: fs.FileSetFileIdentifier{ Namespace: testNs1ID, @@ -541,8 +574,9 @@ func TestReadDeleteOnError(t *testing.T) { } shard := uint32(0) - writeTSDBFiles(t, dir, testNs1ID, shard, testStart, - "foo", nil, []byte{0x1}) + writeTSDBFiles(t, dir, testNs1ID, shard, testStart, []testSeries{ + {"foo", nil, []byte{0x1}}, + }) rOpts := fs.ReaderOpenOptionsMatcher{ ID: fs.FileSetFileIdentifier{ @@ -591,10 +625,33 @@ func TestReadDeleteOnError(t *testing.T) { } func TestReadTags(t *testing.T) { - // TODO(r): Add test that tests the normal bootstrap case with - // series cache policy All returns tags for series -} + dir := createTempDir(t) + defer os.RemoveAll(dir) + + id := "foo" + tags := map[string]string{ + "bar": "baz", + "qux": "qaz", + } + data := []byte{0x1} + + writeTSDBFiles(t, dir, testNs1ID, testShard, testStart, []testSeries{ + {id, tags, data}, + }) + + src := newFileSystemSource(newTestOptions(dir)) + res, err := src.ReadData(testNsMetadata(t), testShardTimeRanges(), + testDefaultRunOpts) + require.NoError(t, err) + + require.Equal(t, 1, len(res.ShardResults())) + require.NotNil(t, res.ShardResults()[testShard]) + + series := res.ShardResults()[testShard] + require.Equal(t, int64(1), series.NumSeries()) -func TestBootstrapIndex(t *testing.T) { - // TODO(r): Add test to bootstrap index segments + fooSeries, ok := series.AllSeries().Get(ident.StringID(id)) + require.True(t, ok) + require.True(t, fooSeries.ID.Equal(ident.StringID(id))) + require.True(t, fooSeries.Tags.Equal(sortedTagsFromTagsMap(tags))) } diff --git a/storage/bootstrap/bootstrapper/fs/source_index_test.go b/storage/bootstrap/bootstrapper/fs/source_index_test.go new file mode 100644 index 0000000000..9612b43b8e --- /dev/null +++ b/storage/bootstrap/bootstrapper/fs/source_index_test.go @@ -0,0 +1,157 @@ +// Copyright (c) 2018 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 fs + +import ( + "os" + "testing" + "time" + + xtime "github.com/m3db/m3x/time" + "github.com/stretchr/testify/require" +) + +func TestBootstrapIndex(t *testing.T) { + dir := createTempDir(t) + defer os.RemoveAll(dir) + + fooSeries := struct { + id string + tags map[string]string + }{ + "foo", + map[string]string{"aaa": "bbb", "ccc": "ddd"}, + } + dataBlocks := [][]testSeries{ + []testSeries{ + {fooSeries.id, fooSeries.tags, []byte{0x1}}, + {"bar", map[string]string{"eee": "fff", "ggg": "hhh"}, []byte{0x1}}, + {"baz", map[string]string{"iii": "jjj", "kkk": "lll"}, []byte{0x1}}, + }, + []testSeries{ + {fooSeries.id, fooSeries.tags, []byte{0x1}}, + {"qux", map[string]string{"mmm": "nnn", "ooo": "ppp"}, []byte{0x1}}, + {"qaz", map[string]string{"qqq": "rrr", "sss": "ttt"}, []byte{0x1}}, + }, + []testSeries{ + {fooSeries.id, fooSeries.tags, []byte{0x1}}, + {"qan", map[string]string{"uuu": "vvv", "www": "xxx"}, []byte{0x1}}, + {"qam", map[string]string{"yyy": "zzz", "000": "111"}, []byte{0x1}}, + }, + } + + at := time.Now() + start := at.Truncate(testBlockSize) + indexStart := start.Truncate(testIndexBlockSize) + for !start.Equal(indexStart) { + // make sure data blocks overlap, test block size is 2h + // and test index block size is 4h + start = start.Add(testBlockSize) + indexStart = start.Truncate(testIndexBlockSize) + } + + writeTSDBFiles(t, dir, testNs1ID, testShard, + start, dataBlocks[0]) + writeTSDBFiles(t, dir, testNs1ID, testShard, + start.Add(testBlockSize), dataBlocks[1]) + writeTSDBFiles(t, dir, testNs1ID, testShard, + start.Add(2*testBlockSize), dataBlocks[2]) + + shardTimeRanges := map[uint32]xtime.Ranges{ + testShard: xtime.Ranges{}.AddRange(xtime.Range{ + Start: indexStart, + End: indexStart.Add(2 * testIndexBlockSize), + }), + } + + src := newFileSystemSource(newTestOptions(dir)) + res, err := src.ReadIndex(testNsMetadata(t), shardTimeRanges, + testDefaultRunOpts) + require.NoError(t, err) + + indexResults := res.IndexResults() + require.Equal(t, 2, len(indexResults)) + + for _, expected := range []struct { + indexBlockStart time.Time + series map[string]testSeries + }{ + { + indexBlockStart: indexStart, + series: map[string]testSeries{ + dataBlocks[0][0].id: dataBlocks[0][0], + dataBlocks[0][1].id: dataBlocks[0][1], + dataBlocks[0][2].id: dataBlocks[0][2], + dataBlocks[1][1].id: dataBlocks[1][1], + dataBlocks[1][2].id: dataBlocks[1][2], + }, + }, + { + indexBlockStart: indexStart.Add(testIndexBlockSize), + series: map[string]testSeries{ + dataBlocks[2][0].id: dataBlocks[2][0], + dataBlocks[2][1].id: dataBlocks[2][1], + dataBlocks[2][2].id: dataBlocks[2][2], + }, + }, + } { + expectedAt := xtime.ToUnixNano(expected.indexBlockStart) + indexBlock, ok := indexResults[expectedAt] + require.True(t, ok) + require.Equal(t, 1, len(indexBlock.Segments())) + for _, seg := range indexBlock.Segments() { + reader, err := seg.Reader() + require.NoError(t, err) + + docs, err := reader.AllDocs() + require.NoError(t, err) + + matches := map[string]struct{}{} + for docs.Next() { + curr := docs.Current() + + _, ok := matches[string(curr.ID)] + require.False(t, ok) + matches[string(curr.ID)] = struct{}{} + + series, ok := expected.series[string(curr.ID)] + require.True(t, ok) + + matchingTags := map[string]struct{}{} + for _, tag := range curr.Fields { + _, ok := matchingTags[string(tag.Name)] + require.False(t, ok) + matchingTags[string(tag.Name)] = struct{}{} + + tagValue, ok := series.tags[string(tag.Name)] + require.True(t, ok) + + require.Equal(t, tagValue, string(tag.Value)) + } + require.Equal(t, len(series.tags), len(matchingTags)) + } + require.NoError(t, docs.Err()) + require.NoError(t, docs.Close()) + + require.Equal(t, len(expected.series), len(matches)) + } + } +} diff --git a/storage/bootstrap/result/options.go b/storage/bootstrap/result/options.go index e2c3675287..ef4bc5191b 100644 --- a/storage/bootstrap/result/options.go +++ b/storage/bootstrap/result/options.go @@ -43,15 +43,13 @@ type options struct { // NewOptions creates new bootstrap options func NewOptions() Options { - iopts := instrument.NewOptions() return &options{ - clockOpts: clock.NewOptions(), - instrumentOpts: iopts, - blockOpts: block.NewOptions(), - newBlocksLen: defaultNewBlocksLen, - seriesCachePolicy: series.DefaultCachePolicy, - mutableSegmentAllocator: index.NewDefaultMutableSegmentAllocator(index.NewOptions(). - SetInstrumentOptions(iopts)), + clockOpts: clock.NewOptions(), + instrumentOpts: instrument.NewOptions(), + blockOpts: block.NewOptions(), + newBlocksLen: defaultNewBlocksLen, + seriesCachePolicy: series.DefaultCachePolicy, + mutableSegmentAllocator: index.NewDefaultMutableSegmentAllocator(index.NewOptions()), } } diff --git a/storage/bootstrap/result/result_data_test.go b/storage/bootstrap/result/result_data_test.go index 3ea0b6d5c0..b171c92325 100644 --- a/storage/bootstrap/result/result_data_test.go +++ b/storage/bootstrap/result/result_data_test.go @@ -40,7 +40,7 @@ func testResultOptions() Options { return NewOptions() } -func TestResultAddMergesExistingShardResults(t *testing.T) { +func TestDataResultAddMergesExistingShardResults(t *testing.T) { opts := testResultOptions() blopts := opts.DatabaseBlockOptions() @@ -79,7 +79,7 @@ func TestResultAddMergesExistingShardResults(t *testing.T) { assert.True(t, r.ShardResults().Equal(merged.ShardResults())) } -func TestResultAddMergesUnfulfilled(t *testing.T) { +func TestDataResultAddMergesUnfulfilled(t *testing.T) { start := time.Now().Truncate(testBlockSize) r := NewDataBootstrapResult() diff --git a/storage/bootstrap/result/result_index.go b/storage/bootstrap/result/result_index.go index 657dfe11f3..ce667ed591 100644 --- a/storage/bootstrap/result/result_index.go +++ b/storage/bootstrap/result/result_index.go @@ -87,6 +87,7 @@ func (r IndexResults) GetOrAddSegment( ) (segment.MutableSegment, error) { blockStart := t.Truncate(idxopts.BlockSize()) blockStartNanos := xtime.ToUnixNano(blockStart) + block, exists := r[blockStartNanos] if !exists { block = NewIndexBlock(blockStart, nil) diff --git a/storage/bootstrap/result/result_index_test.go b/storage/bootstrap/result/result_index_test.go index bddd47cc54..0032bda3af 100644 --- a/storage/bootstrap/result/result_index_test.go +++ b/storage/bootstrap/result/result_index_test.go @@ -19,3 +19,73 @@ // THE SOFTWARE. package result + +import ( + "fmt" + "testing" + "time" + + "github.com/golang/mock/gomock" + "github.com/m3db/m3ninx/index/segment" + + "github.com/stretchr/testify/assert" +) + +func TestIndexResultAddMergesExistingSegments(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + start := time.Now().Truncate(testBlockSize) + + segments := []segment.Segment{ + segment.NewMockSegment(ctrl), + segment.NewMockSegment(ctrl), + segment.NewMockSegment(ctrl), + segment.NewMockSegment(ctrl), + segment.NewMockSegment(ctrl), + segment.NewMockSegment(ctrl), + } + + times := []time.Time{start, start.Add(testBlockSize)} + + r := NewIndexBootstrapResult() + r.Add(NewIndexBlock(times[0], []segment.Segment{segments[0]}), nil) + r.Add(NewIndexBlock(times[0], []segment.Segment{segments[1]}), nil) + r.Add(NewIndexBlock(times[1], []segment.Segment{segments[2], segments[3]}), nil) + + merged := NewIndexBootstrapResult() + merged.Add(NewIndexBlock(times[0], []segment.Segment{segments[4]}), nil) + merged.Add(NewIndexBlock(times[1], []segment.Segment{segments[5]}), nil) + + expected := NewIndexBootstrapResult() + expected.Add(NewIndexBlock(times[0], []segment.Segment{segments[0], segments[1], segments[4]}), nil) + expected.Add(NewIndexBlock(times[1], []segment.Segment{segments[2], segments[3], segments[5]}), nil) + + assert.True(t, segmentsInResultsSame(expected.IndexResults(), merged.IndexResults())) +} + +func segmentsInResultsSame(a, b IndexResults) bool { + if len(a) != len(b) { + // `` + fmt.Printf("not same len\n") + return false + } + for t, block := range a { + otherBlock, ok := b[t] + if !ok { + fmt.Printf("no block at time %v\n", time.Unix(0, int64(t)).String()) + return false + } + if len(block.Segments()) != len(otherBlock.Segments()) { + fmt.Printf("block segment len not match at %v\n", time.Unix(0, int64(t)).String()) + return false + } + for i, s := range block.Segments() { + if s != otherBlock.Segments()[i] { + fmt.Printf("block segment not match at %v idx %d\n", time.Unix(0, int64(t)).String(), i) + return false + } + } + } + return true +} From 63d92ff65fc9b666c30f1b810f65e8b52dc3faaf Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 8 May 2018 00:28:46 -0400 Subject: [PATCH 11/23] Use FromMetricIter to convert document in bootstrapping path --- storage/bootstrap/bootstrapper/fs/source.go | 15 +++------------ 1 file changed, 3 insertions(+), 12 deletions(-) diff --git a/storage/bootstrap/bootstrapper/fs/source.go b/storage/bootstrap/bootstrapper/fs/source.go index 0065ff558c..8113ba5d50 100644 --- a/storage/bootstrap/bootstrapper/fs/source.go +++ b/storage/bootstrap/bootstrapper/fs/source.go @@ -29,6 +29,7 @@ import ( "github.com/m3db/m3db/storage/block" "github.com/m3db/m3db/storage/bootstrap" "github.com/m3db/m3db/storage/bootstrap/result" + "github.com/m3db/m3db/storage/index/convert" "github.com/m3db/m3db/storage/namespace" "github.com/m3db/m3db/storage/series" "github.com/m3db/m3db/ts" @@ -473,18 +474,8 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( resultLock.Lock() exists, err = indexBlockSegment.ContainsID(idBytes) if err == nil && !exists { - d := doc.Document{ - ID: append([]byte(nil), idBytes...), - Fields: make(doc.Fields, 0, tagsIter.Remaining()), - } - for tagsIter.Next() { - curr := tagsIter.Current() - d.Fields = append(d.Fields, doc.Field{ - Name: append([]byte(nil), curr.Name.Bytes()...), - Value: append([]byte(nil), curr.Value.Bytes()...), - }) - } - err = tagsIter.Err() + var d doc.Document + d, err = convert.FromMetricIter(id, tagsIter) if err == nil { _, err = indexBlockSegment.Insert(d) } From c580dbe98237f9cf2a18023e2941d217b5171988 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 8 May 2018 02:17:59 -0400 Subject: [PATCH 12/23] Address further feedback and split inner fs bootstrapper methods --- storage/bootstrap/bootstrapper/fs/source.go | 462 +++++++++--------- storage/bootstrap/result/result_index_test.go | 66 ++- 2 files changed, 293 insertions(+), 235 deletions(-) diff --git a/storage/bootstrap/bootstrapper/fs/source.go b/storage/bootstrap/bootstrapper/fs/source.go index 8113ba5d50..12216c5a32 100644 --- a/storage/bootstrap/bootstrapper/fs/source.go +++ b/storage/bootstrap/bootstrapper/fs/source.go @@ -33,7 +33,6 @@ import ( "github.com/m3db/m3db/storage/namespace" "github.com/m3db/m3db/storage/series" "github.com/m3db/m3db/ts" - "github.com/m3db/m3ninx/doc" "github.com/m3db/m3ninx/index/segment" "github.com/m3db/m3x/checked" "github.com/m3db/m3x/ident" @@ -43,11 +42,6 @@ import ( xtime "github.com/m3db/m3x/time" ) -type newDataFileSetReaderFn func( - bytesPool pool.CheckedBytesPool, - opts fs.Options, -) (fs.DataFileSetReader, error) - type runType int const ( @@ -55,17 +49,10 @@ const ( bootstrapIndexRunType ) -type runResult struct { - data result.DataBootstrapResult - index result.IndexBootstrapResult -} - -func newRunResult() *runResult { - return &runResult{ - data: result.NewDataBootstrapResult(), - index: result.NewIndexBootstrapResult(), - } -} +type newDataFileSetReaderFn func( + bytesPool pool.CheckedBytesPool, + opts fs.Options, +) (fs.DataFileSetReader, error) type fileSystemSource struct { opts Options @@ -263,13 +250,11 @@ func (s *fileSystemSource) bootstrapFromReaders( readersCh <-chan shardReaders, ) *runResult { var ( - wg sync.WaitGroup - resultLock = &sync.RWMutex{} - shardRetrieverMgr block.DatabaseShardBlockRetrieverManager runResult = newRunResult() resultOpts = s.opts.ResultOptions() + shardRetrieverMgr block.DatabaseShardBlockRetrieverManager + wg sync.WaitGroup ) - if retriever != nil { shardRetrieverMgr = block.NewDatabaseShardBlockRetrieverManager(retriever) } @@ -278,7 +263,7 @@ func (s *fileSystemSource) bootstrapFromReaders( shardReaders := shardReaders wg.Add(1) s.processors.Go(func() { - s.loadShardReadersDataIntoShardResult(ns, run, runResult, resultLock, + s.loadShardReadersDataIntoShardResult(ns, run, runResult, resultOpts, shardRetrieverMgr, shardReaders, readerPool) wg.Done() }) @@ -300,7 +285,6 @@ func (s *fileSystemSource) bootstrapFromReaders( // it looks at any remaining (unfulfilled) ranges and makes sure they're marked // as unfulfilled func (s *fileSystemSource) handleErrorsAndUnfulfilled( - resultLock *sync.RWMutex, runResult *runResult, shard uint32, remainingRanges xtime.Ranges, @@ -313,12 +297,16 @@ func (s *fileSystemSource) handleErrorsAndUnfulfilled( // the current implementation saves the extra overhead of merging temporary map with the // final result. if len(timesWithErrors) > 0 { + timesWithErrorsString := make([]string, len(timesWithErrors)) + for i := range timesWithErrors { + timesWithErrorsString[i] = timesWithErrors[i].String() + } s.log.WithFields( xlog.NewField("shard", shard), - xlog.NewField("timesWithErrors", timesWithErrors), + xlog.NewField("timesWithErrors", timesWithErrorsString), ).Info("deleting entries from results for times with errors") - resultLock.Lock() + runResult.Lock() // Delete all affected times from the data results. shardResult, ok := runResult.data.ShardResults()[shard] if ok { @@ -337,11 +325,11 @@ func (s *fileSystemSource) handleErrorsAndUnfulfilled( // index block as results from data files can be subsets of the index block // and there's no way to definitively delete the entry we added as a result // of just this data file failing. - resultLock.Unlock() + runResult.Unlock() } if !remainingRanges.IsEmpty() { - resultLock.Lock() + runResult.Lock() for _, unfulfilled := range []result.ShardTimeRanges{ runResult.data.Unfulfilled(), runResult.index.Unfulfilled(), @@ -354,7 +342,7 @@ func (s *fileSystemSource) handleErrorsAndUnfulfilled( } unfulfilled[shard] = shardUnfulfilled } - resultLock.Unlock() + runResult.Unlock() } } @@ -373,8 +361,7 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( ns namespace.Metadata, run runType, runResult *runResult, - resultLock *sync.RWMutex, - bopts result.Options, + ropts result.Options, shardRetrieverMgr block.DatabaseShardBlockRetrieverManager, shardReaders shardReaders, readerPool *readerPool, @@ -382,17 +369,16 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( var ( timesWithErrors []time.Time shardResult result.ShardResult - shardSeries *result.Map shardRetriever block.DatabaseShardBlockRetriever - blockPool = bopts.DatabaseBlockOptions().DatabaseBlockPool() - seriesCachePolicy = bopts.SeriesCachePolicy() + blockPool = ropts.DatabaseBlockOptions().DatabaseBlockPool() + seriesCachePolicy = ropts.SeriesCachePolicy() indexBlockSegment segment.MutableSegment ) - shard, tr, readers, err := shardReaders.shard, shardReaders.tr, shardReaders.readers, shardReaders.err + sr := shardReaders + shard, tr, readers, err := sr.shard, sr.tr, sr.readers, sr.err if err != nil { - s.handleErrorsAndUnfulfilled(resultLock, runResult, shard, - tr, timesWithErrors) + s.handleErrorsAndUnfulfilled(runResult, shard, tr, timesWithErrors) return } @@ -401,8 +387,7 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( } if seriesCachePolicy == series.CacheAllMetadata && shardRetriever == nil { s.log.Errorf("shard retriever missing for shard: %d", shard) - s.handleErrorsAndUnfulfilled(resultLock, runResult, shard, - tr, timesWithErrors) + s.handleErrorsAndUnfulfilled(runResult, shard, tr, timesWithErrors) return } @@ -410,209 +395,61 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( var ( timeRange = r.Range() start = timeRange.Start - hasError bool + err error ) switch run { case bootstrapDataRunType: - if shardResult == nil { - resultLock.RLock() - results := runResult.data.ShardResults() - var exists bool - shardResult, exists = results[shard] - resultLock.RUnlock() - - if !exists { - resultLock.Lock() - shardResult, exists = results[shard] - if !exists { - // NB(r): Wait until we have a reader to initialize the shard result - // to be able to somewhat estimate the size of it. - shardResult = result.NewShardResult(r.Entries(), bopts) - results[shard] = shardResult - } - resultLock.Unlock() - } - - shardSeries = shardResult.AllSeries() - } + capacity := r.Entries() + shardResult = runResult.getOrAddDataShardResult(shard, capacity, ropts) case bootstrapIndexRunType: - resultLock.Lock() - results := runResult.index.IndexResults() - indexBlockSegment, err = results.GetOrAddSegment(start, - ns.Options().IndexOptions(), bopts) - resultLock.Unlock() - if err != nil { - s.log.Errorf("unable to create index segment: %v", err) - hasError = true - } + indexBlockSegment, err = runResult.getOrAddIndexSegment(start, ns, ropts) default: // Unreachable unless an internal method calls with a run type casted from int panic(fmt.Errorf("invalid run type: %d", run)) } - var numEntries int - if !hasError { - numEntries = r.Entries() - } - for i := 0; i < numEntries; i++ { - if run == bootstrapIndexRunType { - // If performing index run, then simply read the metadata and add to segment - id, tagsIter, _, _, err := r.ReadMetadata() - if err != nil { - s.log.Errorf("unable read metadata: %v", err) - hasError = true - break - } - - idBytes := id.Bytes() - - resultLock.RLock() - exists, err := indexBlockSegment.ContainsID(idBytes) - resultLock.RUnlock() - - if err == nil && !exists { - resultLock.Lock() - exists, err = indexBlockSegment.ContainsID(idBytes) - if err == nil && !exists { - var d doc.Document - d, err = convert.FromMetricIter(id, tagsIter) - if err == nil { - _, err = indexBlockSegment.Insert(d) - } - } - resultLock.Unlock() - } - - // Finalize the ID and tags - id.Finalize() - tagsIter.Close() - - if err != nil { - s.log.Errorf("unable to add doc to segment: %v", err) - hasError = true - } - continue - } - - var ( - seriesBlock = blockPool.Get() - id ident.ID - tagsIter ident.TagIterator - data checked.Bytes - length int - checksum uint32 - err error - ) - switch seriesCachePolicy { - case series.CacheAll: - id, tagsIter, data, checksum, err = r.Read() - case series.CacheAllMetadata: - id, tagsIter, length, checksum, err = r.ReadMetadata() - default: - s.log.WithFields( - xlog.NewField("shard", shard), - xlog.NewField("seriesCachePolicy", seriesCachePolicy.String()), - ).Error("invalid series cache policy: expected CacheAll or CacheAllMetadata") - hasError = true - } - if hasError { - break - } - - if err != nil { - s.log.WithFields( - xlog.NewField("shard", shard), - xlog.NewField("error", err.Error()), - ).Error("reading data file failed") - hasError = true - break - } - - var ( - entry result.DatabaseSeriesBlocks - tags ident.Tags - exists bool - ) - resultLock.RLock() - entry, exists = shardSeries.Get(id) - resultLock.RUnlock() - - if exists { - // NB(r): In the case the series is already inserted - // we can avoid holding onto this ID and use the already - // allocated ID. - id.Finalize() - id = entry.ID - tags = entry.Tags - } else { - tags, err = s.tagsFromTagsIter(tagsIter) - if err != nil { - s.log.Errorf("unable to decode tags: %v", err) - hasError = true - } - } - tagsIter.Close() - if hasError { - break - } - - switch seriesCachePolicy { - case series.CacheAll: - seg := ts.NewSegment(data, nil, ts.FinalizeHead) - seriesBlock.Reset(start, seg) - case series.CacheAllMetadata: - metadata := block.RetrievableBlockMetadata{ - ID: id, - Length: length, - Checksum: checksum, - } - seriesBlock.ResetRetrievable(start, shardRetriever, metadata) + numEntries := r.Entries() + for i := 0; err == nil && i < numEntries; i++ { + switch run { + case bootstrapDataRunType: + err = s.readNextEntryAndRecordBlock(r, runResult, start, shardResult, + shardRetriever, blockPool, seriesCachePolicy) + case bootstrapIndexRunType: + // We can just read the entry and index if performing an index run + err = s.readNextEntryAndIndex(r, runResult, indexBlockSegment) default: - s.log.WithFields( - xlog.NewField("shard", shard), - xlog.NewField("seriesCachePolicy", seriesCachePolicy.String()), - ).Error("invalid series cache policy: expected CacheAll or CacheAllMetadata") - hasError = true - } - if hasError { - break - } - - resultLock.Lock() - if exists { - entry.Blocks.AddBlock(seriesBlock) - } else { - shardResult.AddBlock(id, tags, seriesBlock) + // Unreachable unless an internal method calls with a run type casted from int + panic(fmt.Errorf("invalid run type: %d", run)) } - resultLock.Unlock() } - if run == bootstrapDataRunType && !hasError { + if err == nil { var validateErr error - switch seriesCachePolicy { - case series.CacheAll: - validateErr = r.Validate() - case series.CacheAllMetadata: + switch run { + case bootstrapDataRunType: + switch seriesCachePolicy { + case series.CacheAll: + validateErr = r.Validate() + case series.CacheAllMetadata: + validateErr = r.ValidateMetadata() + default: + err = fmt.Errorf("invalid series cache policy: %s", seriesCachePolicy.String()) + } + case bootstrapIndexRunType: validateErr = r.ValidateMetadata() default: - s.log.WithFields( - xlog.NewField("shard", shard), - xlog.NewField("seriesCachePolicy", seriesCachePolicy.String()), - ).Error("invalid series cache policy: expected CacheAll or CacheAllMetadata") - hasError = true + // Unreachable unless an internal method calls with a run type casted from int + panic(fmt.Errorf("invalid run type: %d", run)) } if validateErr != nil { - s.log.WithFields( - xlog.NewField("shard", shard), - xlog.NewField("error", validateErr.Error()), - ).Error("data validation failed") - hasError = true + err = fmt.Errorf("data validation failed: %v", validateErr) } } - if !hasError { + if err == nil { tr = tr.RemoveRange(timeRange) } else { + s.log.Errorf("%v", err) timesWithErrors = append(timesWithErrors, timeRange.Start) } } @@ -623,8 +460,142 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( } } - s.handleErrorsAndUnfulfilled(resultLock, runResult, shard, - tr, timesWithErrors) + s.handleErrorsAndUnfulfilled(runResult, shard, tr, timesWithErrors) +} + +func (s *fileSystemSource) readNextEntryAndRecordBlock( + r fs.DataFileSetReader, + runResult *runResult, + blockStart time.Time, + shardResult result.ShardResult, + shardRetriever block.DatabaseShardBlockRetriever, + blockPool block.DatabaseBlockPool, + seriesCachePolicy series.CachePolicy, +) error { + var ( + seriesBlock = blockPool.Get() + id ident.ID + tagsIter ident.TagIterator + data checked.Bytes + length int + checksum uint32 + err error + ) + switch seriesCachePolicy { + case series.CacheAll: + id, tagsIter, data, checksum, err = r.Read() + case series.CacheAllMetadata: + id, tagsIter, length, checksum, err = r.ReadMetadata() + default: + err = fmt.Errorf("invalid series cache policy: %s", seriesCachePolicy.String()) + } + if err != nil { + return fmt.Errorf("error reading data file: %v", err) + } + + var ( + entry result.DatabaseSeriesBlocks + tags ident.Tags + exists bool + ) + runResult.RLock() + entry, exists = shardResult.AllSeries().Get(id) + runResult.RUnlock() + + if exists { + // NB(r): In the case the series is already inserted + // we can avoid holding onto this ID and use the already + // allocated ID. + id.Finalize() + id = entry.ID + tags = entry.Tags + } else { + tags, err = s.tagsFromTagsIter(tagsIter) + if err != nil { + return fmt.Errorf("unable to decode tags: %v", err) + } + } + tagsIter.Close() + + switch seriesCachePolicy { + case series.CacheAll: + seg := ts.NewSegment(data, nil, ts.FinalizeHead) + seriesBlock.Reset(blockStart, seg) + case series.CacheAllMetadata: + metadata := block.RetrievableBlockMetadata{ + ID: id, + Length: length, + Checksum: checksum, + } + seriesBlock.ResetRetrievable(blockStart, shardRetriever, metadata) + default: + return fmt.Errorf("invalid series cache policy: %s", seriesCachePolicy.String()) + } + + runResult.Lock() + if exists { + entry.Blocks.AddBlock(seriesBlock) + } else { + shardResult.AddBlock(id, tags, seriesBlock) + } + runResult.Unlock() + return nil +} + +func (s *fileSystemSource) readNextEntryAndIndex( + r fs.DataFileSetReader, + runResult *runResult, + segment segment.MutableSegment, +) error { + // If performing index run, then simply read the metadata and add to segment + id, tagsIter, _, _, err := r.ReadMetadata() + if err != nil { + return err + } + + // NB(r): Avoiding defer in the hot path here + release := func() { + // Finalize the ID and tags + id.Finalize() + tagsIter.Close() + } + + idBytes := id.Bytes() + + runResult.RLock() + exists, err := segment.ContainsID(idBytes) + runResult.RUnlock() + if err != nil { + release() + return err + } + if exists { + release() + return nil + } + + d, err := convert.FromMetricIter(id, tagsIter) + if err != nil { + release() + return err + } + + runResult.Lock() + exists, err = segment.ContainsID(idBytes) + // ID and tags no longer required below + release() + if err != nil { + runResult.Unlock() + return err + } + if exists { + runResult.Unlock() + return nil + } + _, err = segment.Insert(d) + runResult.Unlock() + + return err } func (s *fileSystemSource) read( @@ -805,3 +776,54 @@ func (p *readerPool) put(r fs.DataFileSetReader) { p.values = append(p.values, r) p.Unlock() } + +type runResult struct { + sync.RWMutex + data result.DataBootstrapResult + index result.IndexBootstrapResult +} + +func newRunResult() *runResult { + return &runResult{ + data: result.NewDataBootstrapResult(), + index: result.NewIndexBootstrapResult(), + } +} + +func (r *runResult) getOrAddDataShardResult( + shard uint32, + capacity int, + ropts result.Options, +) result.ShardResult { + // Only called once per shard so ok to acquire write lock immediately + r.Lock() + defer r.Unlock() + + dataResults := r.data.ShardResults() + shardResult, exists := dataResults[shard] + if exists { + return shardResult + } + + // NB(r): Wait until we have a reader to initialize the shard result + // to be able to somewhat estimate the size of it. + shardResult = result.NewShardResult(capacity, ropts) + dataResults[shard] = shardResult + + return shardResult +} + +func (r *runResult) getOrAddIndexSegment( + start time.Time, + ns namespace.Metadata, + ropts result.Options, +) (segment.MutableSegment, error) { + // Only called once per shard so ok to acquire write lock immediately + r.Lock() + defer r.Unlock() + + indexResults := r.index.IndexResults() + indexBlockSegment, err := indexResults.GetOrAddSegment(start, + ns.Options().IndexOptions(), ropts) + return indexBlockSegment, err +} diff --git a/storage/bootstrap/result/result_index_test.go b/storage/bootstrap/result/result_index_test.go index 0032bda3af..467de5679c 100644 --- a/storage/bootstrap/result/result_index_test.go +++ b/storage/bootstrap/result/result_index_test.go @@ -21,17 +21,56 @@ package result import ( - "fmt" "testing" "time" - "github.com/golang/mock/gomock" + "github.com/stretchr/testify/require" + + "github.com/m3db/m3db/storage/namespace" "github.com/m3db/m3ninx/index/segment" + "github.com/golang/mock/gomock" "github.com/stretchr/testify/assert" ) -func TestIndexResultAddMergesExistingSegments(t *testing.T) { +func TestIndexResultGetOrAddSegment(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + created := segment.NewMockMutableSegment(ctrl) + allocated := 0 + opts := NewOptions(). + SetIndexMutableSegmentAllocator(func() (segment.MutableSegment, error) { + allocated++ + return created, nil + }) + + now := time.Now() + blockSize := time.Hour + idxOpts := namespace.NewIndexOptions().SetBlockSize(blockSize) + aligned := now.Truncate(blockSize) + + results := IndexResults{} + seg, err := results.GetOrAddSegment(aligned.Add(time.Minute), idxOpts, opts) + require.NoError(t, err) + require.True(t, seg == created) + require.Equal(t, 1, len(results)) + + seg, err = results.GetOrAddSegment(aligned.Add(2*time.Minute), idxOpts, opts) + require.NoError(t, err) + require.True(t, seg == created) + require.Equal(t, 1, len(results)) + + seg, err = results.GetOrAddSegment(aligned.Add(blockSize), idxOpts, opts) + require.NoError(t, err) + require.True(t, seg == created) + require.Equal(t, 2, len(results)) + + // Total allocs should've only been two + require.Equal(t, 2, allocated) +} + +func TestIndexResultMergeMergesExistingSegments(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() @@ -48,14 +87,16 @@ func TestIndexResultAddMergesExistingSegments(t *testing.T) { times := []time.Time{start, start.Add(testBlockSize)} - r := NewIndexBootstrapResult() - r.Add(NewIndexBlock(times[0], []segment.Segment{segments[0]}), nil) - r.Add(NewIndexBlock(times[0], []segment.Segment{segments[1]}), nil) - r.Add(NewIndexBlock(times[1], []segment.Segment{segments[2], segments[3]}), nil) + first := NewIndexBootstrapResult() + first.Add(NewIndexBlock(times[0], []segment.Segment{segments[0]}), nil) + first.Add(NewIndexBlock(times[0], []segment.Segment{segments[1]}), nil) + first.Add(NewIndexBlock(times[1], []segment.Segment{segments[2], segments[3]}), nil) + + second := NewIndexBootstrapResult() + second.Add(NewIndexBlock(times[0], []segment.Segment{segments[4]}), nil) + second.Add(NewIndexBlock(times[1], []segment.Segment{segments[5]}), nil) - merged := NewIndexBootstrapResult() - merged.Add(NewIndexBlock(times[0], []segment.Segment{segments[4]}), nil) - merged.Add(NewIndexBlock(times[1], []segment.Segment{segments[5]}), nil) + merged := MergedIndexBootstrapResult(first, second) expected := NewIndexBootstrapResult() expected.Add(NewIndexBlock(times[0], []segment.Segment{segments[0], segments[1], segments[4]}), nil) @@ -66,23 +107,18 @@ func TestIndexResultAddMergesExistingSegments(t *testing.T) { func segmentsInResultsSame(a, b IndexResults) bool { if len(a) != len(b) { - // `` - fmt.Printf("not same len\n") return false } for t, block := range a { otherBlock, ok := b[t] if !ok { - fmt.Printf("no block at time %v\n", time.Unix(0, int64(t)).String()) return false } if len(block.Segments()) != len(otherBlock.Segments()) { - fmt.Printf("block segment len not match at %v\n", time.Unix(0, int64(t)).String()) return false } for i, s := range block.Segments() { if s != otherBlock.Segments()[i] { - fmt.Printf("block segment not match at %v idx %d\n", time.Unix(0, int64(t)).String(), i) return false } } From 78658f5a2f840e052648f3be5e14369893fdc848 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 8 May 2018 02:28:13 -0400 Subject: [PATCH 13/23] Add sorted order when returning test tags --- persist/fs/read_write_test.go | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/persist/fs/read_write_test.go b/persist/fs/read_write_test.go index 8d6e35c277..c810f0a556 100644 --- a/persist/fs/read_write_test.go +++ b/persist/fs/read_write_test.go @@ -25,6 +25,7 @@ import ( "errors" "os" "path/filepath" + "sort" "testing" "time" @@ -52,10 +53,19 @@ func (e testEntry) Tags() ident.Tags { if e.tags == nil { return nil } + + // Return in sorted order for deterministic order + var keys []string + for key := range e.tags { + keys = append(keys, key) + } + sort.Strings(keys) + var tags ident.Tags - for key, value := range e.tags { - tags = append(tags, ident.StringTag(key, value)) + for _, key := range keys { + tags = append(tags, ident.StringTag(key, e.tags[key])) } + return tags } From 8aadf8581f905f5e33050ee2c777e2ae339b86ed Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 8 May 2018 10:43:43 -0400 Subject: [PATCH 14/23] Add fs tags integration test --- .../admin_session_fetch_blocks_test.go | 4 +- integration/cluster_add_one_node_test.go | 4 +- integration/commitlog_bootstrap_merge_test.go | 6 +- .../commitlog_bootstrap_multi_ns_test.go | 14 +- integration/data.go | 20 ++- integration/disk_flush_multi_ns_test.go | 6 +- integration/disk_flush_test.go | 4 +- integration/disk_snapshot_test.go | 2 +- integration/dynamic_namespace_add_test.go | 4 +- integration/dynamic_namespace_delete_test.go | 4 +- integration/fs_bootstrap_multi_ns_test.go | 10 +- integration/fs_bootstrap_tags_test.go | 126 ++++++++++++++++++ integration/fs_bootstrap_test.go | 4 +- integration/fs_data_expiry_bootstrap_test.go | 2 +- integration/generate/generate.go | 21 +++ integration/generate/types.go | 21 +++ .../peers_bootstrap_high_concurrency_test.go | 8 +- .../peers_bootstrap_merge_local_test.go | 4 +- .../peers_bootstrap_merge_peer_blocks_test.go | 20 +-- integration/peers_bootstrap_node_down_test.go | 20 +-- .../peers_bootstrap_select_best_test.go | 20 +-- integration/peers_bootstrap_simple_test.go | 20 +-- integration/roundtrip_test.go | 4 +- integration/truncate_namespace_test.go | 8 +- storage/index.go | 2 + 25 files changed, 275 insertions(+), 83 deletions(-) create mode 100644 integration/fs_bootstrap_tags_test.go diff --git a/integration/admin_session_fetch_blocks_test.go b/integration/admin_session_fetch_blocks_test.go index bbf9f8597c..ec84620488 100644 --- a/integration/admin_session_fetch_blocks_test.go +++ b/integration/admin_session_fetch_blocks_test.go @@ -70,8 +70,8 @@ func TestAdminSessionFetchBlocksFromPeers(t *testing.T) { now := testSetup.getNowFn() seriesMaps := make(map[xtime.UnixNano]generate.SeriesBlock) inputData := []generate.BlockConfig{ - {[]string{"foo", "bar"}, 100, now}, - {[]string{"foo", "baz"}, 50, now.Add(blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 100, Start: now}, + {IDs: []string{"foo", "baz"}, NumPoints: 50, Start: now.Add(blockSize)}, } for _, input := range inputData { start := input.Start diff --git a/integration/cluster_add_one_node_test.go b/integration/cluster_add_one_node_test.go index 68fb57fdf4..d7e6419dca 100644 --- a/integration/cluster_add_one_node_test.go +++ b/integration/cluster_add_one_node_test.go @@ -152,8 +152,8 @@ func TestClusterAddOneNode(t *testing.T) { now := setups[0].getNowFn() blockSize := namesp.Options().RetentionOptions().BlockSize() seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ - {[]string{ids[0].str, ids[1].str}, 180, now.Add(-blockSize)}, - {[]string{ids[0].str, ids[2].str}, 90, now}, + {IDs: []string{ids[0].str, ids[1].str}, NumPoints: 180, Start: now.Add(-blockSize)}, + {IDs: []string{ids[0].str, ids[2].str}, NumPoints: 90, Start: now}, }) err = writeTestDataToDisk(namesp, setups[0], seriesMaps) require.NoError(t, err) diff --git a/integration/commitlog_bootstrap_merge_test.go b/integration/commitlog_bootstrap_merge_test.go index 8d34023736..83e507cc8f 100644 --- a/integration/commitlog_bootstrap_merge_test.go +++ b/integration/commitlog_bootstrap_merge_test.go @@ -94,9 +94,9 @@ func TestCommitLogAndFSMergeBootstrap(t *testing.T) { t3 = t2.Add(ns1BlockSize) ) blockConfigs := []generate.BlockConfig{ - {[]string{"foo", "bar"}, 20, t0}, - {[]string{"nah", "baz"}, 50, t1}, - {[]string{"hax", "ord"}, 30, t2}, + {IDs: []string{"foo", "bar"}, NumPoints: 20, Start: t0}, + {IDs: []string{"nah", "baz"}, NumPoints: 50, Start: t1}, + {IDs: []string{"hax", "ord"}, NumPoints: 30, Start: t2}, } log.Info("generating data") seriesMaps := generate.BlocksByStart(blockConfigs) diff --git a/integration/commitlog_bootstrap_multi_ns_test.go b/integration/commitlog_bootstrap_multi_ns_test.go index 5050257f07..d2e2de37e4 100644 --- a/integration/commitlog_bootstrap_multi_ns_test.go +++ b/integration/commitlog_bootstrap_multi_ns_test.go @@ -76,9 +76,9 @@ func TestCommitLogBootstrapMultipleNamespaces(t *testing.T) { log.Info("generating data - ns1") now := setup.getNowFn() ns1SeriesMap := generate.BlocksByStart([]generate.BlockConfig{ - {[]string{"foo", "bar"}, 20, now.Add(ns1BlockSize)}, - {[]string{"bar", "baz"}, 50, now.Add(2 * ns1BlockSize)}, - {[]string{"and", "one"}, 40, now.Add(3 * ns1BlockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 20, Start: now.Add(ns1BlockSize)}, + {IDs: []string{"bar", "baz"}, NumPoints: 50, Start: now.Add(2 * ns1BlockSize)}, + {IDs: []string{"and", "one"}, NumPoints: 40, Start: now.Add(3 * ns1BlockSize)}, }) setup.namespaceMetadataOrFail(testNamespaces[0]) @@ -89,10 +89,10 @@ func TestCommitLogBootstrapMultipleNamespaces(t *testing.T) { // Write test data for ns2 log.Info("generating data - ns2") ns2SeriesMap := generate.BlocksByStart([]generate.BlockConfig{ - {[]string{"abc", "def"}, 20, now.Add(ns2BlockSize)}, - {[]string{"xyz", "lmn"}, 50, now.Add(2 * ns2BlockSize)}, - {[]string{"cat", "hax"}, 80, now.Add(3 * ns2BlockSize)}, - {[]string{"why", "this"}, 40, now.Add(4 * ns2BlockSize)}, + {IDs: []string{"abc", "def"}, NumPoints: 20, Start: now.Add(ns2BlockSize)}, + {IDs: []string{"xyz", "lmn"}, NumPoints: 50, Start: now.Add(2 * ns2BlockSize)}, + {IDs: []string{"cat", "hax"}, NumPoints: 80, Start: now.Add(3 * ns2BlockSize)}, + {IDs: []string{"why", "this"}, NumPoints: 40, Start: now.Add(4 * ns2BlockSize)}, }) setup.namespaceMetadataOrFail(testNamespaces[1]) log.Info("writing data - ns2") diff --git a/integration/data.go b/integration/data.go index 843b4f1c07..586661a6e4 100644 --- a/integration/data.go +++ b/integration/data.go @@ -39,9 +39,15 @@ import ( type readableSeries struct { ID string + Tags []readableSeriesTag Data []ts.Datapoint } +type readableSeriesTag struct { + Name string + Value string +} + type readableSeriesList []readableSeries func toDatapoints(fetched *rpc.FetchResult_) []ts.Datapoint { @@ -77,6 +83,7 @@ func verifySeriesMapForRange( require.NoError(t, err) actual[i] = generate.Series{ ID: s.ID, + Tags: s.Tags, Data: fetched, } } @@ -97,7 +104,18 @@ func writeVerifyDebugOutput(t *testing.T, filePath string, start, end time.Time, list := make(readableSeriesList, 0, len(series)) for i := range series { - list = append(list, readableSeries{ID: series[i].ID.String(), Data: series[i].Data}) + tags := make([]readableSeriesTag, len(series[i].Tags)) + for _, tag := range series[i].Tags { + tags = append(tags, readableSeriesTag{ + Name: tag.Name.String(), + Value: tag.Value.String(), + }) + } + list = append(list, readableSeries{ + ID: series[i].ID.String(), + Tags: tags, + Data: series[i].Data, + }) } data, err := json.MarshalIndent(struct { diff --git a/integration/disk_flush_multi_ns_test.go b/integration/disk_flush_multi_ns_test.go index 37ce4e467a..52e32bd8ce 100644 --- a/integration/disk_flush_multi_ns_test.go +++ b/integration/disk_flush_multi_ns_test.go @@ -85,14 +85,14 @@ func TestDiskFlushMultipleNamespace(t *testing.T) { // test data for ns1 ns1SeriesMaps := make(map[xtime.UnixNano]generate.SeriesBlock) ns1InputData := []generate.BlockConfig{ - {[]string{"foo", "bar"}, 100, now}, - {[]string{"foo", "baz"}, 50, now.Add(ns1BlockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 100, Start: now}, + {IDs: []string{"foo", "baz"}, NumPoints: 50, Start: now.Add(ns1BlockSize)}, } // test data for ns2 ns2SeriesMaps := make(map[xtime.UnixNano]generate.SeriesBlock) ns2InputData := []generate.BlockConfig{ - {[]string{"foo", "bar"}, 20, now}, + {IDs: []string{"foo", "bar"}, NumPoints: 20, Start: now}, } for _, ns1Input := range ns1InputData { diff --git a/integration/disk_flush_test.go b/integration/disk_flush_test.go index b4601b861b..9a4118f8a6 100644 --- a/integration/disk_flush_test.go +++ b/integration/disk_flush_test.go @@ -63,8 +63,8 @@ func TestDiskFlushSimple(t *testing.T) { now := testSetup.getNowFn() seriesMaps := make(map[xtime.UnixNano]generate.SeriesBlock) inputData := []generate.BlockConfig{ - {[]string{"foo", "bar"}, 100, now}, - {[]string{"foo", "baz"}, 50, now.Add(blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 100, Start: now}, + {IDs: []string{"foo", "baz"}, NumPoints: 50, Start: now.Add(blockSize)}, } for _, input := range inputData { testSetup.setNowFn(input.Start) diff --git a/integration/disk_snapshot_test.go b/integration/disk_snapshot_test.go index 23f63c90a5..ea656ac9c3 100644 --- a/integration/disk_snapshot_test.go +++ b/integration/disk_snapshot_test.go @@ -64,7 +64,7 @@ func TestDiskSnapshotSimple(t *testing.T) { now := testSetup.getNowFn() seriesMaps := make(map[xtime.UnixNano]generate.SeriesBlock) inputData := []generate.BlockConfig{ - {[]string{"foo", "bar", "baz"}, 100, now}, + {IDs: []string{"foo", "bar", "baz"}, NumPoints: 100, Start: now}, } for _, input := range inputData { testSetup.setNowFn(input.Start) diff --git a/integration/dynamic_namespace_add_test.go b/integration/dynamic_namespace_add_test.go index d77fb44f1d..b813692ee0 100644 --- a/integration/dynamic_namespace_add_test.go +++ b/integration/dynamic_namespace_add_test.go @@ -96,8 +96,8 @@ func TestDynamicNamespaceAdd(t *testing.T) { now := testSetup.getNowFn() seriesMaps := make(map[xtime.UnixNano]generate.SeriesBlock) inputData := []generate.BlockConfig{ - {[]string{"foo", "bar"}, 100, now}, - {[]string{"foo", "baz"}, 50, now.Add(blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 100, Start: now}, + {IDs: []string{"foo", "baz"}, NumPoints: 50, Start: now.Add(blockSize)}, } for _, input := range inputData { start := input.Start diff --git a/integration/dynamic_namespace_delete_test.go b/integration/dynamic_namespace_delete_test.go index aa39468998..20b359e23b 100644 --- a/integration/dynamic_namespace_delete_test.go +++ b/integration/dynamic_namespace_delete_test.go @@ -105,8 +105,8 @@ func TestDynamicNamespaceDelete(t *testing.T) { now := testSetup.getNowFn() seriesMaps := make(map[xtime.UnixNano]generate.SeriesBlock) inputData := []generate.BlockConfig{ - {[]string{"foo", "bar"}, 100, now}, - {[]string{"foo", "baz"}, 50, now.Add(blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 100, Start: now}, + {IDs: []string{"foo", "baz"}, NumPoints: 50, Start: now.Add(blockSize)}, } for _, input := range inputData { start := input.Start diff --git a/integration/fs_bootstrap_multi_ns_test.go b/integration/fs_bootstrap_multi_ns_test.go index de5c8ba22f..bc8344d053 100644 --- a/integration/fs_bootstrap_multi_ns_test.go +++ b/integration/fs_bootstrap_multi_ns_test.go @@ -87,13 +87,13 @@ func TestFilesystemBootstrapMultipleNamespaces(t *testing.T) { // Write test data now := setup.getNowFn() ns1SeriesMaps := generate.BlocksByStart([]generate.BlockConfig{ - {[]string{"foo", "bar"}, 100, now.Add(-ns1BlockSize)}, - {[]string{"foo", "baz"}, 50, now}, + {IDs: []string{"foo", "bar"}, NumPoints: 100, Start: now.Add(-ns1BlockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 50, Start: now}, }) ns2SeriesMaps := generate.BlocksByStart([]generate.BlockConfig{ - {[]string{"bar", "baz"}, 100, now.Add(-2 * ns2BlockSize)}, - {[]string{"foo", "bar"}, 100, now.Add(-ns2BlockSize)}, - {[]string{"foo", "baz"}, 50, now}, + {IDs: []string{"bar", "baz"}, NumPoints: 100, Start: now.Add(-2 * ns2BlockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 100, Start: now.Add(-ns2BlockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 50, Start: now}, }) require.NoError(t, writeTestDataToDisk(ns1, setup, ns1SeriesMaps)) require.NoError(t, writeTestDataToDisk(ns2, setup, ns2SeriesMaps)) diff --git a/integration/fs_bootstrap_tags_test.go b/integration/fs_bootstrap_tags_test.go new file mode 100644 index 0000000000..7d56768b8c --- /dev/null +++ b/integration/fs_bootstrap_tags_test.go @@ -0,0 +1,126 @@ +// +build integration + +// Copyright (c) 2016 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 integration + +import ( + "testing" + "time" + + "github.com/m3db/m3db/integration/generate" + "github.com/m3db/m3db/retention" + "github.com/m3db/m3db/storage/bootstrap" + "github.com/m3db/m3db/storage/bootstrap/bootstrapper" + "github.com/m3db/m3db/storage/bootstrap/bootstrapper/fs" + "github.com/m3db/m3db/storage/bootstrap/result" + "github.com/m3db/m3db/storage/namespace" + "github.com/m3db/m3x/ident" + + "github.com/stretchr/testify/require" +) + +func TestFilesystemBootstrapTags(t *testing.T) { + if testing.Short() { + t.SkipNow() // Just skip if we're doing a short run + } + + var ( + blockSize = 2 * time.Hour + rOpts = retention.NewOptions().SetRetentionPeriod(2 * time.Hour).SetBlockSize(blockSize) + ) + ns1, err := namespace.NewMetadata(testNamespaces[0], namespace.NewOptions().SetRetentionOptions(rOpts)) + require.NoError(t, err) + ns2, err := namespace.NewMetadata(testNamespaces[1], namespace.NewOptions().SetRetentionOptions(rOpts)) + require.NoError(t, err) + + opts := newTestOptions(t). + SetCommitLogRetentionPeriod(rOpts.RetentionPeriod()). + SetCommitLogBlockSize(blockSize). + SetNamespaces([]namespace.Metadata{ns1, ns2}) + + // Test setup + setup, err := newTestSetup(t, opts, nil) + require.NoError(t, err) + defer setup.close() + + fsOpts := setup.storageOpts.CommitLogOptions().FilesystemOptions() + + noOpAll := bootstrapper.NewNoOpAllBootstrapperProvider() + bsOpts := result.NewOptions(). + SetSeriesCachePolicy(setup.storageOpts.SeriesCachePolicy()) + bfsOpts := fs.NewOptions(). + SetResultOptions(bsOpts). + SetFilesystemOptions(fsOpts). + SetDatabaseBlockRetrieverManager(setup.storageOpts.DatabaseBlockRetrieverManager()) + bs := fs.NewFileSystemBootstrapperProvider(bfsOpts, noOpAll) + processProvider := bootstrap.NewProcessProvider(bs, bsOpts) + + setup.storageOpts = setup.storageOpts. + SetBootstrapProcessProvider(processProvider) + + // Write test data + now := setup.getNowFn() + seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ + { + IDs: []string{"foo"}, + Tags: ident.Tags{ident.StringTag("aaa", "bbb"), ident.StringTag("ccc", "ddd")}, + NumPoints: 100, + Start: now.Add(-blockSize), + }, + { + IDs: []string{"bar"}, + Tags: ident.Tags{ident.StringTag("eee", "fff")}, + NumPoints: 100, + Start: now.Add(-blockSize), + }, + { + IDs: []string{"foo"}, + Tags: ident.Tags{ident.StringTag("aaa", "bbb"), ident.StringTag("ccc", "ddd")}, + NumPoints: 50, + Start: now, + }, + { + IDs: []string{"baz"}, + Tags: ident.Tags{ident.StringTag("ggg", "hhh")}, + NumPoints: 50, + Start: now, + }, + }) + require.NoError(t, writeTestDataToDisk(ns1, setup, seriesMaps)) + require.NoError(t, writeTestDataToDisk(ns2, setup, nil)) + + // Start the server with filesystem bootstrapper + log := setup.storageOpts.InstrumentOptions().Logger() + log.Debug("filesystem bootstrap test") + require.NoError(t, setup.startServer()) + log.Debug("server is now up") + + // Stop the server + defer func() { + require.NoError(t, setup.stopServer()) + log.Debug("server is now down") + }() + + // Verify in-memory data match what we expect + verifySeriesMaps(t, setup, testNamespaces[0], seriesMaps) + verifySeriesMaps(t, setup, testNamespaces[1], nil) +} diff --git a/integration/fs_bootstrap_test.go b/integration/fs_bootstrap_test.go index 8e74b00949..6932e656b2 100644 --- a/integration/fs_bootstrap_test.go +++ b/integration/fs_bootstrap_test.go @@ -79,8 +79,8 @@ func TestFilesystemBootstrap(t *testing.T) { // Write test data now := setup.getNowFn() seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ - {[]string{"foo", "bar"}, 100, now.Add(-blockSize)}, - {[]string{"foo", "baz"}, 50, now}, + {IDs: []string{"foo", "bar"}, NumPoints: 100, Start: now.Add(-blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 50, Start: now}, }) require.NoError(t, writeTestDataToDisk(ns1, setup, seriesMaps)) require.NoError(t, writeTestDataToDisk(ns2, setup, nil)) diff --git a/integration/fs_data_expiry_bootstrap_test.go b/integration/fs_data_expiry_bootstrap_test.go index 51a7e1b188..1872eb4d5e 100644 --- a/integration/fs_data_expiry_bootstrap_test.go +++ b/integration/fs_data_expiry_bootstrap_test.go @@ -97,7 +97,7 @@ func TestFilesystemDataExpiryBootstrap(t *testing.T) { // Write test data now := setup.getNowFn() seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ - {[]string{"foo", "bar"}, 100, now.Add(-blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 100, Start: now.Add(-blockSize)}, }) require.NoError(t, writeTestDataToDisk(namesp, setup, seriesMaps)) diff --git a/integration/generate/generate.go b/integration/generate/generate.go index 5e9c2fc265..ace26924df 100644 --- a/integration/generate/generate.go +++ b/integration/generate/generate.go @@ -1,3 +1,23 @@ +// Copyright (c) 2018 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 generate import ( @@ -37,6 +57,7 @@ func Block(conf BlockConfig) SeriesBlock { } testData[i] = Series{ ID: ident.StringID(name), + Tags: conf.Tags, Data: datapoints, } } diff --git a/integration/generate/types.go b/integration/generate/types.go index 217140aaec..53bece6339 100644 --- a/integration/generate/types.go +++ b/integration/generate/types.go @@ -1,3 +1,23 @@ +// Copyright (c) 2018 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 generate import ( @@ -15,6 +35,7 @@ import ( // BlockConfig represents the configuration to generate a SeriesBlock type BlockConfig struct { IDs []string + Tags ident.Tags NumPoints int Start time.Time } diff --git a/integration/peers_bootstrap_high_concurrency_test.go b/integration/peers_bootstrap_high_concurrency_test.go index 4304597a88..b89863911b 100644 --- a/integration/peers_bootstrap_high_concurrency_test.go +++ b/integration/peers_bootstrap_high_concurrency_test.go @@ -88,10 +88,10 @@ func testPeersBootstrapHighConcurrency( now := setups[0].getNowFn() blockSize := retentionOpts.BlockSize() seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ - {shardIDs, 3, now.Add(-3 * blockSize)}, - {shardIDs, 3, now.Add(-2 * blockSize)}, - {shardIDs, 3, now.Add(-blockSize)}, - {shardIDs, 3, now}, + {IDs: shardIDs, NumPoints: 3, Start: now.Add(-3 * blockSize)}, + {IDs: shardIDs, NumPoints: 3, Start: now.Add(-2 * blockSize)}, + {IDs: shardIDs, NumPoints: 3, Start: now.Add(-blockSize)}, + {IDs: shardIDs, NumPoints: 3, Start: now}, }) err = writeTestDataToDisk(namesp, setups[0], seriesMaps) require.NoError(t, err) diff --git a/integration/peers_bootstrap_merge_local_test.go b/integration/peers_bootstrap_merge_local_test.go index 49892983b4..a7246018e2 100644 --- a/integration/peers_bootstrap_merge_local_test.go +++ b/integration/peers_bootstrap_merge_local_test.go @@ -82,8 +82,8 @@ func testPeersBootstrapMergeLocal( completeAt := now.Add(180 * time.Second) blockSize := retentionOpts.BlockSize() seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ - {[]string{"foo", "bar"}, 180, now.Add(-blockSize)}, - {[]string{"foo", "baz"}, int(completeAt.Sub(now) / time.Second), now}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: int(completeAt.Sub(now) / time.Second), Start: now}, }) firstNodeSeriesMaps := map[xtime.UnixNano]generate.SeriesBlock{} directWritesSeriesMaps := map[xtime.UnixNano]generate.SeriesBlock{} diff --git a/integration/peers_bootstrap_merge_peer_blocks_test.go b/integration/peers_bootstrap_merge_peer_blocks_test.go index 7d330297ea..b7d096d371 100644 --- a/integration/peers_bootstrap_merge_peer_blocks_test.go +++ b/integration/peers_bootstrap_merge_peer_blocks_test.go @@ -74,16 +74,16 @@ func testPeersBootstrapMergePeerBlocks( // Make sure we have multiple blocks of data for multiple series to exercise // the grouping and aggregating logic in the client peer bootstrapping process seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ - {[]string{"foo", "bar"}, 180, now.Add(-4 * blockSize)}, - {[]string{"foo", "bar"}, 180, now.Add(-3 * blockSize)}, - {[]string{"foo", "bar"}, 180, now.Add(-2 * blockSize)}, - {[]string{"foo", "bar"}, 180, now.Add(-blockSize)}, - {[]string{"foo", "bar"}, 180, now}, - {[]string{"foo", "baz"}, 90, now.Add(-4 * blockSize)}, - {[]string{"foo", "baz"}, 90, now.Add(-3 * blockSize)}, - {[]string{"foo", "baz"}, 90, now.Add(-2 * blockSize)}, - {[]string{"foo", "baz"}, 90, now.Add(-blockSize)}, - {[]string{"foo", "baz"}, 90, now}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-4 * blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-3 * blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-2 * blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-4 * blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-3 * blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-2 * blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now}, }) left := make(map[xtime.UnixNano]generate.SeriesBlock) right := make(map[xtime.UnixNano]generate.SeriesBlock) diff --git a/integration/peers_bootstrap_node_down_test.go b/integration/peers_bootstrap_node_down_test.go index 8c3caf304d..f2642aed9f 100644 --- a/integration/peers_bootstrap_node_down_test.go +++ b/integration/peers_bootstrap_node_down_test.go @@ -74,16 +74,16 @@ func testPeersBootstrapNodeDown( // Make sure we have multiple blocks of data for multiple series to exercise // the grouping and aggregating logic in the client peer bootstrapping process seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ - {[]string{"foo", "bar"}, 180, now.Add(-4 * blockSize)}, - {[]string{"foo", "bar"}, 180, now.Add(-3 * blockSize)}, - {[]string{"foo", "bar"}, 180, now.Add(-2 * blockSize)}, - {[]string{"foo", "bar"}, 180, now.Add(-blockSize)}, - {[]string{"foo", "bar"}, 180, now}, - {[]string{"foo", "baz"}, 90, now.Add(-4 * blockSize)}, - {[]string{"foo", "baz"}, 90, now.Add(-3 * blockSize)}, - {[]string{"foo", "baz"}, 90, now.Add(-2 * blockSize)}, - {[]string{"foo", "baz"}, 90, now.Add(-blockSize)}, - {[]string{"foo", "baz"}, 90, now}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-4 * blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-3 * blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-2 * blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-4 * blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-3 * blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-2 * blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now}, }) err = writeTestDataToDisk(namesp, setups[0], seriesMaps) require.NoError(t, err) diff --git a/integration/peers_bootstrap_select_best_test.go b/integration/peers_bootstrap_select_best_test.go index a3eb2aa756..609ca60fc9 100644 --- a/integration/peers_bootstrap_select_best_test.go +++ b/integration/peers_bootstrap_select_best_test.go @@ -74,16 +74,16 @@ func testPeerBootstrapSelectBest( // Make sure we have multiple blocks of data for multiple series to exercise // the grouping and aggregating logic in the client peer bootstrapping process seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ - {[]string{"foo", "bar"}, 180, now.Add(-4 * blockSize)}, - {[]string{"foo", "bar"}, 180, now.Add(-3 * blockSize)}, - {[]string{"foo", "bar"}, 180, now.Add(-2 * blockSize)}, - {[]string{"foo", "bar"}, 180, now.Add(-blockSize)}, - {[]string{"foo", "bar"}, 180, now}, - {[]string{"foo", "baz"}, 90, now.Add(-4 * blockSize)}, - {[]string{"foo", "baz"}, 90, now.Add(-3 * blockSize)}, - {[]string{"foo", "baz"}, 90, now.Add(-2 * blockSize)}, - {[]string{"foo", "baz"}, 90, now.Add(-blockSize)}, - {[]string{"foo", "baz"}, 90, now}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-4 * blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-3 * blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-2 * blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-4 * blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-3 * blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-2 * blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now}, }) left := make(map[xtime.UnixNano]generate.SeriesBlock) right := make(map[xtime.UnixNano]generate.SeriesBlock) diff --git a/integration/peers_bootstrap_simple_test.go b/integration/peers_bootstrap_simple_test.go index 2439075df4..1c67d122bc 100644 --- a/integration/peers_bootstrap_simple_test.go +++ b/integration/peers_bootstrap_simple_test.go @@ -71,16 +71,16 @@ func testPeerBootstrapSimple( // Make sure we have multiple blocks of data for multiple series to exercise // the grouping and aggregating logic in the client peer bootstrapping process seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ - {[]string{"foo", "bar"}, 180, now.Add(-4 * blockSize)}, - {[]string{"foo", "bar"}, 180, now.Add(-3 * blockSize)}, - {[]string{"foo", "bar"}, 180, now.Add(-2 * blockSize)}, - {[]string{"foo", "bar"}, 180, now.Add(-blockSize)}, - {[]string{"foo", "bar"}, 180, now}, - {[]string{"foo", "baz"}, 90, now.Add(-4 * blockSize)}, - {[]string{"foo", "baz"}, 90, now.Add(-3 * blockSize)}, - {[]string{"foo", "baz"}, 90, now.Add(-2 * blockSize)}, - {[]string{"foo", "baz"}, 90, now.Add(-blockSize)}, - {[]string{"foo", "baz"}, 90, now}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-4 * blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-3 * blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-2 * blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-4 * blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-3 * blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-2 * blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-blockSize)}, + {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now}, }) require.NoError(t, writeTestDataToDisk(namesp, setups[0], seriesMaps)) diff --git a/integration/roundtrip_test.go b/integration/roundtrip_test.go index 6550b80132..168074ecac 100644 --- a/integration/roundtrip_test.go +++ b/integration/roundtrip_test.go @@ -61,8 +61,8 @@ func TestRoundtrip(t *testing.T) { now := testSetup.getNowFn() seriesMaps := make(map[xtime.UnixNano]generate.SeriesBlock) inputData := []generate.BlockConfig{ - {[]string{"foo", "bar"}, 100, now}, - {[]string{"foo", "baz"}, 50, now.Add(blockSize)}, + {IDs: []string{"foo", "bar"}, NumPoints: 100, Start: now}, + {IDs: []string{"foo", "baz"}, NumPoints: 50, Start: now.Add(blockSize)}, } for _, input := range inputData { testSetup.setNowFn(input.Start) diff --git a/integration/truncate_namespace_test.go b/integration/truncate_namespace_test.go index a2bb8e1938..f2578598ec 100644 --- a/integration/truncate_namespace_test.go +++ b/integration/truncate_namespace_test.go @@ -65,8 +65,12 @@ func TestTruncateNamespace(t *testing.T) { namespace ident.ID conf generate.BlockConfig }{ - {testNamespaces[0], generate.BlockConfig{[]string{"foo"}, 100, now}}, - {testNamespaces[1], generate.BlockConfig{[]string{"bar"}, 50, now.Add(blockSize)}}, + {testNamespaces[0], generate.BlockConfig{ + IDs: []string{"foo"}, NumPoints: 100, Start: now}, + }, + {testNamespaces[1], generate.BlockConfig{ + IDs: []string{"bar"}, NumPoints: 50, Start: now.Add(blockSize)}, + }, } for _, input := range inputData { testSetup.setNowFn(input.conf.Start) diff --git a/storage/index.go b/storage/index.go index b3e278e9d4..f8ba92a953 100644 --- a/storage/index.go +++ b/storage/index.go @@ -495,6 +495,8 @@ func (i *nsIndex) Query( break } + fmt.Printf("\n\n!! issuing query against block %v\n\n", block.StartTime()) + exhaustive, err = block.Query(query, opts, results) if err != nil { return index.QueryResults{}, err From 64bf392151fba38df58e7a4a119f6b147274e63c Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 8 May 2018 11:36:07 -0400 Subject: [PATCH 15/23] Add integration test for testing index queries after bootstrap --- integration/fs_bootstrap_index_test.go | 249 +++++++++++++++++++++++++ integration/generate/generate.go | 3 +- storage/index.go | 9 +- storage/namespace.go | 27 ++- storage/types.go | 3 +- 5 files changed, 275 insertions(+), 16 deletions(-) create mode 100644 integration/fs_bootstrap_index_test.go diff --git a/integration/fs_bootstrap_index_test.go b/integration/fs_bootstrap_index_test.go new file mode 100644 index 0000000000..57c7aa99bd --- /dev/null +++ b/integration/fs_bootstrap_index_test.go @@ -0,0 +1,249 @@ +// +build integration + +// Copyright (c) 2016 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 integration + +import ( + "fmt" + "testing" + "time" + + "github.com/m3db/m3ninx/idx" + + "github.com/m3db/m3db/client" + "github.com/m3db/m3db/integration/generate" + "github.com/m3db/m3db/retention" + "github.com/m3db/m3db/storage/bootstrap" + "github.com/m3db/m3db/storage/bootstrap/bootstrapper" + "github.com/m3db/m3db/storage/bootstrap/bootstrapper/fs" + "github.com/m3db/m3db/storage/bootstrap/result" + "github.com/m3db/m3db/storage/index" + "github.com/m3db/m3db/storage/namespace" + "github.com/m3db/m3x/ident" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestFilesystemBootstrapIndex(t *testing.T) { + if testing.Short() { + t.SkipNow() // Just skip if we're doing a short run + } + + var ( + blockSize = 2 * time.Hour + rOpts = retention.NewOptions().SetRetentionPeriod(2 * blockSize).SetBlockSize(blockSize) + idxOpts = namespace.NewIndexOptions().SetEnabled(true).SetBlockSize(2 * blockSize) + nOpts = namespace.NewOptions().SetRetentionOptions(rOpts).SetIndexOptions(idxOpts) + ) + ns1, err := namespace.NewMetadata(testNamespaces[0], nOpts) + require.NoError(t, err) + ns2, err := namespace.NewMetadata(testNamespaces[1], nOpts) + require.NoError(t, err) + + opts := newTestOptions(t). + SetCommitLogRetentionPeriod(rOpts.RetentionPeriod()). + SetCommitLogBlockSize(blockSize). + SetNamespaces([]namespace.Metadata{ns1, ns2}). + SetIndexingEnabled(true) + + // Test setup + setup, err := newTestSetup(t, opts, nil) + require.NoError(t, err) + defer setup.close() + + fsOpts := setup.storageOpts.CommitLogOptions().FilesystemOptions() + + noOpAll := bootstrapper.NewNoOpAllBootstrapperProvider() + bsOpts := result.NewOptions(). + SetSeriesCachePolicy(setup.storageOpts.SeriesCachePolicy()) + bfsOpts := fs.NewOptions(). + SetResultOptions(bsOpts). + SetFilesystemOptions(fsOpts). + SetDatabaseBlockRetrieverManager(setup.storageOpts.DatabaseBlockRetrieverManager()) + bs := fs.NewFileSystemBootstrapperProvider(bfsOpts, noOpAll) + processProvider := bootstrap.NewProcessProvider(bs, bsOpts) + + setup.storageOpts = setup.storageOpts. + SetBootstrapProcessProvider(processProvider) + + // Write test data + now := setup.getNowFn() + + fooSeries := generate.Series{ + ID: ident.StringID("foo"), + Tags: ident.Tags{ident.StringTag("city", "new_york"), ident.StringTag("foo", "foo")}, + } + + barSeries := generate.Series{ + ID: ident.StringID("bar"), + Tags: ident.Tags{ident.StringTag("city", "new_jersey")}, + } + + bazSeries := generate.Series{ + ID: ident.StringID("baz"), + Tags: ident.Tags{ident.StringTag("city", "seattle")}, + } + + seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ + { + IDs: []string{fooSeries.ID.String()}, + Tags: fooSeries.Tags, + NumPoints: 100, + Start: now.Add(-blockSize), + }, + { + IDs: []string{barSeries.ID.String()}, + Tags: barSeries.Tags, + NumPoints: 100, + Start: now.Add(-blockSize), + }, + { + IDs: []string{fooSeries.ID.String()}, + Tags: fooSeries.Tags, + NumPoints: 50, + Start: now, + }, + { + IDs: []string{bazSeries.ID.String()}, + Tags: bazSeries.Tags, + NumPoints: 50, + Start: now, + }, + }) + + require.NoError(t, writeTestDataToDisk(ns1, setup, seriesMaps)) + require.NoError(t, writeTestDataToDisk(ns2, setup, nil)) + + // Start the server with filesystem bootstrapper + log := setup.storageOpts.InstrumentOptions().Logger() + log.Debug("filesystem bootstrap test") + require.NoError(t, setup.startServer()) + log.Debug("server is now up") + + // Stop the server + defer func() { + require.NoError(t, setup.stopServer()) + log.Debug("server is now down") + }() + + // Verify data matches what we expect + verifySeriesMaps(t, setup, testNamespaces[0], seriesMaps) + verifySeriesMaps(t, setup, testNamespaces[1], nil) + + // Issue some index queries + session, err := setup.m3dbClient.DefaultSession() + require.NoError(t, err) + + start := now.Add(-rOpts.RetentionPeriod()) + end := now.Add(blockSize) + queryOpts := index.QueryOptions{StartInclusive: start, EndExclusive: end} + + // Match all new_*r* from namespace 1 + regexpQuery, err := idx.NewRegexpQuery([]byte("city"), []byte("^new_.*r.*$")) + require.NoError(t, err) + iter, exhausitive, err := session.FetchTaggedIDs(ns1.ID(), + index.Query{regexpQuery}, queryOpts) + require.NoError(t, err) + defer iter.Finalize() + + verifyQueryMetadataResults(t, iter, exhausitive, verifyQueryMetadataResultsOptions{ + namespace: ns1.ID(), + exhausitive: true, + expected: []generate.Series{fooSeries, barSeries}, + }) + + // Match all *e*e* from namespace 2 + regexpQuery, err = idx.NewRegexpQuery([]byte("city"), []byte("^.*e.*e.*$")) + require.NoError(t, err) + iter, exhausitive, err = session.FetchTaggedIDs(ns1.ID(), + index.Query{regexpQuery}, queryOpts) + require.NoError(t, err) + defer iter.Finalize() + + verifyQueryMetadataResults(t, iter, exhausitive, verifyQueryMetadataResultsOptions{ + namespace: ns1.ID(), + exhausitive: true, + expected: []generate.Series{barSeries, bazSeries}, + }) +} + +type verifyQueryMetadataResultsOptions struct { + namespace ident.ID + exhausitive bool + expected []generate.Series +} + +type verifyQueryMetadataResult struct { + series generate.Series + matched bool +} + +func verifyQueryMetadataResults( + t *testing.T, + iter client.TaggedIDsIterator, + exhausitive bool, + opts verifyQueryMetadataResultsOptions, +) { + assert.Equal(t, opts.exhausitive, exhausitive) + + expected := map[string]*verifyQueryMetadataResult{} + for _, series := range opts.expected { + expected[series.ID.String()] = &verifyQueryMetadataResult{ + series: series, + matched: false, + } + } + + compared := 0 + for iter.Next() { + compared++ + + ns, id, tags := iter.Current() + assert.True(t, opts.namespace.Equal(ns)) + + idStr := id.String() + result, ok := expected[idStr] + require.True(t, ok, + fmt.Sprintf("not expecting ID: %s", idStr)) + + expectedTagsIter := ident.NewTagSliceIterator(result.series.Tags) + matcher := ident.NewTagIterMatcher(expectedTagsIter) + assert.True(t, matcher.Matches(tags), + fmt.Sprintf("tags not matching for ID: %s", idStr)) + + result.matched = true + } + require.NoError(t, iter.Err()) + + var matched, notMatched []string + for _, elem := range expected { + if elem.matched { + matched = append(matched, elem.series.ID.String()) + continue + } + notMatched = append(notMatched, elem.series.ID.String()) + } + + assert.Equal(t, len(expected), compared, + fmt.Sprintf("matched: %v, not matched: %v", matched, notMatched)) +} diff --git a/integration/generate/generate.go b/integration/generate/generate.go index ace26924df..71dd21bed9 100644 --- a/integration/generate/generate.go +++ b/integration/generate/generate.go @@ -69,7 +69,8 @@ func Block(conf BlockConfig) SeriesBlock { func BlocksByStart(confs []BlockConfig) SeriesBlocksByStart { seriesMaps := make(map[xtime.UnixNano]SeriesBlock) for _, conf := range confs { - seriesMaps[xtime.ToUnixNano(conf.Start)] = Block(conf) + key := xtime.ToUnixNano(conf.Start) + seriesMaps[key] = append(seriesMaps[key], Block(conf)...) } return seriesMaps } diff --git a/storage/index.go b/storage/index.go index f8ba92a953..e859290709 100644 --- a/storage/index.go +++ b/storage/index.go @@ -29,12 +29,12 @@ import ( "github.com/m3db/m3db/clock" "github.com/m3db/m3db/retention" + "github.com/m3db/m3db/storage/bootstrap/result" m3dberrors "github.com/m3db/m3db/storage/errors" "github.com/m3db/m3db/storage/index" "github.com/m3db/m3db/storage/index/convert" "github.com/m3db/m3db/storage/namespace" "github.com/m3db/m3ninx/doc" - "github.com/m3db/m3ninx/index/segment" "github.com/m3db/m3x/context" xerrors "github.com/m3db/m3x/errors" "github.com/m3db/m3x/ident" @@ -362,7 +362,7 @@ func (i *nsIndex) writeBatchForBlockStartWithRLock( // Bootstrap bootstraps the index with the provide blocks. func (i *nsIndex) Bootstrap( - segmentsByBlockStart map[xtime.UnixNano][]segment.Segment, + bootstrapResults result.IndexResults, ) error { i.state.Lock() if i.state.bootstrapState == Bootstrapping { @@ -381,7 +381,8 @@ func (i *nsIndex) Bootstrap( }() var multiErr xerrors.MultiError - for blockStart, segments := range segmentsByBlockStart { + for blockStart, block := range bootstrapResults { + segments := block.Segments() block, err := i.ensureBlockPresentWithRLock(blockStart.ToTime()) if err != nil { // should never happen multiErr = multiErr.Add(i.unableToAllocBlockInvariantError(err)) @@ -495,8 +496,6 @@ func (i *nsIndex) Query( break } - fmt.Printf("\n\n!! issuing query against block %v\n\n", block.StartTime()) - exhaustive, err = block.Query(query, opts, results) if err != nil { return index.QueryResults{}, err diff --git a/storage/namespace.go b/storage/namespace.go index 0aefa0ff56..e674dceb2e 100644 --- a/storage/namespace.go +++ b/storage/namespace.go @@ -757,15 +757,26 @@ func (n *dbNamespace) Bootstrap(start time.Time, process bootstrap.Process) erro wg.Wait() - // Counter, tag this with namespace - unfulfilled := int64(len(bootstrapResult.DataResult.Unfulfilled())) - n.metrics.unfulfilled.Inc(unfulfilled) - if unfulfilled > 0 { - str := bootstrapResult.DataResult.Unfulfilled().SummaryString() - msgFmt := "bootstrap for namespace %s completed with unfulfilled ranges: %s" - multiErr = multiErr.Add(fmt.Errorf(msgFmt, n.id.String(), str)) - n.log.Errorf(msgFmt, n.id.String(), str) + if n.reverseIndex != nil { + err := n.reverseIndex.Bootstrap(bootstrapResult.IndexResult.IndexResults()) + multiErr = multiErr.Add(err) + } + + markAnyUnfulfilled := func(label string, unfulfilled result.ShardTimeRanges) { + shardsUnfulfilled := int64(len(unfulfilled)) + n.metrics.unfulfilled.Inc(shardsUnfulfilled) + if shardsUnfulfilled > 0 { + str := unfulfilled.SummaryString() + err := fmt.Errorf("bootstrap completed with unfulfilled ranges: %s", str) + multiErr = multiErr.Add(err) + n.log.WithFields( + xlog.NewField("namespace", n.id.String()), + xlog.NewField("bootstrap-type", label), + ).Errorf(err.Error()) + } } + markAnyUnfulfilled("data", bootstrapResult.DataResult.Unfulfilled()) + markAnyUnfulfilled("index", bootstrapResult.IndexResult.Unfulfilled()) err = multiErr.FinalError() n.metrics.bootstrap.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) diff --git a/storage/types.go b/storage/types.go index cb114df243..d5c0c6f8c8 100644 --- a/storage/types.go +++ b/storage/types.go @@ -41,7 +41,6 @@ import ( "github.com/m3db/m3db/x/xcounter" "github.com/m3db/m3db/x/xio" "github.com/m3db/m3ninx/doc" - "github.com/m3db/m3ninx/index/segment" "github.com/m3db/m3x/context" "github.com/m3db/m3x/ident" "github.com/m3db/m3x/instrument" @@ -447,7 +446,7 @@ type namespaceIndex interface { // Bootstrap bootstraps the index the provided segments. Bootstrap( - segmentsByBlockStart map[xtime.UnixNano][]segment.Segment, + bootstrapResults result.IndexResults, ) error // Tick performs internal house keeping in the index, including block rotation, From 29e52475e7a67b36bcf5292487b50012c34ae41a Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 8 May 2018 11:43:01 -0400 Subject: [PATCH 16/23] Test test build errors --- storage/index_block_test.go | 25 +++++++++++++------------ storage/storage_mock.go | 9 ++++----- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/storage/index_block_test.go b/storage/index_block_test.go index 4839ff2557..b8f5d49cce 100644 --- a/storage/index_block_test.go +++ b/storage/index_block_test.go @@ -27,6 +27,7 @@ import ( "time" "github.com/m3db/m3db/retention" + "github.com/m3db/m3db/storage/bootstrap/result" "github.com/m3db/m3db/storage/index" "github.com/m3db/m3db/storage/namespace" "github.com/m3db/m3ninx/doc" @@ -233,14 +234,14 @@ func TestNamespaceIndexBootstrap(t *testing.T) { seg1 := segment.NewMockSegment(ctrl) seg2 := segment.NewMockSegment(ctrl) seg3 := segment.NewMockSegment(ctrl) - bootstrapSegments := map[xtime.UnixNano][]segment.Segment{ - t0Nanos: []segment.Segment{seg1}, - t1Nanos: []segment.Segment{seg2, seg3}, + bootstrapResults := result.IndexResults{ + t0Nanos: result.NewIndexBlock(t0, []segment.Segment{seg1}), + t1Nanos: result.NewIndexBlock(t1, []segment.Segment{seg2, seg3}), } - b0.EXPECT().Bootstrap(bootstrapSegments[t0Nanos]).Return(nil) - b1.EXPECT().Bootstrap(bootstrapSegments[t1Nanos]).Return(nil) - require.NoError(t, idx.Bootstrap(bootstrapSegments)) + b0.EXPECT().Bootstrap(bootstrapResults[t0Nanos].Segments()).Return(nil) + b1.EXPECT().Bootstrap(bootstrapResults[t1Nanos].Segments()).Return(nil) + require.NoError(t, idx.Bootstrap(bootstrapResults)) } func TestNamespaceIndexTickExpire(t *testing.T) { @@ -402,14 +403,14 @@ func TestNamespaceIndexBlockQuery(t *testing.T) { seg1 := segment.NewMockSegment(ctrl) seg2 := segment.NewMockSegment(ctrl) seg3 := segment.NewMockSegment(ctrl) - bootstrapSegments := map[xtime.UnixNano][]segment.Segment{ - t0Nanos: []segment.Segment{seg1}, - t1Nanos: []segment.Segment{seg2, seg3}, + bootstrapResults := result.IndexResults{ + t0Nanos: result.NewIndexBlock(t0, []segment.Segment{seg1}), + t1Nanos: result.NewIndexBlock(t1, []segment.Segment{seg2, seg3}), } - b0.EXPECT().Bootstrap(bootstrapSegments[t0Nanos]).Return(nil) - b1.EXPECT().Bootstrap(bootstrapSegments[t1Nanos]).Return(nil) - require.NoError(t, idx.Bootstrap(bootstrapSegments)) + b0.EXPECT().Bootstrap(bootstrapResults[t0Nanos].Segments()).Return(nil) + b1.EXPECT().Bootstrap(bootstrapResults[t1Nanos].Segments()).Return(nil) + require.NoError(t, idx.Bootstrap(bootstrapResults)) // only queries as much as is needed (wrt to time) ctx := context.NewContext() diff --git a/storage/storage_mock.go b/storage/storage_mock.go index 45d065875d..dd4f6c190b 100644 --- a/storage/storage_mock.go +++ b/storage/storage_mock.go @@ -45,7 +45,6 @@ import ( "github.com/m3db/m3db/x/xcounter" "github.com/m3db/m3db/x/xio" "github.com/m3db/m3ninx/doc" - "github.com/m3db/m3ninx/index/segment" "github.com/m3db/m3x/context" "github.com/m3db/m3x/ident" "github.com/m3db/m3x/instrument" @@ -1410,15 +1409,15 @@ func (mr *MocknamespaceIndexMockRecorder) Query(ctx, query, opts interface{}) *g } // Bootstrap mocks base method -func (m *MocknamespaceIndex) Bootstrap(segmentsByBlockStart map[time0.UnixNano][]segment.Segment) error { - ret := m.ctrl.Call(m, "Bootstrap", segmentsByBlockStart) +func (m *MocknamespaceIndex) Bootstrap(bootstrapResults result.IndexResults) error { + ret := m.ctrl.Call(m, "Bootstrap", bootstrapResults) ret0, _ := ret[0].(error) return ret0 } // Bootstrap indicates an expected call of Bootstrap -func (mr *MocknamespaceIndexMockRecorder) Bootstrap(segmentsByBlockStart interface{}) *gomock.Call { - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Bootstrap", reflect.TypeOf((*MocknamespaceIndex)(nil).Bootstrap), segmentsByBlockStart) +func (mr *MocknamespaceIndexMockRecorder) Bootstrap(bootstrapResults interface{}) *gomock.Call { + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Bootstrap", reflect.TypeOf((*MocknamespaceIndex)(nil).Bootstrap), bootstrapResults) } // Tick mocks base method From b219f44e3625127c005c37f28462b33f12c6c9a2 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 8 May 2018 11:57:53 -0400 Subject: [PATCH 17/23] Address feedback --- docs/architecture/storage.md | 4 +++- integration/fs_bootstrap_index_test.go | 4 ++-- integration/fs_bootstrap_tags_test.go | 10 ++++++---- integration/peers_bootstrap_merge_peer_blocks_test.go | 5 ----- persist/fs/msgpack/encoder.go | 5 ++--- 5 files changed, 13 insertions(+), 15 deletions(-) diff --git a/docs/architecture/storage.md b/docs/architecture/storage.md index 92231eff75..f75843ef59 100644 --- a/docs/architecture/storage.md +++ b/docs/architecture/storage.md @@ -29,7 +29,9 @@ A fileset has the following files: │- Major Version │ │- Index Entry Offset ├──┘ │- Checksum │ │- Summaries (Num) │ └─────────────────────┘ │- Data Entry Offset ├──┐ │- BloomFilter (K/M) │ │- Encoded Tags | | -└─────────────────────┘ └─────────────────────┘ │ +│- Snapshot Time │ └─────────────────────┘ │ +│- Type (Flush/Snap) │ │ +└─────────────────────┘ │ │ ┌─────────────────────┐ ┌───────────────────────────┘ ┌─────────────────────┐ │ Bloom Filter File │ │ diff --git a/integration/fs_bootstrap_index_test.go b/integration/fs_bootstrap_index_test.go index 57c7aa99bd..c3070c2ea7 100644 --- a/integration/fs_bootstrap_index_test.go +++ b/integration/fs_bootstrap_index_test.go @@ -1,6 +1,6 @@ // +build integration -// Copyright (c) 2016 Uber Technologies, Inc. +// Copyright (c) 2018 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 @@ -44,7 +44,7 @@ import ( "github.com/stretchr/testify/require" ) -func TestFilesystemBootstrapIndex(t *testing.T) { +func TestFilesystemBootstrapIndexWithIndexingEnabled(t *testing.T) { if testing.Short() { t.SkipNow() // Just skip if we're doing a short run } diff --git a/integration/fs_bootstrap_tags_test.go b/integration/fs_bootstrap_tags_test.go index 7d56768b8c..aa988d5011 100644 --- a/integration/fs_bootstrap_tags_test.go +++ b/integration/fs_bootstrap_tags_test.go @@ -1,6 +1,6 @@ // +build integration -// Copyright (c) 2016 Uber Technologies, Inc. +// Copyright (c) 2018 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 @@ -38,7 +38,7 @@ import ( "github.com/stretchr/testify/require" ) -func TestFilesystemBootstrapTags(t *testing.T) { +func TestFilesystemBootstrapTagsWithIndexingDisabled(t *testing.T) { if testing.Short() { t.SkipNow() // Just skip if we're doing a short run } @@ -46,10 +46,12 @@ func TestFilesystemBootstrapTags(t *testing.T) { var ( blockSize = 2 * time.Hour rOpts = retention.NewOptions().SetRetentionPeriod(2 * time.Hour).SetBlockSize(blockSize) + idxOpts = namespace.NewIndexOptions().SetEnabled(false) + nOpts = namespace.NewOptions().SetRetentionOptions(rOpts).SetIndexOptions(idxOpts) ) - ns1, err := namespace.NewMetadata(testNamespaces[0], namespace.NewOptions().SetRetentionOptions(rOpts)) + ns1, err := namespace.NewMetadata(testNamespaces[0], nOpts) require.NoError(t, err) - ns2, err := namespace.NewMetadata(testNamespaces[1], namespace.NewOptions().SetRetentionOptions(rOpts)) + ns2, err := namespace.NewMetadata(testNamespaces[1], nOpts) require.NoError(t, err) opts := newTestOptions(t). diff --git a/integration/peers_bootstrap_merge_peer_blocks_test.go b/integration/peers_bootstrap_merge_peer_blocks_test.go index b7d096d371..2c3b15baae 100644 --- a/integration/peers_bootstrap_merge_peer_blocks_test.go +++ b/integration/peers_bootstrap_merge_peer_blocks_test.go @@ -74,11 +74,6 @@ func testPeersBootstrapMergePeerBlocks( // Make sure we have multiple blocks of data for multiple series to exercise // the grouping and aggregating logic in the client peer bootstrapping process seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-4 * blockSize)}, - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-3 * blockSize)}, - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-2 * blockSize)}, - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-blockSize)}, - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now}, {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-4 * blockSize)}, {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-3 * blockSize)}, {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-2 * blockSize)}, diff --git a/persist/fs/msgpack/encoder.go b/persist/fs/msgpack/encoder.go index 45278a0b05..4cf2977224 100644 --- a/persist/fs/msgpack/encoder.go +++ b/persist/fs/msgpack/encoder.go @@ -170,7 +170,7 @@ func (enc *Encoder) EncodeLogMetadata(entry schema.LogMetadata) error { // backwards-compatbility func (enc *Encoder) encodeIndexInfoV1(info schema.IndexInfo) { // Manually encode num fields for testing purposes - enc.encodeArrayLenFn(minNumIndexInfoFields) + enc.encodeArrayLenFn(6) // v1 had 6 fields enc.encodeVarintFn(info.BlockStart) enc.encodeVarintFn(info.BlockSize) enc.encodeVarintFn(info.Entries) @@ -206,7 +206,7 @@ func (enc *Encoder) encodeIndexBloomFilterInfo(info schema.IndexBloomFilterInfo) // backwards-compatbility func (enc *Encoder) encodeIndexEntryV1(entry schema.IndexEntry) { // Manually encode num fields for testing purposes - enc.encodeArrayLenFn(minNumIndexEntryFields) + enc.encodeArrayLenFn(5) // v1 had 5 fields enc.encodeVarintFn(entry.Index) enc.encodeBytesFn(entry.ID) enc.encodeVarintFn(entry.Size) @@ -215,7 +215,6 @@ func (enc *Encoder) encodeIndexEntryV1(entry schema.IndexEntry) { } func (enc *Encoder) encodeIndexEntryV2(entry schema.IndexEntry) { - // Manually encode num fields for testing purposes enc.encodeNumObjectFieldsForFn(indexEntryType) enc.encodeVarintFn(entry.Index) enc.encodeBytesFn(entry.ID) From 37d4f77d546ea06947cb7c5d2c4c10d26053dab3 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 8 May 2018 11:59:11 -0400 Subject: [PATCH 18/23] Address nit naming comment --- persist/fs/read.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/persist/fs/read.go b/persist/fs/read.go index 0df5b7e90d..122593b0e4 100644 --- a/persist/fs/read.go +++ b/persist/fs/read.go @@ -356,7 +356,7 @@ func (r *reader) Read() (ident.ID, ident.TagIterator, checked.Bytes, uint32, err } id := r.entryClonedID(entry.ID) - tags := r.entryClonedEncodedTagsTagIter(entry.EncodedTags) + tags := r.entryClonedEncodedTagsIter(entry.EncodedTags) r.entriesRead++ return id, tags, data, uint32(entry.Checksum), nil @@ -369,7 +369,7 @@ func (r *reader) ReadMetadata() (ident.ID, ident.TagIterator, int, uint32, error entry := r.indexEntriesByOffsetAsc[r.metadataRead] id := r.entryClonedID(entry.ID) - tags := r.entryClonedEncodedTagsTagIter(entry.EncodedTags) + tags := r.entryClonedEncodedTagsIter(entry.EncodedTags) length := int(entry.Size) checksum := uint32(entry.Checksum) @@ -404,7 +404,7 @@ func (r *reader) entryClonedID(id []byte) ident.ID { return ident.BinaryID(r.entryClonedBytes(id)) } -func (r *reader) entryClonedEncodedTagsTagIter(encodedTags []byte) ident.TagIterator { +func (r *reader) entryClonedEncodedTagsIter(encodedTags []byte) ident.TagIterator { if len(encodedTags) == 0 { // No tags set for this entry, return an empty tag iterator return ident.EmptyTagIterator From 6170c63fbc227994803387bc66c774c321160150 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 8 May 2018 12:12:55 -0400 Subject: [PATCH 19/23] Fix TestPeersBootstrapNodeDown integration test --- integration/peers_bootstrap_node_down_test.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/integration/peers_bootstrap_node_down_test.go b/integration/peers_bootstrap_node_down_test.go index f2642aed9f..e7e41e328d 100644 --- a/integration/peers_bootstrap_node_down_test.go +++ b/integration/peers_bootstrap_node_down_test.go @@ -74,11 +74,6 @@ func testPeersBootstrapNodeDown( // Make sure we have multiple blocks of data for multiple series to exercise // the grouping and aggregating logic in the client peer bootstrapping process seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-4 * blockSize)}, - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-3 * blockSize)}, - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-2 * blockSize)}, - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-blockSize)}, - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now}, {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-4 * blockSize)}, {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-3 * blockSize)}, {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-2 * blockSize)}, From be3a1fd231ad20d5f9e0ec9f8c3f94e5636d1ffa Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 8 May 2018 12:35:13 -0400 Subject: [PATCH 20/23] Fix remaining integration tests --- integration/peers_bootstrap_select_best_test.go | 5 ----- integration/peers_bootstrap_simple_test.go | 5 ----- 2 files changed, 10 deletions(-) diff --git a/integration/peers_bootstrap_select_best_test.go b/integration/peers_bootstrap_select_best_test.go index 609ca60fc9..48741d5e47 100644 --- a/integration/peers_bootstrap_select_best_test.go +++ b/integration/peers_bootstrap_select_best_test.go @@ -74,11 +74,6 @@ func testPeerBootstrapSelectBest( // Make sure we have multiple blocks of data for multiple series to exercise // the grouping and aggregating logic in the client peer bootstrapping process seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-4 * blockSize)}, - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-3 * blockSize)}, - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-2 * blockSize)}, - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-blockSize)}, - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now}, {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-4 * blockSize)}, {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-3 * blockSize)}, {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-2 * blockSize)}, diff --git a/integration/peers_bootstrap_simple_test.go b/integration/peers_bootstrap_simple_test.go index 1c67d122bc..b8db641995 100644 --- a/integration/peers_bootstrap_simple_test.go +++ b/integration/peers_bootstrap_simple_test.go @@ -71,11 +71,6 @@ func testPeerBootstrapSimple( // Make sure we have multiple blocks of data for multiple series to exercise // the grouping and aggregating logic in the client peer bootstrapping process seriesMaps := generate.BlocksByStart([]generate.BlockConfig{ - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-4 * blockSize)}, - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-3 * blockSize)}, - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-2 * blockSize)}, - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now.Add(-blockSize)}, - {IDs: []string{"foo", "bar"}, NumPoints: 180, Start: now}, {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-4 * blockSize)}, {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-3 * blockSize)}, {IDs: []string{"foo", "baz"}, NumPoints: 90, Start: now.Add(-2 * blockSize)}, From 1a8f1f980b3aa42954230b7716cc90daca080212 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 8 May 2018 13:19:13 -0400 Subject: [PATCH 21/23] Address feedback --- integration/fs_bootstrap_index_test.go | 7 +++---- storage/bootstrap/bootstrapper/fs/source.go | 7 +++---- storage/bootstrap/result/result_index.go | 4 ++++ 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/integration/fs_bootstrap_index_test.go b/integration/fs_bootstrap_index_test.go index c3070c2ea7..3800489600 100644 --- a/integration/fs_bootstrap_index_test.go +++ b/integration/fs_bootstrap_index_test.go @@ -27,8 +27,6 @@ import ( "testing" "time" - "github.com/m3db/m3ninx/idx" - "github.com/m3db/m3db/client" "github.com/m3db/m3db/integration/generate" "github.com/m3db/m3db/retention" @@ -38,6 +36,7 @@ import ( "github.com/m3db/m3db/storage/bootstrap/result" "github.com/m3db/m3db/storage/index" "github.com/m3db/m3db/storage/namespace" + "github.com/m3db/m3ninx/idx" "github.com/m3db/m3x/ident" "github.com/stretchr/testify/assert" @@ -158,7 +157,7 @@ func TestFilesystemBootstrapIndexWithIndexingEnabled(t *testing.T) { end := now.Add(blockSize) queryOpts := index.QueryOptions{StartInclusive: start, EndExclusive: end} - // Match all new_*r* from namespace 1 + // Match all new_*r* regexpQuery, err := idx.NewRegexpQuery([]byte("city"), []byte("^new_.*r.*$")) require.NoError(t, err) iter, exhausitive, err := session.FetchTaggedIDs(ns1.ID(), @@ -172,7 +171,7 @@ func TestFilesystemBootstrapIndexWithIndexingEnabled(t *testing.T) { expected: []generate.Series{fooSeries, barSeries}, }) - // Match all *e*e* from namespace 2 + // Match all *e*e* regexpQuery, err = idx.NewRegexpQuery([]byte("city"), []byte("^.*e.*e.*$")) require.NoError(t, err) iter, exhausitive, err = session.FetchTaggedIDs(ns1.ID(), diff --git a/storage/bootstrap/bootstrapper/fs/source.go b/storage/bootstrap/bootstrapper/fs/source.go index 12216c5a32..b914636ba0 100644 --- a/storage/bootstrap/bootstrapper/fs/source.go +++ b/storage/bootstrap/bootstrapper/fs/source.go @@ -321,7 +321,7 @@ func (s *fileSystemSource) handleErrorsAndUnfulfilled( // as they are additive and get merged together with results from other // bootstrappers by just appending the result (unlike data bootstrap // results that when merged replace the block with the current block). - // It would also be difficult to remove only series that was added to the + // It would also be difficult to remove only series that were added to the // index block as results from data files can be subsets of the index block // and there's no way to definitively delete the entry we added as a result // of just this data file failing. @@ -575,15 +575,14 @@ func (s *fileSystemSource) readNextEntryAndIndex( } d, err := convert.FromMetricIter(id, tagsIter) + release() if err != nil { - release() return err } runResult.Lock() - exists, err = segment.ContainsID(idBytes) + exists, err = segment.ContainsID(d.ID) // ID and tags no longer required below - release() if err != nil { runResult.Unlock() return err diff --git a/storage/bootstrap/result/result_index.go b/storage/bootstrap/result/result_index.go index ce667ed591..67c31f438e 100644 --- a/storage/bootstrap/result/result_index.go +++ b/storage/bootstrap/result/result_index.go @@ -85,6 +85,10 @@ func (r IndexResults) GetOrAddSegment( idxopts namespace.IndexOptions, opts Options, ) (segment.MutableSegment, error) { + // NB(r): The reason we can align by the retention block size and guarentee + // there is only one entry for this time is because index blocks must be a + // positive multiple of the data block size, making it easy to map a data + // block entry to at most one index block entry. blockStart := t.Truncate(idxopts.BlockSize()) blockStartNanos := xtime.ToUnixNano(blockStart) From 22b56808b94e1999445f54bc645e83d71418fb8a Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 8 May 2018 14:24:04 -0400 Subject: [PATCH 22/23] Fix forward compatible encoder/decoder tests --- persist/fs/msgpack/decoder.go | 70 ++++++++++++++------ persist/fs/msgpack/decoder_test.go | 40 ++++++------ persist/fs/msgpack/encoder.go | 32 ++++++---- persist/fs/msgpack/encoder_test.go | 2 +- persist/fs/msgpack/roundtrip_test.go | 96 +++++++++++++++------------- 5 files changed, 140 insertions(+), 100 deletions(-) diff --git a/persist/fs/msgpack/decoder.go b/persist/fs/msgpack/decoder.go index cc8b2b1b8b..d4dc19ddca 100644 --- a/persist/fs/msgpack/decoder.go +++ b/persist/fs/msgpack/decoder.go @@ -51,10 +51,16 @@ type Decoder struct { dec *msgpack.Decoder err error allocDecodedBytes bool + + legacy legacyEncodingOptions } // NewDecoder creates a new decoder func NewDecoder(opts DecodingOptions) *Decoder { + return newDecoder(defaultlegacyEncodingOptions, opts) +} + +func newDecoder(legacy legacyEncodingOptions, opts DecodingOptions) *Decoder { if opts == nil { opts = NewDecodingOptions() } @@ -63,6 +69,7 @@ func NewDecoder(opts DecodingOptions) *Decoder { allocDecodedBytes: opts.AllocDecodedBytes(), reader: reader, dec: msgpack.NewDecoder(reader), + legacy: legacy, } } @@ -161,7 +168,7 @@ func (dec *Decoder) DecodeLogEntryUniqueIndex() (DecodeLogEntryRemainingToken, u } _, numFieldsToSkip1 := dec.decodeRootObject(logEntryVersion, logEntryType) - numFieldsToSkip2, _, ok := dec.checkNumFieldsFor(logEntryType) + numFieldsToSkip2, _, ok := dec.checkNumFieldsFor(logEntryType, checkNumFieldsOptions{}) if !ok { return emptyLogEntryRemainingToken, 0, errorUnableToDetermineNumFieldsToSkip } @@ -217,7 +224,14 @@ func (dec *Decoder) DecodeLogMetadata() (schema.LogMetadata, error) { } func (dec *Decoder) decodeIndexInfo() schema.IndexInfo { - numFieldsToSkip, actual, ok := dec.checkNumFieldsFor(indexInfoType) + var opts checkNumFieldsOptions + if dec.legacy.decodeLegacyV1IndexInfo { + // v1 had 6 fields + opts.override = true + opts.numExpectedMinFields = 6 + opts.numExpectedCurrFields = 6 + } + numFieldsToSkip, actual, ok := dec.checkNumFieldsFor(indexInfoType, opts) if !ok { return emptyIndexInfo } @@ -230,21 +244,20 @@ func (dec *Decoder) decodeIndexInfo() schema.IndexInfo { indexInfo.Summaries = dec.decodeIndexSummariesInfo() indexInfo.BloomFilter = dec.decodeIndexBloomFilterInfo() - if actual < 8 { + if dec.legacy.decodeLegacyV1IndexInfo || actual < 8 { + dec.skip(numFieldsToSkip) return indexInfo } + indexInfo.SnapshotTime = dec.decodeVarint() indexInfo.FileType = persist.FileSetType(dec.decodeVarint()) dec.skip(numFieldsToSkip) - if dec.err != nil { - return emptyIndexInfo - } return indexInfo } func (dec *Decoder) decodeIndexSummariesInfo() schema.IndexSummariesInfo { - numFieldsToSkip, _, ok := dec.checkNumFieldsFor(indexSummariesInfoType) + numFieldsToSkip, _, ok := dec.checkNumFieldsFor(indexSummariesInfoType, checkNumFieldsOptions{}) if !ok { return emptyIndexSummariesInfo } @@ -258,7 +271,7 @@ func (dec *Decoder) decodeIndexSummariesInfo() schema.IndexSummariesInfo { } func (dec *Decoder) decodeIndexBloomFilterInfo() schema.IndexBloomFilterInfo { - numFieldsToSkip, _, ok := dec.checkNumFieldsFor(indexBloomFilterInfoType) + numFieldsToSkip, _, ok := dec.checkNumFieldsFor(indexBloomFilterInfoType, checkNumFieldsOptions{}) if !ok { return emptyIndexBloomFilterInfo } @@ -273,7 +286,14 @@ func (dec *Decoder) decodeIndexBloomFilterInfo() schema.IndexBloomFilterInfo { } func (dec *Decoder) decodeIndexEntry() schema.IndexEntry { - numFieldsToSkip, actual, ok := dec.checkNumFieldsFor(indexEntryType) + var opts checkNumFieldsOptions + if dec.legacy.decodeLegacyV1IndexInfo { + // v1 had 5 fields + opts.override = true + opts.numExpectedMinFields = 5 + opts.numExpectedCurrFields = 5 + } + numFieldsToSkip, actual, ok := dec.checkNumFieldsFor(indexEntryType, opts) if !ok { return emptyIndexEntry } @@ -285,20 +305,19 @@ func (dec *Decoder) decodeIndexEntry() schema.IndexEntry { indexEntry.Offset = dec.decodeVarint() indexEntry.Checksum = dec.decodeVarint() - if actual < 6 { + if dec.legacy.decodeLegacyV1IndexEntry || actual < 6 { + dec.skip(numFieldsToSkip) return indexEntry } + indexEntry.EncodedTags, _, _ = dec.decodeBytes() dec.skip(numFieldsToSkip) - if dec.err != nil { - return emptyIndexEntry - } return indexEntry } func (dec *Decoder) decodeIndexSummary() (schema.IndexSummary, IndexSummaryToken) { - numFieldsToSkip, _, ok := dec.checkNumFieldsFor(indexSummaryType) + numFieldsToSkip, _, ok := dec.checkNumFieldsFor(indexSummaryType, checkNumFieldsOptions{}) if !ok { return emptyIndexSummary, emptyIndexSummaryToken } @@ -326,7 +345,7 @@ func (dec *Decoder) decodeIndexSummary() (schema.IndexSummary, IndexSummaryToken } func (dec *Decoder) decodeLogInfo() schema.LogInfo { - numFieldsToSkip, _, ok := dec.checkNumFieldsFor(logInfoType) + numFieldsToSkip, _, ok := dec.checkNumFieldsFor(logInfoType, checkNumFieldsOptions{}) if !ok { return emptyLogInfo } @@ -342,7 +361,7 @@ func (dec *Decoder) decodeLogInfo() schema.LogInfo { } func (dec *Decoder) decodeLogEntry() schema.LogEntry { - numFieldsToSkip, _, ok := dec.checkNumFieldsFor(logEntryType) + numFieldsToSkip, _, ok := dec.checkNumFieldsFor(logEntryType, checkNumFieldsOptions{}) if !ok { return emptyLogEntry } @@ -362,7 +381,7 @@ func (dec *Decoder) decodeLogEntry() schema.LogEntry { } func (dec *Decoder) decodeLogMetadata() schema.LogMetadata { - numFieldsToSkip, _, ok := dec.checkNumFieldsFor(logMetadataType) + numFieldsToSkip, _, ok := dec.checkNumFieldsFor(logMetadataType, checkNumFieldsOptions{}) if !ok { return emptyLogMetadata } @@ -382,7 +401,7 @@ func (dec *Decoder) decodeRootObject(expectedVersion int, expectedType objectTyp if dec.err != nil { return 0, 0 } - numFieldsToSkip, _, ok := dec.checkNumFieldsFor(rootObjectType) + numFieldsToSkip, _, ok := dec.checkNumFieldsFor(rootObjectType, checkNumFieldsOptions{}) if !ok { return 0, 0 } @@ -410,12 +429,25 @@ func (dec *Decoder) checkVersion(expected int) int { return version } -func (dec *Decoder) checkNumFieldsFor(objType objectType) (numToSkip int, actual int, ok bool) { +type checkNumFieldsOptions struct { + override bool + numExpectedMinFields int + numExpectedCurrFields int +} + +func (dec *Decoder) checkNumFieldsFor( + objType objectType, + opts checkNumFieldsOptions, +) (numToSkip int, actual int, ok bool) { actual = dec.decodeNumObjectFields() if dec.err != nil { return 0, 0, false } min, curr := numFieldsForType(objType) + if opts.override { + min = opts.numExpectedMinFields + curr = opts.numExpectedCurrFields + } if min > actual { dec.err = fmt.Errorf("number of fields mismatch: expected minimum of %d actual %d", min, actual) return 0, 0, false diff --git a/persist/fs/msgpack/decoder_test.go b/persist/fs/msgpack/decoder_test.go index e8c05b6c82..6f6afca8a0 100644 --- a/persist/fs/msgpack/decoder_test.go +++ b/persist/fs/msgpack/decoder_test.go @@ -43,8 +43,8 @@ func testGenEncodeNumObjectFieldsForFn( func TestDecodeNewerVersionThanExpected(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) ) // Intentionally bump client-side version @@ -85,8 +85,8 @@ func TestDecodeNewerVersionThanExpected(t *testing.T) { func TestDecodeRootObjectMoreFieldsThanExpected(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) ) // Intentionally bump number of fields for the root object @@ -103,8 +103,8 @@ func TestDecodeRootObjectMoreFieldsThanExpected(t *testing.T) { func TestDecodeIndexInfoMoreFieldsThanExpected(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) ) // Intentionally bump number of fields for the index info object @@ -121,8 +121,8 @@ func TestDecodeIndexInfoMoreFieldsThanExpected(t *testing.T) { func TestDecodeIndexEntryMoreFieldsThanExpected(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) ) // Intentionally bump number of fields for the index entry object @@ -139,8 +139,8 @@ func TestDecodeIndexEntryMoreFieldsThanExpected(t *testing.T) { func TestDecodeLogInfoMoreFieldsThanExpected(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) ) // Intentionally bump number of fields for the log info object @@ -157,8 +157,8 @@ func TestDecodeLogInfoMoreFieldsThanExpected(t *testing.T) { func TestDecodeLogEntryMoreFieldsThanExpected(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) ) // Intentionally bump number of fields for the log entry object @@ -175,8 +175,8 @@ func TestDecodeLogEntryMoreFieldsThanExpected(t *testing.T) { func TestDecodeLogMetadataMoreFieldsThanExpected(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) ) // Intentionally bump number of fields for the log metadata object @@ -193,8 +193,8 @@ func TestDecodeLogMetadataMoreFieldsThanExpected(t *testing.T) { func TestDecodeLogEntryFewerFieldsThanExpected(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) ) // Intentionally bump number of fields for the log entry object @@ -209,8 +209,8 @@ func TestDecodeLogEntryFewerFieldsThanExpected(t *testing.T) { func TestDecodeBytesNoAlloc(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) ) require.NoError(t, enc.EncodeIndexEntry(testIndexEntry)) @@ -229,8 +229,8 @@ func TestDecodeBytesNoAlloc(t *testing.T) { func TestDecodeBytesAllocNew(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, NewDecodingOptions().SetAllocDecodedBytes(true)) + enc = NewEncoder() + dec = NewDecoder(NewDecodingOptions().SetAllocDecodedBytes(true)) ) require.NoError(t, enc.EncodeIndexEntry(testIndexEntry)) diff --git a/persist/fs/msgpack/encoder.go b/persist/fs/msgpack/encoder.go index 4cf2977224..6adea00419 100644 --- a/persist/fs/msgpack/encoder.go +++ b/persist/fs/msgpack/encoder.go @@ -50,24 +50,29 @@ type Encoder struct { encodeBytesFn encodeBytesFn encodeArrayLenFn encodeArrayLenFn + legacy legacyEncodingOptions +} + +type legacyEncodingOptions struct { encodeLegacyV1IndexInfo bool encodeLegacyV1IndexEntry bool + decodeLegacyV1IndexInfo bool + decodeLegacyV1IndexEntry bool } -// NewEncoder creates a new encoder -func NewEncoder() *Encoder { - return newEncoder(newEncoderOptions{ - encodeLegacyV1IndexInfo: false, - encodeLegacyV1IndexEntry: false, - }) +var defaultlegacyEncodingOptions = legacyEncodingOptions{ + encodeLegacyV1IndexInfo: false, + encodeLegacyV1IndexEntry: false, + decodeLegacyV1IndexInfo: false, + decodeLegacyV1IndexEntry: false, } -type newEncoderOptions struct { - encodeLegacyV1IndexInfo bool - encodeLegacyV1IndexEntry bool +// NewEncoder creates a new encoder +func NewEncoder() *Encoder { + return newEncoder(defaultlegacyEncodingOptions) } -func newEncoder(opts newEncoderOptions) *Encoder { +func newEncoder(legacy legacyEncodingOptions) *Encoder { buf := bytes.NewBuffer(nil) enc := &Encoder{ buf: buf, @@ -83,8 +88,7 @@ func newEncoder(opts newEncoderOptions) *Encoder { enc.encodeArrayLenFn = enc.encodeArrayLen // Used primarily for testing - enc.encodeLegacyV1IndexInfo = opts.encodeLegacyV1IndexInfo - enc.encodeLegacyV1IndexEntry = opts.encodeLegacyV1IndexEntry + enc.legacy = legacy return enc } @@ -104,7 +108,7 @@ func (enc *Encoder) EncodeIndexInfo(info schema.IndexInfo) error { return enc.err } enc.encodeRootObject(indexInfoVersion, indexInfoType) - if enc.encodeLegacyV1IndexInfo { + if enc.legacy.encodeLegacyV1IndexInfo { enc.encodeIndexInfoV1(info) } else { enc.encodeIndexInfoV2(info) @@ -118,7 +122,7 @@ func (enc *Encoder) EncodeIndexEntry(entry schema.IndexEntry) error { return enc.err } enc.encodeRootObject(indexEntryVersion, indexEntryType) - if enc.encodeLegacyV1IndexEntry { + if enc.legacy.encodeLegacyV1IndexEntry { enc.encodeIndexEntryV1(entry) } else { enc.encodeIndexEntryV2(entry) diff --git a/persist/fs/msgpack/encoder_test.go b/persist/fs/msgpack/encoder_test.go index a440b0687b..a0e17558c0 100644 --- a/persist/fs/msgpack/encoder_test.go +++ b/persist/fs/msgpack/encoder_test.go @@ -37,7 +37,7 @@ var ( ) func testCapturingEncoder(t *testing.T) (*Encoder, *[]interface{}) { - encoder := testEncoder(t) + encoder := NewEncoder() var result []interface{} encoder.encodeVarintFn = func(value int64) { diff --git a/persist/fs/msgpack/roundtrip_test.go b/persist/fs/msgpack/roundtrip_test.go index e45accd0c9..53c0277a71 100644 --- a/persist/fs/msgpack/roundtrip_test.go +++ b/persist/fs/msgpack/roundtrip_test.go @@ -85,18 +85,10 @@ var ( } ) -func testEncoder(t *testing.T) *Encoder { - return NewEncoder() -} - -func testDecoder(t *testing.T, opts DecodingOptions) *Decoder { - return NewDecoder(opts) -} - func TestIndexInfoRoundtrip(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) ) require.NoError(t, enc.EncodeIndexInfo(testIndexInfo)) dec.Reset(NewDecoderStream(enc.Bytes())) @@ -108,21 +100,22 @@ func TestIndexInfoRoundtrip(t *testing.T) { // Make sure the new decoding code can handle the old file format func TestIndexInfoRoundTripBackwardsCompatibilityV1(t *testing.T) { var ( - enc = newEncoder(newEncoderOptions{encodeLegacyV1IndexInfo: true}) - dec = testDecoder(t, nil) + opts = legacyEncodingOptions{encodeLegacyV1IndexInfo: true} + enc = newEncoder(opts) + dec = newDecoder(opts, nil) ) // Set the default values on the fields that did not exist in V1 // and then restore them at the end of the test - This is required // because the new decoder won't try and read the new fields from // the old file format - oldSnapshotTime := testIndexInfo.SnapshotTime - oldFileType := testIndexInfo.FileType + currSnapshotTime := testIndexInfo.SnapshotTime + currFileType := testIndexInfo.FileType testIndexInfo.SnapshotTime = 0 testIndexInfo.FileType = 0 defer func() { - testIndexInfo.SnapshotTime = oldSnapshotTime - testIndexInfo.FileType = oldFileType + testIndexInfo.SnapshotTime = currSnapshotTime + testIndexInfo.FileType = currFileType }() enc.EncodeIndexInfo(testIndexInfo) @@ -135,23 +128,28 @@ func TestIndexInfoRoundTripBackwardsCompatibilityV1(t *testing.T) { // Make sure the old decoder code can handle the new file format func TestIndexInfoRoundTripForwardsCompatibilityV2(t *testing.T) { var ( - enc = newEncoder(newEncoderOptions{encodeLegacyV1IndexInfo: false}) - dec = testDecoder(t, nil) + opts = legacyEncodingOptions{decodeLegacyV1IndexInfo: true} + enc = newEncoder(opts) + dec = newDecoder(opts, nil) ) // Set the default values on the fields that did not exist in V1 // and then restore them at the end of the test - This is required // because the old decoder won't read the new fields - oldSnapshotTime := testIndexInfo.SnapshotTime - oldFileType := testIndexInfo.FileType + currSnapshotTime := testIndexInfo.SnapshotTime + currFileType := testIndexInfo.FileType + + enc.EncodeIndexInfo(testIndexInfo) + + // Make sure to zero them before we compare, but after we have + // encoded the data testIndexInfo.SnapshotTime = 0 testIndexInfo.FileType = 0 defer func() { - testIndexInfo.SnapshotTime = oldSnapshotTime - testIndexInfo.FileType = oldFileType + testIndexInfo.SnapshotTime = currSnapshotTime + testIndexInfo.FileType = currFileType }() - enc.EncodeIndexInfo(testIndexInfo) dec.Reset(NewDecoderStream(enc.Bytes())) res, err := dec.DecodeIndexInfo() require.NoError(t, err) @@ -160,8 +158,8 @@ func TestIndexInfoRoundTripForwardsCompatibilityV2(t *testing.T) { func TestIndexEntryRoundtrip(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) ) require.NoError(t, enc.EncodeIndexEntry(testIndexEntry)) dec.Reset(NewDecoderStream(enc.Bytes())) @@ -173,18 +171,19 @@ func TestIndexEntryRoundtrip(t *testing.T) { // Make sure the new decoding code can handle the old file format func TestIndexEntryRoundTripBackwardsCompatibilityV1(t *testing.T) { var ( - enc = newEncoder(newEncoderOptions{encodeLegacyV1IndexEntry: true}) - dec = testDecoder(t, nil) + opts = legacyEncodingOptions{encodeLegacyV1IndexEntry: true} + enc = newEncoder(opts) + dec = newDecoder(opts, nil) ) // Set the default values on the fields that did not exist in V1 // and then restore them at the end of the test - This is required // because the new decoder won't try and read the new fields from // the old file format - oldEncodedTags := testIndexEntry.EncodedTags + currEncodedTags := testIndexEntry.EncodedTags testIndexEntry.EncodedTags = nil defer func() { - testIndexEntry.EncodedTags = oldEncodedTags + testIndexEntry.EncodedTags = currEncodedTags }() enc.EncodeIndexEntry(testIndexEntry) @@ -197,20 +196,25 @@ func TestIndexEntryRoundTripBackwardsCompatibilityV1(t *testing.T) { // Make sure the old decoder code can handle the new file format func TestIndexEntryRoundTripForwardsCompatibilityV2(t *testing.T) { var ( - enc = newEncoder(newEncoderOptions{encodeLegacyV1IndexEntry: false}) - dec = testDecoder(t, nil) + opts = legacyEncodingOptions{decodeLegacyV1IndexEntry: true} + enc = newEncoder(opts) + dec = newDecoder(opts, nil) ) // Set the default values on the fields that did not exist in V1 // and then restore them at the end of the test - This is required // because the old decoder won't read the new fields - oldEncodedTags := testIndexEntry.EncodedTags + currEncodedTags := testIndexEntry.EncodedTags + + enc.EncodeIndexEntry(testIndexEntry) + + // Make sure to zero them before we compare, but after we have + // encoded the data testIndexEntry.EncodedTags = nil defer func() { - testIndexEntry.EncodedTags = oldEncodedTags + testIndexEntry.EncodedTags = currEncodedTags }() - enc.EncodeIndexEntry(testIndexEntry) dec.Reset(NewDecoderStream(enc.Bytes())) res, err := dec.DecodeIndexEntry() require.NoError(t, err) @@ -219,8 +223,8 @@ func TestIndexEntryRoundTripForwardsCompatibilityV2(t *testing.T) { func TestIndexSummaryRoundtrip(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) ) require.NoError(t, enc.EncodeIndexSummary(testIndexSummary)) dec.Reset(NewDecoderStream(enc.Bytes())) @@ -231,8 +235,8 @@ func TestIndexSummaryRoundtrip(t *testing.T) { func TestLogInfoRoundtrip(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) ) require.NoError(t, enc.EncodeLogInfo(testLogInfo)) dec.Reset(NewDecoderStream(enc.Bytes())) @@ -243,8 +247,8 @@ func TestLogInfoRoundtrip(t *testing.T) { func TestLogEntryRoundtrip(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) ) require.NoError(t, enc.EncodeLogEntry(testLogEntry)) dec.Reset(NewDecoderStream(enc.Bytes())) @@ -279,8 +283,8 @@ func BenchmarkLogEntryDecoder(b *testing.B) { func TestLogEntryRoundtripUniqueIndexAndRemaining(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) ) require.NoError(t, enc.EncodeLogEntry(testLogEntry)) dec.Reset(NewDecoderStream(enc.Bytes())) @@ -296,8 +300,8 @@ func TestLogEntryRoundtripUniqueIndexAndRemaining(t *testing.T) { func TestLogMetadataRoundtrip(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) ) require.NoError(t, enc.EncodeLogMetadata(testLogMetadata)) dec.Reset(NewDecoderStream(enc.Bytes())) @@ -308,8 +312,8 @@ func TestLogMetadataRoundtrip(t *testing.T) { func TestMultiTypeRoundtripStress(t *testing.T) { var ( - enc = testEncoder(t) - dec = testDecoder(t, nil) + enc = NewEncoder() + dec = NewDecoder(nil) iter = 10000 res interface{} err error From da4bcb628901715fea1d3e7054ef6fd8e701df23 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Tue, 8 May 2018 14:50:28 -0400 Subject: [PATCH 23/23] Fix lint --- storage/bootstrap/bootstrapper/fs/source.go | 9 ++++++--- storage/bootstrap/result/result_index.go | 2 +- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/storage/bootstrap/bootstrapper/fs/source.go b/storage/bootstrap/bootstrapper/fs/source.go index b914636ba0..5751434df5 100644 --- a/storage/bootstrap/bootstrapper/fs/source.go +++ b/storage/bootstrap/bootstrapper/fs/source.go @@ -385,8 +385,11 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( if shardRetrieverMgr != nil { shardRetriever = shardRetrieverMgr.ShardRetriever(shard) } - if seriesCachePolicy == series.CacheAllMetadata && shardRetriever == nil { - s.log.Errorf("shard retriever missing for shard: %d", shard) + if run == bootstrapDataRunType && seriesCachePolicy == series.CacheAllMetadata && shardRetriever == nil { + s.log.WithFields( + xlog.NewField("has-shard-retriever-mgr", shardRetrieverMgr != nil), + xlog.NewField("has-shard-retriever", shardRetriever != nil), + ).Errorf("shard retriever missing for shard: %d", shard) s.handleErrorsAndUnfulfilled(runResult, shard, tr, timesWithErrors) return } @@ -653,7 +656,7 @@ func (s *fileSystemSource) read( s.log.WithFields( xlog.NewField("shards", len(shardsTimeRanges)), xlog.NewField("concurrency", s.opts.NumProcessors()), - xlog.NewField("metadataOnly", blockRetriever != nil), + xlog.NewField("metadata-only", blockRetriever != nil), ).Infof("filesystem bootstrapper bootstrapping shards for ranges") bytesPool := s.opts.ResultOptions().DatabaseBlockOptions().BytesPool() diff --git a/storage/bootstrap/result/result_index.go b/storage/bootstrap/result/result_index.go index 67c31f438e..9c1f86a131 100644 --- a/storage/bootstrap/result/result_index.go +++ b/storage/bootstrap/result/result_index.go @@ -85,7 +85,7 @@ func (r IndexResults) GetOrAddSegment( idxopts namespace.IndexOptions, opts Options, ) (segment.MutableSegment, error) { - // NB(r): The reason we can align by the retention block size and guarentee + // NB(r): The reason we can align by the retention block size and guarantee // there is only one entry for this time is because index blocks must be a // positive multiple of the data block size, making it easy to map a data // block entry to at most one index block entry.