From 7d2dc739ce33176246a0a1fd896f036a122d9915 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 22 Jul 2019 09:39:15 -0400 Subject: [PATCH 01/22] Add shard.Load() API --- src/dbnode/storage/shard.go | 108 ++++++++++++++++++++++-------------- 1 file changed, 66 insertions(+), 42 deletions(-) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 67457822d1..823ab625e4 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -1788,14 +1788,67 @@ func (s *dbShard) Bootstrap( // (will be passed to series.Bootstrap() as BlockState). s.bootstrapFlushStates() + multiErr := xerrors.NewMultiError() + bootstrapResult, err := s.loadSeries(bootstrappedSeries, true) + if err != nil { + multiErr = multiErr.Add(err) + } + s.emitBootstrapResult(bootstrapResult) + + // From this point onwards, all newly created series that aren't in + // the existing map should be considered bootstrapped because they + // have no data within the retention period. + s.Lock() + s.newSeriesBootstrapped = true + s.Unlock() + + // Find the series with no data within the retention period but has + // buffered data points since server start. Any new series added + // after this will be marked as bootstrapped. + s.forEachShardEntry(func(entry *lookup.Entry) bool { + series := entry.Series + if series.IsBootstrapped() { + return true + } + _, err := series.Bootstrap(nil, nil) + multiErr = multiErr.Add(err) + return true + }) + + // Now that this shard has finished bootstrapping, attempt to cache all of its seekers. Cannot call + // this earlier as block lease verification will fail due to the shards not being bootstrapped + // (and as a result no leases can be verified since the flush state is not yet known). + if err := s.cacheShardIndices(); err != nil { + multiErr = multiErr.Add(err) + } + + s.Lock() + s.bootstrapState = Bootstrapped + s.Unlock() + + return multiErr.FinalError() +} + +func (s *dbShard) Load( + series *result.Map, +) error { + _, err := s.loadSeries(series, false) + return err +} + +func (s *dbShard) loadSeries( + series *result.Map, + bootstrap bool, +) (dbShardBootstrapResult, error) { var ( + // Only used for the bootstrap path. shardBootstrapResult = dbShardBootstrapResult{} multiErr = xerrors.NewMultiError() // Safe to use the same snapshot for all the series since the block states can't change while // this is running since no warm/cold flushes can occur while the bootstrap is ongoing. blockStates = s.BlockStatesSnapshot() ) - for _, elem := range bootstrappedSeries.Iter() { + for _, elem := range series.Iter() { dbBlocks := elem.Value() // First lookup if series already exists @@ -1805,8 +1858,8 @@ func (s *dbShard) Bootstrap( continue } if entry == nil { - // Synchronously insert to avoid waiting for - // the insert queue potential delayed insert + // Synchronously insert to avoid waiting for the insert queue which could potentially + // delay the insert. entry, err = s.insertSeriesSync(dbBlocks.ID, newTagsArg(dbBlocks.Tags), insertSyncIncReaderWriterCount) if err != nil { @@ -1824,51 +1877,22 @@ func (s *dbShard) Bootstrap( dbBlocks.Tags.Finalize() } - // Cannot close blocks once done as series takes ref to these - bsResult, err := entry.Series.Bootstrap(dbBlocks.Blocks, blockStates) - if err != nil { - multiErr = multiErr.Add(err) + if bootstrap { + bsResult, err := entry.Series.Bootstrap(dbBlocks.Blocks, blockStates) + if err != nil { + multiErr = multiErr.Add(err) + } + shardBootstrapResult.update(bsResult) + } else { + entry.Series.Load(dbBlocks.Blocks, blockStates) } - shardBootstrapResult.update(bsResult) + // Cannot close blocks once done as series takes ref to them. // Always decrement the writer count, avoid continue on bootstrap error entry.DecrementReaderWriterCount() } - s.emitBootstrapResult(shardBootstrapResult) - - // From this point onwards, all newly created series that aren't in - // the existing map should be considered bootstrapped because they - // have no data within the retention period. - s.Lock() - s.newSeriesBootstrapped = true - s.Unlock() - - // Find the series with no data within the retention period but has - // buffered data points since server start. Any new series added - // after this will be marked as bootstrapped. - s.forEachShardEntry(func(entry *lookup.Entry) bool { - series := entry.Series - if series.IsBootstrapped() { - return true - } - _, err := series.Bootstrap(nil, nil) - multiErr = multiErr.Add(err) - return true - }) - - // Now that this shard has finished bootstrapping, attempt to cache all of its seekers. Cannot call - // this earlier as block lease verification will fail due to the shards not being bootstrapped - // (and as a result no leases can be verified since the flush state is not yet known). - if err := s.cacheShardIndices(); err != nil { - multiErr = multiErr.Add(err) - } - - s.Lock() - s.bootstrapState = Bootstrapped - s.Unlock() - - return multiErr.FinalError() + return shardBootstrapResult, multiErr.FinalError() } func (s *dbShard) bootstrapFlushStates() { From 6bdfc56e745cf94c07c987da3a9f22b6876b6910 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 22 Jul 2019 09:42:14 -0400 Subject: [PATCH 02/22] Add shard.Load() to shard interface --- src/dbnode/storage/types.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 6f1396171e..c7d3c7ecf4 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -466,6 +466,12 @@ type databaseShard interface { bootstrappedSeries *result.Map, ) error + // Load does the same thing as Bootstrap, except it can be called more than once + // and after a shard is bootstrapped already. + Load( + series *result.Map, + ) error + // WarmFlush flushes the WarmWrites in this shard. WarmFlush( blockStart time.Time, From 1dd069c040dead8077a26c98aef019cf70e39507 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 22 Jul 2019 09:49:52 -0400 Subject: [PATCH 03/22] Add guards against using flushState before its bootstrapped --- src/dbnode/storage/shard.go | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 823ab625e4..f885db93cf 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -72,6 +72,8 @@ var ( errShardInvalidPageToken = errors.New("shard could not unmarshal page token") errNewShardEntryTagsTypeInvalid = errors.New("new shard entry options error: tags type invalid") errNewShardEntryTagsIterNotAtIndexZero = errors.New("new shard entry options error: tags iter not at index zero") + errShardIsNotBootstrapped = errors.New("shard is not bootstrapped") + errFlushStateIsNotBootstrapped = errors.New("flush state is not bootstrapped") ) type filesetsFn func( @@ -232,6 +234,7 @@ type shardListElement *list.Element type shardFlushState struct { sync.RWMutex statesByTime map[xtime.UnixNano]fileOpState + bootstrapped bool } func newShardFlushState() shardFlushState { @@ -382,6 +385,10 @@ func (s *dbShard) BlockStatesSnapshot() map[xtime.UnixNano]series.BlockState { s.flushState.RLock() defer s.flushState.RUnlock() + if !s.flushState.bootstrapped { + return errFlushStateIsNotBootstrapped + } + states := s.flushState.statesByTime snapshot := make(map[xtime.UnixNano]series.BlockState, len(states)) for time, state := range states { @@ -1773,6 +1780,7 @@ func (s *dbShard) Bootstrap( ) error { s.Lock() if s.bootstrapState == Bootstrapped { + // TODO(rartoul): This should return an error instead of failing silently. s.Unlock() return nil } @@ -1832,6 +1840,15 @@ func (s *dbShard) Bootstrap( func (s *dbShard) Load( series *result.Map, ) error { + s.Lock() + // Don't allow loads until the shard is bootstrapped because the shard flush states need to be + // bootstrapped in order to safely load blocks. This also keeps things simpler to reason about. + if !s.bootstrapState == Bootstrapped { + s.Unlock() + return errShardIsNotBootstrapped + } + s.Unlock() + _, err := s.loadSeries(series, false) return err } @@ -1896,6 +1913,12 @@ func (s *dbShard) loadSeries( } func (s *dbShard) bootstrapFlushStates() { + s.FlushState.Lock() + defer func() { + s.flushState.bootstrapped = true + s.flushState.Unlock() + }() + fsOpts := s.opts.CommitLogOptions().FilesystemOptions() readInfoFilesResults := fs.ReadInfoFiles(fsOpts.FilePathPrefix(), s.namespace.ID(), s.shard, fsOpts.InfoReaderBufferSize(), fsOpts.DecodingOptions()) From f8318f60d06ccbaea20da8a29e960936bf9d3135 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 22 Jul 2019 10:06:55 -0400 Subject: [PATCH 04/22] make BlockStateSnapshot() check flushState bootstrap status --- src/dbnode/storage/series/series_mock.go | 5 +-- src/dbnode/storage/series/types.go | 2 +- src/dbnode/storage/shard.go | 39 ++++++++++++++---------- 3 files changed, 27 insertions(+), 19 deletions(-) diff --git a/src/dbnode/storage/series/series_mock.go b/src/dbnode/storage/series/series_mock.go index 166981075e..c38cfa9cd7 100644 --- a/src/dbnode/storage/series/series_mock.go +++ b/src/dbnode/storage/series/series_mock.go @@ -365,11 +365,12 @@ func (m *MockQueryableBlockRetriever) EXPECT() *MockQueryableBlockRetrieverMockR } // BlockStatesSnapshot mocks base method -func (m *MockQueryableBlockRetriever) BlockStatesSnapshot() map[time0.UnixNano]BlockState { +func (m *MockQueryableBlockRetriever) BlockStatesSnapshot() (map[time0.UnixNano]BlockState, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BlockStatesSnapshot") ret0, _ := ret[0].(map[time0.UnixNano]BlockState) - return ret0 + ret1, _ := ret[1].(error) + return ret0, ret1 } // BlockStatesSnapshot indicates an expected call of BlockStatesSnapshot diff --git a/src/dbnode/storage/series/types.go b/src/dbnode/storage/series/types.go index cf840cac5d..faa5473d1b 100644 --- a/src/dbnode/storage/series/types.go +++ b/src/dbnode/storage/series/types.go @@ -173,7 +173,7 @@ type QueryableBlockRetriever interface { // Flushes may occur and change the actual block state while iterating // through this snapshot, so any logic using this function should take this // into account. - BlockStatesSnapshot() map[xtime.UnixNano]BlockState + BlockStatesSnapshot() (map[xtime.UnixNano]BlockState, error) } // BlockState contains the state of a block. diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index f885db93cf..51f0dfdf28 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -381,12 +381,13 @@ func (s *dbShard) RetrievableBlockColdVersion(blockStart time.Time) int { } // BlockStatesSnapshot implements series.QueryableBlockRetriever -func (s *dbShard) BlockStatesSnapshot() map[xtime.UnixNano]series.BlockState { +func (s *dbShard) BlockStatesSnapshot() (map[xtime.UnixNano]series.BlockState, error) { s.flushState.RLock() defer s.flushState.RUnlock() if !s.flushState.bootstrapped { - return errFlushStateIsNotBootstrapped + // Safeguard against attempting to use the flush state before it has been bootstrapped. + return nil, errFlushStateIsNotBootstrapped } states := s.flushState.statesByTime @@ -398,7 +399,7 @@ func (s *dbShard) BlockStatesSnapshot() map[xtime.UnixNano]series.BlockState { } } - return snapshot + return snapshot, nil } func (s *dbShard) OnRetrieveBlock( @@ -672,8 +673,11 @@ func (s *dbShard) tickAndExpire( tickSleepPerSeries := s.currRuntimeOptions.tickSleepPerSeries // Acquire snapshot of block states here to avoid releasing the // RLock and acquiring it right after. - blockStates := s.BlockStatesSnapshot() + blockStates, err := s.BlockStatesSnapshot() s.RUnlock() + if err != nil { + return tickResult{}, err + } s.forEachShardEntryBatch(func(currEntries []*lookup.Entry) bool { // re-using `expired` to amortize allocs, still need to reset it // to be safe for re-use. @@ -1843,7 +1847,7 @@ func (s *dbShard) Load( s.Lock() // Don't allow loads until the shard is bootstrapped because the shard flush states need to be // bootstrapped in order to safely load blocks. This also keeps things simpler to reason about. - if !s.bootstrapState == Bootstrapped { + if s.bootstrapState != Bootstrapped { s.Unlock() return errShardIsNotBootstrapped } @@ -1861,10 +1865,13 @@ func (s *dbShard) loadSeries( // Only used for the bootstrap path. shardBootstrapResult = dbShardBootstrapResult{} multiErr = xerrors.NewMultiError() - // Safe to use the same snapshot for all the series since the block states can't change while - // this is running since no warm/cold flushes can occur while the bootstrap is ongoing. - blockStates = s.BlockStatesSnapshot() ) + // Safe to use the same snapshot for all the series since the block states can't change while + // this is running since no warm/cold flushes can occur while the bootstrap is ongoing. + blockStates, err := s.BlockStatesSnapshot() + if err != nil { + return dbShardBootstrapResult{}, err + } for _, elem := range series.Iter() { dbBlocks := elem.Value() @@ -1913,12 +1920,6 @@ func (s *dbShard) loadSeries( } func (s *dbShard) bootstrapFlushStates() { - s.FlushState.Lock() - defer func() { - s.flushState.bootstrapped = true - s.flushState.Unlock() - }() - fsOpts := s.opts.CommitLogOptions().FilesystemOptions() readInfoFilesResults := fs.ReadInfoFiles(fsOpts.FilePathPrefix(), s.namespace.ID(), s.shard, fsOpts.InfoReaderBufferSize(), fsOpts.DecodingOptions()) @@ -2064,7 +2065,10 @@ func (s *dbShard) ColdFlush( idElementPool = resources.idElementPool ) - blockStates := s.BlockStatesSnapshot() + blockStates, err := s.BlockStatesSnapshot() + if err != nil { + return err + } // First, loop through all series to capture data on which blocks have dirty // series and add them to the resources for further processing. s.forEachShardEntry(func(entry *lookup.Entry) bool { @@ -2305,7 +2309,10 @@ func (s *dbShard) CleanupCompactedFileSets() error { // Get a snapshot of all states here to prevent constantly getting/releasing // locks in a tight loop below. This snapshot won't become stale halfway // through this because flushing and cleanup never happen in parallel. - blockStates := s.BlockStatesSnapshot() + blockStates, err := s.BlockStatesSnapshot() + if err != nil { + return err + } toDelete := fs.FileSetFilesSlice(make([]fs.FileSetFile, 0, len(filesets))) for _, datafile := range filesets { fileID := datafile.ID From 4d500c8be60600b52f69d51b457d8c04080a7238 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 22 Jul 2019 21:09:25 -0400 Subject: [PATCH 05/22] fix broken tests --- src/dbnode/storage/cleanup_test.go | 10 +-- src/dbnode/storage/flush.go | 46 ++++++++--- src/dbnode/storage/flush_test.go | 25 +++--- src/dbnode/storage/fs_merge_with_mem.go | 6 +- src/dbnode/storage/fs_merge_with_mem_test.go | 4 +- src/dbnode/storage/index.go | 20 +++-- src/dbnode/storage/index_test.go | 16 ++-- src/dbnode/storage/namespace.go | 32 ++++++-- src/dbnode/storage/namespace_test.go | 80 +++++++++++-------- src/dbnode/storage/series/reader.go | 18 ++++- src/dbnode/storage/series/reader_test.go | 12 +-- src/dbnode/storage/series/series_mock.go | 10 ++- src/dbnode/storage/series/types.go | 4 +- src/dbnode/storage/shard.go | 81 ++++++++++++++++---- src/dbnode/storage/shard_test.go | 45 +++++++---- src/dbnode/storage/types.go | 4 +- 16 files changed, 287 insertions(+), 126 deletions(-) diff --git a/src/dbnode/storage/cleanup_test.go b/src/dbnode/storage/cleanup_test.go index cdecdf9008..3557a7bf10 100644 --- a/src/dbnode/storage/cleanup_test.go +++ b/src/dbnode/storage/cleanup_test.go @@ -295,7 +295,7 @@ func TestCleanupManagerCleanupCommitlogsAndSnapshots(t *testing.T) { ns := NewMockdatabaseNamespace(ctrl) ns.EXPECT().ID().Return(ident.StringID(fmt.Sprintf("ns%d", i))).AnyTimes() ns.EXPECT().Options().Return(nsOpts).AnyTimes() - ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(false).AnyTimes() + ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(false, nil).AnyTimes() ns.EXPECT().GetOwnedShards().Return(shards).AnyTimes() namespaces = append(namespaces, ns) } @@ -347,7 +347,7 @@ func TestCleanupManagerNamespaceCleanup(t *testing.T) { ns := NewMockdatabaseNamespace(ctrl) ns.EXPECT().ID().Return(ident.StringID("ns")).AnyTimes() ns.EXPECT().Options().Return(nsOpts).AnyTimes() - ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(false).AnyTimes() + ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(false, nil).AnyTimes() ns.EXPECT().GetOwnedShards().Return(nil).AnyTimes() idx := NewMocknamespaceIndex(ctrl) @@ -376,7 +376,7 @@ func TestCleanupManagerDoesntNeedCleanup(t *testing.T) { for range namespaces { ns := NewMockdatabaseNamespace(ctrl) ns.EXPECT().Options().Return(nsOpts).AnyTimes() - ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(false).AnyTimes() + ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(false, nil).AnyTimes() namespaces = append(namespaces, ns) } db := newMockdatabase(ctrl, namespaces...) @@ -411,7 +411,7 @@ func TestCleanupDataAndSnapshotFileSetFiles(t *testing.T) { shard.EXPECT().ID().Return(uint32(0)).AnyTimes() ns.EXPECT().GetOwnedShards().Return([]databaseShard{shard}).AnyTimes() ns.EXPECT().ID().Return(ident.StringID("nsID")).AnyTimes() - ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(false).AnyTimes() + ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(false, nil).AnyTimes() namespaces := []databaseNamespace{ns} db := newMockdatabase(ctrl, namespaces...) @@ -440,7 +440,7 @@ func TestDeleteInactiveDataAndSnapshotFileSetFiles(t *testing.T) { shard.EXPECT().ID().Return(uint32(0)).AnyTimes() ns.EXPECT().GetOwnedShards().Return([]databaseShard{shard}).AnyTimes() ns.EXPECT().ID().Return(ident.StringID("nsID")).AnyTimes() - ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(false).AnyTimes() + ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(false, nil).AnyTimes() namespaces := []databaseNamespace{ns} db := newMockdatabase(ctrl, namespaces...) diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index f43f2ec4f2..1e38265ea1 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -168,7 +168,12 @@ func (m *flushManager) dataWarmFlush( multiErr := xerrors.NewMultiError() for _, ns := range namespaces { // Flush first because we will only snapshot if there are no outstanding flushes - flushTimes := m.namespaceFlushTimes(ns, tickStart) + flushTimes, err := m.namespaceFlushTimes(ns, tickStart) + if err != nil { + multiErr = multiErr.Add(fmt.Errorf( + "error determining namespace flush times for ns: %s, err: %v", ns.ID().String(), err)) + continue + } shardBootstrapTimes, ok := dbBootstrapStateAtTickStart.NamespaceBootstrapStates[ns.ID().String()] if !ok { // Could happen if namespaces are added / removed. @@ -234,7 +239,14 @@ func (m *flushManager) dataSnapshot( multiErr = xerrors.NewMultiError() ) for _, ns := range namespaces { - snapshotBlockStarts := m.namespaceSnapshotTimes(ns, tickStart) + snapshotBlockStarts, err := m.namespaceSnapshotTimes(ns, tickStart) + if err != nil { + detailedErr := fmt.Errorf( + "namespace %s failed to determine snapshot times: %v", + ns.ID().String(), err) + multiErr = multiErr.Add(detailedErr) + continue + } if len(snapshotBlockStarts) > maxBlocksSnapshottedByNamespace { maxBlocksSnapshottedByNamespace = len(snapshotBlockStarts) @@ -244,9 +256,11 @@ func (m *flushManager) dataSnapshot( snapshotBlockStart, tickStart, snapshotPersist) if err != nil { - detailedErr := fmt.Errorf("namespace %s failed to snapshot data: %v", - ns.ID().String(), err) + detailedErr := fmt.Errorf( + "namespace %s failed to snapshot data for blockStart %s: %v", + ns.ID().String(), snapshotBlockStart.String(), err) multiErr = multiErr.Add(detailedErr) + continue } } } @@ -327,7 +341,7 @@ func (m *flushManager) flushRange(rOpts retention.Options, t time.Time) (time.Ti return retention.FlushTimeStart(rOpts, t), retention.FlushTimeEnd(rOpts, t) } -func (m *flushManager) namespaceFlushTimes(ns databaseNamespace, curr time.Time) []time.Time { +func (m *flushManager) namespaceFlushTimes(ns databaseNamespace, curr time.Time) ([]time.Time, error) { var ( rOpts = ns.Options().RetentionOptions() blockSize = rOpts.BlockSize() @@ -335,12 +349,18 @@ func (m *flushManager) namespaceFlushTimes(ns databaseNamespace, curr time.Time) ) candidateTimes := timesInRange(earliest, latest, blockSize) + var loopErr error return filterTimes(candidateTimes, func(t time.Time) bool { - return ns.NeedsFlush(t, t) - }) + needsFlush, err := ns.NeedsFlush(t, t) + if err != nil { + loopErr = err + return false + } + return needsFlush + }), loopErr } -func (m *flushManager) namespaceSnapshotTimes(ns databaseNamespace, curr time.Time) []time.Time { +func (m *flushManager) namespaceSnapshotTimes(ns databaseNamespace, curr time.Time) ([]time.Time, error) { var ( rOpts = ns.Options().RetentionOptions() blockSize = rOpts.BlockSize() @@ -356,10 +376,16 @@ func (m *flushManager) namespaceSnapshotTimes(ns databaseNamespace, curr time.Ti ) candidateTimes := timesInRange(earliest, latest, blockSize) + var loopErr error return filterTimes(candidateTimes, func(t time.Time) bool { // Snapshot anything that is unflushed. - return ns.NeedsFlush(t, t) - }) + needsFlush, err := ns.NeedsFlush(t, t) + if err != nil { + loopErr = err + return false + } + return needsFlush + }), loopErr } // flushWithTime flushes in-memory data for a given namespace, at a given diff --git a/src/dbnode/storage/flush_test.go b/src/dbnode/storage/flush_test.go index ba61e144af..064e04cd0b 100644 --- a/src/dbnode/storage/flush_test.go +++ b/src/dbnode/storage/flush_test.go @@ -280,7 +280,7 @@ func TestFlushManagerSkipNamespaceIndexingDisabled(t *testing.T) { ns := NewMockdatabaseNamespace(ctrl) ns.EXPECT().Options().Return(nsOpts).AnyTimes() ns.EXPECT().ID().Return(defaultTestNs1ID).AnyTimes() - ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(true).AnyTimes() + ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(true, nil).AnyTimes() ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() ns.EXPECT().ColdFlush(gomock.Any()).Return(nil).AnyTimes() ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() @@ -329,7 +329,7 @@ func TestFlushManagerNamespaceIndexingEnabled(t *testing.T) { ns := NewMockdatabaseNamespace(ctrl) ns.EXPECT().Options().Return(nsOpts).AnyTimes() ns.EXPECT().ID().Return(defaultTestNs1ID).AnyTimes() - ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(true).AnyTimes() + ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(true, nil).AnyTimes() ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() ns.EXPECT().ColdFlush(gomock.Any()).Return(nil).AnyTimes() ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() @@ -418,8 +418,9 @@ func TestFlushManagerNamespaceFlushTimesNoNeedFlush(t *testing.T) { fm, ns1, _, _ := newMultipleFlushManagerNeedsFlush(t, ctrl) now := time.Now() - ns1.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(false).AnyTimes() - flushTimes := fm.namespaceFlushTimes(ns1, now) + ns1.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(false, nil).AnyTimes() + flushTimes, err := fm.namespaceFlushTimes(ns1, now) + require.NoError(t, err) require.Empty(t, flushTimes) } @@ -430,8 +431,9 @@ func TestFlushManagerNamespaceFlushTimesAllNeedFlush(t *testing.T) { fm, ns1, _, _ := newMultipleFlushManagerNeedsFlush(t, ctrl) now := time.Now() - ns1.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(true).AnyTimes() - times := fm.namespaceFlushTimes(ns1, now) + ns1.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(true, nil).AnyTimes() + times, err := fm.namespaceFlushTimes(ns1, now) + require.NoError(t, err) sort.Sort(timesInOrder(times)) blockSize := ns1.Options().RetentionOptions().BlockSize() @@ -462,15 +464,16 @@ func TestFlushManagerNamespaceFlushTimesSomeNeedFlush(t *testing.T) { // skip 1/3 of input if i%3 == 0 { - ns1.EXPECT().NeedsFlush(st, st).Return(false) + ns1.EXPECT().NeedsFlush(st, st).Return(false, nil) continue } - ns1.EXPECT().NeedsFlush(st, st).Return(true) + ns1.EXPECT().NeedsFlush(st, st).Return(true, nil) expectedTimes = append(expectedTimes, st) } - times := fm.namespaceFlushTimes(ns1, now) + times, err := fm.namespaceFlushTimes(ns1, now) + require.NoError(t, err) require.NotEmpty(t, times) sort.Sort(timesInOrder(times)) require.Equal(t, expectedTimes, times) @@ -494,7 +497,7 @@ func TestFlushManagerFlushSnapshot(t *testing.T) { for i := 0; i < num; i++ { st := start.Add(time.Duration(i) * blockSize) - ns.EXPECT().NeedsFlush(st, st).Return(false) + ns.EXPECT().NeedsFlush(st, st).Return(false, nil) } ns.EXPECT().ColdFlush(gomock.Any()) @@ -503,7 +506,7 @@ func TestFlushManagerFlushSnapshot(t *testing.T) { num = numIntervals(start, snapshotEnd, blockSize) for i := 0; i < num; i++ { st := start.Add(time.Duration(i) * blockSize) - ns.EXPECT().NeedsFlush(st, st).Return(true) + ns.EXPECT().NeedsFlush(st, st).Return(true, nil) ns.EXPECT().Snapshot(st, now, gomock.Any()) } } diff --git a/src/dbnode/storage/fs_merge_with_mem.go b/src/dbnode/storage/fs_merge_with_mem.go index a384b71d50..7d64d82d57 100644 --- a/src/dbnode/storage/fs_merge_with_mem.go +++ b/src/dbnode/storage/fs_merge_with_mem.go @@ -85,7 +85,11 @@ func (m *fsMergeWithMem) fetchBlocks( nsCtx namespace.Context, ) ([]xio.BlockReader, bool, error) { startTime := blockStart.ToTime() - nextVersion := m.retriever.RetrievableBlockColdVersion(startTime) + 1 + currVersion, err := m.retriever.RetrievableBlockColdVersion(startTime) + if err != nil { + return nil, false, err + } + nextVersion := currVersion + 1 blocks, err := m.shard.FetchBlocksForColdFlush(ctx, id, startTime, nextVersion, nsCtx) if err != nil { diff --git a/src/dbnode/storage/fs_merge_with_mem_test.go b/src/dbnode/storage/fs_merge_with_mem_test.go index 7deab20778..698d285f36 100644 --- a/src/dbnode/storage/fs_merge_with_mem_test.go +++ b/src/dbnode/storage/fs_merge_with_mem_test.go @@ -51,7 +51,7 @@ func TestRead(t *testing.T) { ctx := context.NewContext() nsCtx := namespace.Context{} fetchedBlocks := []xio.BlockReader{xio.BlockReader{}} - retriever.EXPECT().RetrievableBlockColdVersion(gomock.Any()).Return(version).AnyTimes() + retriever.EXPECT().RetrievableBlockColdVersion(gomock.Any()).Return(version, nil).AnyTimes() dirtySeries := newDirtySeriesMap(dirtySeriesMapOptions{}) dirtySeriesToWrite := make(map[xtime.UnixNano]*idList) @@ -129,7 +129,7 @@ func TestForEachRemaining(t *testing.T) { ctx := context.NewContext() nsCtx := namespace.Context{} fetchedBlocks := []xio.BlockReader{xio.BlockReader{}} - retriever.EXPECT().RetrievableBlockColdVersion(gomock.Any()).Return(version).AnyTimes() + retriever.EXPECT().RetrievableBlockColdVersion(gomock.Any()).Return(version, nil).AnyTimes() dirtySeries := newDirtySeriesMap(dirtySeriesMapOptions{}) dirtySeriesToWrite := make(map[xtime.UnixNano]*idList) diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 6081c2e53e..10a39f1362 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -799,7 +799,11 @@ func (i *nsIndex) flushableBlocks( } flushable := make([]index.Block, 0, len(i.state.blocksByTime)) for _, block := range i.state.blocksByTime { - if !i.canFlushBlock(block, shards) { + canFlush, err := i.canFlushBlock(block, shards) + if err != nil { + return nil, err + } + if !canFlush { continue } flushable = append(flushable, block) @@ -810,11 +814,11 @@ func (i *nsIndex) flushableBlocks( func (i *nsIndex) canFlushBlock( block index.Block, shards []databaseShard, -) bool { +) (bool, error) { // Check the block needs flushing because it is sealed and has // any mutable segments that need to be evicted from memory if !block.IsSealed() || !block.NeedsMutableSegmentsEvicted() { - return false + return false, nil } // Check all data files exist for the shards we own @@ -822,13 +826,17 @@ func (i *nsIndex) canFlushBlock( start := block.StartTime() dataBlockSize := i.nsMetadata.Options().RetentionOptions().BlockSize() for t := start; t.Before(block.EndTime()); t = t.Add(dataBlockSize) { - if shard.FlushState(t).WarmStatus != fileOpSuccess { - return false + flushState, err := shard.FlushState(t) + if err != nil { + return false, err + } + if flushState.WarmStatus != fileOpSuccess { + return false, nil } } } - return true + return true, nil } func (i *nsIndex) flushBlock( diff --git a/src/dbnode/storage/index_test.go b/src/dbnode/storage/index_test.go index 0f7664d509..0d3b6a6dd6 100644 --- a/src/dbnode/storage/index_test.go +++ b/src/dbnode/storage/index_test.go @@ -118,8 +118,8 @@ func TestNamespaceIndexFlushSuccess(t *testing.T) { mockShard := NewMockdatabaseShard(ctrl) mockShard.EXPECT().ID().Return(uint32(0)).AnyTimes() - mockShard.EXPECT().FlushState(blockTime).Return(fileOpState{WarmStatus: fileOpSuccess}) - mockShard.EXPECT().FlushState(blockTime.Add(test.blockSize)).Return(fileOpState{WarmStatus: fileOpSuccess}) + mockShard.EXPECT().FlushState(blockTime).Return(fileOpState{WarmStatus: fileOpSuccess}, nil) + mockShard.EXPECT().FlushState(blockTime.Add(test.blockSize)).Return(fileOpState{WarmStatus: fileOpSuccess}, nil) shards := []databaseShard{mockShard} mockFlush := persist.NewMockIndexFlush(ctrl) @@ -185,8 +185,8 @@ func TestNamespaceIndexFlushShardStateNotSuccess(t *testing.T) { mockShard := NewMockdatabaseShard(ctrl) mockShard.EXPECT().ID().Return(uint32(0)).AnyTimes() - mockShard.EXPECT().FlushState(blockTime).Return(fileOpState{WarmStatus: fileOpSuccess}) - mockShard.EXPECT().FlushState(blockTime.Add(test.blockSize)).Return(fileOpState{WarmStatus: fileOpFailed}) + mockShard.EXPECT().FlushState(blockTime).Return(fileOpState{WarmStatus: fileOpSuccess}, nil) + mockShard.EXPECT().FlushState(blockTime.Add(test.blockSize)).Return(fileOpState{WarmStatus: fileOpFailed}, nil) shards := []databaseShard{mockShard} mockFlush := persist.NewMockIndexFlush(ctrl) @@ -220,13 +220,13 @@ func TestNamespaceIndexFlushSuccessMultipleShards(t *testing.T) { mockShard1 := NewMockdatabaseShard(ctrl) mockShard1.EXPECT().ID().Return(uint32(0)).AnyTimes() - mockShard1.EXPECT().FlushState(blockTime).Return(fileOpState{WarmStatus: fileOpSuccess}) - mockShard1.EXPECT().FlushState(blockTime.Add(test.blockSize)).Return(fileOpState{WarmStatus: fileOpSuccess}) + mockShard1.EXPECT().FlushState(blockTime).Return(fileOpState{WarmStatus: fileOpSuccess}, nil) + mockShard1.EXPECT().FlushState(blockTime.Add(test.blockSize)).Return(fileOpState{WarmStatus: fileOpSuccess}, nil) mockShard2 := NewMockdatabaseShard(ctrl) mockShard2.EXPECT().ID().Return(uint32(1)).AnyTimes() - mockShard2.EXPECT().FlushState(blockTime).Return(fileOpState{WarmStatus: fileOpSuccess}) - mockShard2.EXPECT().FlushState(blockTime.Add(test.blockSize)).Return(fileOpState{WarmStatus: fileOpSuccess}) + mockShard2.EXPECT().FlushState(blockTime).Return(fileOpState{WarmStatus: fileOpSuccess}, nil) + mockShard2.EXPECT().FlushState(blockTime.Add(test.blockSize)).Return(fileOpState{WarmStatus: fileOpSuccess}, nil) shards := []databaseShard{mockShard1, mockShard2} diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index 82d8c87002..71d7af2b4a 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -957,10 +957,15 @@ func (n *dbNamespace) WarmFlush( continue } + flushState, err := shard.FlushState(blockStart) + if err != nil { + return err + } // skip flushing if the shard has already flushed data for the `blockStart` - if s := shard.FlushState(blockStart); s.WarmStatus == fileOpSuccess { + if flushState.WarmStatus == fileOpSuccess { continue } + // NB(xichen): we still want to proceed if a shard fails to flush its data. // Probably want to emit a counter here, but for now just log it. if err := shard.WarmFlush(blockStart, flushPersist, nsCtx); err != nil { @@ -1142,7 +1147,9 @@ func (n *dbNamespace) Snapshot( } func (n *dbNamespace) NeedsFlush( - alignedInclusiveStart time.Time, alignedInclusiveEnd time.Time) bool { + alignedInclusiveStart time.Time, + alignedInclusiveEnd time.Time, +) (bool, error) { // NB(r): Essentially if all are success, we don't need to flush, if any // are failed with the minimum num failures less than max retries then // we need to flush - otherwise if any in progress we can't flush and if @@ -1152,7 +1159,10 @@ func (n *dbNamespace) NeedsFlush( return n.needsFlushWithLock(alignedInclusiveStart, alignedInclusiveEnd) } -func (n *dbNamespace) needsFlushWithLock(alignedInclusiveStart time.Time, alignedInclusiveEnd time.Time) bool { +func (n *dbNamespace) needsFlushWithLock( + alignedInclusiveStart time.Time, + alignedInclusiveEnd time.Time, +) (bool, error) { var ( blockSize = n.nopts.RetentionOptions().BlockSize() blockStarts = timesInRange(alignedInclusiveStart, alignedInclusiveEnd, blockSize) @@ -1167,14 +1177,18 @@ func (n *dbNamespace) needsFlushWithLock(alignedInclusiveStart time.Time, aligne continue } for _, blockStart := range blockStarts { - if shard.FlushState(blockStart).WarmStatus != fileOpSuccess { - return true + flushState, err := shard.FlushState(blockStart) + if err != nil { + return false, err + } + if flushState.WarmStatus != fileOpSuccess { + return true, nil } } } // All success or failed and reached max retries - return false + return false, nil } func (n *dbNamespace) Truncate() (int64, error) { @@ -1410,7 +1424,11 @@ func (n *dbNamespace) FlushState(shardID uint32, blockStart time.Time) (fileOpSt if err != nil { return fileOpState{}, err } - return shard.FlushState(blockStart), nil + flushState, err := shard.FlushState(blockStart) + if err != nil { + return fileOpState{}, err + } + return flushState, nil } func (n *dbNamespace) nsContextWithRLock() namespace.Context { diff --git a/src/dbnode/storage/namespace_test.go b/src/dbnode/storage/namespace_test.go index 1a6cc3c5d3..a90535e40c 100644 --- a/src/dbnode/storage/namespace_test.go +++ b/src/dbnode/storage/namespace_test.go @@ -455,7 +455,7 @@ func TestNamespaceFlushSkipFlushed(t *testing.T) { for i, s := range states { shard := NewMockdatabaseShard(ctrl) shard.EXPECT().ID().Return(testShardIDs[i].ID()) - shard.EXPECT().FlushState(blockStart).Return(s) + shard.EXPECT().FlushState(blockStart).Return(s, nil) if s.WarmStatus != fileOpSuccess { shard.EXPECT().WarmFlush(blockStart, gomock.Any(), gomock.Any()).Return(nil) } @@ -780,11 +780,11 @@ func setShardExpects(ns *dbNamespace, ctrl *gomock.Controller, cases []needsFlus if needFlush { shard.EXPECT().FlushState(t.ToTime()).Return(fileOpState{ WarmStatus: fileOpNotStarted, - }).AnyTimes() + }, nil).AnyTimes() } else { shard.EXPECT().FlushState(t.ToTime()).Return(fileOpState{ WarmStatus: fileOpSuccess, - }).AnyTimes() + }, nil).AnyTimes() } } ns.shards[cs.shardNum] = shard @@ -813,10 +813,11 @@ func TestNamespaceNeedsFlushRange(t *testing.T) { } setShardExpects(ns, ctrl, inputCases) - assert.False(t, ns.NeedsFlush(t0, t0)) - assert.True(t, ns.NeedsFlush(t0, t1)) - assert.True(t, ns.NeedsFlush(t1, t1)) - assert.False(t, ns.NeedsFlush(t1, t0)) + + assertNeedsFlush(t, ns, t0, t0, false) + assertNeedsFlush(t, ns, t0, t1, true) + assertNeedsFlush(t, ns, t1, t1, true) + assertNeedsFlush(t, ns, t1, t0, false) } func TestNamespaceNeedsFlushRangeMultipleShardConflict(t *testing.T) { @@ -843,14 +844,14 @@ func TestNamespaceNeedsFlushRangeMultipleShardConflict(t *testing.T) { } setShardExpects(ns, ctrl, inputCases) - assert.True(t, ns.NeedsFlush(t0, t0)) - assert.True(t, ns.NeedsFlush(t1, t1)) - assert.True(t, ns.NeedsFlush(t2, t2)) - assert.True(t, ns.NeedsFlush(t0, t1)) - assert.True(t, ns.NeedsFlush(t0, t2)) - assert.True(t, ns.NeedsFlush(t1, t2)) - assert.False(t, ns.NeedsFlush(t2, t1)) - assert.False(t, ns.NeedsFlush(t2, t0)) + assertNeedsFlush(t, ns, t0, t0, true) + assertNeedsFlush(t, ns, t1, t1, true) + assertNeedsFlush(t, ns, t2, t2, true) + assertNeedsFlush(t, ns, t0, t1, true) + assertNeedsFlush(t, ns, t0, t2, true) + assertNeedsFlush(t, ns, t1, t2, true) + assertNeedsFlush(t, ns, t2, t1, false) + assertNeedsFlush(t, ns, t2, t0, false) } func TestNamespaceNeedsFlushRangeSingleShardConflict(t *testing.T) { ctrl := gomock.NewController(t) @@ -876,14 +877,14 @@ func TestNamespaceNeedsFlushRangeSingleShardConflict(t *testing.T) { } setShardExpects(ns, ctrl, inputCases) - assert.True(t, ns.NeedsFlush(t0, t0)) - assert.False(t, ns.NeedsFlush(t1, t1)) - assert.True(t, ns.NeedsFlush(t2, t2)) - assert.True(t, ns.NeedsFlush(t0, t1)) - assert.True(t, ns.NeedsFlush(t0, t2)) - assert.True(t, ns.NeedsFlush(t1, t2)) - assert.False(t, ns.NeedsFlush(t2, t1)) - assert.False(t, ns.NeedsFlush(t2, t0)) + assertNeedsFlush(t, ns, t0, t0, true) + assertNeedsFlush(t, ns, t1, t1, false) + assertNeedsFlush(t, ns, t2, t2, true) + assertNeedsFlush(t, ns, t0, t1, true) + assertNeedsFlush(t, ns, t0, t2, true) + assertNeedsFlush(t, ns, t1, t2, true) + assertNeedsFlush(t, ns, t2, t1, false) + assertNeedsFlush(t, ns, t2, t0, false) } func TestNamespaceNeedsFlushAllSuccess(t *testing.T) { @@ -920,11 +921,11 @@ func TestNamespaceNeedsFlushAllSuccess(t *testing.T) { shard.EXPECT().ID().Return(s.ID()).AnyTimes() shard.EXPECT().FlushState(blockStart).Return(fileOpState{ WarmStatus: fileOpSuccess, - }).AnyTimes() + }, nil).AnyTimes() ns.shards[s.ID()] = shard } - assert.False(t, ns.NeedsFlush(blockStart, blockStart)) + assertNeedsFlush(t, ns, blockStart, blockStart, false) } func TestNamespaceNeedsFlushAnyFailed(t *testing.T) { @@ -962,21 +963,21 @@ func TestNamespaceNeedsFlushAnyFailed(t *testing.T) { case shards[0].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ WarmStatus: fileOpSuccess, - }).AnyTimes() + }, nil).AnyTimes() case shards[1].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ WarmStatus: fileOpSuccess, - }).AnyTimes() + }, nil).AnyTimes() case shards[2].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ WarmStatus: fileOpFailed, NumFailures: 999, - }).AnyTimes() + }, nil).AnyTimes() } ns.shards[s.ID()] = shard } - assert.True(t, ns.NeedsFlush(blockStart, blockStart)) + assertNeedsFlush(t, ns, blockStart, blockStart, true) } func TestNamespaceNeedsFlushAnyNotStarted(t *testing.T) { @@ -1014,20 +1015,20 @@ func TestNamespaceNeedsFlushAnyNotStarted(t *testing.T) { case shards[0].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ WarmStatus: fileOpSuccess, - }).AnyTimes() + }, nil).AnyTimes() case shards[1].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ WarmStatus: fileOpNotStarted, - }).AnyTimes() + }, nil).AnyTimes() case shards[2].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ WarmStatus: fileOpSuccess, - }).AnyTimes() + }, nil).AnyTimes() } ns.shards[s.ID()] = shard } - assert.True(t, ns.NeedsFlush(blockStart, blockStart)) + assertNeedsFlush(t, ns, blockStart, blockStart, true) } func TestNamespaceCloseWillCloseShard(t *testing.T) { @@ -1191,6 +1192,11 @@ func TestNamespaceTicksIndex(t *testing.T) { idx := NewMocknamespaceIndex(ctrl) ns, closer := newTestNamespaceWithIndex(t, idx) defer closer() + for _, s := range ns.shards { + if s != nil { + s.Bootstrap(nil) + } + } ctx := context.NewCancellable() idx.EXPECT().Tick(ctx, gomock.Any()).Return(namespaceIndexTickResult{}, nil) @@ -1251,7 +1257,7 @@ func TestNamespaceFlushState(t *testing.T) { } shard0 = NewMockdatabaseShard(ctrl) ) - shard0.EXPECT().FlushState(blockStart).Return(expectedFlushState) + shard0.EXPECT().FlushState(blockStart).Return(expectedFlushState, nil) ns.shards[0] = shard0 flushState, err := ns.FlushState(0, blockStart) @@ -1274,3 +1280,9 @@ func waitForStats( wg.Wait() } + +func assertNeedsFlush(t *testing.T, ns *dbNamespace, t0, t1 time.Time, assertTrue bool) { + needsFlush, err := ns.NeedsFlush(t1, t1) + require.NoError(t, err) + require.Equal(t, assertTrue, needsFlush) +} diff --git a/src/dbnode/storage/series/reader.go b/src/dbnode/storage/series/reader.go index 4850d3086f..7feaa532e4 100644 --- a/src/dbnode/storage/series/reader.go +++ b/src/dbnode/storage/series/reader.go @@ -167,7 +167,11 @@ func (r Reader) readersWithBlocksMapAndBuffer( // No-op, block metadata should have been in-memory case r.retriever != nil: // Try to stream from disk - if r.retriever.IsBlockRetrievable(blockAt) { + isRetrievable, err := r.retriever.IsBlockRetrievable(blockAt) + if err != nil { + return nil, err + } + if isRetrievable { streamedBlock, err := r.retriever.Stream(ctx, r.id, blockAt, r.onRetrieve, nsCtx) if err != nil { return nil, err @@ -272,7 +276,17 @@ func (r Reader) fetchBlocksWithBlocksMapAndBuffer( // No-op, block metadata should have been in-memory case r.retriever != nil: // Try to stream from disk - if r.retriever.IsBlockRetrievable(start) { + isRetrievable, err := r.retriever.IsBlockRetrievable(start) + if err != nil { + // Short-circuit this entire blockstart if an error was encountered. + r := block.NewFetchBlockResult(start, nil, + fmt.Errorf("unable to retrieve block stream for series %s time %v: %v", + r.id.String(), start, err)) + res = append(res, r) + continue + } + + if isRetrievable { streamedBlock, err := r.retriever.Stream(ctx, r.id, start, onRetrieve, nsCtx) if err != nil { // Short-circuit this entire blockstart if an error was encountered. diff --git a/src/dbnode/storage/series/reader_test.go b/src/dbnode/storage/series/reader_test.go index 98f3c8b3be..75ebb4602f 100644 --- a/src/dbnode/storage/series/reader_test.go +++ b/src/dbnode/storage/series/reader_test.go @@ -51,8 +51,8 @@ func TestReaderUsingRetrieverReadEncoded(t *testing.T) { onRetrieveBlock := block.NewMockOnRetrieveBlock(ctrl) retriever := NewMockQueryableBlockRetriever(ctrl) - retriever.EXPECT().IsBlockRetrievable(start).Return(true) - retriever.EXPECT().IsBlockRetrievable(start.Add(ropts.BlockSize())).Return(true) + retriever.EXPECT().IsBlockRetrievable(start).Return(true, nil) + retriever.EXPECT().IsBlockRetrievable(start.Add(ropts.BlockSize())).Return(true, nil) var blockReaders []xio.BlockReader for i := 0; i < 2; i++ { @@ -476,9 +476,9 @@ func TestReaderFetchBlocksRobust(t *testing.T) { diskCache.EXPECT().BlockAt(currTime).Return(nil, false) diskBlocks, ok := tc.diskBlocks[xtime.ToUnixNano(currTime)] if !ok { - retriever.EXPECT().IsBlockRetrievable(currTime).Return(false) + retriever.EXPECT().IsBlockRetrievable(currTime).Return(false, nil) } else { - retriever.EXPECT().IsBlockRetrievable(currTime).Return(true) + retriever.EXPECT().IsBlockRetrievable(currTime).Return(true, nil) if diskBlocks.err != nil { retriever.EXPECT(). Stream(ctx, ident.NewIDMatcher("foo"), currTime, nil, gomock.Any()). @@ -572,9 +572,9 @@ func TestReaderReadEncodedRobust(t *testing.T) { diskCache.EXPECT().BlockAt(currTime).Return(nil, false) diskBlocks, ok := tc.diskBlocks[xtime.ToUnixNano(currTime)] if !ok { - retriever.EXPECT().IsBlockRetrievable(currTime).Return(false) + retriever.EXPECT().IsBlockRetrievable(currTime).Return(false, nil) } else { - retriever.EXPECT().IsBlockRetrievable(currTime).Return(true) + retriever.EXPECT().IsBlockRetrievable(currTime).Return(true, nil) if diskBlocks.err != nil { retriever.EXPECT(). Stream(ctx, ident.NewIDMatcher("foo"), currTime, onRetrieveBlock, gomock.Any()). diff --git a/src/dbnode/storage/series/series_mock.go b/src/dbnode/storage/series/series_mock.go index c38cfa9cd7..7690bd5de8 100644 --- a/src/dbnode/storage/series/series_mock.go +++ b/src/dbnode/storage/series/series_mock.go @@ -380,11 +380,12 @@ func (mr *MockQueryableBlockRetrieverMockRecorder) BlockStatesSnapshot() *gomock } // IsBlockRetrievable mocks base method -func (m *MockQueryableBlockRetriever) IsBlockRetrievable(arg0 time.Time) bool { +func (m *MockQueryableBlockRetriever) IsBlockRetrievable(arg0 time.Time) (bool, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "IsBlockRetrievable", arg0) ret0, _ := ret[0].(bool) - return ret0 + ret1, _ := ret[1].(error) + return ret0, ret1 } // IsBlockRetrievable indicates an expected call of IsBlockRetrievable @@ -394,11 +395,12 @@ func (mr *MockQueryableBlockRetrieverMockRecorder) IsBlockRetrievable(arg0 inter } // RetrievableBlockColdVersion mocks base method -func (m *MockQueryableBlockRetriever) RetrievableBlockColdVersion(arg0 time.Time) int { +func (m *MockQueryableBlockRetriever) RetrievableBlockColdVersion(arg0 time.Time) (int, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "RetrievableBlockColdVersion", arg0) ret0, _ := ret[0].(int) - return ret0 + ret1, _ := ret[1].(error) + return ret0, ret1 } // RetrievableBlockColdVersion indicates an expected call of RetrievableBlockColdVersion diff --git a/src/dbnode/storage/series/types.go b/src/dbnode/storage/series/types.go index faa5473d1b..fb3b1a562f 100644 --- a/src/dbnode/storage/series/types.go +++ b/src/dbnode/storage/series/types.go @@ -160,11 +160,11 @@ type QueryableBlockRetriever interface { // IsBlockRetrievable returns whether a block is retrievable // for a given block start time. - IsBlockRetrievable(blockStart time.Time) bool + IsBlockRetrievable(blockStart time.Time) (bool, error) // RetrievableBlockColdVersion returns the cold version that was // successfully persisted. - RetrievableBlockColdVersion(blockStart time.Time) int + RetrievableBlockColdVersion(blockStart time.Time) (int, error) // BlockStatesSnapshot returns a snapshot of the whether blocks are // retrievable and their flush versions for each block start. This is used diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 51f0dfdf28..425068523c 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -74,6 +74,7 @@ var ( errNewShardEntryTagsIterNotAtIndexZero = errors.New("new shard entry options error: tags iter not at index zero") errShardIsNotBootstrapped = errors.New("shard is not bootstrapped") errFlushStateIsNotBootstrapped = errors.New("flush state is not bootstrapped") + errFlushStateAlreadyBootstrapped = errors.New("flush state is already bootstrapped") ) type filesetsFn func( @@ -354,13 +355,16 @@ func (s *dbShard) Stream( } // IsBlockRetrievable implements series.QueryableBlockRetriever -func (s *dbShard) IsBlockRetrievable(blockStart time.Time) bool { +func (s *dbShard) IsBlockRetrievable(blockStart time.Time) (bool, error) { return s.hasWarmFlushed(blockStart) } -func (s *dbShard) hasWarmFlushed(blockStart time.Time) bool { - flushState := s.FlushState(blockStart) - return statusIsRetrievable(flushState.WarmStatus) +func (s *dbShard) hasWarmFlushed(blockStart time.Time) (bool, error) { + flushState, err := s.FlushState(blockStart) + if err != nil { + return false, err + } + return statusIsRetrievable(flushState.WarmStatus), nil } func statusIsRetrievable(status fileOpStatus) bool { @@ -375,9 +379,12 @@ func statusIsRetrievable(status fileOpStatus) bool { } // RetrievableBlockColdVersion implements series.QueryableBlockRetriever -func (s *dbShard) RetrievableBlockColdVersion(blockStart time.Time) int { - flushState := s.FlushState(blockStart) - return flushState.ColdVersion +func (s *dbShard) RetrievableBlockColdVersion(blockStart time.Time) (int, error) { + flushState, err := s.FlushState(blockStart) + if err != nil { + return -1, err + } + return flushState.ColdVersion, nil } // BlockStatesSnapshot implements series.QueryableBlockRetriever @@ -1919,7 +1926,20 @@ func (s *dbShard) loadSeries( return shardBootstrapResult, multiErr.FinalError() } -func (s *dbShard) bootstrapFlushStates() { +func (s *dbShard) bootstrapFlushStates() error { + s.flushState.RLock() + if s.flushState.bootstrapped { + s.RUnlock() + return errFlushStateAlreadyBootstrapped + } + s.flushState.RUnlock() + + defer func() { + s.Lock() + s.flushState.bootstrapped = true + s.Unlock() + }() + fsOpts := s.opts.CommitLogOptions().FilesystemOptions() readInfoFilesResults := fs.ReadInfoFiles(fsOpts.FilePathPrefix(), s.namespace.ID(), s.shard, fsOpts.InfoReaderBufferSize(), fsOpts.DecodingOptions()) @@ -1936,7 +1956,7 @@ func (s *dbShard) bootstrapFlushStates() { } info := result.Info at := xtime.FromNanoseconds(info.BlockStart) - fs := s.FlushState(at) + fs := s.flushStateNoBootstrapCheck(at) if fs.WarmStatus != fileOpSuccess { s.markWarmFlushStateSuccess(at) } @@ -1952,6 +1972,8 @@ func (s *dbShard) bootstrapFlushStates() { s.setFlushStateColdVersion(at, info.VolumeIndex) } } + + return nil } func (s *dbShard) cacheShardIndices() error { @@ -2069,6 +2091,12 @@ func (s *dbShard) ColdFlush( if err != nil { return err } + + var ( + // forEachShardEntry should not execute in parallel, but protect with a lock anyways for paranoia. + loopErrLock sync.Mutex + loopErr error + ) // First, loop through all series to capture data on which blocks have dirty // series and add them to the resources for further processing. s.forEachShardEntry(func(entry *lookup.Entry) bool { @@ -2079,7 +2107,14 @@ func (s *dbShard) ColdFlush( // Cold flushes can only happen on blockStarts that have been // warm flushed, because warm flush logic does not currently // perform any merging logic. - if !s.hasWarmFlushed(t.ToTime()) { + hasWarmFlushed, err := s.hasWarmFlushed(t.ToTime()) + if err != nil { + loopErrLock.Lock() + loopErr = err + loopErrLock.Unlock() + return + } + if !hasWarmFlushed { return } @@ -2095,6 +2130,9 @@ func (s *dbShard) ColdFlush( return true }) + if loopErr != nil { + return loopErr + } if dirtySeries.Len() == 0 { // Early exit if there is nothing dirty to merge. dirtySeriesToWrite @@ -2113,7 +2151,11 @@ func (s *dbShard) ColdFlush( // a block, we continue to try persisting other blocks. for blockStart := range dirtySeriesToWrite { startTime := blockStart.ToTime() - coldVersion := s.RetrievableBlockColdVersion(startTime) + coldVersion, err := s.RetrievableBlockColdVersion(startTime) + if err != nil { + multiErr = multiErr.Add(err) + continue + } fsID := fs.FileSetFileIdentifier{ Namespace: s.namespace.ID(), Shard: s.ID(), @@ -2122,7 +2164,7 @@ func (s *dbShard) ColdFlush( } nextVersion := coldVersion + 1 - err := merger.Merge(fsID, mergeWithMem, nextVersion, flushPreparer, nsCtx) + err = merger.Merge(fsID, mergeWithMem, nextVersion, flushPreparer, nsCtx) if err != nil { multiErr = multiErr.Add(err) continue @@ -2222,10 +2264,23 @@ func (s *dbShard) Snapshot( return multiErr.FinalError() } -func (s *dbShard) FlushState(blockStart time.Time) fileOpState { +func (s *dbShard) FlushState(blockStart time.Time) (fileOpState, error) { s.flushState.RLock() defer s.flushState.RUnlock() + if !s.flushState.bootstrapped { + return fileOpState{}, errFlushStateIsNotBootstrapped + } + + return s.flushStateWithRLock(blockStart), nil +} + +func (s *dbShard) flushStateNoBootstrapCheck(blockStart time.Time) fileOpState { + s.flushState.RLock() + defer s.flushState.RUnlock() + return s.flushStateWithRLock(blockStart) +} +func (s *dbShard) flushStateWithRLock(blockStart time.Time) fileOpState { state, ok := s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] if !ok { return fileOpState{WarmStatus: fileOpNotStarted} diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index cdad995065..37e506a3bc 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -140,7 +140,9 @@ func TestShardFlushStateNotStarted(t *testing.T) { notStarted := fileOpState{WarmStatus: fileOpNotStarted} for st := earliest; !st.After(latest); st = st.Add(ropts.BlockSize()) { - assert.Equal(t, notStarted, s.FlushState(earliest)) + flushState, err := s.FlushState(earliest) + require.NoError(t, err) + assert.Equal(t, notStarted, flushState) } } @@ -267,7 +269,9 @@ func TestShardBootstrapWithFlushVersion(t *testing.T) { require.Equal(t, Bootstrapped, s.bootstrapState) for i, blockStart := range blockStarts { - require.Equal(t, i, s.FlushState(blockStart).ColdVersion) + flushState, err := s.FlushState(blockStart) + require.NoError(t, err) + require.Equal(t, i, flushState.ColdVersion) } } @@ -326,7 +330,9 @@ func TestShardBootstrapWithFlushVersionNoCleanUp(t *testing.T) { require.NoError(t, err) require.Equal(t, Bootstrapped, s.bootstrapState) - require.Equal(t, numVolumes-1, s.FlushState(start).ColdVersion) + flushState, err := s.FlushState(start) + require.NoError(t, err) + require.Equal(t, numVolumes-1, flushState.ColdVersion) } // TestShardBootstrapWithCacheShardIndices ensures that the shard is able to bootstrap @@ -432,7 +438,8 @@ func TestShardFlushSeriesFlushError(t *testing.T) { require.NotNil(t, err) require.Equal(t, "error bar", err.Error()) - flushState := s.FlushState(blockStart) + flushState, err := s.FlushState(blockStart) + require.NoError(t, err) require.Equal(t, fileOpState{ WarmStatus: fileOpFailed, NumFailures: 2, @@ -498,7 +505,8 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { require.True(t, closed) require.Nil(t, err) - flushState := s.FlushState(blockStart) + flushState, err := s.FlushState(blockStart) + require.NoError(t, err) require.Equal(t, fileOpState{ WarmStatus: fileOpSuccess, ColdVersion: 0, @@ -580,16 +588,22 @@ func TestShardColdFlush(t *testing.T) { // Assert that flush state cold versions all start at 0. for i := t0; i.Before(t7.Add(blockSize)); i = i.Add(blockSize) { - assert.Equal(t, 0, shard.RetrievableBlockColdVersion(i)) + coldVersion, err := shard.RetrievableBlockColdVersion(i) + require.NoError(t, err) + assert.Equal(t, 0, coldVersion) } shard.ColdFlush(preparer, resources, nsCtx) // After a cold flush, t0-t6 previously dirty block starts should be updated // to version 1. for i := t0; i.Before(t6.Add(blockSize)); i = i.Add(blockSize) { - assert.Equal(t, 1, shard.RetrievableBlockColdVersion(i)) + coldVersion, err := shard.RetrievableBlockColdVersion(i) + require.NoError(t, err) + assert.Equal(t, 1, coldVersion) } // t7 shouldn't be cold flushed because it hasn't been warm flushed. - assert.Equal(t, 0, shard.RetrievableBlockColdVersion(t7)) + coldVersion, err := shard.RetrievableBlockColdVersion(t7) + require.NoError(t, err) + assert.Equal(t, 0, coldVersion) } func TestShardColdFlushNoMergeIfNothingDirty(t *testing.T) { @@ -603,7 +617,7 @@ func TestShardColdFlushNoMergeIfNothingDirty(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(nowFn)) blockSize := opts.SeriesOptions().RetentionOptions().BlockSize() shard := testDatabaseShard(t, opts) - shard.bootstrapState = Bootstrapped + shard.Bootstrap(nil) shard.newMergerFn = newMergerTestFn shard.newFSMergeWithMemFn = newFSMergeWithMemTestFn @@ -642,7 +656,9 @@ func TestShardColdFlushNoMergeIfNothingDirty(t *testing.T) { // After a cold flush, t0-t3 should remain version 0, since nothing should // actually be merged. for i := t0; i.Before(t3.Add(blockSize)); i = i.Add(blockSize) { - assert.Equal(t, 0, shard.RetrievableBlockColdVersion(i)) + coldVersion, err := shard.RetrievableBlockColdVersion(i) + require.NoError(t, err) + assert.Equal(t, 0, coldVersion) } } @@ -847,11 +863,12 @@ func TestShardTick(t *testing.T) { sleepPerSeries := time.Microsecond shard := testDatabaseShard(t, opts) + shard.Bootstrap(nil) shard.SetRuntimeOptions(runtime.NewOptions(). SetTickPerSeriesSleepDuration(sleepPerSeries). SetTickSeriesBatchSize(1)) retriever := series.NewMockQueryableBlockRetriever(ctrl) - retriever.EXPECT().IsBlockRetrievable(gomock.Any()).Return(false).AnyTimes() + retriever.EXPECT().IsBlockRetrievable(gomock.Any()).Return(false, nil).AnyTimes() shard.seriesBlockRetriever = retriever defer shard.Close() @@ -1007,12 +1024,13 @@ func testShardWriteAsync(t *testing.T, writes []testWrite) { sleepPerSeries := time.Microsecond shard := testDatabaseShard(t, opts) + shard.Bootstrap(nil) shard.SetRuntimeOptions(runtime.NewOptions(). SetWriteNewSeriesAsync(true). SetTickPerSeriesSleepDuration(sleepPerSeries). SetTickSeriesBatchSize(1)) retriever := series.NewMockQueryableBlockRetriever(ctrl) - retriever.EXPECT().IsBlockRetrievable(gomock.Any()).Return(false).AnyTimes() + retriever.EXPECT().IsBlockRetrievable(gomock.Any()).Return(false, nil).AnyTimes() shard.seriesBlockRetriever = retriever defer shard.Close() @@ -1091,6 +1109,7 @@ func TestShardTickRace(t *testing.T) { func TestShardTickCleanupSmallBatchSize(t *testing.T) { opts := DefaultTestOptions() shard := testDatabaseShard(t, opts) + shard.Bootstrap(nil) addTestSeries(shard, ident.StringID("foo")) shard.Tick(context.NewNoOpCanncellable(), time.Now(), namespace.Context{}) require.Equal(t, 0, shard.lookup.Len()) @@ -1231,7 +1250,7 @@ func TestPurgeExpiredSeriesNonEmptySeries(t *testing.T) { opts := DefaultTestOptions() shard := testDatabaseShard(t, opts) retriever := series.NewMockQueryableBlockRetriever(ctrl) - retriever.EXPECT().IsBlockRetrievable(gomock.Any()).Return(false).AnyTimes() + retriever.EXPECT().IsBlockRetrievable(gomock.Any()).Return(false, nil).AnyTimes() shard.seriesBlockRetriever = retriever defer shard.Close() ctx := opts.ContextPool().Get() diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index c7d3c7ecf4..b80fce2d1c 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -358,7 +358,7 @@ type databaseNamespace interface { // NeedsFlush returns true if the namespace needs a flush for the // period: [start, end] (both inclusive). // NB: The start/end times are assumed to be aligned to block size boundary. - NeedsFlush(alignedInclusiveStart time.Time, alignedInclusiveEnd time.Time) bool + NeedsFlush(alignedInclusiveStart time.Time, alignedInclusiveEnd time.Time) (bool, error) // Truncate truncates the in-memory data for this namespace. Truncate() (int64, error) @@ -495,7 +495,7 @@ type databaseShard interface { ) error // FlushState returns the flush state for this shard at block start. - FlushState(blockStart time.Time) fileOpState + FlushState(blockStart time.Time) (fileOpState, error) // CleanupExpiredFileSets removes expired fileset files. CleanupExpiredFileSets(earliestToRetain time.Time) error From 4e19911dab226688f5919920c3cb937115924813 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Tue, 23 Jul 2019 16:06:03 -0400 Subject: [PATCH 06/22] fix all the interfaces and tests --- src/dbnode/storage/namespace_test.go | 2 +- src/dbnode/storage/series/buffer.go | 67 +++---- src/dbnode/storage/series/buffer_mock.go | 2 +- src/dbnode/storage/series/buffer_test.go | 28 ++- src/dbnode/storage/series/series.go | 61 +++--- src/dbnode/storage/series/series_mock.go | 15 +- src/dbnode/storage/series/series_test.go | 174 ++++++++++-------- src/dbnode/storage/series/types.go | 40 +++- .../series_wired_list_interaction_test.go | 12 +- src/dbnode/storage/shard.go | 57 +++--- src/dbnode/storage/shard_race_prop_test.go | 1 + src/dbnode/storage/shard_test.go | 112 +++++++++-- 12 files changed, 363 insertions(+), 208 deletions(-) diff --git a/src/dbnode/storage/namespace_test.go b/src/dbnode/storage/namespace_test.go index a90535e40c..1de5e156bc 100644 --- a/src/dbnode/storage/namespace_test.go +++ b/src/dbnode/storage/namespace_test.go @@ -1282,7 +1282,7 @@ func waitForStats( } func assertNeedsFlush(t *testing.T, ns *dbNamespace, t0, t1 time.Time, assertTrue bool) { - needsFlush, err := ns.NeedsFlush(t1, t1) + needsFlush, err := ns.NeedsFlush(t0, t1) require.NoError(t, err) require.Equal(t, assertTrue, needsFlush) } diff --git a/src/dbnode/storage/series/buffer.go b/src/dbnode/storage/series/buffer.go index 183f124abb..9abbd4ee15 100644 --- a/src/dbnode/storage/series/buffer.go +++ b/src/dbnode/storage/series/buffer.go @@ -133,7 +133,7 @@ type databaseBuffer interface { Stats() bufferStats - Tick(versions map[xtime.UnixNano]BlockState, nsCtx namespace.Context) bufferTickResult + Tick(versions ShardBlockStateSnapshot, nsCtx namespace.Context) bufferTickResult Load(bl block.DatabaseBlock, writeType WriteType) @@ -365,7 +365,7 @@ func (b *dbBuffer) Stats() bufferStats { } } -func (b *dbBuffer) Tick(blockStates map[xtime.UnixNano]BlockState, nsCtx namespace.Context) bufferTickResult { +func (b *dbBuffer) Tick(blockStates ShardBlockStateSnapshot, nsCtx namespace.Context) bufferTickResult { mergedOutOfOrder := 0 var evictedBucketTimes OptimizedTimes for tNano, buckets := range b.bucketsMap { @@ -377,35 +377,40 @@ func (b *dbBuffer) Tick(blockStates map[xtime.UnixNano]BlockState, nsCtx namespa // 2) remove a lower versioned bucket. // Retrievable and higher versioned buckets will be left to be // collected in the next tick. - blockState := blockStates[tNano] - if coldVersion := blockState.ColdVersion; blockState.WarmRetrievable || coldVersion > 0 { - if blockState.WarmRetrievable { - // Buckets for WarmWrites that are retrievable will only be version 1, since - // they only get successfully persisted once. - buckets.removeBucketsUpToVersion(WarmWrite, 1) - } - if coldVersion > 0 { - buckets.removeBucketsUpToVersion(ColdWrite, coldVersion) - } - - if buckets.streamsLen() == 0 { - t := tNano.ToTime() - // All underlying buckets have been flushed successfully, so we can - // just remove the buckets from the bucketsMap. - b.removeBucketVersionsAt(t) - // Pass which bucket got evicted from the buffer to the series. - // Data gets read in order of precedence: buffer -> cache -> disk. - // After a bucket gets removed from the buffer, data from the cache - // will be served. However, since data just got persisted to disk, - // the cached block is now stale. To correct this, we can either: - // 1) evict the stale block from cache so that new data will - // be retrieved from disk, or - // 2) merge the new data into the cached block. - // It's unclear whether recently flushed data would frequently be - // read soon afterward, so we're choosing (1) here, since it has a - // simpler implementation (just removing from a map). - evictedBucketTimes.Add(tNano) - continue + blockStateSnapshot, bootstrapped := blockStates.Snapshot() + // Only use block state snapshot information to make eviction decisions if the block state + // has been properly bootstrapped already. + if bootstrapped { + blockState := blockStateSnapshot.Snapshot[tNano] + if coldVersion := blockState.ColdVersion; blockState.WarmRetrievable || coldVersion > 0 { + if blockState.WarmRetrievable { + // Buckets for WarmWrites that are retrievable will only be version 1, since + // they only get successfully persisted once. + buckets.removeBucketsUpToVersion(WarmWrite, 1) + } + if coldVersion > 0 { + buckets.removeBucketsUpToVersion(ColdWrite, coldVersion) + } + + if buckets.streamsLen() == 0 { + t := tNano.ToTime() + // All underlying buckets have been flushed successfully, so we can + // just remove the buckets from the bucketsMap. + b.removeBucketVersionsAt(t) + // Pass which bucket got evicted from the buffer to the series. + // Data gets read in order of precedence: buffer -> cache -> disk. + // After a bucket gets removed from the buffer, data from the cache + // will be served. However, since data just got persisted to disk, + // the cached block is now stale. To correct this, we can either: + // 1) evict the stale block from cache so that new data will + // be retrieved from disk, or + // 2) merge the new data into the cached block. + // It's unclear whether recently flushed data would frequently be + // read soon afterward, so we're choosing (1) here, since it has a + // simpler implementation (just removing from a map). + evictedBucketTimes.Add(tNano) + continue + } } } diff --git a/src/dbnode/storage/series/buffer_mock.go b/src/dbnode/storage/series/buffer_mock.go index 15d53c3d63..0d6afeaa7f 100644 --- a/src/dbnode/storage/series/buffer_mock.go +++ b/src/dbnode/storage/series/buffer_mock.go @@ -208,7 +208,7 @@ func (mr *MockdatabaseBufferMockRecorder) Stats() *gomock.Call { } // Tick mocks base method -func (m *MockdatabaseBuffer) Tick(versions map[time0.UnixNano]BlockState, nsCtx namespace.Context) bufferTickResult { +func (m *MockdatabaseBuffer) Tick(versions ShardBlockStateSnapshot, nsCtx namespace.Context) bufferTickResult { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Tick", versions, nsCtx) ret0, _ := ret[0].(bufferTickResult) diff --git a/src/dbnode/storage/series/buffer_test.go b/src/dbnode/storage/series/buffer_test.go index 82c39bf689..03e2b274a3 100644 --- a/src/dbnode/storage/series/buffer_test.go +++ b/src/dbnode/storage/series/buffer_test.go @@ -825,13 +825,17 @@ func TestBufferTickReordersOutOfOrderBuffers(t *testing.T) { assert.Equal(t, 2, len(encoders)) - blockStates := make(map[xtime.UnixNano]BlockState) - blockStates[xtime.ToUnixNano(start)] = BlockState{ - WarmRetrievable: true, - ColdVersion: 1, + blockStates := BlockStateSnapshot{ + Snapshot: map[xtime.UnixNano]BlockState{ + xtime.ToUnixNano(start): BlockState{ + WarmRetrievable: true, + ColdVersion: 1, + }, + }, } + shardBlockState := NewShardBlockStateSnapshot(true, blockStates) // Perform a tick and ensure merged out of order blocks. - r := buffer.Tick(blockStates, namespace.Context{}) + r := buffer.Tick(shardBlockState, namespace.Context{}) assert.Equal(t, 1, r.mergedOutOfOrderBlocks) // Check values correct. @@ -901,17 +905,21 @@ func TestBufferRemoveBucket(t *testing.T) { // Simulate that a flush has fully completed on this bucket so that it will. // get removed from the bucket. - blockStates := make(map[xtime.UnixNano]BlockState) - blockStates[xtime.ToUnixNano(start)] = BlockState{ - WarmRetrievable: true, - ColdVersion: 1, + blockStates := BlockStateSnapshot{ + Snapshot: map[xtime.UnixNano]BlockState{ + xtime.ToUnixNano(start): BlockState{ + WarmRetrievable: true, + ColdVersion: 1, + }, + }, } + shardBlockState := NewShardBlockStateSnapshot(true, blockStates) bucket.version = 1 // False because we just wrote to it. assert.False(t, buffer.IsEmpty()) // Perform a tick to remove the bucket which has been flushed. - buffer.Tick(blockStates, namespace.Context{}) + buffer.Tick(shardBlockState, namespace.Context{}) // True because we just removed the bucket. assert.True(t, buffer.IsEmpty()) } diff --git a/src/dbnode/storage/series/series.go b/src/dbnode/storage/series/series.go index b447040125..141f490eaa 100644 --- a/src/dbnode/storage/series/series.go +++ b/src/dbnode/storage/series/series.go @@ -49,8 +49,9 @@ var ( // ErrSeriesAllDatapointsExpired is returned on tick when all datapoints are expired ErrSeriesAllDatapointsExpired = errors.New("series datapoints are all expired") - errSeriesAlreadyBootstrapped = errors.New("series is already bootstrapped") - errSeriesNotBootstrapped = errors.New("series is not yet bootstrapped") + errSeriesAlreadyBootstrapped = errors.New("series is already bootstrapped") + errSeriesNotBootstrapped = errors.New("series is not yet bootstrapped") + errBlockStateSnapshotNotBootstrapped = errors.New("block state snapshot is not bootstrapped") ) type dbSeries struct { @@ -117,7 +118,7 @@ func (s *dbSeries) Tags() ident.Tags { return tags } -func (s *dbSeries) Tick(blockStates map[xtime.UnixNano]BlockState, nsCtx namespace.Context) (TickResult, error) { +func (s *dbSeries) Tick(blockStates ShardBlockStateSnapshot, nsCtx namespace.Context) (TickResult, error) { var r TickResult s.Lock() @@ -149,7 +150,7 @@ type updateBlocksResult struct { } func (s *dbSeries) updateBlocksWithLock( - blockStates map[xtime.UnixNano]BlockState, + blockStates ShardBlockStateSnapshot, evictedBucketTimes OptimizedTimes, ) (updateBlocksResult, error) { var ( @@ -204,24 +205,28 @@ func (s *dbSeries) updateBlocksWithLock( // Potentially unwire var unwired, shouldUnwire bool - // Makes sure that the block has been flushed, which - // prevents us from unwiring blocks that haven't been flushed yet which - // would cause data loss. - if blockState := blockStates[startNano]; blockState.WarmRetrievable { - switch cachePolicy { - case CacheNone: - shouldUnwire = true - case CacheRecentlyRead: - sinceLastRead := now.Sub(currBlock.LastReadTime()) - shouldUnwire = sinceLastRead >= wiredTimeout - case CacheLRU: - // The tick is responsible for managing the lifecycle of blocks that were not - // read from disk (not retrieved), and the WiredList will manage those that were - // retrieved from disk. - shouldUnwire = !currBlock.WasRetrievedFromDisk() - default: - s.opts.InstrumentOptions().Logger().Fatal( - "unhandled cache policy in series tick", zap.Any("policy", cachePolicy)) + blockStatesSnapshot, bootstrapped := blockStates.Snapshot() + // Only use the block states data to make decision if it has been bootstrapped already. + if bootstrapped { + // Makes sure that the block has been flushed, which + // prevents us from unwiring blocks that haven't been flushed yet which + // would cause data loss. + if blockState := blockStatesSnapshot.Snapshot[startNano]; blockState.WarmRetrievable { + switch cachePolicy { + case CacheNone: + shouldUnwire = true + case CacheRecentlyRead: + sinceLastRead := now.Sub(currBlock.LastReadTime()) + shouldUnwire = sinceLastRead >= wiredTimeout + case CacheLRU: + // The tick is responsible for managing the lifecycle of blocks that were not + // read from disk (not retrieved), and the WiredList will manage those that were + // retrieved from disk. + shouldUnwire = !currBlock.WasRetrievedFromDisk() + default: + s.opts.InstrumentOptions().Logger().Fatal( + "unhandled cache policy in series tick", zap.Any("policy", cachePolicy)) + } } } @@ -411,7 +416,7 @@ func (s *dbSeries) addBlockWithLock(b block.DatabaseBlock) { func (s *dbSeries) Bootstrap( bootstrappedBlocks block.DatabaseSeriesBlocks, - blockStates map[xtime.UnixNano]BlockState, + blockStates BlockStateSnapshot, ) (BootstrapResult, error) { s.Lock() defer func() { @@ -436,7 +441,7 @@ func (s *dbSeries) Bootstrap( func (s *dbSeries) Load( bootstrappedBlocks block.DatabaseSeriesBlocks, - blockStates map[xtime.UnixNano]BlockState, + blockStates BlockStateSnapshot, ) { s.Lock() s.loadWithLock(bootstrappedBlocks, blockStates) @@ -445,11 +450,11 @@ func (s *dbSeries) Load( func (s *dbSeries) loadWithLock( bootstrappedBlocks block.DatabaseSeriesBlocks, - blockStates map[xtime.UnixNano]BlockState, + blockStates BlockStateSnapshot, ) { for _, block := range bootstrappedBlocks.AllBlocks() { blStartNano := xtime.ToUnixNano(block.StartTime()) - blState := blockStates[blStartNano] + blState := blockStates.Snapshot[blStartNano] if !blState.WarmRetrievable { // If the block being bootstrapped has never been warm flushed before then the block // can be loaded into the buffer as a WarmWrite because a subsequent warm flush will @@ -603,11 +608,11 @@ func (s *dbSeries) Snapshot( return s.buffer.Snapshot(ctx, blockStart, s.id, s.tags, persistFn, nsCtx) } -func (s *dbSeries) ColdFlushBlockStarts(blockStates map[xtime.UnixNano]BlockState) OptimizedTimes { +func (s *dbSeries) ColdFlushBlockStarts(blockStates BlockStateSnapshot) OptimizedTimes { s.RLock() defer s.RUnlock() - return s.buffer.ColdFlushBlockStarts(blockStates) + return s.buffer.ColdFlushBlockStarts(blockStates.Snapshot) } func (s *dbSeries) Close() { diff --git a/src/dbnode/storage/series/series_mock.go b/src/dbnode/storage/series/series_mock.go index 7690bd5de8..22a5f81dfa 100644 --- a/src/dbnode/storage/series/series_mock.go +++ b/src/dbnode/storage/series/series_mock.go @@ -64,7 +64,7 @@ func (m *MockDatabaseSeries) EXPECT() *MockDatabaseSeriesMockRecorder { } // Bootstrap mocks base method -func (m *MockDatabaseSeries) Bootstrap(arg0 block.DatabaseSeriesBlocks, arg1 map[time0.UnixNano]BlockState) (BootstrapResult, error) { +func (m *MockDatabaseSeries) Bootstrap(arg0 block.DatabaseSeriesBlocks, arg1 BlockStateSnapshot) (BootstrapResult, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Bootstrap", arg0, arg1) ret0, _ := ret[0].(BootstrapResult) @@ -91,7 +91,7 @@ func (mr *MockDatabaseSeriesMockRecorder) Close() *gomock.Call { } // ColdFlushBlockStarts mocks base method -func (m *MockDatabaseSeries) ColdFlushBlockStarts(arg0 map[time0.UnixNano]BlockState) OptimizedTimes { +func (m *MockDatabaseSeries) ColdFlushBlockStarts(arg0 BlockStateSnapshot) OptimizedTimes { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ColdFlushBlockStarts", arg0) ret0, _ := ret[0].(OptimizedTimes) @@ -192,7 +192,7 @@ func (mr *MockDatabaseSeriesMockRecorder) IsEmpty() *gomock.Call { } // Load mocks base method -func (m *MockDatabaseSeries) Load(arg0 block.DatabaseSeriesBlocks, arg1 map[time0.UnixNano]BlockState) { +func (m *MockDatabaseSeries) Load(arg0 block.DatabaseSeriesBlocks, arg1 BlockStateSnapshot) { m.ctrl.T.Helper() m.ctrl.Call(m, "Load", arg0, arg1) } @@ -297,7 +297,7 @@ func (mr *MockDatabaseSeriesMockRecorder) Tags() *gomock.Call { } // Tick mocks base method -func (m *MockDatabaseSeries) Tick(arg0 map[time0.UnixNano]BlockState, arg1 namespace.Context) (TickResult, error) { +func (m *MockDatabaseSeries) Tick(arg0 ShardBlockStateSnapshot, arg1 namespace.Context) (TickResult, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Tick", arg0, arg1) ret0, _ := ret[0].(TickResult) @@ -365,12 +365,11 @@ func (m *MockQueryableBlockRetriever) EXPECT() *MockQueryableBlockRetrieverMockR } // BlockStatesSnapshot mocks base method -func (m *MockQueryableBlockRetriever) BlockStatesSnapshot() (map[time0.UnixNano]BlockState, error) { +func (m *MockQueryableBlockRetriever) BlockStatesSnapshot() ShardBlockStateSnapshot { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "BlockStatesSnapshot") - ret0, _ := ret[0].(map[time0.UnixNano]BlockState) - ret1, _ := ret[1].(error) - return ret0, ret1 + ret0, _ := ret[0].(ShardBlockStateSnapshot) + return ret0 } // BlockStatesSnapshot indicates an expected call of BlockStatesSnapshot diff --git a/src/dbnode/storage/series/series_test.go b/src/dbnode/storage/series/series_test.go index ec9d788296..01a9a40f4f 100644 --- a/src/dbnode/storage/series/series_test.go +++ b/src/dbnode/storage/series/series_test.go @@ -84,7 +84,7 @@ func newSeriesTestOptions() Options { func TestSeriesEmpty(t *testing.T) { opts := newSeriesTestOptions() series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(nil, BlockStateSnapshot{}) assert.NoError(t, err) assert.True(t, series.IsEmpty()) } @@ -106,7 +106,7 @@ func TestSeriesWriteFlush(t *testing.T) { return curr })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(nil, BlockStateSnapshot{}) assert.NoError(t, err) data := []value{ @@ -141,7 +141,7 @@ func TestSeriesSamePointDoesNotWrite(t *testing.T) { return curr })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(nil, BlockStateSnapshot{}) assert.NoError(t, err) data := []value{ @@ -184,7 +184,7 @@ func TestSeriesWriteFlushRead(t *testing.T) { return curr })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(nil, BlockStateSnapshot{}) assert.NoError(t, err) data := []value{ @@ -228,13 +228,13 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { f func( series DatabaseSeries, blocks block.DatabaseSeriesBlocks, - blockStates map[xtime.UnixNano]BlockState) + blockStates BlockStateSnapshot) }{ { title: "load", f: func(series DatabaseSeries, blocks block.DatabaseSeriesBlocks, - blockStates map[xtime.UnixNano]BlockState, + blockStates BlockStateSnapshot, ) { series.Load(blocks, blockStates) }}, @@ -242,7 +242,7 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { title: "bootstrap", f: func(series DatabaseSeries, blocks block.DatabaseSeriesBlocks, - blockStates map[xtime.UnixNano]BlockState, + blockStates BlockStateSnapshot, ) { _, err := series.Bootstrap(blocks, blockStates) require.NoError(t, err) @@ -285,13 +285,15 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { blocks = block.NewDatabaseSeriesBlocks(len(loadWrites)) alreadyWarmFlushedBlockStart = curr.Add(blockSize).Truncate(blockSize) notYetWarmFlushedBlockStart = curr.Add(2 * blockSize).Truncate(blockSize) - blockStates = map[xtime.UnixNano]BlockState{ - // Exercise both code paths. - xtime.ToUnixNano(alreadyWarmFlushedBlockStart): BlockState{ - WarmRetrievable: true, - }, - xtime.ToUnixNano(notYetWarmFlushedBlockStart): BlockState{ - WarmRetrievable: false, + blockStates = BlockStateSnapshot{ + Snapshot: map[xtime.UnixNano]BlockState{ + // Exercise both code paths. + xtime.ToUnixNano(alreadyWarmFlushedBlockStart): BlockState{ + WarmRetrievable: true, + }, + xtime.ToUnixNano(notYetWarmFlushedBlockStart): BlockState{ + WarmRetrievable: false, + }, }, } ) @@ -338,7 +340,7 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { func TestSeriesReadEndBeforeStart(t *testing.T) { opts := newSeriesTestOptions() series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(nil, BlockStateSnapshot{}) assert.NoError(t, err) ctx := context.NewContext() @@ -354,7 +356,7 @@ func TestSeriesReadEndBeforeStart(t *testing.T) { func TestSeriesFlushNoBlock(t *testing.T) { opts := newSeriesTestOptions() series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(nil, BlockStateSnapshot{}) assert.NoError(t, err) flushTime := time.Unix(7200, 0) outcome, err := series.WarmFlush(nil, flushTime, nil, namespace.Context{}) @@ -373,7 +375,7 @@ func TestSeriesFlush(t *testing.T) { })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(nil, BlockStateSnapshot{}) assert.NoError(t, err) ctx := context.NewContext() @@ -400,9 +402,9 @@ func TestSeriesFlush(t *testing.T) { func TestSeriesTickEmptySeries(t *testing.T) { opts := newSeriesTestOptions() series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(nil, BlockStateSnapshot{}) assert.NoError(t, err) - _, err = series.Tick(nil, namespace.Context{}) + _, err = series.Tick(NewShardBlockStateSnapshot(true, BlockStateSnapshot{}), namespace.Context{}) require.Equal(t, ErrSeriesAllDatapointsExpired, err) } @@ -412,13 +414,13 @@ func TestSeriesTickDrainAndResetBuffer(t *testing.T) { opts := newSeriesTestOptions() series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(nil, BlockStateSnapshot{}) assert.NoError(t, err) buffer := NewMockdatabaseBuffer(ctrl) series.buffer = buffer - buffer.EXPECT().Tick(nil, gomock.Any()).Return(bufferTickResult{}) + buffer.EXPECT().Tick(gomock.Any(), gomock.Any()).Return(bufferTickResult{}) buffer.EXPECT().Stats().Return(bufferStats{wiredBlocks: 1}) - r, err := series.Tick(nil, namespace.Context{}) + r, err := series.Tick(NewShardBlockStateSnapshot(true, BlockStateSnapshot{}), namespace.Context{}) require.NoError(t, err) assert.Equal(t, 1, r.ActiveBlocks) assert.Equal(t, 1, r.WiredBlocks) @@ -436,7 +438,7 @@ func TestSeriesTickNeedsBlockExpiry(t *testing.T) { return curr })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(nil, BlockStateSnapshot{}) assert.NoError(t, err) blockStart := curr.Add(-ropts.RetentionPeriod()).Add(-ropts.BlockSize()) b := block.NewMockDatabaseBlock(ctrl) @@ -453,16 +455,19 @@ func TestSeriesTickNeedsBlockExpiry(t *testing.T) { series.buffer = buffer buffer.EXPECT().Tick(gomock.Any(), gomock.Any()).Return(bufferTickResult{}) buffer.EXPECT().Stats().Return(bufferStats{wiredBlocks: 1}) - blockStates := make(map[xtime.UnixNano]BlockState) - blockStates[xtime.ToUnixNano(blockStart)] = BlockState{ - WarmRetrievable: false, - ColdVersion: 0, - } - blockStates[xtime.ToUnixNano(curr)] = BlockState{ - WarmRetrievable: false, - ColdVersion: 0, + blockStates := BlockStateSnapshot{ + Snapshot: map[xtime.UnixNano]BlockState{ + xtime.ToUnixNano(blockStart): BlockState{ + WarmRetrievable: false, + ColdVersion: 0, + }, + xtime.ToUnixNano(curr): BlockState{ + WarmRetrievable: false, + ColdVersion: 0, + }, + }, } - r, err := series.Tick(blockStates, namespace.Context{}) + r, err := series.Tick(NewShardBlockStateSnapshot(true, blockStates), namespace.Context{}) require.NoError(t, err) require.Equal(t, 2, r.ActiveBlocks) require.Equal(t, 2, r.WiredBlocks) @@ -489,7 +494,7 @@ func TestSeriesTickRecentlyRead(t *testing.T) { series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(nil, BlockStateSnapshot{}) assert.NoError(t, err) // Test case where block has been read within expiry period - won't be removed @@ -500,12 +505,16 @@ func TestSeriesTickRecentlyRead(t *testing.T) { b.EXPECT().HasMergeTarget().Return(true) series.cachedBlocks.AddBlock(b) - blockStates := make(map[xtime.UnixNano]BlockState) - blockStates[xtime.ToUnixNano(curr)] = BlockState{ - WarmRetrievable: true, - ColdVersion: 1, + blockStates := BlockStateSnapshot{ + Snapshot: map[xtime.UnixNano]BlockState{ + xtime.ToUnixNano(curr): BlockState{ + WarmRetrievable: true, + ColdVersion: 1, + }, + }, } - tickResult, err := series.Tick(blockStates, namespace.Context{}) + shardBlockStates := NewShardBlockStateSnapshot(true, blockStates) + tickResult, err := series.Tick(shardBlockStates, namespace.Context{}) require.NoError(t, err) require.Equal(t, 0, tickResult.UnwiredBlocks) require.Equal(t, 1, tickResult.PendingMergeBlocks) @@ -518,7 +527,7 @@ func TestSeriesTickRecentlyRead(t *testing.T) { b.EXPECT().Close().Return() series.cachedBlocks.AddBlock(b) - tickResult, err = series.Tick(blockStates, namespace.Context{}) + tickResult, err = series.Tick(shardBlockStates, namespace.Context{}) require.NoError(t, err) require.Equal(t, 1, tickResult.UnwiredBlocks) require.Equal(t, 0, tickResult.PendingMergeBlocks) @@ -529,12 +538,16 @@ func TestSeriesTickRecentlyRead(t *testing.T) { b.EXPECT().HasMergeTarget().Return(true) series.cachedBlocks.AddBlock(b) - blockStates = make(map[xtime.UnixNano]BlockState) - blockStates[xtime.ToUnixNano(curr)] = BlockState{ - WarmRetrievable: false, - ColdVersion: 0, + blockStates = BlockStateSnapshot{ + Snapshot: map[xtime.UnixNano]BlockState{ + xtime.ToUnixNano(curr): BlockState{ + WarmRetrievable: false, + ColdVersion: 0, + }, + }, } - tickResult, err = series.Tick(blockStates, namespace.Context{}) + shardBlockStates = NewShardBlockStateSnapshot(true, blockStates) + tickResult, err = series.Tick(shardBlockStates, namespace.Context{}) require.NoError(t, err) require.Equal(t, 0, tickResult.UnwiredBlocks) require.Equal(t, 1, tickResult.PendingMergeBlocks) @@ -557,7 +570,7 @@ func TestSeriesTickCacheLRU(t *testing.T) { series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(nil, BlockStateSnapshot{}) assert.NoError(t, err) // Test case where block was not retrieved from disk - Will be removed @@ -567,12 +580,16 @@ func TestSeriesTickCacheLRU(t *testing.T) { b.EXPECT().Close().Return() series.cachedBlocks.AddBlock(b) - blockStates := make(map[xtime.UnixNano]BlockState) - blockStates[xtime.ToUnixNano(curr)] = BlockState{ - WarmRetrievable: true, - ColdVersion: 1, + blockStates := BlockStateSnapshot{ + Snapshot: map[xtime.UnixNano]BlockState{ + xtime.ToUnixNano(curr): BlockState{ + WarmRetrievable: true, + ColdVersion: 1, + }, + }, } - tickResult, err := series.Tick(blockStates, namespace.Context{}) + shardBlockStates := NewShardBlockStateSnapshot(true, blockStates) + tickResult, err := series.Tick(shardBlockStates, namespace.Context{}) require.NoError(t, err) require.Equal(t, 1, tickResult.UnwiredBlocks) require.Equal(t, 0, tickResult.PendingMergeBlocks) @@ -584,7 +601,7 @@ func TestSeriesTickCacheLRU(t *testing.T) { b.EXPECT().WasRetrievedFromDisk().Return(true) series.cachedBlocks.AddBlock(b) - tickResult, err = series.Tick(blockStates, namespace.Context{}) + tickResult, err = series.Tick(shardBlockStates, namespace.Context{}) require.NoError(t, err) require.Equal(t, 0, tickResult.UnwiredBlocks) require.Equal(t, 1, tickResult.PendingMergeBlocks) @@ -603,12 +620,16 @@ func TestSeriesTickCacheLRU(t *testing.T) { _, expiredBlockExists := series.cachedBlocks.BlockAt(curr.Add(-2 * retentionPeriod)) require.Equal(t, true, expiredBlockExists) - blockStates = make(map[xtime.UnixNano]BlockState) - blockStates[xtime.ToUnixNano(curr)] = BlockState{ - WarmRetrievable: false, - ColdVersion: 0, + blockStates = BlockStateSnapshot{ + Snapshot: map[xtime.UnixNano]BlockState{ + xtime.ToUnixNano(curr): BlockState{ + WarmRetrievable: false, + ColdVersion: 0, + }, + }, } - tickResult, err = series.Tick(blockStates, namespace.Context{}) + shardBlockStates = NewShardBlockStateSnapshot(true, blockStates) + tickResult, err = series.Tick(shardBlockStates, namespace.Context{}) require.NoError(t, err) require.Equal(t, 0, tickResult.UnwiredBlocks) require.Equal(t, 1, tickResult.PendingMergeBlocks) @@ -632,7 +653,7 @@ func TestSeriesTickCacheNone(t *testing.T) { series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(nil, BlockStateSnapshot{}) assert.NoError(t, err) // Retrievable blocks should be removed @@ -641,12 +662,16 @@ func TestSeriesTickCacheNone(t *testing.T) { b.EXPECT().Close().Return() series.cachedBlocks.AddBlock(b) - blockStates := make(map[xtime.UnixNano]BlockState) - blockStates[xtime.ToUnixNano(curr)] = BlockState{ - WarmRetrievable: true, - ColdVersion: 1, + blockStates := BlockStateSnapshot{ + Snapshot: map[xtime.UnixNano]BlockState{ + xtime.ToUnixNano(curr): BlockState{ + WarmRetrievable: true, + ColdVersion: 1, + }, + }, } - tickResult, err := series.Tick(blockStates, namespace.Context{}) + shardBlockStates := NewShardBlockStateSnapshot(true, blockStates) + tickResult, err := series.Tick(shardBlockStates, namespace.Context{}) require.NoError(t, err) require.Equal(t, 1, tickResult.UnwiredBlocks) require.Equal(t, 0, tickResult.PendingMergeBlocks) @@ -657,12 +682,16 @@ func TestSeriesTickCacheNone(t *testing.T) { b.EXPECT().HasMergeTarget().Return(true) series.cachedBlocks.AddBlock(b) - blockStates = make(map[xtime.UnixNano]BlockState) - blockStates[xtime.ToUnixNano(curr)] = BlockState{ - WarmRetrievable: false, - ColdVersion: 0, + blockStates = BlockStateSnapshot{ + Snapshot: map[xtime.UnixNano]BlockState{ + xtime.ToUnixNano(curr): BlockState{ + WarmRetrievable: false, + ColdVersion: 0, + }, + }, } - tickResult, err = series.Tick(blockStates, namespace.Context{}) + shardBlockStates = NewShardBlockStateSnapshot(true, blockStates) + tickResult, err = series.Tick(shardBlockStates, namespace.Context{}) require.NoError(t, err) require.Equal(t, 0, tickResult.UnwiredBlocks) require.Equal(t, 1, tickResult.PendingMergeBlocks) @@ -716,8 +745,9 @@ func TestSeriesTickCachedBlockRemove(t *testing.T) { series.buffer = buffer assert.Equal(t, 3, series.cachedBlocks.Len()) - blockStates := make(map[xtime.UnixNano]BlockState) - _, err := series.Tick(blockStates, namespace.Context{}) + blockStates := BlockStateSnapshot{} + shardBlockStates := NewShardBlockStateSnapshot(true, blockStates) + _, err := series.Tick(shardBlockStates, namespace.Context{}) require.NoError(t, err) assert.Equal(t, 1, series.cachedBlocks.Len()) } @@ -753,7 +783,7 @@ func TestSeriesFetchBlocks(t *testing.T) { Return([]block.FetchBlockResult{block.NewFetchBlockResult(starts[2], nil, nil)}) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(nil, BlockStateSnapshot{}) assert.NoError(t, err) series.cachedBlocks = blocks @@ -823,7 +853,7 @@ func TestSeriesFetchBlocksMetadata(t *testing.T) { Return(expectedResults, nil) series := NewDatabaseSeries(ident.StringID("bar"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(nil, BlockStateSnapshot{}) assert.NoError(t, err) mockBlocks := block.NewMockDatabaseSeriesBlocks(ctrl) mockBlocks.EXPECT().AllBlocks().Return(blocks) @@ -961,7 +991,7 @@ func TestSeriesWriteReadFromTheSameBucket(t *testing.T) { return curr })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(nil, BlockStateSnapshot{}) assert.NoError(t, err) ctx := context.NewContext() diff --git a/src/dbnode/storage/series/types.go b/src/dbnode/storage/series/types.go index fb3b1a562f..d283da7490 100644 --- a/src/dbnode/storage/series/types.go +++ b/src/dbnode/storage/series/types.go @@ -50,7 +50,7 @@ type DatabaseSeries interface { Tags() ident.Tags // Tick executes async updates - Tick(blockStates map[xtime.UnixNano]BlockState, nsCtx namespace.Context) (TickResult, error) + Tick(blockStates ShardBlockStateSnapshot, nsCtx namespace.Context) (TickResult, error) // Write writes a new value. Write( @@ -103,11 +103,11 @@ type DatabaseSeries interface { IsBootstrapped() bool // Bootstrap merges the raw series bootstrapped along with any buffered data. - Bootstrap(blocks block.DatabaseSeriesBlocks, blockStates map[xtime.UnixNano]BlockState) (BootstrapResult, error) + Bootstrap(blocks block.DatabaseSeriesBlocks, blockStates BlockStateSnapshot) (BootstrapResult, error) // Load does the same thing as Bootstrap except it should be used for data that did // not originate from the Bootstrap process (like background repairs). - Load(blocks block.DatabaseSeriesBlocks, blockStates map[xtime.UnixNano]BlockState) + Load(blocks block.DatabaseSeriesBlocks, blockStates BlockStateSnapshot) // WarmFlush flushes the WarmWrites of this series for a given start time. WarmFlush( @@ -127,7 +127,7 @@ type DatabaseSeries interface { ) error // ColdFlushBlockStarts returns the block starts that need cold flushes. - ColdFlushBlockStarts(blockStates map[xtime.UnixNano]BlockState) OptimizedTimes + ColdFlushBlockStarts(blockStates BlockStateSnapshot) OptimizedTimes // Close will close the series and if pooled returned to the pool. Close() @@ -173,7 +173,37 @@ type QueryableBlockRetriever interface { // Flushes may occur and change the actual block state while iterating // through this snapshot, so any logic using this function should take this // into account. - BlockStatesSnapshot() (map[xtime.UnixNano]BlockState, error) + BlockStatesSnapshot() ShardBlockStateSnapshot +} + +// ShardBlockStateSnapshot represents a snapshot of a shard's block state at +// a moment in time. +type ShardBlockStateSnapshot struct { + bootstrapped bool + snapshot BlockStateSnapshot +} + +// NewShardBlockStateSnapshot constructs a new NewShardBlockStateSnapshot. +func NewShardBlockStateSnapshot( + bootstrapped bool, + snapshot BlockStateSnapshot, +) ShardBlockStateSnapshot { + return ShardBlockStateSnapshot{ + bootstrapped: bootstrapped, + snapshot: snapshot, + } +} + +// Snapshot returns a BlockStateSnapshot and a boolean indicating whether the +// snapshot is bootstrapped or not. +func (s *ShardBlockStateSnapshot) Snapshot() (BlockStateSnapshot, bool) { + return s.snapshot, s.bootstrapped +} + +// BlockStateSnapshot represents a bootstrapped shard block state snapshot. +// TODO(rartoul): Rename? +type BlockStateSnapshot struct { + Snapshot map[xtime.UnixNano]BlockState } // BlockState contains the state of a block. diff --git a/src/dbnode/storage/series_wired_list_interaction_test.go b/src/dbnode/storage/series_wired_list_interaction_test.go index 6c1bb64380..2f98d24554 100644 --- a/src/dbnode/storage/series_wired_list_interaction_test.go +++ b/src/dbnode/storage/series_wired_list_interaction_test.go @@ -85,15 +85,15 @@ func TestSeriesWiredListConcurrentInteractions(t *testing.T) { SetWiredList(wl). SetDatabaseBlockPool(blPool), ) - shard = testDatabaseShard(t, opts) - id = ident.StringID("foo") - series = series.NewDatabaseSeries(id, ident.Tags{}, shard.seriesOpts) + shard = testDatabaseShard(t, opts) + id = ident.StringID("foo") + seriesEntry = series.NewDatabaseSeries(id, ident.Tags{}, shard.seriesOpts) ) - series.Reset(id, ident.Tags{}, nil, shard.seriesOnRetrieveBlock, shard, shard.seriesOpts) - series.Bootstrap(nil, nil) + seriesEntry.Reset(id, ident.Tags{}, nil, shard.seriesOnRetrieveBlock, shard, shard.seriesOpts) + seriesEntry.Bootstrap(nil, series.BlockStateSnapshot{}) shard.Lock() - shard.insertNewShardEntryWithLock(lookup.NewEntry(series, 0)) + shard.insertNewShardEntryWithLock(lookup.NewEntry(seriesEntry, 0)) shard.Unlock() var ( diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 425068523c..357febfb2f 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -388,13 +388,12 @@ func (s *dbShard) RetrievableBlockColdVersion(blockStart time.Time) (int, error) } // BlockStatesSnapshot implements series.QueryableBlockRetriever -func (s *dbShard) BlockStatesSnapshot() (map[xtime.UnixNano]series.BlockState, error) { +func (s *dbShard) BlockStatesSnapshot() series.ShardBlockStateSnapshot { s.flushState.RLock() defer s.flushState.RUnlock() if !s.flushState.bootstrapped { - // Safeguard against attempting to use the flush state before it has been bootstrapped. - return nil, errFlushStateIsNotBootstrapped + return series.NewShardBlockStateSnapshot(false, series.BlockStateSnapshot{}) } states := s.flushState.statesByTime @@ -406,7 +405,9 @@ func (s *dbShard) BlockStatesSnapshot() (map[xtime.UnixNano]series.BlockState, e } } - return snapshot, nil + return series.NewShardBlockStateSnapshot(true, series.BlockStateSnapshot{ + Snapshot: snapshot, + }) } func (s *dbShard) OnRetrieveBlock( @@ -680,11 +681,8 @@ func (s *dbShard) tickAndExpire( tickSleepPerSeries := s.currRuntimeOptions.tickSleepPerSeries // Acquire snapshot of block states here to avoid releasing the // RLock and acquiring it right after. - blockStates, err := s.BlockStatesSnapshot() + blockStates := s.BlockStatesSnapshot() s.RUnlock() - if err != nil { - return tickResult{}, err - } s.forEachShardEntryBatch(func(currEntries []*lookup.Entry) bool { // re-using `expired` to amortize allocs, still need to reset it // to be safe for re-use. @@ -1259,7 +1257,7 @@ func (s *dbShard) insertSeriesSync( } if s.newSeriesBootstrapped { - _, err := entry.Series.Bootstrap(nil, nil) + _, err := entry.Series.Bootstrap(nil, series.BlockStateSnapshot{}) if err != nil { entry = nil // Don't increment the writer count for this series return nil, err @@ -1345,7 +1343,7 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { // Insert still pending, perform the insert entry = inserts[i].entry if s.newSeriesBootstrapped { - _, err := entry.Series.Bootstrap(nil, nil) + _, err := entry.Series.Bootstrap(nil, series.BlockStateSnapshot{}) if err != nil { s.metrics.insertAsyncBootstrapErrors.Inc(1) } @@ -1825,11 +1823,11 @@ func (s *dbShard) Bootstrap( // buffered data points since server start. Any new series added // after this will be marked as bootstrapped. s.forEachShardEntry(func(entry *lookup.Entry) bool { - series := entry.Series - if series.IsBootstrapped() { + seriesEntry := entry.Series + if seriesEntry.IsBootstrapped() { return true } - _, err := series.Bootstrap(nil, nil) + _, err := seriesEntry.Bootstrap(nil, series.BlockStateSnapshot{}) multiErr = multiErr.Add(err) return true }) @@ -1868,6 +1866,10 @@ func (s *dbShard) loadSeries( series *result.Map, bootstrap bool, ) (dbShardBootstrapResult, error) { + if series == nil { + return dbShardBootstrapResult{}, nil + } + var ( // Only used for the bootstrap path. shardBootstrapResult = dbShardBootstrapResult{} @@ -1875,9 +1877,10 @@ func (s *dbShard) loadSeries( ) // Safe to use the same snapshot for all the series since the block states can't change while // this is running since no warm/cold flushes can occur while the bootstrap is ongoing. - blockStates, err := s.BlockStatesSnapshot() - if err != nil { - return dbShardBootstrapResult{}, err + blockStates := s.BlockStatesSnapshot() + blockStatesSnapshot, bootstrapped := blockStates.Snapshot() + if !bootstrapped { + return dbShardBootstrapResult{}, errFlushStateIsNotBootstrapped } for _, elem := range series.Iter() { dbBlocks := elem.Value() @@ -1909,13 +1912,13 @@ func (s *dbShard) loadSeries( } if bootstrap { - bsResult, err := entry.Series.Bootstrap(dbBlocks.Blocks, blockStates) + bsResult, err := entry.Series.Bootstrap(dbBlocks.Blocks, blockStatesSnapshot) if err != nil { multiErr = multiErr.Add(err) } shardBootstrapResult.update(bsResult) } else { - entry.Series.Load(dbBlocks.Blocks, blockStates) + entry.Series.Load(dbBlocks.Blocks, blockStatesSnapshot) } // Cannot close blocks once done as series takes ref to them. @@ -2087,9 +2090,10 @@ func (s *dbShard) ColdFlush( idElementPool = resources.idElementPool ) - blockStates, err := s.BlockStatesSnapshot() - if err != nil { - return err + blockStates := s.BlockStatesSnapshot() + blockStatesSnapshot, bootstrapped := blockStates.Snapshot() + if !bootstrapped { + return errFlushStateIsNotBootstrapped } var ( @@ -2102,7 +2106,7 @@ func (s *dbShard) ColdFlush( s.forEachShardEntry(func(entry *lookup.Entry) bool { curr := entry.Series seriesID := curr.ID() - blockStarts := curr.ColdFlushBlockStarts(blockStates) + blockStarts := curr.ColdFlushBlockStarts(blockStatesSnapshot) blockStarts.ForEach(func(t xtime.UnixNano) { // Cold flushes can only happen on blockStarts that have been // warm flushed, because warm flush logic does not currently @@ -2364,14 +2368,15 @@ func (s *dbShard) CleanupCompactedFileSets() error { // Get a snapshot of all states here to prevent constantly getting/releasing // locks in a tight loop below. This snapshot won't become stale halfway // through this because flushing and cleanup never happen in parallel. - blockStates, err := s.BlockStatesSnapshot() - if err != nil { - return err + blockStates := s.BlockStatesSnapshot() + blockStatesSnapshot, bootstrapped := blockStates.Snapshot() + if !bootstrapped { + return errFlushStateIsNotBootstrapped } toDelete := fs.FileSetFilesSlice(make([]fs.FileSetFile, 0, len(filesets))) for _, datafile := range filesets { fileID := datafile.ID - blockState := blockStates[xtime.ToUnixNano(fileID.BlockStart)] + blockState := blockStatesSnapshot.Snapshot[xtime.ToUnixNano(fileID.BlockStart)] if fileID.VolumeIndex < blockState.ColdVersion { toDelete = append(toDelete, datafile) } diff --git a/src/dbnode/storage/shard_race_prop_test.go b/src/dbnode/storage/shard_race_prop_test.go index e0a2de401e..6dc371a755 100644 --- a/src/dbnode/storage/shard_race_prop_test.go +++ b/src/dbnode/storage/shard_race_prop_test.go @@ -110,6 +110,7 @@ var fetchBlocksMetadataV2ShardFn testShardReadFn = func(shard *dbShard) { func propTestDatabaseShard(t *testing.T, tickBatchSize int) (*dbShard, Options) { opts := DefaultTestOptions().SetRuntimeOptionsManager(runtime.NewOptionsManager()) shard := testDatabaseShard(t, opts) + shard.Bootstrap(nil) shard.currRuntimeOptions.tickSleepPerSeries = time.Microsecond shard.currRuntimeOptions.tickSleepSeriesBatchSize = tickBatchSize return shard, opts diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index 37e506a3bc..e1a034b854 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -124,25 +124,35 @@ func TestShardBootstrapState(t *testing.T) { } func TestShardFlushStateNotStarted(t *testing.T) { + dir, err := ioutil.TempDir("", "testdir") + require.NoError(t, err) + defer os.RemoveAll(dir) + now := time.Now() nowFn := func() time.Time { return now } opts := DefaultTestOptions() - opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(nowFn)) + fsOpts := opts.CommitLogOptions().FilesystemOptions(). + SetFilePathPrefix(dir) + opts = opts. + SetClockOptions(opts.ClockOptions().SetNowFn(nowFn)). + SetCommitLogOptions(opts.CommitLogOptions(). + SetFilesystemOptions(fsOpts)) ropts := defaultTestRetentionOpts earliest, latest := retention.FlushTimeStart(ropts, now), retention.FlushTimeEnd(ropts, now) s := testDatabaseShard(t, opts) defer s.Close() + s.Bootstrap(nil) notStarted := fileOpState{WarmStatus: fileOpNotStarted} for st := earliest; !st.After(latest); st = st.Add(ropts.BlockSize()) { flushState, err := s.FlushState(earliest) require.NoError(t, err) - assert.Equal(t, notStarted, flushState) + require.Equal(t, notStarted, flushState) } } @@ -255,9 +265,11 @@ func TestShardBootstrapWithFlushVersion(t *testing.T) { // Ensure that the bootstrapped flush/block states get passed to the series.Bootstrap() // method properly. - blockStateSnapshot := map[xtime.UnixNano]series.BlockState{} + blockStateSnapshot := series.BlockStateSnapshot{ + Snapshot: map[xtime.UnixNano]series.BlockState{}, + } for i, blockStart := range blockStarts { - blockStateSnapshot[xtime.ToUnixNano(blockStart)] = series.BlockState{ + blockStateSnapshot.Snapshot[xtime.ToUnixNano(blockStart)] = series.BlockState{ WarmRetrievable: true, ColdVersion: i, } @@ -388,7 +400,7 @@ func TestShardFlushSeriesFlushError(t *testing.T) { s := testDatabaseShard(t, DefaultTestOptions()) defer s.Close() - s.bootstrapState = Bootstrapped + s.Bootstrap(nil) s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] = fileOpState{ WarmStatus: fileOpFailed, NumFailures: 1, @@ -459,7 +471,7 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(nowFn)) s := testDatabaseShard(t, opts) defer s.Close() - s.bootstrapState = Bootstrapped + s.Bootstrap(nil) s.flushState.statesByTime[xtime.ToUnixNano(blockStart)] = fileOpState{ WarmStatus: fileOpFailed, NumFailures: 1, @@ -528,6 +540,10 @@ func optimizedTimesFromTimes(times []time.Time) series.OptimizedTimes { } func TestShardColdFlush(t *testing.T) { + dir, err := ioutil.TempDir("", "testdir") + require.NoError(t, err) + defer os.RemoveAll(dir) + ctrl := gomock.NewController(t) defer ctrl.Finish() now := time.Now() @@ -535,10 +551,16 @@ func TestShardColdFlush(t *testing.T) { return now } opts := DefaultTestOptions() - opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(nowFn)) + fsOpts := opts.CommitLogOptions().FilesystemOptions(). + SetFilePathPrefix(dir) + opts = opts. + SetClockOptions(opts.ClockOptions().SetNowFn(nowFn)). + SetCommitLogOptions(opts.CommitLogOptions(). + SetFilesystemOptions(fsOpts)) + blockSize := opts.SeriesOptions().RetentionOptions().BlockSize() shard := testDatabaseShard(t, opts) - shard.bootstrapState = Bootstrapped + shard.Bootstrap(nil) shard.newMergerFn = newMergerTestFn shard.newFSMergeWithMemFn = newFSMergeWithMemTestFn @@ -590,7 +612,7 @@ func TestShardColdFlush(t *testing.T) { for i := t0; i.Before(t7.Add(blockSize)); i = i.Add(blockSize) { coldVersion, err := shard.RetrievableBlockColdVersion(i) require.NoError(t, err) - assert.Equal(t, 0, coldVersion) + require.Equal(t, 0, coldVersion) } shard.ColdFlush(preparer, resources, nsCtx) // After a cold flush, t0-t6 previously dirty block starts should be updated @@ -598,12 +620,12 @@ func TestShardColdFlush(t *testing.T) { for i := t0; i.Before(t6.Add(blockSize)); i = i.Add(blockSize) { coldVersion, err := shard.RetrievableBlockColdVersion(i) require.NoError(t, err) - assert.Equal(t, 1, coldVersion) + require.Equal(t, 1, coldVersion) } // t7 shouldn't be cold flushed because it hasn't been warm flushed. coldVersion, err := shard.RetrievableBlockColdVersion(t7) require.NoError(t, err) - assert.Equal(t, 0, coldVersion) + require.Equal(t, 0, coldVersion) } func TestShardColdFlushNoMergeIfNothingDirty(t *testing.T) { @@ -803,18 +825,18 @@ func addTestSeriesWithCount(shard *dbShard, id ident.ID, count int32) series.Dat } func addTestSeriesWithCountAndBootstrap(shard *dbShard, id ident.ID, count int32, bootstrap bool) series.DatabaseSeries { - series := series.NewDatabaseSeries(id, ident.Tags{}, shard.seriesOpts) + seriesEntry := series.NewDatabaseSeries(id, ident.Tags{}, shard.seriesOpts) if bootstrap { - series.Bootstrap(nil, nil) + seriesEntry.Bootstrap(nil, series.BlockStateSnapshot{}) } shard.Lock() - entry := lookup.NewEntry(series, 0) + entry := lookup.NewEntry(seriesEntry, 0) for i := int32(0); i < count; i++ { entry.IncrementReaderWriterCount() } shard.insertNewShardEntryWithLock(entry) shard.Unlock() - return series + return seriesEntry } func writeShardAndVerify( @@ -837,6 +859,10 @@ func writeShardAndVerify( } func TestShardTick(t *testing.T) { + dir, err := ioutil.TempDir("", "testdir") + require.NoError(t, err) + defer os.RemoveAll(dir) + ctrl := gomock.NewController(t) defer ctrl.Finish() @@ -857,6 +883,12 @@ func TestShardTick(t *testing.T) { opts := DefaultTestOptions() opts = opts.SetClockOptions(opts.ClockOptions().SetNowFn(nowFn)) + fsOpts := opts.CommitLogOptions().FilesystemOptions(). + SetFilePathPrefix(dir) + opts = opts. + SetCommitLogOptions(opts.CommitLogOptions(). + SetFilesystemOptions(fsOpts)) + earliestFlush := retention.FlushTimeStart(defaultTestRetentionOpts, now) beforeEarliestFlush := earliestFlush.Add(-defaultTestRetentionOpts.BlockSize()) @@ -968,6 +1000,10 @@ func TestShardWriteAsyncWithAnnotations(t *testing.T) { } func testShardWriteAsync(t *testing.T, writes []testWrite) { + dir, err := ioutil.TempDir("", "testdir") + require.NoError(t, err) + defer os.RemoveAll(dir) + ctrl := gomock.NewController(t) defer ctrl.Finish() @@ -1010,13 +1046,17 @@ func testShardWriteAsync(t *testing.T, writes []testWrite) { opts := DefaultTestOptions(). SetBytesPool(mockBytesPool) + fsOpts := opts.CommitLogOptions().FilesystemOptions(). + SetFilePathPrefix(dir) opts = opts. SetInstrumentOptions( opts.InstrumentOptions(). SetMetricsScope(scope). SetReportInterval(100 * time.Millisecond)). SetClockOptions( - opts.ClockOptions().SetNowFn(nowFn)) + opts.ClockOptions().SetNowFn(nowFn)). + SetCommitLogOptions(opts.CommitLogOptions(). + SetFilesystemOptions(fsOpts)) earliestFlush := retention.FlushTimeStart(defaultTestRetentionOpts, now) beforeEarliestFlush := earliestFlush.Add(-defaultTestRetentionOpts.BlockSize()) @@ -1081,18 +1121,25 @@ func TestShardTickRace(t *testing.T) { opts := DefaultTestOptions() shard := testDatabaseShard(t, opts) defer shard.Close() + shard.Bootstrap(nil) addTestSeries(shard, ident.StringID("foo")) var wg sync.WaitGroup wg.Add(2) go func() { - shard.Tick(context.NewNoOpCanncellable(), time.Now(), namespace.Context{}) + _, err := shard.Tick(context.NewNoOpCanncellable(), time.Now(), namespace.Context{}) + if err != nil { + panic(err) + } wg.Done() }() go func() { - shard.Tick(context.NewNoOpCanncellable(), time.Now(), namespace.Context{}) + _, err := shard.Tick(context.NewNoOpCanncellable(), time.Now(), namespace.Context{}) + if err != nil { + panic(err) + } wg.Done() }() @@ -1117,11 +1164,22 @@ func TestShardTickCleanupSmallBatchSize(t *testing.T) { // This tests ensures the shard returns an error if two ticks are triggered concurrently. func TestShardReturnsErrorForConcurrentTicks(t *testing.T) { + dir, err := ioutil.TempDir("", "testdir") + require.NoError(t, err) + defer os.RemoveAll(dir) + ctrl := gomock.NewController(t) defer ctrl.Finish() opts := DefaultTestOptions() + fsOpts := opts.CommitLogOptions().FilesystemOptions(). + SetFilePathPrefix(dir) + opts = opts. + SetCommitLogOptions(opts.CommitLogOptions(). + SetFilesystemOptions(fsOpts)) + shard := testDatabaseShard(t, opts) + shard.Bootstrap(nil) shard.currRuntimeOptions.tickSleepSeriesBatchSize = 1 shard.currRuntimeOptions.tickSleepPerSeries = time.Millisecond @@ -1144,7 +1202,9 @@ func TestShardReturnsErrorForConcurrentTicks(t *testing.T) { go func() { _, err := shard.Tick(context.NewNoOpCanncellable(), time.Now(), namespace.Context{}) - require.NoError(t, err) + if err != nil { + panic(err) + } closeWg.Done() }() @@ -1453,12 +1513,24 @@ func TestShardRegisterRuntimeOptionsListeners(t *testing.T) { } func TestShardReadEncodedCachesSeriesWithRecentlyReadPolicy(t *testing.T) { + dir, err := ioutil.TempDir("", "testdir") + require.NoError(t, err) + defer os.RemoveAll(dir) + ctrl := gomock.NewController(t) defer ctrl.Finish() - opts := DefaultTestOptions().SetSeriesCachePolicy(series.CacheRecentlyRead) + opts := DefaultTestOptions(). + SetSeriesCachePolicy(series.CacheRecentlyRead) + fsOpts := opts.CommitLogOptions().FilesystemOptions(). + SetFilePathPrefix(dir) + opts = opts. + SetCommitLogOptions(opts.CommitLogOptions(). + SetFilesystemOptions(fsOpts)) + shard := testDatabaseShard(t, opts) defer shard.Close() + require.NoError(t, shard.Bootstrap(nil)) ropts := shard.seriesOpts.RetentionOptions() end := opts.ClockOptions().NowFn()().Truncate(ropts.BlockSize()) From 0927e73111d12cbb7204a12d8857f09562d39c21 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Wed, 24 Jul 2019 17:22:53 -0400 Subject: [PATCH 07/22] fix lint issue --- src/dbnode/storage/types.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index b80fce2d1c..6336556736 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -468,9 +468,7 @@ type databaseShard interface { // Load does the same thing as Bootstrap, except it can be called more than once // and after a shard is bootstrapped already. - Load( - series *result.Map, - ) error + Load(series *result.Map) error // WarmFlush flushes the WarmWrites in this shard. WarmFlush( From a3fa7d047ed3a3675cd711ba08e766f1b3296356 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Wed, 24 Jul 2019 17:33:17 -0400 Subject: [PATCH 08/22] error out if bootstrapped more than once --- src/dbnode/storage/shard.go | 4 ++-- src/dbnode/storage/shard_race_prop_test.go | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 357febfb2f..dc5739dceb 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -73,6 +73,7 @@ var ( errNewShardEntryTagsTypeInvalid = errors.New("new shard entry options error: tags type invalid") errNewShardEntryTagsIterNotAtIndexZero = errors.New("new shard entry options error: tags iter not at index zero") errShardIsNotBootstrapped = errors.New("shard is not bootstrapped") + errShardAlreadyBootstrapped = errors.New("shard is already bootstrapped") errFlushStateIsNotBootstrapped = errors.New("flush state is not bootstrapped") errFlushStateAlreadyBootstrapped = errors.New("flush state is already bootstrapped") ) @@ -1789,9 +1790,8 @@ func (s *dbShard) Bootstrap( ) error { s.Lock() if s.bootstrapState == Bootstrapped { - // TODO(rartoul): This should return an error instead of failing silently. s.Unlock() - return nil + return errShardAlreadyBootstrapped } if s.bootstrapState == Bootstrapping { s.Unlock() diff --git a/src/dbnode/storage/shard_race_prop_test.go b/src/dbnode/storage/shard_race_prop_test.go index 6dc371a755..e0a2de401e 100644 --- a/src/dbnode/storage/shard_race_prop_test.go +++ b/src/dbnode/storage/shard_race_prop_test.go @@ -110,7 +110,6 @@ var fetchBlocksMetadataV2ShardFn testShardReadFn = func(shard *dbShard) { func propTestDatabaseShard(t *testing.T, tickBatchSize int) (*dbShard, Options) { opts := DefaultTestOptions().SetRuntimeOptionsManager(runtime.NewOptionsManager()) shard := testDatabaseShard(t, opts) - shard.Bootstrap(nil) shard.currRuntimeOptions.tickSleepPerSeries = time.Microsecond shard.currRuntimeOptions.tickSleepSeriesBatchSize = tickBatchSize return shard, opts From d490c1427b060504185d97a05fd208af4d0d026d Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Wed, 24 Jul 2019 17:36:14 -0400 Subject: [PATCH 09/22] add unit test for double shard bootstrap --- src/dbnode/storage/shard_test.go | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index e1a034b854..13e1405656 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -110,7 +110,7 @@ func TestShardDontNeedBootstrap(t *testing.T) { require.True(t, shard.newSeriesBootstrapped) } -func TestShardBootstrapState(t *testing.T) { +func TestShardErrorIfDoubleBootstrap(t *testing.T) { opts := DefaultTestOptions() testNs, closer := newTestNamespace(t) defer closer() @@ -120,7 +120,15 @@ func TestShardBootstrapState(t *testing.T) { defer shard.Close() require.Equal(t, Bootstrapped, shard.bootstrapState) - require.Equal(t, Bootstrapped, shard.BootstrapState()) + require.True(t, shard.newSeriesBootstrapped) +} + +func TestShardBootstrapState(t *testing.T) { + opts := DefaultTestOptions() + s := testDatabaseShard(t, opts) + defer s.Close() + require.NoError(t, s.Bootstrap(nil)) + require.Error(t, s.Bootstrap(nil)) } func TestShardFlushStateNotStarted(t *testing.T) { From d5a44967e85b7a6f7407149eca24eb1162c7a3ef Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Wed, 24 Jul 2019 17:37:13 -0400 Subject: [PATCH 10/22] add newline --- src/dbnode/storage/shard.go | 1 + 1 file changed, 1 insertion(+) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index dc5739dceb..23be734fb7 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -2160,6 +2160,7 @@ func (s *dbShard) ColdFlush( multiErr = multiErr.Add(err) continue } + fsID := fs.FileSetFileIdentifier{ Namespace: s.namespace.ID(), Shard: s.ID(), From 1f65061e51c1a8826b48b8f511f777e6cdb61314 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Wed, 24 Jul 2019 17:37:46 -0400 Subject: [PATCH 11/22] newlines --- src/dbnode/storage/shard.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 23be734fb7..2a8f0e263e 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -2366,6 +2366,7 @@ func (s *dbShard) CleanupCompactedFileSets() error { return fmt.Errorf("encountered errors when getting fileset files for prefix %s namespace %s shard %d: %v", filePathPrefix, s.namespace.ID(), s.ID(), err) } + // Get a snapshot of all states here to prevent constantly getting/releasing // locks in a tight loop below. This snapshot won't become stale halfway // through this because flushing and cleanup never happen in parallel. @@ -2374,6 +2375,7 @@ func (s *dbShard) CleanupCompactedFileSets() error { if !bootstrapped { return errFlushStateIsNotBootstrapped } + toDelete := fs.FileSetFilesSlice(make([]fs.FileSetFile, 0, len(filesets))) for _, datafile := range filesets { fileID := datafile.ID From e311f397e30402fd60994ec917bd6f31a97e8ced Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Wed, 24 Jul 2019 17:50:49 -0400 Subject: [PATCH 12/22] rename struct --- src/dbnode/storage/series/buffer_test.go | 4 +- src/dbnode/storage/series/series.go | 8 +-- src/dbnode/storage/series/series_mock.go | 6 +- src/dbnode/storage/series/series_test.go | 60 +++++++++---------- src/dbnode/storage/series/types.go | 19 +++--- .../series_wired_list_interaction_test.go | 2 +- src/dbnode/storage/shard.go | 10 ++-- src/dbnode/storage/shard_test.go | 4 +- 8 files changed, 56 insertions(+), 57 deletions(-) diff --git a/src/dbnode/storage/series/buffer_test.go b/src/dbnode/storage/series/buffer_test.go index 03e2b274a3..3404002112 100644 --- a/src/dbnode/storage/series/buffer_test.go +++ b/src/dbnode/storage/series/buffer_test.go @@ -825,7 +825,7 @@ func TestBufferTickReordersOutOfOrderBuffers(t *testing.T) { assert.Equal(t, 2, len(encoders)) - blockStates := BlockStateSnapshot{ + blockStates := BootstrappedBlockStateSnapshot{ Snapshot: map[xtime.UnixNano]BlockState{ xtime.ToUnixNano(start): BlockState{ WarmRetrievable: true, @@ -905,7 +905,7 @@ func TestBufferRemoveBucket(t *testing.T) { // Simulate that a flush has fully completed on this bucket so that it will. // get removed from the bucket. - blockStates := BlockStateSnapshot{ + blockStates := BootstrappedBlockStateSnapshot{ Snapshot: map[xtime.UnixNano]BlockState{ xtime.ToUnixNano(start): BlockState{ WarmRetrievable: true, diff --git a/src/dbnode/storage/series/series.go b/src/dbnode/storage/series/series.go index 141f490eaa..5962e28945 100644 --- a/src/dbnode/storage/series/series.go +++ b/src/dbnode/storage/series/series.go @@ -416,7 +416,7 @@ func (s *dbSeries) addBlockWithLock(b block.DatabaseBlock) { func (s *dbSeries) Bootstrap( bootstrappedBlocks block.DatabaseSeriesBlocks, - blockStates BlockStateSnapshot, + blockStates BootstrappedBlockStateSnapshot, ) (BootstrapResult, error) { s.Lock() defer func() { @@ -441,7 +441,7 @@ func (s *dbSeries) Bootstrap( func (s *dbSeries) Load( bootstrappedBlocks block.DatabaseSeriesBlocks, - blockStates BlockStateSnapshot, + blockStates BootstrappedBlockStateSnapshot, ) { s.Lock() s.loadWithLock(bootstrappedBlocks, blockStates) @@ -450,7 +450,7 @@ func (s *dbSeries) Load( func (s *dbSeries) loadWithLock( bootstrappedBlocks block.DatabaseSeriesBlocks, - blockStates BlockStateSnapshot, + blockStates BootstrappedBlockStateSnapshot, ) { for _, block := range bootstrappedBlocks.AllBlocks() { blStartNano := xtime.ToUnixNano(block.StartTime()) @@ -608,7 +608,7 @@ func (s *dbSeries) Snapshot( return s.buffer.Snapshot(ctx, blockStart, s.id, s.tags, persistFn, nsCtx) } -func (s *dbSeries) ColdFlushBlockStarts(blockStates BlockStateSnapshot) OptimizedTimes { +func (s *dbSeries) ColdFlushBlockStarts(blockStates BootstrappedBlockStateSnapshot) OptimizedTimes { s.RLock() defer s.RUnlock() diff --git a/src/dbnode/storage/series/series_mock.go b/src/dbnode/storage/series/series_mock.go index 22a5f81dfa..166cb91f23 100644 --- a/src/dbnode/storage/series/series_mock.go +++ b/src/dbnode/storage/series/series_mock.go @@ -64,7 +64,7 @@ func (m *MockDatabaseSeries) EXPECT() *MockDatabaseSeriesMockRecorder { } // Bootstrap mocks base method -func (m *MockDatabaseSeries) Bootstrap(arg0 block.DatabaseSeriesBlocks, arg1 BlockStateSnapshot) (BootstrapResult, error) { +func (m *MockDatabaseSeries) Bootstrap(arg0 block.DatabaseSeriesBlocks, arg1 BootstrappedBlockStateSnapshot) (BootstrapResult, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Bootstrap", arg0, arg1) ret0, _ := ret[0].(BootstrapResult) @@ -91,7 +91,7 @@ func (mr *MockDatabaseSeriesMockRecorder) Close() *gomock.Call { } // ColdFlushBlockStarts mocks base method -func (m *MockDatabaseSeries) ColdFlushBlockStarts(arg0 BlockStateSnapshot) OptimizedTimes { +func (m *MockDatabaseSeries) ColdFlushBlockStarts(arg0 BootstrappedBlockStateSnapshot) OptimizedTimes { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ColdFlushBlockStarts", arg0) ret0, _ := ret[0].(OptimizedTimes) @@ -192,7 +192,7 @@ func (mr *MockDatabaseSeriesMockRecorder) IsEmpty() *gomock.Call { } // Load mocks base method -func (m *MockDatabaseSeries) Load(arg0 block.DatabaseSeriesBlocks, arg1 BlockStateSnapshot) { +func (m *MockDatabaseSeries) Load(arg0 block.DatabaseSeriesBlocks, arg1 BootstrappedBlockStateSnapshot) { m.ctrl.T.Helper() m.ctrl.Call(m, "Load", arg0, arg1) } diff --git a/src/dbnode/storage/series/series_test.go b/src/dbnode/storage/series/series_test.go index 01a9a40f4f..a28e1d9975 100644 --- a/src/dbnode/storage/series/series_test.go +++ b/src/dbnode/storage/series/series_test.go @@ -84,7 +84,7 @@ func newSeriesTestOptions() Options { func TestSeriesEmpty(t *testing.T) { opts := newSeriesTestOptions() series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BlockStateSnapshot{}) + _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) assert.True(t, series.IsEmpty()) } @@ -106,7 +106,7 @@ func TestSeriesWriteFlush(t *testing.T) { return curr })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BlockStateSnapshot{}) + _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) data := []value{ @@ -141,7 +141,7 @@ func TestSeriesSamePointDoesNotWrite(t *testing.T) { return curr })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BlockStateSnapshot{}) + _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) data := []value{ @@ -184,7 +184,7 @@ func TestSeriesWriteFlushRead(t *testing.T) { return curr })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BlockStateSnapshot{}) + _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) data := []value{ @@ -228,13 +228,13 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { f func( series DatabaseSeries, blocks block.DatabaseSeriesBlocks, - blockStates BlockStateSnapshot) + blockStates BootstrappedBlockStateSnapshot) }{ { title: "load", f: func(series DatabaseSeries, blocks block.DatabaseSeriesBlocks, - blockStates BlockStateSnapshot, + blockStates BootstrappedBlockStateSnapshot, ) { series.Load(blocks, blockStates) }}, @@ -242,7 +242,7 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { title: "bootstrap", f: func(series DatabaseSeries, blocks block.DatabaseSeriesBlocks, - blockStates BlockStateSnapshot, + blockStates BootstrappedBlockStateSnapshot, ) { _, err := series.Bootstrap(blocks, blockStates) require.NoError(t, err) @@ -285,7 +285,7 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { blocks = block.NewDatabaseSeriesBlocks(len(loadWrites)) alreadyWarmFlushedBlockStart = curr.Add(blockSize).Truncate(blockSize) notYetWarmFlushedBlockStart = curr.Add(2 * blockSize).Truncate(blockSize) - blockStates = BlockStateSnapshot{ + blockStates = BootstrappedBlockStateSnapshot{ Snapshot: map[xtime.UnixNano]BlockState{ // Exercise both code paths. xtime.ToUnixNano(alreadyWarmFlushedBlockStart): BlockState{ @@ -340,7 +340,7 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { func TestSeriesReadEndBeforeStart(t *testing.T) { opts := newSeriesTestOptions() series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BlockStateSnapshot{}) + _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) ctx := context.NewContext() @@ -356,7 +356,7 @@ func TestSeriesReadEndBeforeStart(t *testing.T) { func TestSeriesFlushNoBlock(t *testing.T) { opts := newSeriesTestOptions() series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BlockStateSnapshot{}) + _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) flushTime := time.Unix(7200, 0) outcome, err := series.WarmFlush(nil, flushTime, nil, namespace.Context{}) @@ -375,7 +375,7 @@ func TestSeriesFlush(t *testing.T) { })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BlockStateSnapshot{}) + _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) ctx := context.NewContext() @@ -402,9 +402,9 @@ func TestSeriesFlush(t *testing.T) { func TestSeriesTickEmptySeries(t *testing.T) { opts := newSeriesTestOptions() series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BlockStateSnapshot{}) + _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) - _, err = series.Tick(NewShardBlockStateSnapshot(true, BlockStateSnapshot{}), namespace.Context{}) + _, err = series.Tick(NewShardBlockStateSnapshot(true, BootstrappedBlockStateSnapshot{}), namespace.Context{}) require.Equal(t, ErrSeriesAllDatapointsExpired, err) } @@ -414,13 +414,13 @@ func TestSeriesTickDrainAndResetBuffer(t *testing.T) { opts := newSeriesTestOptions() series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BlockStateSnapshot{}) + _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) buffer := NewMockdatabaseBuffer(ctrl) series.buffer = buffer buffer.EXPECT().Tick(gomock.Any(), gomock.Any()).Return(bufferTickResult{}) buffer.EXPECT().Stats().Return(bufferStats{wiredBlocks: 1}) - r, err := series.Tick(NewShardBlockStateSnapshot(true, BlockStateSnapshot{}), namespace.Context{}) + r, err := series.Tick(NewShardBlockStateSnapshot(true, BootstrappedBlockStateSnapshot{}), namespace.Context{}) require.NoError(t, err) assert.Equal(t, 1, r.ActiveBlocks) assert.Equal(t, 1, r.WiredBlocks) @@ -438,7 +438,7 @@ func TestSeriesTickNeedsBlockExpiry(t *testing.T) { return curr })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BlockStateSnapshot{}) + _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) blockStart := curr.Add(-ropts.RetentionPeriod()).Add(-ropts.BlockSize()) b := block.NewMockDatabaseBlock(ctrl) @@ -455,7 +455,7 @@ func TestSeriesTickNeedsBlockExpiry(t *testing.T) { series.buffer = buffer buffer.EXPECT().Tick(gomock.Any(), gomock.Any()).Return(bufferTickResult{}) buffer.EXPECT().Stats().Return(bufferStats{wiredBlocks: 1}) - blockStates := BlockStateSnapshot{ + blockStates := BootstrappedBlockStateSnapshot{ Snapshot: map[xtime.UnixNano]BlockState{ xtime.ToUnixNano(blockStart): BlockState{ WarmRetrievable: false, @@ -494,7 +494,7 @@ func TestSeriesTickRecentlyRead(t *testing.T) { series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever - _, err := series.Bootstrap(nil, BlockStateSnapshot{}) + _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) // Test case where block has been read within expiry period - won't be removed @@ -505,7 +505,7 @@ func TestSeriesTickRecentlyRead(t *testing.T) { b.EXPECT().HasMergeTarget().Return(true) series.cachedBlocks.AddBlock(b) - blockStates := BlockStateSnapshot{ + blockStates := BootstrappedBlockStateSnapshot{ Snapshot: map[xtime.UnixNano]BlockState{ xtime.ToUnixNano(curr): BlockState{ WarmRetrievable: true, @@ -538,7 +538,7 @@ func TestSeriesTickRecentlyRead(t *testing.T) { b.EXPECT().HasMergeTarget().Return(true) series.cachedBlocks.AddBlock(b) - blockStates = BlockStateSnapshot{ + blockStates = BootstrappedBlockStateSnapshot{ Snapshot: map[xtime.UnixNano]BlockState{ xtime.ToUnixNano(curr): BlockState{ WarmRetrievable: false, @@ -570,7 +570,7 @@ func TestSeriesTickCacheLRU(t *testing.T) { series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever - _, err := series.Bootstrap(nil, BlockStateSnapshot{}) + _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) // Test case where block was not retrieved from disk - Will be removed @@ -580,7 +580,7 @@ func TestSeriesTickCacheLRU(t *testing.T) { b.EXPECT().Close().Return() series.cachedBlocks.AddBlock(b) - blockStates := BlockStateSnapshot{ + blockStates := BootstrappedBlockStateSnapshot{ Snapshot: map[xtime.UnixNano]BlockState{ xtime.ToUnixNano(curr): BlockState{ WarmRetrievable: true, @@ -620,7 +620,7 @@ func TestSeriesTickCacheLRU(t *testing.T) { _, expiredBlockExists := series.cachedBlocks.BlockAt(curr.Add(-2 * retentionPeriod)) require.Equal(t, true, expiredBlockExists) - blockStates = BlockStateSnapshot{ + blockStates = BootstrappedBlockStateSnapshot{ Snapshot: map[xtime.UnixNano]BlockState{ xtime.ToUnixNano(curr): BlockState{ WarmRetrievable: false, @@ -653,7 +653,7 @@ func TestSeriesTickCacheNone(t *testing.T) { series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever - _, err := series.Bootstrap(nil, BlockStateSnapshot{}) + _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) // Retrievable blocks should be removed @@ -662,7 +662,7 @@ func TestSeriesTickCacheNone(t *testing.T) { b.EXPECT().Close().Return() series.cachedBlocks.AddBlock(b) - blockStates := BlockStateSnapshot{ + blockStates := BootstrappedBlockStateSnapshot{ Snapshot: map[xtime.UnixNano]BlockState{ xtime.ToUnixNano(curr): BlockState{ WarmRetrievable: true, @@ -682,7 +682,7 @@ func TestSeriesTickCacheNone(t *testing.T) { b.EXPECT().HasMergeTarget().Return(true) series.cachedBlocks.AddBlock(b) - blockStates = BlockStateSnapshot{ + blockStates = BootstrappedBlockStateSnapshot{ Snapshot: map[xtime.UnixNano]BlockState{ xtime.ToUnixNano(curr): BlockState{ WarmRetrievable: false, @@ -745,7 +745,7 @@ func TestSeriesTickCachedBlockRemove(t *testing.T) { series.buffer = buffer assert.Equal(t, 3, series.cachedBlocks.Len()) - blockStates := BlockStateSnapshot{} + blockStates := BootstrappedBlockStateSnapshot{} shardBlockStates := NewShardBlockStateSnapshot(true, blockStates) _, err := series.Tick(shardBlockStates, namespace.Context{}) require.NoError(t, err) @@ -783,7 +783,7 @@ func TestSeriesFetchBlocks(t *testing.T) { Return([]block.FetchBlockResult{block.NewFetchBlockResult(starts[2], nil, nil)}) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BlockStateSnapshot{}) + _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) series.cachedBlocks = blocks @@ -853,7 +853,7 @@ func TestSeriesFetchBlocksMetadata(t *testing.T) { Return(expectedResults, nil) series := NewDatabaseSeries(ident.StringID("bar"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BlockStateSnapshot{}) + _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) mockBlocks := block.NewMockDatabaseSeriesBlocks(ctrl) mockBlocks.EXPECT().AllBlocks().Return(blocks) @@ -991,7 +991,7 @@ func TestSeriesWriteReadFromTheSameBucket(t *testing.T) { return curr })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BlockStateSnapshot{}) + _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) ctx := context.NewContext() diff --git a/src/dbnode/storage/series/types.go b/src/dbnode/storage/series/types.go index d283da7490..f1991d626b 100644 --- a/src/dbnode/storage/series/types.go +++ b/src/dbnode/storage/series/types.go @@ -103,11 +103,11 @@ type DatabaseSeries interface { IsBootstrapped() bool // Bootstrap merges the raw series bootstrapped along with any buffered data. - Bootstrap(blocks block.DatabaseSeriesBlocks, blockStates BlockStateSnapshot) (BootstrapResult, error) + Bootstrap(blocks block.DatabaseSeriesBlocks, blockStates BootstrappedBlockStateSnapshot) (BootstrapResult, error) // Load does the same thing as Bootstrap except it should be used for data that did // not originate from the Bootstrap process (like background repairs). - Load(blocks block.DatabaseSeriesBlocks, blockStates BlockStateSnapshot) + Load(blocks block.DatabaseSeriesBlocks, blockStates BootstrappedBlockStateSnapshot) // WarmFlush flushes the WarmWrites of this series for a given start time. WarmFlush( @@ -127,7 +127,7 @@ type DatabaseSeries interface { ) error // ColdFlushBlockStarts returns the block starts that need cold flushes. - ColdFlushBlockStarts(blockStates BlockStateSnapshot) OptimizedTimes + ColdFlushBlockStarts(blockStates BootstrappedBlockStateSnapshot) OptimizedTimes // Close will close the series and if pooled returned to the pool. Close() @@ -180,13 +180,13 @@ type QueryableBlockRetriever interface { // a moment in time. type ShardBlockStateSnapshot struct { bootstrapped bool - snapshot BlockStateSnapshot + snapshot BootstrappedBlockStateSnapshot } // NewShardBlockStateSnapshot constructs a new NewShardBlockStateSnapshot. func NewShardBlockStateSnapshot( bootstrapped bool, - snapshot BlockStateSnapshot, + snapshot BootstrappedBlockStateSnapshot, ) ShardBlockStateSnapshot { return ShardBlockStateSnapshot{ bootstrapped: bootstrapped, @@ -194,15 +194,14 @@ func NewShardBlockStateSnapshot( } } -// Snapshot returns a BlockStateSnapshot and a boolean indicating whether the +// Snapshot returns a BootstrappedBlockStateSnapshot and a boolean indicating whether the // snapshot is bootstrapped or not. -func (s *ShardBlockStateSnapshot) Snapshot() (BlockStateSnapshot, bool) { +func (s *ShardBlockStateSnapshot) Snapshot() (BootstrappedBlockStateSnapshot, bool) { return s.snapshot, s.bootstrapped } -// BlockStateSnapshot represents a bootstrapped shard block state snapshot. -// TODO(rartoul): Rename? -type BlockStateSnapshot struct { +// BootstrappedBlockStateSnapshot represents a bootstrapped shard block state snapshot. +type BootstrappedBlockStateSnapshot struct { Snapshot map[xtime.UnixNano]BlockState } diff --git a/src/dbnode/storage/series_wired_list_interaction_test.go b/src/dbnode/storage/series_wired_list_interaction_test.go index 2f98d24554..ccf5fe9a17 100644 --- a/src/dbnode/storage/series_wired_list_interaction_test.go +++ b/src/dbnode/storage/series_wired_list_interaction_test.go @@ -91,7 +91,7 @@ func TestSeriesWiredListConcurrentInteractions(t *testing.T) { ) seriesEntry.Reset(id, ident.Tags{}, nil, shard.seriesOnRetrieveBlock, shard, shard.seriesOpts) - seriesEntry.Bootstrap(nil, series.BlockStateSnapshot{}) + seriesEntry.Bootstrap(nil, series.BootstrappedBlockStateSnapshot{}) shard.Lock() shard.insertNewShardEntryWithLock(lookup.NewEntry(seriesEntry, 0)) shard.Unlock() diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 2a8f0e263e..0292fe7fa3 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -394,7 +394,7 @@ func (s *dbShard) BlockStatesSnapshot() series.ShardBlockStateSnapshot { defer s.flushState.RUnlock() if !s.flushState.bootstrapped { - return series.NewShardBlockStateSnapshot(false, series.BlockStateSnapshot{}) + return series.NewShardBlockStateSnapshot(false, series.BootstrappedBlockStateSnapshot{}) } states := s.flushState.statesByTime @@ -406,7 +406,7 @@ func (s *dbShard) BlockStatesSnapshot() series.ShardBlockStateSnapshot { } } - return series.NewShardBlockStateSnapshot(true, series.BlockStateSnapshot{ + return series.NewShardBlockStateSnapshot(true, series.BootstrappedBlockStateSnapshot{ Snapshot: snapshot, }) } @@ -1258,7 +1258,7 @@ func (s *dbShard) insertSeriesSync( } if s.newSeriesBootstrapped { - _, err := entry.Series.Bootstrap(nil, series.BlockStateSnapshot{}) + _, err := entry.Series.Bootstrap(nil, series.BootstrappedBlockStateSnapshot{}) if err != nil { entry = nil // Don't increment the writer count for this series return nil, err @@ -1344,7 +1344,7 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { // Insert still pending, perform the insert entry = inserts[i].entry if s.newSeriesBootstrapped { - _, err := entry.Series.Bootstrap(nil, series.BlockStateSnapshot{}) + _, err := entry.Series.Bootstrap(nil, series.BootstrappedBlockStateSnapshot{}) if err != nil { s.metrics.insertAsyncBootstrapErrors.Inc(1) } @@ -1827,7 +1827,7 @@ func (s *dbShard) Bootstrap( if seriesEntry.IsBootstrapped() { return true } - _, err := seriesEntry.Bootstrap(nil, series.BlockStateSnapshot{}) + _, err := seriesEntry.Bootstrap(nil, series.BootstrappedBlockStateSnapshot{}) multiErr = multiErr.Add(err) return true }) diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index 13e1405656..884a009238 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -273,7 +273,7 @@ func TestShardBootstrapWithFlushVersion(t *testing.T) { // Ensure that the bootstrapped flush/block states get passed to the series.Bootstrap() // method properly. - blockStateSnapshot := series.BlockStateSnapshot{ + blockStateSnapshot := series.BootstrappedBlockStateSnapshot{ Snapshot: map[xtime.UnixNano]series.BlockState{}, } for i, blockStart := range blockStarts { @@ -835,7 +835,7 @@ func addTestSeriesWithCount(shard *dbShard, id ident.ID, count int32) series.Dat func addTestSeriesWithCountAndBootstrap(shard *dbShard, id ident.ID, count int32, bootstrap bool) series.DatabaseSeries { seriesEntry := series.NewDatabaseSeries(id, ident.Tags{}, shard.seriesOpts) if bootstrap { - seriesEntry.Bootstrap(nil, series.BlockStateSnapshot{}) + seriesEntry.Bootstrap(nil, series.BootstrappedBlockStateSnapshot{}) } shard.Lock() entry := lookup.NewEntry(seriesEntry, 0) From bb88a6168b334116e3d828b2a31f07ae48dd7198 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Wed, 24 Jul 2019 18:00:31 -0400 Subject: [PATCH 13/22] fix comments --- src/dbnode/storage/series/series.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/dbnode/storage/series/series.go b/src/dbnode/storage/series/series.go index 5962e28945..558c921fbf 100644 --- a/src/dbnode/storage/series/series.go +++ b/src/dbnode/storage/series/series.go @@ -206,7 +206,8 @@ func (s *dbSeries) updateBlocksWithLock( // Potentially unwire var unwired, shouldUnwire bool blockStatesSnapshot, bootstrapped := blockStates.Snapshot() - // Only use the block states data to make decision if it has been bootstrapped already. + // Only use block state snapshot information to make eviction decisions if the block state + // has been properly bootstrapped already. if bootstrapped { // Makes sure that the block has been flushed, which // prevents us from unwiring blocks that haven't been flushed yet which From 0aeca80777450ca9656cf6e8252c13a411c5e7a5 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 29 Jul 2019 09:49:05 -0400 Subject: [PATCH 14/22] Convert method to not be pointer type --- src/dbnode/storage/series/types.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/dbnode/storage/series/types.go b/src/dbnode/storage/series/types.go index f1991d626b..3789357e39 100644 --- a/src/dbnode/storage/series/types.go +++ b/src/dbnode/storage/series/types.go @@ -196,7 +196,7 @@ func NewShardBlockStateSnapshot( // Snapshot returns a BootstrappedBlockStateSnapshot and a boolean indicating whether the // snapshot is bootstrapped or not. -func (s *ShardBlockStateSnapshot) Snapshot() (BootstrappedBlockStateSnapshot, bool) { +func (s ShardBlockStateSnapshot) Snapshot() (BootstrappedBlockStateSnapshot, bool) { return s.snapshot, s.bootstrapped } From 6e2c944c636c5509b62c12423fb26505007a6553 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 29 Jul 2019 12:23:40 -0400 Subject: [PATCH 15/22] Address feedback --- .../mocks/gomock_reflect_239576158/prog.go | 66 ++++++++++++++ .../fs/gomock_reflect_525677702/prog.go | 88 +++++++++++++++++++ src/dbnode/storage/series/buffer.go | 2 +- src/dbnode/storage/series/series.go | 20 ++++- src/dbnode/storage/series/series_mock.go | 26 ++---- .../storage/series/series_parallel_test.go | 2 +- src/dbnode/storage/series/series_test.go | 36 ++++---- src/dbnode/storage/series/types.go | 47 ++++++---- .../series_wired_list_interaction_test.go | 5 +- src/dbnode/storage/shard.go | 54 ++++++++---- src/dbnode/storage/shard_test.go | 10 +-- 11 files changed, 272 insertions(+), 84 deletions(-) create mode 100644 src/dbnode/generated/mocks/gomock_reflect_239576158/prog.go create mode 100644 src/dbnode/persist/fs/gomock_reflect_525677702/prog.go diff --git a/src/dbnode/generated/mocks/gomock_reflect_239576158/prog.go b/src/dbnode/generated/mocks/gomock_reflect_239576158/prog.go new file mode 100644 index 0000000000..16d86e34e1 --- /dev/null +++ b/src/dbnode/generated/mocks/gomock_reflect_239576158/prog.go @@ -0,0 +1,66 @@ + +package main + +import ( + "encoding/gob" + "flag" + "fmt" + "os" + "path" + "reflect" + + "github.com/golang/mock/mockgen/model" + + pkg_ "github.com/m3db/m3/src/cmd/services/m3dbnode/config" +) + +var output = flag.String("output", "", "The output file name, or empty to use stdout.") + +func main() { + flag.Parse() + + its := []struct{ + sym string + typ reflect.Type + }{ + + { "BootstrapConfigurationValidator", reflect.TypeOf((*pkg_.BootstrapConfigurationValidator)(nil)).Elem()}, + + } + pkg := &model.Package{ + // NOTE: This behaves contrary to documented behaviour if the + // package name is not the final component of the import path. + // The reflect package doesn't expose the package name, though. + Name: path.Base("github.com/m3db/m3/src/cmd/services/m3dbnode/config"), + } + + for _, it := range its { + intf, err := model.InterfaceFromInterfaceType(it.typ) + if err != nil { + fmt.Fprintf(os.Stderr, "Reflection: %v\n", err) + os.Exit(1) + } + intf.Name = it.sym + pkg.Interfaces = append(pkg.Interfaces, intf) + } + + outfile := os.Stdout + if len(*output) != 0 { + var err error + outfile, err = os.Create(*output) + if err != nil { + fmt.Fprintf(os.Stderr, "failed to open output file %q", *output) + } + defer func() { + if err := outfile.Close(); err != nil { + fmt.Fprintf(os.Stderr, "failed to close output file %q", *output) + os.Exit(1) + } + }() + } + + if err := gob.NewEncoder(outfile).Encode(pkg); err != nil { + fmt.Fprintf(os.Stderr, "gob encode: %v\n", err) + os.Exit(1) + } +} diff --git a/src/dbnode/persist/fs/gomock_reflect_525677702/prog.go b/src/dbnode/persist/fs/gomock_reflect_525677702/prog.go new file mode 100644 index 0000000000..bdf503de9c --- /dev/null +++ b/src/dbnode/persist/fs/gomock_reflect_525677702/prog.go @@ -0,0 +1,88 @@ + +package main + +import ( + "encoding/gob" + "flag" + "fmt" + "os" + "path" + "reflect" + + "github.com/golang/mock/mockgen/model" + + pkg_ "github.com/m3db/m3/src/dbnode/persist/fs" +) + +var output = flag.String("output", "", "The output file name, or empty to use stdout.") + +func main() { + flag.Parse() + + its := []struct{ + sym string + typ reflect.Type + }{ + + { "DataFileSetWriter", reflect.TypeOf((*pkg_.DataFileSetWriter)(nil)).Elem()}, + + { "DataFileSetReader", reflect.TypeOf((*pkg_.DataFileSetReader)(nil)).Elem()}, + + { "DataFileSetSeeker", reflect.TypeOf((*pkg_.DataFileSetSeeker)(nil)).Elem()}, + + { "IndexFileSetWriter", reflect.TypeOf((*pkg_.IndexFileSetWriter)(nil)).Elem()}, + + { "IndexFileSetReader", reflect.TypeOf((*pkg_.IndexFileSetReader)(nil)).Elem()}, + + { "IndexSegmentFileSetWriter", reflect.TypeOf((*pkg_.IndexSegmentFileSetWriter)(nil)).Elem()}, + + { "IndexSegmentFileSet", reflect.TypeOf((*pkg_.IndexSegmentFileSet)(nil)).Elem()}, + + { "IndexSegmentFile", reflect.TypeOf((*pkg_.IndexSegmentFile)(nil)).Elem()}, + + { "SnapshotMetadataFileWriter", reflect.TypeOf((*pkg_.SnapshotMetadataFileWriter)(nil)).Elem()}, + + { "DataFileSetSeekerManager", reflect.TypeOf((*pkg_.DataFileSetSeekerManager)(nil)).Elem()}, + + { "ConcurrentDataFileSetSeeker", reflect.TypeOf((*pkg_.ConcurrentDataFileSetSeeker)(nil)).Elem()}, + + { "MergeWith", reflect.TypeOf((*pkg_.MergeWith)(nil)).Elem()}, + + } + pkg := &model.Package{ + // NOTE: This behaves contrary to documented behaviour if the + // package name is not the final component of the import path. + // The reflect package doesn't expose the package name, though. + Name: path.Base("github.com/m3db/m3/src/dbnode/persist/fs"), + } + + for _, it := range its { + intf, err := model.InterfaceFromInterfaceType(it.typ) + if err != nil { + fmt.Fprintf(os.Stderr, "Reflection: %v\n", err) + os.Exit(1) + } + intf.Name = it.sym + pkg.Interfaces = append(pkg.Interfaces, intf) + } + + outfile := os.Stdout + if len(*output) != 0 { + var err error + outfile, err = os.Create(*output) + if err != nil { + fmt.Fprintf(os.Stderr, "failed to open output file %q", *output) + } + defer func() { + if err := outfile.Close(); err != nil { + fmt.Fprintf(os.Stderr, "failed to close output file %q", *output) + os.Exit(1) + } + }() + } + + if err := gob.NewEncoder(outfile).Encode(pkg); err != nil { + fmt.Fprintf(os.Stderr, "gob encode: %v\n", err) + os.Exit(1) + } +} diff --git a/src/dbnode/storage/series/buffer.go b/src/dbnode/storage/series/buffer.go index 9abbd4ee15..36426ee45d 100644 --- a/src/dbnode/storage/series/buffer.go +++ b/src/dbnode/storage/series/buffer.go @@ -377,7 +377,7 @@ func (b *dbBuffer) Tick(blockStates ShardBlockStateSnapshot, nsCtx namespace.Con // 2) remove a lower versioned bucket. // Retrievable and higher versioned buckets will be left to be // collected in the next tick. - blockStateSnapshot, bootstrapped := blockStates.Snapshot() + blockStateSnapshot, bootstrapped := blockStates.UnwrapValue() // Only use block state snapshot information to make eviction decisions if the block state // has been properly bootstrapped already. if bootstrapped { diff --git a/src/dbnode/storage/series/series.go b/src/dbnode/storage/series/series.go index 558c921fbf..31fef27f6e 100644 --- a/src/dbnode/storage/series/series.go +++ b/src/dbnode/storage/series/series.go @@ -205,7 +205,7 @@ func (s *dbSeries) updateBlocksWithLock( // Potentially unwire var unwired, shouldUnwire bool - blockStatesSnapshot, bootstrapped := blockStates.Snapshot() + blockStatesSnapshot, bootstrapped := blockStates.UnwrapValue() // Only use block state snapshot information to make eviction decisions if the block state // has been properly bootstrapped already. if bootstrapped { @@ -415,7 +415,21 @@ func (s *dbSeries) addBlockWithLock(b block.DatabaseBlock) { s.cachedBlocks.AddBlock(b) } -func (s *dbSeries) Bootstrap( +func (s *dbSeries) Load( + opts LoadOptions, + bootstrappedBlocks block.DatabaseSeriesBlocks, + blockStates BootstrappedBlockStateSnapshot, +) (LoadResult, error) { + if opts.Bootstrap { + bsResult, err := s.bootstrap(bootstrappedBlocks, blockStates) + return LoadResult{Bootstrap: bsResult}, err + } + + s.load(bootstrappedBlocks, blockStates) + return LoadResult{}, nil +} + +func (s *dbSeries) bootstrap( bootstrappedBlocks block.DatabaseSeriesBlocks, blockStates BootstrappedBlockStateSnapshot, ) (BootstrapResult, error) { @@ -440,7 +454,7 @@ func (s *dbSeries) Bootstrap( return result, nil } -func (s *dbSeries) Load( +func (s *dbSeries) load( bootstrappedBlocks block.DatabaseSeriesBlocks, blockStates BootstrappedBlockStateSnapshot, ) { diff --git a/src/dbnode/storage/series/series_mock.go b/src/dbnode/storage/series/series_mock.go index 166cb91f23..b845c5f905 100644 --- a/src/dbnode/storage/series/series_mock.go +++ b/src/dbnode/storage/series/series_mock.go @@ -63,21 +63,6 @@ func (m *MockDatabaseSeries) EXPECT() *MockDatabaseSeriesMockRecorder { return m.recorder } -// Bootstrap mocks base method -func (m *MockDatabaseSeries) Bootstrap(arg0 block.DatabaseSeriesBlocks, arg1 BootstrappedBlockStateSnapshot) (BootstrapResult, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Bootstrap", arg0, arg1) - ret0, _ := ret[0].(BootstrapResult) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// Bootstrap indicates an expected call of Bootstrap -func (mr *MockDatabaseSeriesMockRecorder) Bootstrap(arg0, arg1 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Bootstrap", reflect.TypeOf((*MockDatabaseSeries)(nil).Bootstrap), arg0, arg1) -} - // Close mocks base method func (m *MockDatabaseSeries) Close() { m.ctrl.T.Helper() @@ -192,15 +177,18 @@ func (mr *MockDatabaseSeriesMockRecorder) IsEmpty() *gomock.Call { } // Load mocks base method -func (m *MockDatabaseSeries) Load(arg0 block.DatabaseSeriesBlocks, arg1 BootstrappedBlockStateSnapshot) { +func (m *MockDatabaseSeries) Load(arg0 LoadOptions, arg1 block.DatabaseSeriesBlocks, arg2 BootstrappedBlockStateSnapshot) (LoadResult, error) { m.ctrl.T.Helper() - m.ctrl.Call(m, "Load", arg0, arg1) + ret := m.ctrl.Call(m, "Load", arg0, arg1, arg2) + ret0, _ := ret[0].(LoadResult) + ret1, _ := ret[1].(error) + return ret0, ret1 } // Load indicates an expected call of Load -func (mr *MockDatabaseSeriesMockRecorder) Load(arg0, arg1 interface{}) *gomock.Call { +func (mr *MockDatabaseSeriesMockRecorder) Load(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Load", reflect.TypeOf((*MockDatabaseSeries)(nil).Load), arg0, arg1) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Load", reflect.TypeOf((*MockDatabaseSeries)(nil).Load), arg0, arg1, arg2) } // NumActiveBlocks mocks base method diff --git a/src/dbnode/storage/series/series_parallel_test.go b/src/dbnode/storage/series/series_parallel_test.go index 5bb4dd55fd..9e4afe2b52 100644 --- a/src/dbnode/storage/series/series_parallel_test.go +++ b/src/dbnode/storage/series/series_parallel_test.go @@ -46,7 +46,7 @@ func TestSeriesWriteReadParallel(t *testing.T) { series = NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) ) - _, err := series.Bootstrap(nil, nil) + _, err := series.Bootstrap(LoadOptions{Bootstrap: true}, nil, nil) assert.NoError(t, err) ctx := context.NewContext() diff --git a/src/dbnode/storage/series/series_test.go b/src/dbnode/storage/series/series_test.go index a28e1d9975..666a5707d1 100644 --- a/src/dbnode/storage/series/series_test.go +++ b/src/dbnode/storage/series/series_test.go @@ -84,7 +84,7 @@ func newSeriesTestOptions() Options { func TestSeriesEmpty(t *testing.T) { opts := newSeriesTestOptions() series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) assert.True(t, series.IsEmpty()) } @@ -106,7 +106,7 @@ func TestSeriesWriteFlush(t *testing.T) { return curr })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) data := []value{ @@ -141,7 +141,7 @@ func TestSeriesSamePointDoesNotWrite(t *testing.T) { return curr })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) data := []value{ @@ -184,7 +184,7 @@ func TestSeriesWriteFlushRead(t *testing.T) { return curr })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) data := []value{ @@ -236,7 +236,7 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { blocks block.DatabaseSeriesBlocks, blockStates BootstrappedBlockStateSnapshot, ) { - series.Load(blocks, blockStates) + series.Load(LoadOptions{}, blocks, blockStates) }}, { title: "bootstrap", @@ -244,7 +244,7 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { blocks block.DatabaseSeriesBlocks, blockStates BootstrappedBlockStateSnapshot, ) { - _, err := series.Bootstrap(blocks, blockStates) + _, err := series.Load(LoadOptions{Bootstrap: true}, blocks, blockStates) require.NoError(t, err) }}, } @@ -340,7 +340,7 @@ func TestSeriesBootstrapAndLoad(t *testing.T) { func TestSeriesReadEndBeforeStart(t *testing.T) { opts := newSeriesTestOptions() series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) ctx := context.NewContext() @@ -356,7 +356,7 @@ func TestSeriesReadEndBeforeStart(t *testing.T) { func TestSeriesFlushNoBlock(t *testing.T) { opts := newSeriesTestOptions() series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) flushTime := time.Unix(7200, 0) outcome, err := series.WarmFlush(nil, flushTime, nil, namespace.Context{}) @@ -375,7 +375,7 @@ func TestSeriesFlush(t *testing.T) { })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) ctx := context.NewContext() @@ -402,7 +402,7 @@ func TestSeriesFlush(t *testing.T) { func TestSeriesTickEmptySeries(t *testing.T) { opts := newSeriesTestOptions() series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) _, err = series.Tick(NewShardBlockStateSnapshot(true, BootstrappedBlockStateSnapshot{}), namespace.Context{}) require.Equal(t, ErrSeriesAllDatapointsExpired, err) @@ -414,7 +414,7 @@ func TestSeriesTickDrainAndResetBuffer(t *testing.T) { opts := newSeriesTestOptions() series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) buffer := NewMockdatabaseBuffer(ctrl) series.buffer = buffer @@ -438,7 +438,7 @@ func TestSeriesTickNeedsBlockExpiry(t *testing.T) { return curr })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) blockStart := curr.Add(-ropts.RetentionPeriod()).Add(-ropts.BlockSize()) b := block.NewMockDatabaseBlock(ctrl) @@ -494,7 +494,7 @@ func TestSeriesTickRecentlyRead(t *testing.T) { series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever - _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) // Test case where block has been read within expiry period - won't be removed @@ -570,7 +570,7 @@ func TestSeriesTickCacheLRU(t *testing.T) { series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever - _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) // Test case where block was not retrieved from disk - Will be removed @@ -653,7 +653,7 @@ func TestSeriesTickCacheNone(t *testing.T) { series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) blockRetriever := NewMockQueryableBlockRetriever(ctrl) series.blockRetriever = blockRetriever - _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) // Retrievable blocks should be removed @@ -783,7 +783,7 @@ func TestSeriesFetchBlocks(t *testing.T) { Return([]block.FetchBlockResult{block.NewFetchBlockResult(starts[2], nil, nil)}) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) series.cachedBlocks = blocks @@ -853,7 +853,7 @@ func TestSeriesFetchBlocksMetadata(t *testing.T) { Return(expectedResults, nil) series := NewDatabaseSeries(ident.StringID("bar"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) mockBlocks := block.NewMockDatabaseSeriesBlocks(ctrl) mockBlocks.EXPECT().AllBlocks().Return(blocks) @@ -991,7 +991,7 @@ func TestSeriesWriteReadFromTheSameBucket(t *testing.T) { return curr })) series := NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) - _, err := series.Bootstrap(nil, BootstrappedBlockStateSnapshot{}) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) ctx := context.NewContext() diff --git a/src/dbnode/storage/series/types.go b/src/dbnode/storage/series/types.go index 3789357e39..2fc064f168 100644 --- a/src/dbnode/storage/series/types.go +++ b/src/dbnode/storage/series/types.go @@ -102,12 +102,12 @@ type DatabaseSeries interface { // IsBootstrapped returns whether the series is bootstrapped or not. IsBootstrapped() bool - // Bootstrap merges the raw series bootstrapped along with any buffered data. - Bootstrap(blocks block.DatabaseSeriesBlocks, blockStates BootstrappedBlockStateSnapshot) (BootstrapResult, error) - - // Load does the same thing as Bootstrap except it should be used for data that did - // not originate from the Bootstrap process (like background repairs). - Load(blocks block.DatabaseSeriesBlocks, blockStates BootstrappedBlockStateSnapshot) + // Load loads data into the series. + Load( + opts LoadOptions, + blocks block.DatabaseSeriesBlocks, + blockStates BootstrappedBlockStateSnapshot, + ) (LoadResult, error) // WarmFlush flushes the WarmWrites of this series for a given start time. WarmFlush( @@ -194,9 +194,9 @@ func NewShardBlockStateSnapshot( } } -// Snapshot returns a BootstrappedBlockStateSnapshot and a boolean indicating whether the +// UnwrapValue returns a BootstrappedBlockStateSnapshot and a boolean indicating whether the // snapshot is bootstrapped or not. -func (s ShardBlockStateSnapshot) Snapshot() (BootstrappedBlockStateSnapshot, bool) { +func (s ShardBlockStateSnapshot) UnwrapValue() (BootstrappedBlockStateSnapshot, bool) { return s.snapshot, s.bootstrapped } @@ -264,15 +264,6 @@ const ( FlushOutcomeFlushedToDisk ) -// BootstrapResult contains information about the result of bootstrapping a series. -// It is returned from the series Bootstrap method primarily so the caller can aggregate -// and emit metrics instead of the series itself having to store additional fields (which -// would be costly because we have millions of them.) -type BootstrapResult struct { - NumBlocksMovedToBuffer int64 - NumBlocksMerged int64 -} - // Options represents the options for series type Options interface { // Validate validates the options @@ -417,3 +408,25 @@ type WriteOptions struct { // TransformOptions describes transformation options for incoming writes. TransformOptions WriteTransformOptions } + +// LoadOptions contains the options for the Load() method. +type LoadOptions struct { + // Whether the call to Bootstrap should be considered a "true" bootstrap + // or if additional data is being loaded after the fact (as in the case + // of repairs). + Bootstrap bool +} + +// LoadResult contains the return information for the Load() method. +type LoadResult struct { + Bootstrap BootstrapResult +} + +// BootstrapResult contains information about the result of bootstrapping a series. +// It is returned from the series Bootstrap method primarily so the caller can aggregate +// and emit metrics instead of the series itself having to store additional fields (which +// would be costly because we have millions of them.) +type BootstrapResult struct { + NumBlocksMovedToBuffer int64 + NumBlocksMerged int64 +} diff --git a/src/dbnode/storage/series_wired_list_interaction_test.go b/src/dbnode/storage/series_wired_list_interaction_test.go index ccf5fe9a17..86c2fcc6b1 100644 --- a/src/dbnode/storage/series_wired_list_interaction_test.go +++ b/src/dbnode/storage/series_wired_list_interaction_test.go @@ -91,7 +91,10 @@ func TestSeriesWiredListConcurrentInteractions(t *testing.T) { ) seriesEntry.Reset(id, ident.Tags{}, nil, shard.seriesOnRetrieveBlock, shard, shard.seriesOpts) - seriesEntry.Bootstrap(nil, series.BootstrappedBlockStateSnapshot{}) + seriesEntry.Load( + series.LoadOptions{Bootstrap: true}, + nil, + series.BootstrappedBlockStateSnapshot{}) shard.Lock() shard.insertNewShardEntryWithLock(lookup.NewEntry(seriesEntry, 0)) shard.Unlock() diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 0292fe7fa3..299fcf16ec 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -76,6 +76,7 @@ var ( errShardAlreadyBootstrapped = errors.New("shard is already bootstrapped") errFlushStateIsNotBootstrapped = errors.New("flush state is not bootstrapped") errFlushStateAlreadyBootstrapped = errors.New("flush state is already bootstrapped") + errTriedToLoadNilSeries = errors.New("tried to load nil series into shard") ) type filesetsFn func( @@ -1258,7 +1259,10 @@ func (s *dbShard) insertSeriesSync( } if s.newSeriesBootstrapped { - _, err := entry.Series.Bootstrap(nil, series.BootstrappedBlockStateSnapshot{}) + _, err := entry.Series.Load( + series.LoadOptions{Bootstrap: true}, + nil, + series.BootstrappedBlockStateSnapshot{}) if err != nil { entry = nil // Don't increment the writer count for this series return nil, err @@ -1344,7 +1348,10 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { // Insert still pending, perform the insert entry = inserts[i].entry if s.newSeriesBootstrapped { - _, err := entry.Series.Bootstrap(nil, series.BootstrappedBlockStateSnapshot{}) + _, err := entry.Series.Load( + series.LoadOptions{Bootstrap: true}, + nil, + series.BootstrappedBlockStateSnapshot{}) if err != nil { s.metrics.insertAsyncBootstrapErrors.Inc(1) } @@ -1802,7 +1809,7 @@ func (s *dbShard) Bootstrap( // Iterate flushed time ranges to determine which blocks are retrievable. This step happens // first because the flushState information is required for bootstrapping individual series - // (will be passed to series.Bootstrap() as BlockState). + // (will be passed to series.Load() as BlockState). s.bootstrapFlushStates() multiErr := xerrors.NewMultiError() @@ -1827,7 +1834,10 @@ func (s *dbShard) Bootstrap( if seriesEntry.IsBootstrapped() { return true } - _, err := seriesEntry.Bootstrap(nil, series.BootstrappedBlockStateSnapshot{}) + _, err := seriesEntry.Load( + series.LoadOptions{Bootstrap: true}, + nil, + series.BootstrappedBlockStateSnapshot{}) multiErr = multiErr.Add(err) return true }) @@ -1847,8 +1857,12 @@ func (s *dbShard) Bootstrap( } func (s *dbShard) Load( - series *result.Map, + seriesToLoad *result.Map, ) error { + if seriesToLoad == nil { + return errTriedToLoadNilSeries + } + s.Lock() // Don't allow loads until the shard is bootstrapped because the shard flush states need to be // bootstrapped in order to safely load blocks. This also keeps things simpler to reason about. @@ -1858,15 +1872,15 @@ func (s *dbShard) Load( } s.Unlock() - _, err := s.loadSeries(series, false) + _, err := s.loadSeries(seriesToLoad, false) return err } func (s *dbShard) loadSeries( - series *result.Map, + seriesToLoad *result.Map, bootstrap bool, ) (dbShardBootstrapResult, error) { - if series == nil { + if seriesToLoad == nil { return dbShardBootstrapResult{}, nil } @@ -1878,11 +1892,11 @@ func (s *dbShard) loadSeries( // Safe to use the same snapshot for all the series since the block states can't change while // this is running since no warm/cold flushes can occur while the bootstrap is ongoing. blockStates := s.BlockStatesSnapshot() - blockStatesSnapshot, bootstrapped := blockStates.Snapshot() + blockStatesSnapshot, bootstrapped := blockStates.UnwrapValue() if !bootstrapped { return dbShardBootstrapResult{}, errFlushStateIsNotBootstrapped } - for _, elem := range series.Iter() { + for _, elem := range seriesToLoad.Iter() { dbBlocks := elem.Value() // First lookup if series already exists @@ -1911,14 +1925,16 @@ func (s *dbShard) loadSeries( dbBlocks.Tags.Finalize() } + loadOpts := series.LoadOptions{Bootstrap: bootstrap} + loadResult, err := entry.Series.Load( + loadOpts, + dbBlocks.Blocks, + blockStatesSnapshot) + if err != nil { + multiErr = multiErr.Add(err) + } if bootstrap { - bsResult, err := entry.Series.Bootstrap(dbBlocks.Blocks, blockStatesSnapshot) - if err != nil { - multiErr = multiErr.Add(err) - } - shardBootstrapResult.update(bsResult) - } else { - entry.Series.Load(dbBlocks.Blocks, blockStatesSnapshot) + shardBootstrapResult.update(loadResult.Bootstrap) } // Cannot close blocks once done as series takes ref to them. @@ -2091,7 +2107,7 @@ func (s *dbShard) ColdFlush( ) blockStates := s.BlockStatesSnapshot() - blockStatesSnapshot, bootstrapped := blockStates.Snapshot() + blockStatesSnapshot, bootstrapped := blockStates.UnwrapValue() if !bootstrapped { return errFlushStateIsNotBootstrapped } @@ -2371,7 +2387,7 @@ func (s *dbShard) CleanupCompactedFileSets() error { // locks in a tight loop below. This snapshot won't become stale halfway // through this because flushing and cleanup never happen in parallel. blockStates := s.BlockStatesSnapshot() - blockStatesSnapshot, bootstrapped := blockStates.Snapshot() + blockStatesSnapshot, bootstrapped := blockStates.UnwrapValue() if !bootstrapped { return errFlushStateIsNotBootstrapped } diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index 884a009238..999ab674c7 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -185,9 +185,9 @@ func TestShardBootstrapWithError(t *testing.T) { fooBlocks := block.NewMockDatabaseSeriesBlocks(ctrl) barBlocks := block.NewMockDatabaseSeriesBlocks(ctrl) - fooSeries.EXPECT().Bootstrap(fooBlocks, gomock.Any()).Return(series.BootstrapResult{}, nil) + fooSeries.EXPECT().Load(series.LoadOptions{Bootstrap: true}, fooBlocks, gomock.Any()).Return(series.LoadResult{}, nil) fooSeries.EXPECT().IsBootstrapped().Return(true) - barSeries.EXPECT().Bootstrap(barBlocks, gomock.Any()).Return(series.BootstrapResult{}, errors.New("series error")) + barSeries.EXPECT().Load(series.LoadOptions{Bootstrap: true}, barBlocks, gomock.Any()).Return(series.LoadResult{}, errors.New("series error")) barSeries.EXPECT().IsBootstrapped().Return(true) fooID := ident.StringID("foo") @@ -271,7 +271,7 @@ func TestShardBootstrapWithFlushVersion(t *testing.T) { Blocks: blocks, }) - // Ensure that the bootstrapped flush/block states get passed to the series.Bootstrap() + // Ensure that the bootstrapped flush/block states get passed to the series.Load() // method properly. blockStateSnapshot := series.BootstrappedBlockStateSnapshot{ Snapshot: map[xtime.UnixNano]series.BlockState{}, @@ -282,7 +282,7 @@ func TestShardBootstrapWithFlushVersion(t *testing.T) { ColdVersion: i, } } - mockSeries.EXPECT().Bootstrap(blocks, blockStateSnapshot) + mockSeries.EXPECT().Load(series.LoadOptions{Bootstrap: true}, blocks, blockStateSnapshot) err = s.Bootstrap(bootstrappedSeries) require.NoError(t, err) @@ -835,7 +835,7 @@ func addTestSeriesWithCount(shard *dbShard, id ident.ID, count int32) series.Dat func addTestSeriesWithCountAndBootstrap(shard *dbShard, id ident.ID, count int32, bootstrap bool) series.DatabaseSeries { seriesEntry := series.NewDatabaseSeries(id, ident.Tags{}, shard.seriesOpts) if bootstrap { - seriesEntry.Bootstrap(nil, series.BootstrappedBlockStateSnapshot{}) + seriesEntry.Load(series.LoadOptions{Bootstrap: true}, nil, series.BootstrappedBlockStateSnapshot{}) } shard.Lock() entry := lookup.NewEntry(seriesEntry, 0) From 7e2f9b9db9a6a0b3167bb7d91f3947a9bd4f5704 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 29 Jul 2019 12:26:25 -0400 Subject: [PATCH 16/22] delete temp mock --- .../mocks/gomock_reflect_239576158/prog.go | 66 ------------------- 1 file changed, 66 deletions(-) delete mode 100644 src/dbnode/generated/mocks/gomock_reflect_239576158/prog.go diff --git a/src/dbnode/generated/mocks/gomock_reflect_239576158/prog.go b/src/dbnode/generated/mocks/gomock_reflect_239576158/prog.go deleted file mode 100644 index 16d86e34e1..0000000000 --- a/src/dbnode/generated/mocks/gomock_reflect_239576158/prog.go +++ /dev/null @@ -1,66 +0,0 @@ - -package main - -import ( - "encoding/gob" - "flag" - "fmt" - "os" - "path" - "reflect" - - "github.com/golang/mock/mockgen/model" - - pkg_ "github.com/m3db/m3/src/cmd/services/m3dbnode/config" -) - -var output = flag.String("output", "", "The output file name, or empty to use stdout.") - -func main() { - flag.Parse() - - its := []struct{ - sym string - typ reflect.Type - }{ - - { "BootstrapConfigurationValidator", reflect.TypeOf((*pkg_.BootstrapConfigurationValidator)(nil)).Elem()}, - - } - pkg := &model.Package{ - // NOTE: This behaves contrary to documented behaviour if the - // package name is not the final component of the import path. - // The reflect package doesn't expose the package name, though. - Name: path.Base("github.com/m3db/m3/src/cmd/services/m3dbnode/config"), - } - - for _, it := range its { - intf, err := model.InterfaceFromInterfaceType(it.typ) - if err != nil { - fmt.Fprintf(os.Stderr, "Reflection: %v\n", err) - os.Exit(1) - } - intf.Name = it.sym - pkg.Interfaces = append(pkg.Interfaces, intf) - } - - outfile := os.Stdout - if len(*output) != 0 { - var err error - outfile, err = os.Create(*output) - if err != nil { - fmt.Fprintf(os.Stderr, "failed to open output file %q", *output) - } - defer func() { - if err := outfile.Close(); err != nil { - fmt.Fprintf(os.Stderr, "failed to close output file %q", *output) - os.Exit(1) - } - }() - } - - if err := gob.NewEncoder(outfile).Encode(pkg); err != nil { - fmt.Fprintf(os.Stderr, "gob encode: %v\n", err) - os.Exit(1) - } -} From d4e984079a2fb6c632472972ec6aa9b0cfb38c0f Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 29 Jul 2019 12:26:50 -0400 Subject: [PATCH 17/22] delete temp mock --- .../fs/gomock_reflect_525677702/prog.go | 88 ------------------- 1 file changed, 88 deletions(-) delete mode 100644 src/dbnode/persist/fs/gomock_reflect_525677702/prog.go diff --git a/src/dbnode/persist/fs/gomock_reflect_525677702/prog.go b/src/dbnode/persist/fs/gomock_reflect_525677702/prog.go deleted file mode 100644 index bdf503de9c..0000000000 --- a/src/dbnode/persist/fs/gomock_reflect_525677702/prog.go +++ /dev/null @@ -1,88 +0,0 @@ - -package main - -import ( - "encoding/gob" - "flag" - "fmt" - "os" - "path" - "reflect" - - "github.com/golang/mock/mockgen/model" - - pkg_ "github.com/m3db/m3/src/dbnode/persist/fs" -) - -var output = flag.String("output", "", "The output file name, or empty to use stdout.") - -func main() { - flag.Parse() - - its := []struct{ - sym string - typ reflect.Type - }{ - - { "DataFileSetWriter", reflect.TypeOf((*pkg_.DataFileSetWriter)(nil)).Elem()}, - - { "DataFileSetReader", reflect.TypeOf((*pkg_.DataFileSetReader)(nil)).Elem()}, - - { "DataFileSetSeeker", reflect.TypeOf((*pkg_.DataFileSetSeeker)(nil)).Elem()}, - - { "IndexFileSetWriter", reflect.TypeOf((*pkg_.IndexFileSetWriter)(nil)).Elem()}, - - { "IndexFileSetReader", reflect.TypeOf((*pkg_.IndexFileSetReader)(nil)).Elem()}, - - { "IndexSegmentFileSetWriter", reflect.TypeOf((*pkg_.IndexSegmentFileSetWriter)(nil)).Elem()}, - - { "IndexSegmentFileSet", reflect.TypeOf((*pkg_.IndexSegmentFileSet)(nil)).Elem()}, - - { "IndexSegmentFile", reflect.TypeOf((*pkg_.IndexSegmentFile)(nil)).Elem()}, - - { "SnapshotMetadataFileWriter", reflect.TypeOf((*pkg_.SnapshotMetadataFileWriter)(nil)).Elem()}, - - { "DataFileSetSeekerManager", reflect.TypeOf((*pkg_.DataFileSetSeekerManager)(nil)).Elem()}, - - { "ConcurrentDataFileSetSeeker", reflect.TypeOf((*pkg_.ConcurrentDataFileSetSeeker)(nil)).Elem()}, - - { "MergeWith", reflect.TypeOf((*pkg_.MergeWith)(nil)).Elem()}, - - } - pkg := &model.Package{ - // NOTE: This behaves contrary to documented behaviour if the - // package name is not the final component of the import path. - // The reflect package doesn't expose the package name, though. - Name: path.Base("github.com/m3db/m3/src/dbnode/persist/fs"), - } - - for _, it := range its { - intf, err := model.InterfaceFromInterfaceType(it.typ) - if err != nil { - fmt.Fprintf(os.Stderr, "Reflection: %v\n", err) - os.Exit(1) - } - intf.Name = it.sym - pkg.Interfaces = append(pkg.Interfaces, intf) - } - - outfile := os.Stdout - if len(*output) != 0 { - var err error - outfile, err = os.Create(*output) - if err != nil { - fmt.Fprintf(os.Stderr, "failed to open output file %q", *output) - } - defer func() { - if err := outfile.Close(); err != nil { - fmt.Fprintf(os.Stderr, "failed to close output file %q", *output) - os.Exit(1) - } - }() - } - - if err := gob.NewEncoder(outfile).Encode(pkg); err != nil { - fmt.Fprintf(os.Stderr, "gob encode: %v\n", err) - os.Exit(1) - } -} From 066af05f23a5d1eadfeb1be66cb82964d481b5f5 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 29 Jul 2019 12:52:42 -0400 Subject: [PATCH 18/22] regen mocks --- src/dbnode/storage/storage_mock.go | 24 ++++++++++++++++++++---- 1 file changed, 20 insertions(+), 4 deletions(-) diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index 45661b98fe..bc7c765a06 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -1342,11 +1342,12 @@ func (mr *MockdatabaseNamespaceMockRecorder) Snapshot(blockStart, snapshotTime, } // NeedsFlush mocks base method -func (m *MockdatabaseNamespace) NeedsFlush(alignedInclusiveStart, alignedInclusiveEnd time.Time) bool { +func (m *MockdatabaseNamespace) NeedsFlush(alignedInclusiveStart, alignedInclusiveEnd time.Time) (bool, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "NeedsFlush", alignedInclusiveStart, alignedInclusiveEnd) ret0, _ := ret[0].(bool) - return ret0 + ret1, _ := ret[1].(error) + return ret0, ret1 } // NeedsFlush indicates an expected call of NeedsFlush @@ -1719,6 +1720,20 @@ func (mr *MockdatabaseShardMockRecorder) Bootstrap(bootstrappedSeries interface{ return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Bootstrap", reflect.TypeOf((*MockdatabaseShard)(nil).Bootstrap), bootstrappedSeries) } +// Load mocks base method +func (m *MockdatabaseShard) Load(series *result.Map) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Load", series) + ret0, _ := ret[0].(error) + return ret0 +} + +// Load indicates an expected call of Load +func (mr *MockdatabaseShardMockRecorder) Load(series interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Load", reflect.TypeOf((*MockdatabaseShard)(nil).Load), series) +} + // WarmFlush mocks base method func (m *MockdatabaseShard) WarmFlush(blockStart time.Time, flush persist.FlushPreparer, nsCtx namespace.Context) error { m.ctrl.T.Helper() @@ -1762,11 +1777,12 @@ func (mr *MockdatabaseShardMockRecorder) Snapshot(blockStart, snapshotStart, flu } // FlushState mocks base method -func (m *MockdatabaseShard) FlushState(blockStart time.Time) fileOpState { +func (m *MockdatabaseShard) FlushState(blockStart time.Time) (fileOpState, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "FlushState", blockStart) ret0, _ := ret[0].(fileOpState) - return ret0 + ret1, _ := ret[1].(error) + return ret0, ret1 } // FlushState indicates an expected call of FlushState From 8e95c31a5af11a0e7b535a9f7335fe50f1ac7cb1 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 29 Jul 2019 13:40:12 -0400 Subject: [PATCH 19/22] fix broken integration test --- src/dbnode/integration/truncate_namespace_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/dbnode/integration/truncate_namespace_test.go b/src/dbnode/integration/truncate_namespace_test.go index ad32910c7f..8c20a07fde 100644 --- a/src/dbnode/integration/truncate_namespace_test.go +++ b/src/dbnode/integration/truncate_namespace_test.go @@ -114,8 +114,7 @@ func TestTruncateNamespace(t *testing.T) { log.Sugar().Debugf("fetching data from namespace %s again", testNamespaces[0]) res, err = testSetup.fetch(fetchReq) - require.NoError(t, err) - require.Equal(t, 0, len(res)) + require.Error(t, err) log.Sugar().Debugf("fetching data from a different namespace %s", testNamespaces[1]) fetchReq.ID = "bar" From d6273fa87531b0442cfb94c1dddbd5ed31ce7d1e Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 29 Jul 2019 14:16:34 -0400 Subject: [PATCH 20/22] fix broken test --- src/dbnode/storage/series/series_parallel_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/dbnode/storage/series/series_parallel_test.go b/src/dbnode/storage/series/series_parallel_test.go index 9e4afe2b52..56866bd5d7 100644 --- a/src/dbnode/storage/series/series_parallel_test.go +++ b/src/dbnode/storage/series/series_parallel_test.go @@ -46,7 +46,7 @@ func TestSeriesWriteReadParallel(t *testing.T) { series = NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) ) - _, err := series.Bootstrap(LoadOptions{Bootstrap: true}, nil, nil) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, nil) assert.NoError(t, err) ctx := context.NewContext() From 4fc88ae1cfa540dcc94e46dface890f4ef63ab42 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 29 Jul 2019 14:58:12 -0400 Subject: [PATCH 21/22] fix broken test --- src/dbnode/storage/shard_test.go | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index 999ab674c7..2584bc44ee 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -1136,18 +1136,12 @@ func TestShardTickRace(t *testing.T) { wg.Add(2) go func() { - _, err := shard.Tick(context.NewNoOpCanncellable(), time.Now(), namespace.Context{}) - if err != nil { - panic(err) - } + shard.Tick(context.NewNoOpCanncellable(), time.Now(), namespace.Context{}) wg.Done() }() go func() { - _, err := shard.Tick(context.NewNoOpCanncellable(), time.Now(), namespace.Context{}) - if err != nil { - panic(err) - } + shard.Tick(context.NewNoOpCanncellable(), time.Now(), namespace.Context{}) wg.Done() }() From 844b94135e2e2d07f3865e744e30075472eecac9 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Mon, 29 Jul 2019 15:16:47 -0400 Subject: [PATCH 22/22] fix broken test --- src/dbnode/storage/series/series_parallel_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/dbnode/storage/series/series_parallel_test.go b/src/dbnode/storage/series/series_parallel_test.go index 56866bd5d7..161ba34fcb 100644 --- a/src/dbnode/storage/series/series_parallel_test.go +++ b/src/dbnode/storage/series/series_parallel_test.go @@ -46,7 +46,7 @@ func TestSeriesWriteReadParallel(t *testing.T) { series = NewDatabaseSeries(ident.StringID("foo"), ident.Tags{}, opts).(*dbSeries) ) - _, err := series.Load(LoadOptions{Bootstrap: true}, nil, nil) + _, err := series.Load(LoadOptions{Bootstrap: true}, nil, BootstrappedBlockStateSnapshot{}) assert.NoError(t, err) ctx := context.NewContext()