From de7e6b906b9474bec55c4d36de98e5a49bd981a6 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Wed, 28 Jul 2021 19:06:15 -0400 Subject: [PATCH 01/36] WIP - refactor flush state marking to occur after index flush --- src/dbnode/storage/flush.go | 51 ++++++++++++++++++++++++++------- src/dbnode/storage/namespace.go | 29 +++++++++++++++---- src/dbnode/storage/shard.go | 7 ++--- src/dbnode/storage/types.go | 5 +++- 4 files changed, 70 insertions(+), 22 deletions(-) diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index 242c676d83..fd464aeb30 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -54,6 +54,12 @@ const ( flushManagerIndexFlushInProgress ) +type namespaceFlush struct { + ns databaseNamespace + t xtime.UnixNano + shard databaseShard +} + type flushManagerMetrics struct { isFlushing tally.Gauge isSnapshotting tally.Gauge @@ -144,7 +150,8 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { // will attempt to snapshot blocks w/ unflushed data which would be wasteful if // the block is already flushable. multiErr := xerrors.NewMultiError() - if err = m.dataWarmFlush(namespaces, startTime); err != nil { + flushes, err := m.dataWarmFlush(namespaces, startTime) + if err != nil { multiErr = multiErr.Add(err) } @@ -163,22 +170,31 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { multiErr = multiErr.Add(err) } - return multiErr.FinalError() + err = multiErr.FinalError() + + // Mark all flush states at the very end to ensure this + // happens after both data and index flushing. + for _, f := range flushes { + f.shard.MarkWarmFlushStateSuccessOrError(f.t, err) + } + + return err } func (m *flushManager) dataWarmFlush( namespaces []databaseNamespace, startTime xtime.UnixNano, -) error { +) ([]namespaceFlush, error) { flushPersist, err := m.pm.StartFlushPersist() if err != nil { - return err + return nil, err } m.setState(flushManagerFlushInProgress) var ( - start = m.nowFn() - multiErr = xerrors.NewMultiError() + start = m.nowFn() + multiErr = xerrors.NewMultiError() + allFlushes = make([]namespaceFlush, 0) ) for _, ns := range namespaces { // Flush first because we will only snapshot if there are no outstanding flushes. @@ -187,10 +203,13 @@ func (m *flushManager) dataWarmFlush( multiErr = multiErr.Add(err) continue } - err = m.flushNamespaceWithTimes(ns, flushTimes, flushPersist) + flushes, err := m.flushNamespaceWithTimes(ns, flushTimes, flushPersist) if err != nil { multiErr = multiErr.Add(err) } + for _, f := range flushes { + allFlushes = append(allFlushes, f) + } } err = flushPersist.DoneFlush() @@ -199,7 +218,7 @@ func (m *flushManager) dataWarmFlush( } m.metrics.dataWarmFlushDuration.Record(m.nowFn().Sub(start)) - return multiErr.FinalError() + return allFlushes, multiErr.FinalError() } func (m *flushManager) dataSnapshot( @@ -361,18 +380,28 @@ func (m *flushManager) flushNamespaceWithTimes( ns databaseNamespace, times []xtime.UnixNano, flushPreparer persist.FlushPreparer, -) error { +) ([]namespaceFlush, error) { + flushes := make([]namespaceFlush, 0) multiErr := xerrors.NewMultiError() for _, t := range times { // NB(xichen): we still want to proceed if a namespace fails to flush its data. // Probably want to emit a counter here, but for now just log it. - if err := ns.WarmFlush(t, flushPreparer); err != nil { + shards, err := ns.WarmFlush(t, flushPreparer) + if err != nil { detailedErr := fmt.Errorf("namespace %s failed to flush data: %v", ns.ID().String(), err) multiErr = multiErr.Add(detailedErr) + } else { + for _, s := range shards { + flushes = append(flushes, namespaceFlush{ + ns: ns, + t: t, + shard: s, + }) + } } } - return multiErr.FinalError() + return flushes, multiErr.FinalError() } func (m *flushManager) LastSuccessfulSnapshotStartTime() (xtime.UnixNano, bool) { diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index 0cd2a3b63b..cedf318217 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -1181,7 +1181,7 @@ func (n *dbNamespace) Bootstrap( func (n *dbNamespace) WarmFlush( blockStart xtime.UnixNano, flushPersist persist.FlushPreparer, -) error { +) ([]databaseShard, error) { // NB(rartoul): This value can be used for emitting metrics, but should not be used // for business logic. callStart := n.nowFn() @@ -1190,24 +1190,25 @@ func (n *dbNamespace) WarmFlush( if n.bootstrapState != Bootstrapped { n.RUnlock() n.metrics.flushWarmData.ReportError(n.nowFn().Sub(callStart)) - return errNamespaceNotBootstrapped + return nil, errNamespaceNotBootstrapped } nsCtx := n.nsContextWithRLock() n.RUnlock() if n.ReadOnly() || !n.nopts.FlushEnabled() { n.metrics.flushWarmData.ReportSuccess(n.nowFn().Sub(callStart)) - return nil + return nil, nil } // check if blockStart is aligned with the namespace's retention options bs := n.nopts.RetentionOptions().BlockSize() if t := blockStart.Truncate(bs); !blockStart.Equal(t) { - return fmt.Errorf("failed to flush at time %v, not aligned to blockSize", blockStart.String()) + return nil, fmt.Errorf("failed to flush at time %v, not aligned to blockSize", blockStart.String()) } multiErr := xerrors.NewMultiError() shards := n.OwnedShards() + flushedShards := make([]databaseShard, 0) for _, shard := range shards { if !shard.IsBootstrapped() { n.log. @@ -1218,7 +1219,7 @@ func (n *dbNamespace) WarmFlush( flushState, err := shard.FlushState(blockStart) if err != nil { - return err + return nil, err } // skip flushing if the shard has already flushed data for the `blockStart` if flushState.WarmStatus == fileOpSuccess { @@ -1231,12 +1232,28 @@ func (n *dbNamespace) WarmFlush( detailedErr := fmt.Errorf("shard %d failed to flush data: %v", shard.ID(), err) multiErr = multiErr.Add(detailedErr) + + // Errors so mark as failed here. + // We mark as success later in time to ensure that happens after index flushing as well. + shard.MarkWarmFlushStateSuccessOrError(blockStart, err) + } else { + flushedShards = append(flushedShards, shard) } } res := multiErr.FinalError() n.metrics.flushWarmData.ReportSuccessOrError(res, n.nowFn().Sub(callStart)) - return res + return flushedShards, res +} + +func (n *dbNamespace) MarkWarmFlushStateSuccessOrError( + blockStart xtime.UnixNano, + shards []databaseShard, + err error, +) { + for _, shard := range shards { + shard.MarkWarmFlushStateSuccessOrError(blockStart, err) + } } // idAndBlockStart is the composite key for the genny map used to keep track of diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 10bc935234..71e2a05cd2 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -2202,7 +2202,7 @@ func (s *dbShard) WarmFlush( } prepared, err := flushPreparer.PrepareData(prepareOpts) if err != nil { - return s.markWarmFlushStateSuccessOrError(blockStart, err) + return err } var multiErr xerrors.MultiError @@ -2236,7 +2236,7 @@ func (s *dbShard) WarmFlush( multiErr = multiErr.Add(err) } - return s.markWarmFlushStateSuccessOrError(blockStart, multiErr.FinalError()) + return multiErr.FinalError() } func (s *dbShard) ColdFlush( @@ -2498,14 +2498,13 @@ func (s *dbShard) flushStateWithRLock(blockStart xtime.UnixNano) fileOpState { return state } -func (s *dbShard) markWarmFlushStateSuccessOrError(blockStart xtime.UnixNano, err error) error { +func (s *dbShard) MarkWarmFlushStateSuccessOrError(blockStart xtime.UnixNano, err error) { // Track flush state for block state if err == nil { s.markWarmFlushStateSuccess(blockStart) } else { s.markWarmFlushStateFail(blockStart) } - return err } func (s *dbShard) markWarmFlushStateSuccess(blockStart xtime.UnixNano) { diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 61317be6e4..47b3dd0a92 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -425,7 +425,7 @@ type databaseNamespace interface { Bootstrap(ctx context.Context, bootstrapResult bootstrap.NamespaceResult) error // WarmFlush flushes in-memory WarmWrites. - WarmFlush(blockStart xtime.UnixNano, flush persist.FlushPreparer) error + WarmFlush(blockStart xtime.UnixNano, flush persist.FlushPreparer) ([]databaseShard, error) // FlushIndex flushes in-memory index data. FlushIndex( @@ -611,6 +611,9 @@ type databaseShard interface { nsCtx namespace.Context, ) error + // MarkWarmFlushStateSuccessOrError marks the flush state as success or error. + MarkWarmFlushStateSuccessOrError(blockStart xtime.UnixNano, err error) + // ColdFlush flushes the unflushed ColdWrites in this shard. ColdFlush( flush persist.FlushPreparer, From 9fc493431cffdac7ff17f27e51c9df3156d664af Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Wed, 28 Jul 2021 19:10:04 -0400 Subject: [PATCH 02/36] WIP - remove unused ns marking func --- src/dbnode/storage/namespace.go | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index cedf318217..b552232348 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -1246,16 +1246,6 @@ func (n *dbNamespace) WarmFlush( return flushedShards, res } -func (n *dbNamespace) MarkWarmFlushStateSuccessOrError( - blockStart xtime.UnixNano, - shards []databaseShard, - err error, -) { - for _, shard := range shards { - shard.MarkWarmFlushStateSuccessOrError(blockStart, err) - } -} - // idAndBlockStart is the composite key for the genny map used to keep track of // dirty series that need to be ColdFlushed. type idAndBlockStart struct { From e19a2acd273b1adaf96f67eddf2e9fabfda71bf4 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Thu, 29 Jul 2021 13:56:42 -0400 Subject: [PATCH 03/36] WIP - do not remove from index if series not empty --- src/dbnode/storage/series/lookup/entry.go | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/dbnode/storage/series/lookup/entry.go b/src/dbnode/storage/series/lookup/entry.go index 31209dd20f..5303980079 100644 --- a/src/dbnode/storage/series/lookup/entry.go +++ b/src/dbnode/storage/series/lookup/entry.go @@ -235,6 +235,17 @@ func (entry *Entry) IfAlreadyIndexedMarkIndexSuccessAndFinalize( func (entry *Entry) RemoveIndexedForBlockStarts( blockStarts map[xtime.UnixNano]struct{}, ) index.RemoveIndexedForBlockStartsResult { + // Cannot remove from index if the series is not yet empty. This presumes that + // TSDB series data is never marked as flushed to disk prior to index series data + // which is why warm and cold flushing ensures marking data as flushed entails + // both data + index, and not just data. + if !entry.Series.IsEmpty() { + return index.RemoveIndexedForBlockStartsResult{ + IndexedBlockStartsRemoved: 0, + IndexedBlockStartsRemaining: len(blockStarts), + } + } + var result index.RemoveIndexedForBlockStartsResult entry.reverseIndex.Lock() for k, state := range entry.reverseIndex.states { From ca70206b3aabaeb256240a8073ad78017882b728 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Thu, 29 Jul 2021 14:54:36 -0400 Subject: [PATCH 04/36] WIP - remove flushed block checks --- src/dbnode/storage/index.go | 21 ----- src/dbnode/storage/index/block.go | 9 -- src/dbnode/storage/index/mutable_segments.go | 97 ++++---------------- src/dbnode/storage/index/types.go | 10 +- src/dbnode/storage/series/lookup/entry.go | 58 +++--------- src/dbnode/storage/shard.go | 7 -- 6 files changed, 32 insertions(+), 170 deletions(-) diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 7892c0e06c..fba25185bc 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -929,9 +929,6 @@ func (i *nsIndex) Tick( // such as notify of sealed blocks. tickingBlocks, multiErr := i.tickingBlocks(startTime) - // Track blocks that are flushed (and have their bootstrapped results). - flushedBlocks := make([]xtime.UnixNano, 0, len(tickingBlocks.tickingBlocks)) - result.NumBlocks = int64(tickingBlocks.totalBlocks) for _, block := range tickingBlocks.tickingBlocks { if c.IsCancelled() { @@ -946,10 +943,6 @@ func (i *nsIndex) Tick( result.NumSegmentsMutable += blockTickResult.NumSegmentsMutable result.NumTotalDocs += blockTickResult.NumDocs result.FreeMmap += blockTickResult.FreeMmap - - if tickErr == nil && blockTickResult.NumSegmentsBootstrapped != 0 { - flushedBlocks = append(flushedBlocks, block.StartTime()) - } } blockTickResult, tickErr := tickingBlocks.activeBlock.Tick(c) @@ -960,18 +953,6 @@ func (i *nsIndex) Tick( result.NumTotalDocs += blockTickResult.NumDocs result.FreeMmap += blockTickResult.FreeMmap - // Notify in memory block of sealed and flushed blocks - // and make sure to do this out of the lock since - // this can take a considerable amount of time - // and is an expensive task that doesn't require - // holding the index lock. - notifyErr := tickingBlocks.activeBlock.ActiveBlockNotifyFlushedBlocks(flushedBlocks) - if notifyErr != nil { - multiErr = multiErr.Add(notifyErr) - } else { - i.metrics.blocksNotifyFlushed.Inc(int64(len(flushedBlocks))) - } - i.metrics.tick.Inc(1) return result, multiErr.FinalError() @@ -2508,7 +2489,6 @@ type nsIndexMetrics struct { forwardIndexCounter tally.Counter insertEndToEndLatency tally.Timer blocksEvictedMutableSegments tally.Counter - blocksNotifyFlushed tally.Counter blockMetrics nsIndexBlocksMetrics indexingConcurrencyMin tally.Gauge indexingConcurrencyMax tally.Gauge @@ -2576,7 +2556,6 @@ func newNamespaceIndexMetrics( insertEndToEndLatency: instrument.NewTimer(scope, "insert-end-to-end-latency", iopts.TimerOptions()), blocksEvictedMutableSegments: scope.Counter("blocks-evicted-mutable-segments"), - blocksNotifyFlushed: scope.Counter("blocks-notify-flushed"), blockMetrics: newNamespaceIndexBlocksMetrics(opts, blocksScope), indexingConcurrencyMin: scope.Tagged(map[string]string{ "stat": "min", diff --git a/src/dbnode/storage/index/block.go b/src/dbnode/storage/index/block.go index e4dfe18aec..ee8894b1ad 100644 --- a/src/dbnode/storage/index/block.go +++ b/src/dbnode/storage/index/block.go @@ -280,15 +280,6 @@ func NewBlock( return b, nil } -func (b *block) ActiveBlockNotifyFlushedBlocks( - flushed []xtime.UnixNano, -) error { - if !b.blockOpts.ActiveBlock { - return fmt.Errorf("block not in-memory block: start=%v", b.StartTime()) - } - return b.mutableSegments.NotifyFlushedBlocks(flushed) -} - func (b *block) StartTime() xtime.UnixNano { return b.blockStart } diff --git a/src/dbnode/storage/index/mutable_segments.go b/src/dbnode/storage/index/mutable_segments.go index 72c3e5f2ba..c92d64d188 100644 --- a/src/dbnode/storage/index/mutable_segments.go +++ b/src/dbnode/storage/index/mutable_segments.go @@ -82,10 +82,6 @@ type mutableSegments struct { optsListener xresource.SimpleCloser writeIndexingConcurrency int - flushedBlockStarts map[xtime.UnixNano]struct{} - backgroundCompactGCPending bool - backgroundCompactDisable bool - metrics mutableSegmentsMetrics logger *zap.Logger } @@ -130,48 +126,19 @@ func newMutableSegments( iopts instrument.Options, ) *mutableSegments { m := &mutableSegments{ - blockStart: blockStart, - blockSize: md.Options().IndexOptions().BlockSize(), - opts: opts, - blockOpts: blockOpts, - compact: mutableSegmentsCompact{opts: opts, blockOpts: blockOpts}, - flushedBlockStarts: make(map[xtime.UnixNano]struct{}), - iopts: iopts, - metrics: newMutableSegmentsMetrics(iopts.MetricsScope()), - logger: iopts.Logger(), + blockStart: blockStart, + blockSize: md.Options().IndexOptions().BlockSize(), + opts: opts, + blockOpts: blockOpts, + compact: mutableSegmentsCompact{opts: opts, blockOpts: blockOpts}, + iopts: iopts, + metrics: newMutableSegmentsMetrics(iopts.MetricsScope()), + logger: iopts.Logger(), } m.optsListener = namespaceRuntimeOptsMgr.RegisterListener(m) return m } -func (m *mutableSegments) NotifyFlushedBlocks( - flushed []xtime.UnixNano, -) error { - if len(flushed) == 0 { - return nil - } - - m.Lock() - updated := false - for _, blockStart := range flushed { - _, exists := m.flushedBlockStarts[blockStart] - if exists { - continue - } - m.flushedBlockStarts[blockStart] = struct{}{} - updated = true - } - if updated { - // Only trigger background compact GC if - // and only if updated the sealed block starts. - m.backgroundCompactGCPending = true - m.maybeBackgroundCompactWithLock() - } - m.Unlock() - - return nil -} - func (m *mutableSegments) SetNamespaceRuntimeOptions(opts namespace.RuntimeOptions) { m.Lock() // Update current runtime opts for segment builders created in future. @@ -393,9 +360,6 @@ func (m *mutableSegments) Close() { } func (m *mutableSegments) maybeBackgroundCompactWithLock() { - if m.backgroundCompactDisable { - return - } if m.compact.compactingBackgroundStandard { return } @@ -429,19 +393,13 @@ func (m *mutableSegments) backgroundCompactWithLock() { } var ( - gcRequired = false - gcPlan = &compaction.Plan{} - gcAlreadyRunning = m.compact.compactingBackgroundGarbageCollect - flushedBlockStarts = make(map[xtime.UnixNano]struct{}, len(m.flushedBlockStarts)) + gcRequired = false + gcPlan = &compaction.Plan{} + gcAlreadyRunning = m.compact.compactingBackgroundGarbageCollect ) - // Take copy of sealed block starts so can act on this - // async. - for k, v := range m.flushedBlockStarts { - flushedBlockStarts[k] = v - } - if !gcAlreadyRunning && m.backgroundCompactGCPending { + + if !gcAlreadyRunning { gcRequired = true - m.backgroundCompactGCPending = false for _, seg := range m.backgroundSegments { alreadyHasTask := false @@ -488,8 +446,7 @@ func (m *mutableSegments) backgroundCompactWithLock() { // Kick off compaction. m.compact.compactingBackgroundStandard = true go func() { - m.backgroundCompactWithPlan(plan, m.compact.backgroundCompactors, - gcRequired, flushedBlockStarts) + m.backgroundCompactWithPlan(plan, m.compact.backgroundCompactors, gcRequired) m.Lock() m.compact.compactingBackgroundStandard = false @@ -508,8 +465,7 @@ func (m *mutableSegments) backgroundCompactWithLock() { l.Error("error background gc segments", zap.Error(err)) }) } else { - m.backgroundCompactWithPlan(gcPlan, compactors, - gcRequired, flushedBlockStarts) + m.backgroundCompactWithPlan(gcPlan, compactors, gcRequired) m.closeCompactors(compactors) } @@ -580,7 +536,6 @@ func (m *mutableSegments) backgroundCompactWithPlan( plan *compaction.Plan, compactors chan *compaction.Compactor, gcRequired bool, - flushedBlocks map[xtime.UnixNano]struct{}, ) { sw := m.metrics.backgroundCompactionPlanRunLatency.Start() defer sw.Stop() @@ -617,7 +572,7 @@ func (m *mutableSegments) backgroundCompactWithPlan( wg.Done() }() err := m.backgroundCompactWithTask(task, compactor, gcRequired, - flushedBlocks, log, logger.With(zap.Int("task", i))) + log, logger.With(zap.Int("task", i))) if err != nil { instrument.EmitAndLogInvariantViolation(m.iopts, func(l *zap.Logger) { l.Error("error compacting segments", zap.Error(err)) @@ -638,7 +593,6 @@ func (m *mutableSegments) backgroundCompactWithTask( task compaction.Task, compactor *compaction.Compactor, gcRequired bool, - flushedBlocks map[xtime.UnixNano]struct{}, log bool, logger *zap.Logger, ) error { @@ -672,23 +626,8 @@ func (m *mutableSegments) backgroundCompactWithTask( return true } - latestEntry, ok := entry.RelookupAndIncrementReaderWriterCount() - if !ok { - // Should not happen since shard will not expire until - // no more block starts are indexed. - // We do not GC this series if shard is missing since - // we open up a race condition where the entry is not - // in the shard yet and we GC it since we can't find it - // due to an asynchronous insert. - return true - } - - result := latestEntry.RemoveIndexedForBlockStarts(flushedBlocks) - latestEntry.DecrementReaderWriterCount() - - // Keep the series if and only if there are remaining - // index block starts outside of the sealed blocks starts. - return result.IndexedBlockStartsRemaining > 0 + // Keep if not yet empty (i.e. there is still in-memory data associated with the series). + return !entry.IsEmpty() }) } diff --git a/src/dbnode/storage/index/types.go b/src/dbnode/storage/index/types.go index f617242dc8..c628fdaa03 100644 --- a/src/dbnode/storage/index/types.go +++ b/src/dbnode/storage/index/types.go @@ -400,11 +400,7 @@ type OnIndexSeries interface { blockStart xtime.UnixNano, ) bool - RemoveIndexedForBlockStarts( - blockStarts map[xtime.UnixNano]struct{}, - ) RemoveIndexedForBlockStartsResult - - RelookupAndIncrementReaderWriterCount() (OnIndexSeries, bool) + IsEmpty() bool DecrementReaderWriterCount() @@ -459,10 +455,6 @@ type Block interface { // AddResults adds bootstrap results to the block. AddResults(resultsByVolumeType result.IndexBlockByVolumeType) error - // ActiveBlockNotifyFlushedBlocks notifies an active in-memory block of - // sealed blocks. - ActiveBlockNotifyFlushedBlocks(sealed []xtime.UnixNano) error - // Tick does internal house keeping operations. Tick(c context.Cancellable) (BlockTickResult, error) diff --git a/src/dbnode/storage/series/lookup/entry.go b/src/dbnode/storage/series/lookup/entry.go index 5303980079..23db283ef5 100644 --- a/src/dbnode/storage/series/lookup/entry.go +++ b/src/dbnode/storage/series/lookup/entry.go @@ -79,11 +79,10 @@ var _ bootstrap.SeriesRefResolver = &Entry{} // NewEntryOptions supplies options for a new entry. type NewEntryOptions struct { - RelookupAndIncrementReaderWriterCount func() (index.OnIndexSeries, bool) - Series series.DatabaseSeries - Index uint64 - IndexWriter IndexWriter - NowFn clock.NowFn + Series series.DatabaseSeries + Index uint64 + IndexWriter IndexWriter + NowFn clock.NowFn } // NewEntry returns a new Entry. @@ -93,22 +92,16 @@ func NewEntry(opts NewEntryOptions) *Entry { nowFn = opts.NowFn } entry := &Entry{ - relookupAndIncrementReaderWriterCount: opts.RelookupAndIncrementReaderWriterCount, - Series: opts.Series, - Index: opts.Index, - indexWriter: opts.IndexWriter, - nowFn: nowFn, - pendingIndexBatchSizeOne: make([]writes.PendingIndexInsert, 1), - reverseIndex: newEntryIndexState(), + Series: opts.Series, + Index: opts.Index, + indexWriter: opts.IndexWriter, + nowFn: nowFn, + pendingIndexBatchSizeOne: make([]writes.PendingIndexInsert, 1), + reverseIndex: newEntryIndexState(), } return entry } -// RelookupAndIncrementReaderWriterCount will relookup the entry. -func (entry *Entry) RelookupAndIncrementReaderWriterCount() (index.OnIndexSeries, bool) { - return entry.relookupAndIncrementReaderWriterCount() -} - // ReaderWriterCount returns the current ref count on the Entry. func (entry *Entry) ReaderWriterCount() int32 { return atomic.LoadInt32(&entry.curReadWriters) @@ -231,34 +224,9 @@ func (entry *Entry) IfAlreadyIndexedMarkIndexSuccessAndFinalize( return successAlready } -// RemoveIndexedForBlockStarts removes the entry for the index for all blockStarts. -func (entry *Entry) RemoveIndexedForBlockStarts( - blockStarts map[xtime.UnixNano]struct{}, -) index.RemoveIndexedForBlockStartsResult { - // Cannot remove from index if the series is not yet empty. This presumes that - // TSDB series data is never marked as flushed to disk prior to index series data - // which is why warm and cold flushing ensures marking data as flushed entails - // both data + index, and not just data. - if !entry.Series.IsEmpty() { - return index.RemoveIndexedForBlockStartsResult{ - IndexedBlockStartsRemoved: 0, - IndexedBlockStartsRemaining: len(blockStarts), - } - } - - var result index.RemoveIndexedForBlockStartsResult - entry.reverseIndex.Lock() - for k, state := range entry.reverseIndex.states { - _, ok := blockStarts[k] - if ok && state.success { - delete(entry.reverseIndex.states, k) - result.IndexedBlockStartsRemoved++ - continue - } - result.IndexedBlockStartsRemaining++ - } - entry.reverseIndex.Unlock() - return result +// IsEmpty returns true if the entry has no in-memory series data. +func (entry *Entry) IsEmpty() bool { + return entry.Series.IsEmpty() } // Write writes a new value. diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 71e2a05cd2..dda3205ee7 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -1216,13 +1216,6 @@ func (s *dbShard) newShardEntry( Options: s.seriesOpts, }) return lookup.NewEntry(lookup.NewEntryOptions{ - RelookupAndIncrementReaderWriterCount: func() (index.OnIndexSeries, bool) { - e, _, err := s.tryRetrieveWritableSeries(seriesID) - if err != nil || e == nil { - return nil, false - } - return e, true - }, Series: newSeries, Index: uniqueIndex, IndexWriter: s.reverseIndex, From 6a2eaf1025a4ad3ca4db370748bae083a6197d59 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Thu, 29 Jul 2021 16:51:51 -0400 Subject: [PATCH 05/36] Cleanup 1 --- src/dbnode/storage/flush.go | 32 +++++++++++++++++++------------- 1 file changed, 19 insertions(+), 13 deletions(-) diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index fd464aeb30..9d233dacca 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -55,8 +55,12 @@ const ( ) type namespaceFlush struct { - ns databaseNamespace - t xtime.UnixNano + namespace databaseNamespace + shardFlushes []shardFlush +} + +type shardFlush struct { + time xtime.UnixNano shard databaseShard } @@ -175,7 +179,9 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { // Mark all flush states at the very end to ensure this // happens after both data and index flushing. for _, f := range flushes { - f.shard.MarkWarmFlushStateSuccessOrError(f.t, err) + for _, s := range f.shardFlushes { + s.shard.MarkWarmFlushStateSuccessOrError(s.time, err) + } } return err @@ -203,13 +209,11 @@ func (m *flushManager) dataWarmFlush( multiErr = multiErr.Add(err) continue } - flushes, err := m.flushNamespaceWithTimes(ns, flushTimes, flushPersist) + flush, err := m.flushNamespaceWithTimes(ns, flushTimes, flushPersist) if err != nil { multiErr = multiErr.Add(err) } - for _, f := range flushes { - allFlushes = append(allFlushes, f) - } + allFlushes = append(allFlushes, flush) } err = flushPersist.DoneFlush() @@ -380,8 +384,8 @@ func (m *flushManager) flushNamespaceWithTimes( ns databaseNamespace, times []xtime.UnixNano, flushPreparer persist.FlushPreparer, -) ([]namespaceFlush, error) { - flushes := make([]namespaceFlush, 0) +) (namespaceFlush, error) { + flushes := make([]shardFlush, 0) multiErr := xerrors.NewMultiError() for _, t := range times { // NB(xichen): we still want to proceed if a namespace fails to flush its data. @@ -393,15 +397,17 @@ func (m *flushManager) flushNamespaceWithTimes( multiErr = multiErr.Add(detailedErr) } else { for _, s := range shards { - flushes = append(flushes, namespaceFlush{ - ns: ns, - t: t, + flushes = append(flushes, shardFlush{ + time: t, shard: s, }) } } } - return flushes, multiErr.FinalError() + return namespaceFlush{ + namespace: ns, + shardFlushes: flushes, + }, multiErr.FinalError() } func (m *flushManager) LastSuccessfulSnapshotStartTime() (xtime.UnixNano, bool) { From 5f1be6ccabea8ac10533938bf01ab55209cdd5d2 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Thu, 29 Jul 2021 16:55:24 -0400 Subject: [PATCH 06/36] Mock gen --- src/dbnode/storage/index/index_mock.go | 57 +++++++------------------- src/dbnode/storage/storage_mock.go | 19 +++++++-- 2 files changed, 30 insertions(+), 46 deletions(-) diff --git a/src/dbnode/storage/index/index_mock.go b/src/dbnode/storage/index/index_mock.go index 3c78ffa986..cca1c1cd86 100644 --- a/src/dbnode/storage/index/index_mock.go +++ b/src/dbnode/storage/index/index_mock.go @@ -853,6 +853,20 @@ func (mr *MockOnIndexSeriesMockRecorder) IndexedForBlockStart(indexBlockStart in return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexedForBlockStart", reflect.TypeOf((*MockOnIndexSeries)(nil).IndexedForBlockStart), indexBlockStart) } +// IsEmpty mocks base method. +func (m *MockOnIndexSeries) IsEmpty() bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "IsEmpty") + ret0, _ := ret[0].(bool) + return ret0 +} + +// IsEmpty indicates an expected call of IsEmpty. +func (mr *MockOnIndexSeriesMockRecorder) IsEmpty() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IsEmpty", reflect.TypeOf((*MockOnIndexSeries)(nil).IsEmpty)) +} + // NeedsIndexUpdate mocks base method. func (m *MockOnIndexSeries) NeedsIndexUpdate(indexBlockStartForWrite time0.UnixNano) bool { m.ctrl.T.Helper() @@ -903,35 +917,6 @@ func (mr *MockOnIndexSeriesMockRecorder) OnIndexSuccess(blockStart interface{}) return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnIndexSuccess", reflect.TypeOf((*MockOnIndexSeries)(nil).OnIndexSuccess), blockStart) } -// RelookupAndIncrementReaderWriterCount mocks base method. -func (m *MockOnIndexSeries) RelookupAndIncrementReaderWriterCount() (OnIndexSeries, bool) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "RelookupAndIncrementReaderWriterCount") - ret0, _ := ret[0].(OnIndexSeries) - ret1, _ := ret[1].(bool) - return ret0, ret1 -} - -// RelookupAndIncrementReaderWriterCount indicates an expected call of RelookupAndIncrementReaderWriterCount. -func (mr *MockOnIndexSeriesMockRecorder) RelookupAndIncrementReaderWriterCount() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RelookupAndIncrementReaderWriterCount", reflect.TypeOf((*MockOnIndexSeries)(nil).RelookupAndIncrementReaderWriterCount)) -} - -// RemoveIndexedForBlockStarts mocks base method. -func (m *MockOnIndexSeries) RemoveIndexedForBlockStarts(blockStarts map[time0.UnixNano]struct{}) RemoveIndexedForBlockStartsResult { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "RemoveIndexedForBlockStarts", blockStarts) - ret0, _ := ret[0].(RemoveIndexedForBlockStartsResult) - return ret0 -} - -// RemoveIndexedForBlockStarts indicates an expected call of RemoveIndexedForBlockStarts. -func (mr *MockOnIndexSeriesMockRecorder) RemoveIndexedForBlockStarts(blockStarts interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RemoveIndexedForBlockStarts", reflect.TypeOf((*MockOnIndexSeries)(nil).RemoveIndexedForBlockStarts), blockStarts) -} - // MockBlock is a mock of Block interface. type MockBlock struct { ctrl *gomock.Controller @@ -955,20 +940,6 @@ func (m *MockBlock) EXPECT() *MockBlockMockRecorder { return m.recorder } -// ActiveBlockNotifyFlushedBlocks mocks base method. -func (m *MockBlock) ActiveBlockNotifyFlushedBlocks(sealed []time0.UnixNano) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ActiveBlockNotifyFlushedBlocks", sealed) - ret0, _ := ret[0].(error) - return ret0 -} - -// ActiveBlockNotifyFlushedBlocks indicates an expected call of ActiveBlockNotifyFlushedBlocks. -func (mr *MockBlockMockRecorder) ActiveBlockNotifyFlushedBlocks(sealed interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ActiveBlockNotifyFlushedBlocks", reflect.TypeOf((*MockBlock)(nil).ActiveBlockNotifyFlushedBlocks), sealed) -} - // AddResults mocks base method. func (m *MockBlock) AddResults(resultsByVolumeType result.IndexBlockByVolumeType) error { m.ctrl.T.Helper() diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index 256e576658..72cfa38566 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -1784,11 +1784,12 @@ func (mr *MockdatabaseNamespaceMockRecorder) Truncate() *gomock.Call { } // WarmFlush mocks base method. -func (m *MockdatabaseNamespace) WarmFlush(blockStart time0.UnixNano, flush persist.FlushPreparer) error { +func (m *MockdatabaseNamespace) WarmFlush(blockStart time0.UnixNano, flush persist.FlushPreparer) ([]databaseShard, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "WarmFlush", blockStart, flush) - ret0, _ := ret[0].(error) - return ret0 + ret0, _ := ret[0].([]databaseShard) + ret1, _ := ret[1].(error) + return ret0, ret1 } // WarmFlush indicates an expected call of WarmFlush. @@ -2221,6 +2222,18 @@ func (mr *MockdatabaseShardMockRecorder) LoadBlocks(series interface{}) *gomock. return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LoadBlocks", reflect.TypeOf((*MockdatabaseShard)(nil).LoadBlocks), series) } +// MarkWarmFlushStateSuccessOrError mocks base method. +func (m *MockdatabaseShard) MarkWarmFlushStateSuccessOrError(blockStart time0.UnixNano, err error) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "MarkWarmFlushStateSuccessOrError", blockStart, err) +} + +// MarkWarmFlushStateSuccessOrError indicates an expected call of MarkWarmFlushStateSuccessOrError. +func (mr *MockdatabaseShardMockRecorder) MarkWarmFlushStateSuccessOrError(blockStart, err interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "MarkWarmFlushStateSuccessOrError", reflect.TypeOf((*MockdatabaseShard)(nil).MarkWarmFlushStateSuccessOrError), blockStart, err) +} + // NumSeries mocks base method. func (m *MockdatabaseShard) NumSeries() int64 { m.ctrl.T.Helper() From de4c1655d43feace0be1e14caff7d94edbd0def5 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Thu, 29 Jul 2021 17:20:36 -0400 Subject: [PATCH 07/36] Fix tests 1 --- src/dbnode/storage/flush_test.go | 12 +++++- src/dbnode/storage/index/block_bench_test.go | 4 +- src/dbnode/storage/index_block_test.go | 4 -- src/dbnode/storage/namespace_test.go | 20 +++++++--- src/dbnode/storage/shard_test.go | 39 +++++++++++++++----- 5 files changed, 56 insertions(+), 23 deletions(-) diff --git a/src/dbnode/storage/flush_test.go b/src/dbnode/storage/flush_test.go index 667c1f05ff..cf7fdde0fe 100644 --- a/src/dbnode/storage/flush_test.go +++ b/src/dbnode/storage/flush_test.go @@ -315,12 +315,16 @@ func TestFlushManagerSkipNamespaceIndexingDisabled(t *testing.T) { defer ctrl.Finish() nsOpts := defaultTestNs1Opts.SetIndexOptions(namespace.NewIndexOptions().SetEnabled(false)) + s1 := NewMockdatabaseShard(ctrl) + s2 := NewMockdatabaseShard(ctrl) ns := NewMockdatabaseNamespace(ctrl) ns.EXPECT().Options().Return(nsOpts).AnyTimes() ns.EXPECT().ID().Return(defaultTestNs1ID).AnyTimes() ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(true, nil).AnyTimes() - ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any()).Return(nil).AnyTimes() + ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any()).Return([]databaseShard{s1, s2}, nil).AnyTimes() ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() + s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil).AnyTimes() + s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil).AnyTimes() var ( mockFlushPersist = persist.NewMockFlushPreparer(ctrl) @@ -358,13 +362,17 @@ func TestFlushManagerNamespaceIndexingEnabled(t *testing.T) { defer ctrl.Finish() nsOpts := defaultTestNs1Opts.SetIndexOptions(namespace.NewIndexOptions().SetEnabled(true)) + s1 := NewMockdatabaseShard(ctrl) + s2 := NewMockdatabaseShard(ctrl) ns := NewMockdatabaseNamespace(ctrl) ns.EXPECT().Options().Return(nsOpts).AnyTimes() ns.EXPECT().ID().Return(defaultTestNs1ID).AnyTimes() ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(true, nil).AnyTimes() - ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any()).Return(nil).AnyTimes() + ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any()).Return([]databaseShard{s1, s2}, nil).AnyTimes() ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() ns.EXPECT().FlushIndex(gomock.Any()).Return(nil) + s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil).AnyTimes() + s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil).AnyTimes() var ( mockFlushPersist = persist.NewMockFlushPreparer(ctrl) diff --git a/src/dbnode/storage/index/block_bench_test.go b/src/dbnode/storage/index/block_bench_test.go index 70b1e68aff..3780d1bf01 100644 --- a/src/dbnode/storage/index/block_bench_test.go +++ b/src/dbnode/storage/index/block_bench_test.go @@ -134,6 +134,4 @@ func (m mockOnIndexSeries) RemoveIndexedForBlockStarts( return RemoveIndexedForBlockStartsResult{} } func (m mockOnIndexSeries) IndexedOrAttemptedAny() bool { return false } -func (m mockOnIndexSeries) RelookupAndIncrementReaderWriterCount() (OnIndexSeries, bool) { - return m, false -} +func (m mockOnIndexSeries) IsEmpty() bool { return false } diff --git a/src/dbnode/storage/index_block_test.go b/src/dbnode/storage/index_block_test.go index caa7de660f..3ace4a6210 100644 --- a/src/dbnode/storage/index_block_test.go +++ b/src/dbnode/storage/index_block_test.go @@ -460,7 +460,6 @@ func TestNamespaceIndexTickExpire(t *testing.T) { c := context.NewCancellable() bActive.EXPECT().Tick(c).Return(index.BlockTickResult{}, nil) - bActive.EXPECT().ActiveBlockNotifyFlushedBlocks(gomock.Any()).Return(nil) b0.EXPECT().Close().Return(nil) @@ -531,9 +530,6 @@ func TestNamespaceIndexTick(t *testing.T) { }, nil). AnyTimes() bActive.EXPECT().IsSealed().Return(false).AnyTimes() - bActive.EXPECT().ActiveBlockNotifyFlushedBlocks(gomock.Any()). - Return(nil). - AnyTimes() b0.EXPECT().Tick(c). Return(index.BlockTickResult{ diff --git a/src/dbnode/storage/namespace_test.go b/src/dbnode/storage/namespace_test.go index 74abfcc679..0709661577 100644 --- a/src/dbnode/storage/namespace_test.go +++ b/src/dbnode/storage/namespace_test.go @@ -607,7 +607,9 @@ func testNamespaceBootstrapUnfulfilledShards( func TestNamespaceFlushNotBootstrapped(t *testing.T) { ns, closer := newTestNamespace(t) defer closer() - require.Equal(t, errNamespaceNotBootstrapped, ns.WarmFlush(xtime.Now(), nil)) + flushed, err := ns.WarmFlush(xtime.Now(), nil) + require.Equal(t, errNamespaceNotBootstrapped, err) + require.Equal(t, 0, len(flushed)) require.Equal(t, errNamespaceNotBootstrapped, ns.ColdFlush(nil)) } @@ -617,7 +619,9 @@ func TestNamespaceFlushDontNeedFlush(t *testing.T) { defer close() ns.bootstrapState = Bootstrapped - require.NoError(t, ns.WarmFlush(xtime.Now(), nil)) + flushed, err := ns.WarmFlush(xtime.Now(), nil) + require.NoError(t, err) + require.Equal(t, 0, len(flushed)) require.NoError(t, ns.ColdFlush(nil)) } @@ -627,7 +631,9 @@ func TestNamespaceSkipFlushIfReadOnly(t *testing.T) { ns.bootstrapState = Bootstrapped ns.SetReadOnly(true) - require.NoError(t, ns.WarmFlush(xtime.Now(), nil)) + flushed, err := ns.WarmFlush(xtime.Now(), nil) + require.NoError(t, err) + require.Equal(t, 0, len(flushed)) require.NoError(t, ns.ColdFlush(nil)) } @@ -658,7 +664,9 @@ func TestNamespaceFlushSkipFlushed(t *testing.T) { ns.shards[testShardIDs[i].ID()] = shard } - require.NoError(t, ns.WarmFlush(blockStart, nil)) + flushed, err := ns.WarmFlush(blockStart, nil) + require.NoError(t, err) + require.Equal(t, 1, len(flushed)) } func TestNamespaceFlushSkipShardNotBootstrapped(t *testing.T) { @@ -679,7 +687,9 @@ func TestNamespaceFlushSkipShardNotBootstrapped(t *testing.T) { shard.EXPECT().IsBootstrapped().Return(false) ns.shards[testShardIDs[0].ID()] = shard - require.NoError(t, ns.WarmFlush(blockStart, nil)) + flushed, err := ns.WarmFlush(blockStart, nil) + require.NoError(t, err) + require.Equal(t, 0, len(flushed)) require.NoError(t, ns.ColdFlush(nil)) } diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index 93ab49f37b..deb21e5d09 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -431,8 +431,8 @@ func TestShardFlushSeriesFlushError(t *testing.T) { s.Bootstrap(ctx, nsCtx) s.flushState.statesByTime[blockStart] = fileOpState{ - WarmStatus: fileOpFailed, - NumFailures: 1, + WarmStatus: fileOpNotStarted, + NumFailures: 0, } var closed bool @@ -469,7 +469,7 @@ func TestShardFlushSeriesFlushError(t *testing.T) { })) } - err := s.WarmFlush(blockStart, flush, namespace.Context{}) + flushErr := s.WarmFlush(blockStart, flush, namespace.Context{}) require.Equal(t, len(flushed), 2) for i := 0; i < 2; i++ { @@ -478,14 +478,24 @@ func TestShardFlushSeriesFlushError(t *testing.T) { } require.True(t, closed) - require.NotNil(t, err) - require.Equal(t, "error bar", err.Error()) + require.NotNil(t, flushErr) + require.Equal(t, "error bar", flushErr.Error()) flushState, err := s.FlushState(blockStart) require.NoError(t, err) require.Equal(t, fileOpState{ - WarmStatus: fileOpFailed, - NumFailures: 2, + WarmStatus: fileOpNotStarted, + NumFailures: 0, + }, flushState) + + s.MarkWarmFlushStateSuccessOrError(blockStart, flushErr) + + flushState, err = s.FlushState(blockStart) + require.NoError(t, err) + require.Equal(t, fileOpState{ + WarmStatus: fileOpFailed, + ColdVersionRetrievable: 0, + NumFailures: 1, }, flushState) } @@ -511,8 +521,8 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { s.Bootstrap(ctx, nsCtx) s.flushState.statesByTime[blockStart] = fileOpState{ - WarmStatus: fileOpFailed, - NumFailures: 1, + WarmStatus: fileOpNotStarted, + NumFailures: 0, } var closed bool @@ -557,8 +567,19 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { require.True(t, closed) require.Nil(t, err) + // State not yet updated since an explicit call to MarkWarmFlushStateSuccessOrError is required. flushState, err := s.FlushState(blockStart) require.NoError(t, err) + require.Equal(t, fileOpState{ + WarmStatus: fileOpNotStarted, + ColdVersionRetrievable: 0, + NumFailures: 0, + }, flushState) + + s.MarkWarmFlushStateSuccessOrError(blockStart, nil) + + flushState, err = s.FlushState(blockStart) + require.NoError(t, err) require.Equal(t, fileOpState{ WarmStatus: fileOpSuccess, ColdVersionRetrievable: 0, From f3117c976fb81601c7776dcad33733a1cbb02c62 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Mon, 2 Aug 2021 11:24:32 -0400 Subject: [PATCH 08/36] Fix TestBlockWriteBackgroundCompact --- src/dbnode/storage/index/block_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/dbnode/storage/index/block_test.go b/src/dbnode/storage/index/block_test.go index bac240fd14..151c5bf309 100644 --- a/src/dbnode/storage/index/block_test.go +++ b/src/dbnode/storage/index/block_test.go @@ -1750,6 +1750,9 @@ func TestBlockWriteBackgroundCompact(t *testing.T) { b, ok := blk.(*block) require.True(t, ok) + // Testing compaction only, so mark GC as already running so the test is limited only to compaction. + b.mutableSegments.compact.compactingBackgroundGarbageCollect = true + // First write h1 := NewMockOnIndexSeries(ctrl) h1.EXPECT().OnIndexFinalize(blockStart) From 6568044076a01d3a591f4f13e24ed7678b7ed8ce Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Mon, 2 Aug 2021 11:26:20 -0400 Subject: [PATCH 09/36] Lint --- src/dbnode/storage/series/lookup/entry.go | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/src/dbnode/storage/series/lookup/entry.go b/src/dbnode/storage/series/lookup/entry.go index 23db283ef5..045fc86f43 100644 --- a/src/dbnode/storage/series/lookup/entry.go +++ b/src/dbnode/storage/series/lookup/entry.go @@ -58,14 +58,13 @@ type IndexWriter interface { // members to track lifecycle and minimize indexing overhead. // NB: users are expected to use `NewEntry` to construct these objects. type Entry struct { - relookupAndIncrementReaderWriterCount func() (index.OnIndexSeries, bool) - Series series.DatabaseSeries - Index uint64 - indexWriter IndexWriter - curReadWriters int32 - reverseIndex entryIndexState - nowFn clock.NowFn - pendingIndexBatchSizeOne []writes.PendingIndexInsert + Series series.DatabaseSeries + Index uint64 + indexWriter IndexWriter + curReadWriters int32 + reverseIndex entryIndexState + nowFn clock.NowFn + pendingIndexBatchSizeOne []writes.PendingIndexInsert } // ensure Entry satisfies the `index.OnIndexSeries` interface. From 7c1f06c1aefbd1529b84c6a5227d2dc248864445 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Mon, 2 Aug 2021 17:12:53 -0400 Subject: [PATCH 10/36] WIP - fix index flush conditions --- src/dbnode/storage/flush.go | 29 +++++++++++++++++++++++------ src/dbnode/storage/index.go | 27 +++++++++++++++++++-------- src/dbnode/storage/namespace.go | 10 +++++----- src/dbnode/storage/types.go | 4 ++-- 4 files changed, 49 insertions(+), 21 deletions(-) diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index 9d233dacca..d52526f766 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -170,7 +170,8 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { multiErr = multiErr.Add(fmt.Errorf("error rotating commitlog in mediator tick: %v", err)) } - if err = m.indexFlush(namespaces); err != nil { + indexFlushes, err := m.indexFlush(namespaces) + if err != nil { multiErr = multiErr.Add(err) } @@ -180,7 +181,14 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { // happens after both data and index flushing. for _, f := range flushes { for _, s := range f.shardFlushes { - s.shard.MarkWarmFlushStateSuccessOrError(s.time, err) + for _, ff := range indexFlushes { + for _, ss := range ff.shardFlushes { + if ff.namespace.ID().Equal(f.namespace.ID()) && ss.time.Equal(s.time) && ss.shard.ID() == s.shard.ID() { + s.shard.MarkWarmFlushStateSuccessOrError(s.time, err) + } + } + } + } } @@ -276,16 +284,17 @@ func (m *flushManager) dataSnapshot( func (m *flushManager) indexFlush( namespaces []databaseNamespace, -) error { +) ([]namespaceFlush, error) { indexFlush, err := m.pm.StartIndexPersist() if err != nil { - return err + return nil, err } m.setState(flushManagerIndexFlushInProgress) var ( start = m.nowFn() multiErr = xerrors.NewMultiError() + flushed = make([]namespaceFlush, 0) ) for _, ns := range namespaces { var ( @@ -295,12 +304,19 @@ func (m *flushManager) indexFlush( if !indexEnabled { continue } - multiErr = multiErr.Add(ns.FlushIndex(indexFlush)) + flushedShards, err := ns.FlushIndex(indexFlush) + if err == nil { + flushed = append(flushed, namespaceFlush{ + namespace: ns, + shardFlushes: flushedShards, + }) + } + multiErr = multiErr.Add(err) } multiErr = multiErr.Add(indexFlush.DoneIndex()) m.metrics.indexFlushDuration.Record(m.nowFn().Sub(start)) - return multiErr.FinalError() + return flushed, multiErr.FinalError() } func (m *flushManager) Report() { @@ -387,6 +403,7 @@ func (m *flushManager) flushNamespaceWithTimes( ) (namespaceFlush, error) { flushes := make([]shardFlush, 0) multiErr := xerrors.NewMultiError() + fmt.Println("TIMES", times) for _, t := range times { // NB(xichen): we still want to proceed if a namespace fails to flush its data. // Probably want to emit a counter here, but for now just log it. diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index fba25185bc..ff2cc8e2f4 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -1007,15 +1007,15 @@ func (i *nsIndex) tickingBlocks( func (i *nsIndex) WarmFlush( flush persist.IndexFlush, shards []databaseShard, -) error { +) ([]shardFlush, error) { if len(shards) == 0 { // No-op if no shards currently owned. - return nil + return []shardFlush{}, nil } flushable, err := i.flushableBlocks(shards, series.WarmWrite) if err != nil { - return err + return nil, err } // Determine the current flush indexing concurrency. @@ -1029,7 +1029,7 @@ func (i *nsIndex) WarmFlush( builder, err := builder.NewBuilderFromDocuments(builderOpts) if err != nil { - return err + return nil, err } defer builder.Close() @@ -1039,10 +1039,11 @@ func (i *nsIndex) WarmFlush( defer i.metrics.flushIndexingConcurrency.Update(0) var evicted int + flushed := make([]shardFlush, 0) for _, block := range flushable { immutableSegments, err := i.flushBlock(flush, block, shards, builder) if err != nil { - return err + return nil, err } // Make a result that covers the entire time ranges for the // block for each shard @@ -1059,7 +1060,7 @@ func (i *nsIndex) WarmFlush( results := result.NewIndexBlockByVolumeType(block.StartTime()) results.SetBlock(idxpersist.DefaultIndexVolumeType, blockResult) if err := block.AddResults(results); err != nil { - return err + return nil, err } evicted++ @@ -1074,9 +1075,16 @@ func (i *nsIndex) WarmFlush( zap.Time("blockStart", block.StartTime().ToTime()), ) } + + for _, s := range shards { + flushed = append(flushed, shardFlush{ + time: block.StartTime(), + shard: s, + }) + } } i.metrics.blocksEvictedMutableSegments.Inc(int64(evicted)) - return nil + return flushed, nil } func (i *nsIndex) ColdFlush(shards []databaseShard) (OnColdFlushDone, error) { @@ -1197,7 +1205,10 @@ func (i *nsIndex) canFlushBlockWithRLock( if err != nil { return false, err } - if flushState.WarmStatus != fileOpSuccess { + + // Skip if the data flushing failed. We mark as "success" only once both + // data and index are flushed. + if flushState.WarmStatus == fileOpFailed { return false, nil } } diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index b552232348..206bb69059 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -1394,25 +1394,25 @@ func (n *dbNamespace) ColdFlush(flushPersist persist.FlushPreparer) error { return res } -func (n *dbNamespace) FlushIndex(flush persist.IndexFlush) error { +func (n *dbNamespace) FlushIndex(flush persist.IndexFlush) ([]shardFlush, error) { callStart := n.nowFn() n.RLock() if n.bootstrapState != Bootstrapped { n.RUnlock() n.metrics.flushIndex.ReportError(n.nowFn().Sub(callStart)) - return errNamespaceNotBootstrapped + return nil, errNamespaceNotBootstrapped } n.RUnlock() if !n.nopts.FlushEnabled() || !n.nopts.IndexOptions().Enabled() { n.metrics.flushIndex.ReportSuccess(n.nowFn().Sub(callStart)) - return nil + return nil, nil } shards := n.OwnedShards() - err := n.reverseIndex.WarmFlush(flush, shards) + flushedBlocks, err := n.reverseIndex.WarmFlush(flush, shards) n.metrics.flushIndex.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) - return err + return flushedBlocks, err } func (n *dbNamespace) Snapshot( diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 47b3dd0a92..ec60227c8c 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -430,7 +430,7 @@ type databaseNamespace interface { // FlushIndex flushes in-memory index data. FlushIndex( flush persist.IndexFlush, - ) error + ) ([]shardFlush, error) // ColdFlush flushes unflushed in-memory ColdWrites. ColdFlush( @@ -761,7 +761,7 @@ type NamespaceIndex interface { WarmFlush( flush persist.IndexFlush, shards []databaseShard, - ) error + ) ([]shardFlush, error) // ColdFlush performs any cold flushes that the index has outstanding using // the owned shards of the database. Also returns a callback to be called when From 75842e310b8037fa7032905652d2a11bbd537b50 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Mon, 2 Aug 2021 17:20:26 -0400 Subject: [PATCH 11/36] WIP - fix index flush conditions 2 --- src/dbnode/storage/flush.go | 30 +++++++----------------------- src/dbnode/storage/index.go | 22 +++++++--------------- src/dbnode/storage/namespace.go | 10 +++++----- src/dbnode/storage/types.go | 4 ++-- 4 files changed, 21 insertions(+), 45 deletions(-) diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index d52526f766..52854f787e 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -170,8 +170,7 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { multiErr = multiErr.Add(fmt.Errorf("error rotating commitlog in mediator tick: %v", err)) } - indexFlushes, err := m.indexFlush(namespaces) - if err != nil { + if err = m.indexFlush(namespaces); err != nil { multiErr = multiErr.Add(err) } @@ -181,14 +180,7 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { // happens after both data and index flushing. for _, f := range flushes { for _, s := range f.shardFlushes { - for _, ff := range indexFlushes { - for _, ss := range ff.shardFlushes { - if ff.namespace.ID().Equal(f.namespace.ID()) && ss.time.Equal(s.time) && ss.shard.ID() == s.shard.ID() { - s.shard.MarkWarmFlushStateSuccessOrError(s.time, err) - } - } - } - + s.shard.MarkWarmFlushStateSuccessOrError(s.time, err) } } @@ -284,17 +276,16 @@ func (m *flushManager) dataSnapshot( func (m *flushManager) indexFlush( namespaces []databaseNamespace, -) ([]namespaceFlush, error) { +) error { indexFlush, err := m.pm.StartIndexPersist() if err != nil { - return nil, err + return err } m.setState(flushManagerIndexFlushInProgress) var ( start = m.nowFn() multiErr = xerrors.NewMultiError() - flushed = make([]namespaceFlush, 0) ) for _, ns := range namespaces { var ( @@ -304,19 +295,13 @@ func (m *flushManager) indexFlush( if !indexEnabled { continue } - flushedShards, err := ns.FlushIndex(indexFlush) - if err == nil { - flushed = append(flushed, namespaceFlush{ - namespace: ns, - shardFlushes: flushedShards, - }) - } - multiErr = multiErr.Add(err) + + multiErr = multiErr.Add(ns.FlushIndex(indexFlush)) } multiErr = multiErr.Add(indexFlush.DoneIndex()) m.metrics.indexFlushDuration.Record(m.nowFn().Sub(start)) - return flushed, multiErr.FinalError() + return multiErr.FinalError() } func (m *flushManager) Report() { @@ -403,7 +388,6 @@ func (m *flushManager) flushNamespaceWithTimes( ) (namespaceFlush, error) { flushes := make([]shardFlush, 0) multiErr := xerrors.NewMultiError() - fmt.Println("TIMES", times) for _, t := range times { // NB(xichen): we still want to proceed if a namespace fails to flush its data. // Probably want to emit a counter here, but for now just log it. diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index ff2cc8e2f4..cb30d9ccee 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -1007,15 +1007,15 @@ func (i *nsIndex) tickingBlocks( func (i *nsIndex) WarmFlush( flush persist.IndexFlush, shards []databaseShard, -) ([]shardFlush, error) { +) error { if len(shards) == 0 { // No-op if no shards currently owned. - return []shardFlush{}, nil + return nil } flushable, err := i.flushableBlocks(shards, series.WarmWrite) if err != nil { - return nil, err + return err } // Determine the current flush indexing concurrency. @@ -1029,7 +1029,7 @@ func (i *nsIndex) WarmFlush( builder, err := builder.NewBuilderFromDocuments(builderOpts) if err != nil { - return nil, err + return err } defer builder.Close() @@ -1039,11 +1039,10 @@ func (i *nsIndex) WarmFlush( defer i.metrics.flushIndexingConcurrency.Update(0) var evicted int - flushed := make([]shardFlush, 0) for _, block := range flushable { immutableSegments, err := i.flushBlock(flush, block, shards, builder) if err != nil { - return nil, err + return err } // Make a result that covers the entire time ranges for the // block for each shard @@ -1060,7 +1059,7 @@ func (i *nsIndex) WarmFlush( results := result.NewIndexBlockByVolumeType(block.StartTime()) results.SetBlock(idxpersist.DefaultIndexVolumeType, blockResult) if err := block.AddResults(results); err != nil { - return nil, err + return err } evicted++ @@ -1075,16 +1074,9 @@ func (i *nsIndex) WarmFlush( zap.Time("blockStart", block.StartTime().ToTime()), ) } - - for _, s := range shards { - flushed = append(flushed, shardFlush{ - time: block.StartTime(), - shard: s, - }) - } } i.metrics.blocksEvictedMutableSegments.Inc(int64(evicted)) - return flushed, nil + return nil } func (i *nsIndex) ColdFlush(shards []databaseShard) (OnColdFlushDone, error) { diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index 206bb69059..b552232348 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -1394,25 +1394,25 @@ func (n *dbNamespace) ColdFlush(flushPersist persist.FlushPreparer) error { return res } -func (n *dbNamespace) FlushIndex(flush persist.IndexFlush) ([]shardFlush, error) { +func (n *dbNamespace) FlushIndex(flush persist.IndexFlush) error { callStart := n.nowFn() n.RLock() if n.bootstrapState != Bootstrapped { n.RUnlock() n.metrics.flushIndex.ReportError(n.nowFn().Sub(callStart)) - return nil, errNamespaceNotBootstrapped + return errNamespaceNotBootstrapped } n.RUnlock() if !n.nopts.FlushEnabled() || !n.nopts.IndexOptions().Enabled() { n.metrics.flushIndex.ReportSuccess(n.nowFn().Sub(callStart)) - return nil, nil + return nil } shards := n.OwnedShards() - flushedBlocks, err := n.reverseIndex.WarmFlush(flush, shards) + err := n.reverseIndex.WarmFlush(flush, shards) n.metrics.flushIndex.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) - return flushedBlocks, err + return err } func (n *dbNamespace) Snapshot( diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index ec60227c8c..47b3dd0a92 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -430,7 +430,7 @@ type databaseNamespace interface { // FlushIndex flushes in-memory index data. FlushIndex( flush persist.IndexFlush, - ) ([]shardFlush, error) + ) error // ColdFlush flushes unflushed in-memory ColdWrites. ColdFlush( @@ -761,7 +761,7 @@ type NamespaceIndex interface { WarmFlush( flush persist.IndexFlush, shards []databaseShard, - ) ([]shardFlush, error) + ) error // ColdFlush performs any cold flushes that the index has outstanding using // the owned shards of the database. Also returns a callback to be called when From ee6e6ea927d77158a0dba625c8dd3a516952cae9 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Mon, 2 Aug 2021 17:54:54 -0400 Subject: [PATCH 12/36] Add test to verify warm flush ordering --- src/dbnode/storage/flush_test.go | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/src/dbnode/storage/flush_test.go b/src/dbnode/storage/flush_test.go index cf7fdde0fe..d52cce656c 100644 --- a/src/dbnode/storage/flush_test.go +++ b/src/dbnode/storage/flush_test.go @@ -361,6 +361,7 @@ func TestFlushManagerNamespaceIndexingEnabled(t *testing.T) { ctrl := xtest.NewController(t) defer ctrl.Finish() + blocks := 24 nsOpts := defaultTestNs1Opts.SetIndexOptions(namespace.NewIndexOptions().SetEnabled(true)) s1 := NewMockdatabaseShard(ctrl) s2 := NewMockdatabaseShard(ctrl) @@ -368,11 +369,19 @@ func TestFlushManagerNamespaceIndexingEnabled(t *testing.T) { ns.EXPECT().Options().Return(nsOpts).AnyTimes() ns.EXPECT().ID().Return(defaultTestNs1ID).AnyTimes() ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(true, nil).AnyTimes() - ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any()).Return([]databaseShard{s1, s2}, nil).AnyTimes() - ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() - ns.EXPECT().FlushIndex(gomock.Any()).Return(nil) - s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil).AnyTimes() - s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil).AnyTimes() + + // Validate that the flush state is marked as successful only AFTER all prequisite steps have been run. + // Order is important to avoid any edge case where data is GCed from memory without all flushing operations + // being completed. + steps := make([]*gomock.Call, 0) + steps = append(steps, ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any()).Return([]databaseShard{s1, s2}, nil).Times(blocks)) + steps = append(steps, ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes()) + steps = append(steps, ns.EXPECT().FlushIndex(gomock.Any()).Return(nil)) + for i := 0; i < blocks; i++ { + steps = append(steps, s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil)) + steps = append(steps, s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil)) + } + gomock.InOrder(steps...) var ( mockFlushPersist = persist.NewMockFlushPreparer(ctrl) From db5552ff3874a7a2ab50560800d4f6638b0ab79f Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Mon, 2 Aug 2021 18:03:27 -0400 Subject: [PATCH 13/36] Lint --- src/dbnode/storage/flush_test.go | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/dbnode/storage/flush_test.go b/src/dbnode/storage/flush_test.go index d52cce656c..c5c6e9de8a 100644 --- a/src/dbnode/storage/flush_test.go +++ b/src/dbnode/storage/flush_test.go @@ -374,12 +374,16 @@ func TestFlushManagerNamespaceIndexingEnabled(t *testing.T) { // Order is important to avoid any edge case where data is GCed from memory without all flushing operations // being completed. steps := make([]*gomock.Call, 0) - steps = append(steps, ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any()).Return([]databaseShard{s1, s2}, nil).Times(blocks)) - steps = append(steps, ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes()) - steps = append(steps, ns.EXPECT().FlushIndex(gomock.Any()).Return(nil)) + steps = append(steps, + ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any()).Return([]databaseShard{s1, s2}, nil).Times(blocks), + ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes(), + ns.EXPECT().FlushIndex(gomock.Any()).Return(nil), + ) for i := 0; i < blocks; i++ { - steps = append(steps, s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil)) - steps = append(steps, s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil)) + steps = append(steps, + s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil), + s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil), + ) } gomock.InOrder(steps...) From 1e81687f41f8c081240c76c594ad5ee790452714 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Thu, 5 Aug 2021 16:09:11 -0400 Subject: [PATCH 14/36] Experimental index flush matching --- src/dbnode/storage/flush.go | 36 +++++++++++++++++++++++++-------- src/dbnode/storage/index.go | 22 +++++++++++++------- src/dbnode/storage/namespace.go | 10 ++++----- src/dbnode/storage/types.go | 4 ++-- 4 files changed, 50 insertions(+), 22 deletions(-) diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index 52854f787e..eaff4a38f7 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -170,7 +170,8 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { multiErr = multiErr.Add(fmt.Errorf("error rotating commitlog in mediator tick: %v", err)) } - if err = m.indexFlush(namespaces); err != nil { + indexFlushes, err := m.indexFlush(namespaces) + if err != nil { multiErr = multiErr.Add(err) } @@ -178,9 +179,19 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { // Mark all flush states at the very end to ensure this // happens after both data and index flushing. + // TODO: if this matching approach of data + index ns+shard+time is needed, then reimplement this + // more efficiently w/ hashing for constant lookup cost. for _, f := range flushes { for _, s := range f.shardFlushes { - s.shard.MarkWarmFlushStateSuccessOrError(s.time, err) + for _, ff := range indexFlushes { + for _, ss := range ff.shardFlushes { + if f.namespace.ID().Equal(ff.namespace.ID()) && + s.shard.ID() == ss.shard.ID() && + s.time.Equal(ss.time) { + s.shard.MarkWarmFlushStateSuccessOrError(s.time, err) + } + } + } } } @@ -276,16 +287,17 @@ func (m *flushManager) dataSnapshot( func (m *flushManager) indexFlush( namespaces []databaseNamespace, -) error { +) ([]namespaceFlush, error) { indexFlush, err := m.pm.StartIndexPersist() if err != nil { - return err + return nil, err } m.setState(flushManagerIndexFlushInProgress) var ( - start = m.nowFn() - multiErr = xerrors.NewMultiError() + start = m.nowFn() + multiErr = xerrors.NewMultiError() + namespaceFlushes = make([]namespaceFlush, 0) ) for _, ns := range namespaces { var ( @@ -296,12 +308,20 @@ func (m *flushManager) indexFlush( continue } - multiErr = multiErr.Add(ns.FlushIndex(indexFlush)) + shardFlushes, err := ns.FlushIndex(indexFlush) + if err != nil { + multiErr = multiErr.Add(err) + } else { + namespaceFlushes = append(namespaceFlushes, namespaceFlush{ + namespace: ns, + shardFlushes: shardFlushes, + }) + } } multiErr = multiErr.Add(indexFlush.DoneIndex()) m.metrics.indexFlushDuration.Record(m.nowFn().Sub(start)) - return multiErr.FinalError() + return namespaceFlushes, multiErr.FinalError() } func (m *flushManager) Report() { diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index cb30d9ccee..c2ebba6f9c 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -1007,15 +1007,15 @@ func (i *nsIndex) tickingBlocks( func (i *nsIndex) WarmFlush( flush persist.IndexFlush, shards []databaseShard, -) error { +) ([]shardFlush, error) { if len(shards) == 0 { // No-op if no shards currently owned. - return nil + return []shardFlush{}, nil } flushable, err := i.flushableBlocks(shards, series.WarmWrite) if err != nil { - return err + return nil, err } // Determine the current flush indexing concurrency. @@ -1029,7 +1029,7 @@ func (i *nsIndex) WarmFlush( builder, err := builder.NewBuilderFromDocuments(builderOpts) if err != nil { - return err + return nil, err } defer builder.Close() @@ -1039,10 +1039,11 @@ func (i *nsIndex) WarmFlush( defer i.metrics.flushIndexingConcurrency.Update(0) var evicted int + shardFlushes := make([]shardFlush, 0) for _, block := range flushable { immutableSegments, err := i.flushBlock(flush, block, shards, builder) if err != nil { - return err + return nil, err } // Make a result that covers the entire time ranges for the // block for each shard @@ -1059,7 +1060,7 @@ func (i *nsIndex) WarmFlush( results := result.NewIndexBlockByVolumeType(block.StartTime()) results.SetBlock(idxpersist.DefaultIndexVolumeType, blockResult) if err := block.AddResults(results); err != nil { - return err + return nil, err } evicted++ @@ -1073,10 +1074,17 @@ func (i *nsIndex) WarmFlush( zap.Error(err), zap.Time("blockStart", block.StartTime().ToTime()), ) + } else { + for _, s := range shards { + shardFlushes = append(shardFlushes, shardFlush{ + shard: s, + time: block.StartTime(), + }) + } } } i.metrics.blocksEvictedMutableSegments.Inc(int64(evicted)) - return nil + return shardFlushes, nil } func (i *nsIndex) ColdFlush(shards []databaseShard) (OnColdFlushDone, error) { diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index b552232348..8baf502662 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -1394,25 +1394,25 @@ func (n *dbNamespace) ColdFlush(flushPersist persist.FlushPreparer) error { return res } -func (n *dbNamespace) FlushIndex(flush persist.IndexFlush) error { +func (n *dbNamespace) FlushIndex(flush persist.IndexFlush) ([]shardFlush, error) { callStart := n.nowFn() n.RLock() if n.bootstrapState != Bootstrapped { n.RUnlock() n.metrics.flushIndex.ReportError(n.nowFn().Sub(callStart)) - return errNamespaceNotBootstrapped + return nil, errNamespaceNotBootstrapped } n.RUnlock() if !n.nopts.FlushEnabled() || !n.nopts.IndexOptions().Enabled() { n.metrics.flushIndex.ReportSuccess(n.nowFn().Sub(callStart)) - return nil + return []shardFlush{}, nil } shards := n.OwnedShards() - err := n.reverseIndex.WarmFlush(flush, shards) + shardFlushes, err := n.reverseIndex.WarmFlush(flush, shards) n.metrics.flushIndex.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) - return err + return shardFlushes, err } func (n *dbNamespace) Snapshot( diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 47b3dd0a92..ec60227c8c 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -430,7 +430,7 @@ type databaseNamespace interface { // FlushIndex flushes in-memory index data. FlushIndex( flush persist.IndexFlush, - ) error + ) ([]shardFlush, error) // ColdFlush flushes unflushed in-memory ColdWrites. ColdFlush( @@ -761,7 +761,7 @@ type NamespaceIndex interface { WarmFlush( flush persist.IndexFlush, shards []databaseShard, - ) error + ) ([]shardFlush, error) // ColdFlush performs any cold flushes that the index has outstanding using // the owned shards of the database. Also returns a callback to be called when From 8344c17c490cc00e8bf75d23f355448c0a636698 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Tue, 10 Aug 2021 16:19:01 -0400 Subject: [PATCH 15/36] Use maps for shard flushes --- src/dbnode/storage/flush.go | 60 +++++++++++++++++++-------------- src/dbnode/storage/index.go | 10 +++--- src/dbnode/storage/namespace.go | 4 +-- src/dbnode/storage/types.go | 4 +-- 4 files changed, 44 insertions(+), 34 deletions(-) diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index eaff4a38f7..53c073c43d 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -54,9 +54,13 @@ const ( flushManagerIndexFlushInProgress ) +type namespaceFlushes map[string]namespaceFlush + +type shardFlushes map[shardFlush]bool + type namespaceFlush struct { namespace databaseNamespace - shardFlushes []shardFlush + shardFlushes map[shardFlush]bool } type shardFlush struct { @@ -64,6 +68,22 @@ type shardFlush struct { shard databaseShard } +func (n namespaceFlushes) forEachMatch(nn namespaceFlushes, fn func(shardFlush)) { + for nsID, ns := range n { + otherNS, ok := nn[nsID] + if !ok { + continue + } + + for s := range ns.shardFlushes { + _, ok := otherNS.shardFlushes[s] + if ok { + fn(s) + } + } + } +} + type flushManagerMetrics struct { isFlushing tally.Gauge isSnapshotting tally.Gauge @@ -181,19 +201,9 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { // happens after both data and index flushing. // TODO: if this matching approach of data + index ns+shard+time is needed, then reimplement this // more efficiently w/ hashing for constant lookup cost. - for _, f := range flushes { - for _, s := range f.shardFlushes { - for _, ff := range indexFlushes { - for _, ss := range ff.shardFlushes { - if f.namespace.ID().Equal(ff.namespace.ID()) && - s.shard.ID() == ss.shard.ID() && - s.time.Equal(ss.time) { - s.shard.MarkWarmFlushStateSuccessOrError(s.time, err) - } - } - } - } - } + flushes.forEachMatch(indexFlushes, func(match shardFlush) { + match.shard.MarkWarmFlushStateSuccessOrError(match.time, err) + }) return err } @@ -201,7 +211,7 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { func (m *flushManager) dataWarmFlush( namespaces []databaseNamespace, startTime xtime.UnixNano, -) ([]namespaceFlush, error) { +) (namespaceFlushes, error) { flushPersist, err := m.pm.StartFlushPersist() if err != nil { return nil, err @@ -211,7 +221,7 @@ func (m *flushManager) dataWarmFlush( var ( start = m.nowFn() multiErr = xerrors.NewMultiError() - allFlushes = make([]namespaceFlush, 0) + allFlushes = make(map[string]namespaceFlush, 0) ) for _, ns := range namespaces { // Flush first because we will only snapshot if there are no outstanding flushes. @@ -224,7 +234,7 @@ func (m *flushManager) dataWarmFlush( if err != nil { multiErr = multiErr.Add(err) } - allFlushes = append(allFlushes, flush) + allFlushes[ns.ID().String()] = flush } err = flushPersist.DoneFlush() @@ -287,7 +297,7 @@ func (m *flushManager) dataSnapshot( func (m *flushManager) indexFlush( namespaces []databaseNamespace, -) ([]namespaceFlush, error) { +) (namespaceFlushes, error) { indexFlush, err := m.pm.StartIndexPersist() if err != nil { return nil, err @@ -297,7 +307,7 @@ func (m *flushManager) indexFlush( var ( start = m.nowFn() multiErr = xerrors.NewMultiError() - namespaceFlushes = make([]namespaceFlush, 0) + namespaceFlushes = make(map[string]namespaceFlush, 0) ) for _, ns := range namespaces { var ( @@ -312,10 +322,10 @@ func (m *flushManager) indexFlush( if err != nil { multiErr = multiErr.Add(err) } else { - namespaceFlushes = append(namespaceFlushes, namespaceFlush{ + namespaceFlushes[ns.ID().String()] = namespaceFlush{ namespace: ns, shardFlushes: shardFlushes, - }) + } } } multiErr = multiErr.Add(indexFlush.DoneIndex()) @@ -406,7 +416,7 @@ func (m *flushManager) flushNamespaceWithTimes( times []xtime.UnixNano, flushPreparer persist.FlushPreparer, ) (namespaceFlush, error) { - flushes := make([]shardFlush, 0) + var flushes map[shardFlush]bool multiErr := xerrors.NewMultiError() for _, t := range times { // NB(xichen): we still want to proceed if a namespace fails to flush its data. @@ -418,10 +428,10 @@ func (m *flushManager) flushNamespaceWithTimes( multiErr = multiErr.Add(detailedErr) } else { for _, s := range shards { - flushes = append(flushes, shardFlush{ - time: t, + flushes[shardFlush{ shard: s, - }) + time: t, + }] = true } } } diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index c2ebba6f9c..5c0c0cd155 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -1007,10 +1007,10 @@ func (i *nsIndex) tickingBlocks( func (i *nsIndex) WarmFlush( flush persist.IndexFlush, shards []databaseShard, -) ([]shardFlush, error) { +) (shardFlushes, error) { if len(shards) == 0 { // No-op if no shards currently owned. - return []shardFlush{}, nil + return nil, nil } flushable, err := i.flushableBlocks(shards, series.WarmWrite) @@ -1039,7 +1039,7 @@ func (i *nsIndex) WarmFlush( defer i.metrics.flushIndexingConcurrency.Update(0) var evicted int - shardFlushes := make([]shardFlush, 0) + shardFlushes := make(map[shardFlush]bool, 0) for _, block := range flushable { immutableSegments, err := i.flushBlock(flush, block, shards, builder) if err != nil { @@ -1076,10 +1076,10 @@ func (i *nsIndex) WarmFlush( ) } else { for _, s := range shards { - shardFlushes = append(shardFlushes, shardFlush{ + shardFlushes[shardFlush{ shard: s, time: block.StartTime(), - }) + }] = true } } } diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index 8baf502662..dccc437132 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -1394,7 +1394,7 @@ func (n *dbNamespace) ColdFlush(flushPersist persist.FlushPreparer) error { return res } -func (n *dbNamespace) FlushIndex(flush persist.IndexFlush) ([]shardFlush, error) { +func (n *dbNamespace) FlushIndex(flush persist.IndexFlush) (shardFlushes, error) { callStart := n.nowFn() n.RLock() if n.bootstrapState != Bootstrapped { @@ -1406,7 +1406,7 @@ func (n *dbNamespace) FlushIndex(flush persist.IndexFlush) ([]shardFlush, error) if !n.nopts.FlushEnabled() || !n.nopts.IndexOptions().Enabled() { n.metrics.flushIndex.ReportSuccess(n.nowFn().Sub(callStart)) - return []shardFlush{}, nil + return nil, nil } shards := n.OwnedShards() diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index ec60227c8c..91ddbc8e34 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -430,7 +430,7 @@ type databaseNamespace interface { // FlushIndex flushes in-memory index data. FlushIndex( flush persist.IndexFlush, - ) ([]shardFlush, error) + ) (shardFlushes, error) // ColdFlush flushes unflushed in-memory ColdWrites. ColdFlush( @@ -761,7 +761,7 @@ type NamespaceIndex interface { WarmFlush( flush persist.IndexFlush, shards []databaseShard, - ) ([]shardFlush, error) + ) (shardFlushes, error) // ColdFlush performs any cold flushes that the index has outstanding using // the owned shards of the database. Also returns a callback to be called when From c6c3b4fc1c788f228dfaac33e2e07c190a206373 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Tue, 17 Aug 2021 10:11:09 -0400 Subject: [PATCH 16/36] Mark flushed shards based on block size --- src/dbnode/storage/flush.go | 57 +++++++++++++++++++++---------------- 1 file changed, 32 insertions(+), 25 deletions(-) diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index 53c073c43d..e7573c3781 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -68,22 +68,6 @@ type shardFlush struct { shard databaseShard } -func (n namespaceFlushes) forEachMatch(nn namespaceFlushes, fn func(shardFlush)) { - for nsID, ns := range n { - otherNS, ok := nn[nsID] - if !ok { - continue - } - - for s := range ns.shardFlushes { - _, ok := otherNS.shardFlushes[s] - if ok { - fn(s) - } - } - } -} - type flushManagerMetrics struct { isFlushing tally.Gauge isSnapshotting tally.Gauge @@ -174,7 +158,7 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { // will attempt to snapshot blocks w/ unflushed data which would be wasteful if // the block is already flushable. multiErr := xerrors.NewMultiError() - flushes, err := m.dataWarmFlush(namespaces, startTime) + dataFlushes, err := m.dataWarmFlush(namespaces, startTime) if err != nil { multiErr = multiErr.Add(err) } @@ -197,13 +181,36 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { err = multiErr.FinalError() - // Mark all flush states at the very end to ensure this - // happens after both data and index flushing. - // TODO: if this matching approach of data + index ns+shard+time is needed, then reimplement this - // more efficiently w/ hashing for constant lookup cost. - flushes.forEachMatch(indexFlushes, func(match shardFlush) { - match.shard.MarkWarmFlushStateSuccessOrError(match.time, err) - }) + // Mark all flushed shards as such. + // If index is not enabled, then a shard+blockStart is "flushed" if the data has been flushed. + // If index is enabled, then a shard+blockStart is "flushed" if the data AND index has been flushed. + for _, n := range namespaces { + var ( + indexEnabled = n.Options().IndexOptions().Enabled() + flushedShards map[shardFlush]bool + ) + if indexEnabled { + flushedShards = indexFlushes[n.ID().String()].shardFlushes + } else { + flushedShards = dataFlushes[n.ID().String()].shardFlushes + } + + for s := range flushedShards { + s.shard.MarkWarmFlushStateSuccessOrError(s.time, err) + + // Block sizes for data and index can differ and so if we are driving the flushing by + // the index blockStarts, we must expand them to mark all containing data blockStarts. + // E.g. if blockSize == 2h and indexBlockSize == 4h and the flushed index time is 6:00pm, + // we should mark as flushed [6:00pm, 8:00pm]. + if indexEnabled { + blockSize := n.Options().RetentionOptions().BlockSize() + indexBlockSize := n.Options().IndexOptions().BlockSize() + for start := s.time.Add(indexBlockSize); start < s.time.Add(indexBlockSize); start = start.Add(blockSize) { + s.shard.MarkWarmFlushStateSuccessOrError(start, err) + } + } + } + } return err } @@ -416,7 +423,7 @@ func (m *flushManager) flushNamespaceWithTimes( times []xtime.UnixNano, flushPreparer persist.FlushPreparer, ) (namespaceFlush, error) { - var flushes map[shardFlush]bool + flushes := make(map[shardFlush]bool, 0) multiErr := xerrors.NewMultiError() for _, t := range times { // NB(xichen): we still want to proceed if a namespace fails to flush its data. From 0b28fe7ba4f47fd28560f37ba94a62679bc9144c Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Tue, 17 Aug 2021 10:24:27 -0400 Subject: [PATCH 17/36] Fixup shard marking logic --- .../index_active_block_rotate_test.go | 2 +- src/dbnode/storage/flush.go | 18 +++++++++++++----- 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/src/dbnode/integration/index_active_block_rotate_test.go b/src/dbnode/integration/index_active_block_rotate_test.go index 0abc5e8924..87d96902bb 100644 --- a/src/dbnode/integration/index_active_block_rotate_test.go +++ b/src/dbnode/integration/index_active_block_rotate_test.go @@ -51,7 +51,7 @@ func TestIndexActiveBlockRotate(t *testing.T) { numWrites = 50 numTags = 10 blockSize = 2 * time.Hour - indexBlockSize = blockSize + indexBlockSize = blockSize * 2 retentionPeriod = 12 * blockSize bufferPast = 10 * time.Minute rOpts = retention.NewOptions(). diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index e7573c3781..b75ed60a07 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -190,14 +190,20 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { flushedShards map[shardFlush]bool ) if indexEnabled { - flushedShards = indexFlushes[n.ID().String()].shardFlushes + flushesForNs, ok := indexFlushes[n.ID().String()] + if !ok { + continue + } + flushedShards = flushesForNs.shardFlushes } else { - flushedShards = dataFlushes[n.ID().String()].shardFlushes + flushesForNs, ok := dataFlushes[n.ID().String()] + if !ok { + continue + } + flushedShards = flushesForNs.shardFlushes } for s := range flushedShards { - s.shard.MarkWarmFlushStateSuccessOrError(s.time, err) - // Block sizes for data and index can differ and so if we are driving the flushing by // the index blockStarts, we must expand them to mark all containing data blockStarts. // E.g. if blockSize == 2h and indexBlockSize == 4h and the flushed index time is 6:00pm, @@ -205,9 +211,11 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { if indexEnabled { blockSize := n.Options().RetentionOptions().BlockSize() indexBlockSize := n.Options().IndexOptions().BlockSize() - for start := s.time.Add(indexBlockSize); start < s.time.Add(indexBlockSize); start = start.Add(blockSize) { + for start := s.time; start < s.time.Add(indexBlockSize); start = start.Add(blockSize) { s.shard.MarkWarmFlushStateSuccessOrError(start, err) } + } else { + s.shard.MarkWarmFlushStateSuccessOrError(s.time, err) } } } From 3bf7412b84e426396656f3c76829ea0af68e0834 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Tue, 17 Aug 2021 10:57:27 -0400 Subject: [PATCH 18/36] Mock --- src/dbnode/storage/storage_mock.go | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index 72cfa38566..49c222ad97 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -1425,11 +1425,12 @@ func (mr *MockdatabaseNamespaceMockRecorder) FetchWideEntry(ctx, id, blockStart, } // FlushIndex mocks base method. -func (m *MockdatabaseNamespace) FlushIndex(flush persist.IndexFlush) error { +func (m *MockdatabaseNamespace) FlushIndex(flush persist.IndexFlush) (shardFlushes, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "FlushIndex", flush) - ret0, _ := ret[0].(error) - return ret0 + ret0, _ := ret[0].(shardFlushes) + ret1, _ := ret[1].(error) + return ret0, ret1 } // FlushIndex indicates an expected call of FlushIndex. @@ -2680,11 +2681,12 @@ func (mr *MockNamespaceIndexMockRecorder) Tick(c, startTime interface{}) *gomock } // WarmFlush mocks base method. -func (m *MockNamespaceIndex) WarmFlush(flush persist.IndexFlush, shards []databaseShard) error { +func (m *MockNamespaceIndex) WarmFlush(flush persist.IndexFlush, shards []databaseShard) (shardFlushes, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "WarmFlush", flush, shards) - ret0, _ := ret[0].(error) - return ret0 + ret0, _ := ret[0].(shardFlushes) + ret1, _ := ret[1].(error) + return ret0, ret1 } // WarmFlush indicates an expected call of WarmFlush. From dd4dd0a648ea00183260512cc273c2f9d477dbff Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Tue, 17 Aug 2021 11:24:22 -0400 Subject: [PATCH 19/36] Fix test --- src/dbnode/storage/index_test.go | 38 +++++++++++++++++++++----------- 1 file changed, 25 insertions(+), 13 deletions(-) diff --git a/src/dbnode/storage/index_test.go b/src/dbnode/storage/index_test.go index e5d0d5c543..81c0da47a6 100644 --- a/src/dbnode/storage/index_test.go +++ b/src/dbnode/storage/index_test.go @@ -649,7 +649,9 @@ func TestNamespaceIndexFlushShardStateNotSuccess(t *testing.T) { mockFlush := persist.NewMockIndexFlush(ctrl) - require.NoError(t, idx.WarmFlush(mockFlush, shards)) + flushed, err := idx.WarmFlush(mockFlush, shards) + require.NoError(t, err) + require.Equal(t, shardFlushes{}, flushed) } func TestNamespaceIndexQueryNoMatchingBlocks(t *testing.T) { @@ -829,17 +831,18 @@ func verifyFlushForShards( shards []uint32, ) { var ( - mockFlush = persist.NewMockIndexFlush(ctrl) - shardMap = make(map[uint32]struct{}) - now = xtime.Now() - warmBlockStart = now.Add(-idx.bufferPast).Truncate(idx.blockSize) - mockShards []*MockdatabaseShard - dbShards []databaseShard - numBlocks int - persistClosedTimes int - persistCalledTimes int - actualDocs = make([]doc.Metadata, 0) - expectedDocs = make([]doc.Metadata, 0) + mockFlush = persist.NewMockIndexFlush(ctrl) + shardMap = make(map[uint32]struct{}) + now = xtime.Now() + warmBlockStart = now.Add(-idx.bufferPast).Truncate(idx.blockSize) + mockShards []*MockdatabaseShard + dbShards []databaseShard + numBlocks int + persistClosedTimes int + persistCalledTimes int + actualDocs = make([]doc.Metadata, 0) + expectedDocs = make([]doc.Metadata, 0) + expectedShardFlushes = make(map[shardFlush]bool, 0) ) // NB(bodu): Always align now w/ the index's view of now. idx.nowFn = func() time.Time { @@ -925,16 +928,25 @@ func verifyFlushForShards( mockShard.EXPECT().FetchBlocksMetadataV2(gomock.Any(), blockStart, blockStart.Add(idx.blockSize), gomock.Any(), gomock.Any(), block.FetchBlocksMetadataOptions{OnlyDisk: true}).Return(results, nil, nil) + + expectedShardFlushes[shardFlush{shard: mockShard, time: blockStart}] = true } mockBlock.EXPECT().IsSealed().Return(true) mockBlock.EXPECT().AddResults(gomock.Any()).Return(nil) mockBlock.EXPECT().EvictMutableSegments().Return(nil) } - require.NoError(t, idx.WarmFlush(mockFlush, dbShards)) + flushed, err := idx.WarmFlush(mockFlush, dbShards) + require.NoError(t, err) + require.Equal(t, len(expectedShardFlushes), len(flushed)) require.Equal(t, numBlocks, persistClosedTimes) require.Equal(t, numBlocks, persistCalledTimes) require.Equal(t, expectedDocs, actualDocs) + + for flushedShard := range flushed { + _, ok := expectedShardFlushes[flushedShard] + require.Equal(t, true, ok) + } } func newReadIndexInfoFileResult( From 4084517fb36f92ef7885cb5b72c02684f87ff160 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Tue, 17 Aug 2021 11:44:06 -0400 Subject: [PATCH 20/36] Fix test TestFlushManagerNamespaceIndexingEnabled --- src/dbnode/storage/flush_test.go | 18 +++++++++++------- src/dbnode/storage/namespace.go | 4 ++-- 2 files changed, 13 insertions(+), 9 deletions(-) diff --git a/src/dbnode/storage/flush_test.go b/src/dbnode/storage/flush_test.go index c5c6e9de8a..f321cedaf8 100644 --- a/src/dbnode/storage/flush_test.go +++ b/src/dbnode/storage/flush_test.go @@ -373,18 +373,22 @@ func TestFlushManagerNamespaceIndexingEnabled(t *testing.T) { // Validate that the flush state is marked as successful only AFTER all prequisite steps have been run. // Order is important to avoid any edge case where data is GCed from memory without all flushing operations // being completed. + mockFlushedShards := map[shardFlush]bool{ + shardFlush{shard: s1, time: xtime.Now().Add(time.Minute * 1)}: true, + shardFlush{shard: s1, time: xtime.Now().Add(time.Minute * 2)}: true, + shardFlush{shard: s2, time: xtime.Now().Add(time.Minute * 1)}: true, + shardFlush{shard: s2, time: xtime.Now().Add(time.Minute * 2)}: true, + } steps := make([]*gomock.Call, 0) steps = append(steps, ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any()).Return([]databaseShard{s1, s2}, nil).Times(blocks), ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes(), - ns.EXPECT().FlushIndex(gomock.Any()).Return(nil), + ns.EXPECT().FlushIndex(gomock.Any()).Return(mockFlushedShards, nil), ) - for i := 0; i < blocks; i++ { - steps = append(steps, - s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil), - s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil), - ) - } + steps = append(steps, s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil)) + steps = append(steps, s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil)) + steps = append(steps, s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil)) + steps = append(steps, s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil)) gomock.InOrder(steps...) var ( diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index dccc437132..3920621219 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -1410,9 +1410,9 @@ func (n *dbNamespace) FlushIndex(flush persist.IndexFlush) (shardFlushes, error) } shards := n.OwnedShards() - shardFlushes, err := n.reverseIndex.WarmFlush(flush, shards) + flushes, err := n.reverseIndex.WarmFlush(flush, shards) n.metrics.flushIndex.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) - return shardFlushes, err + return flushes, err } func (n *dbNamespace) Snapshot( From f83f864d8ed0e9637a934e9ecab78d76c9d6fafd Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Tue, 17 Aug 2021 11:54:57 -0400 Subject: [PATCH 21/36] Lint --- src/dbnode/storage/flush.go | 8 ++++---- src/dbnode/storage/flush_test.go | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index b75ed60a07..b7c1e6a8aa 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -236,7 +236,7 @@ func (m *flushManager) dataWarmFlush( var ( start = m.nowFn() multiErr = xerrors.NewMultiError() - allFlushes = make(map[string]namespaceFlush, 0) + allFlushes = make(map[string]namespaceFlush) ) for _, ns := range namespaces { // Flush first because we will only snapshot if there are no outstanding flushes. @@ -322,7 +322,7 @@ func (m *flushManager) indexFlush( var ( start = m.nowFn() multiErr = xerrors.NewMultiError() - namespaceFlushes = make(map[string]namespaceFlush, 0) + namespaceFlushes = make(map[string]namespaceFlush) ) for _, ns := range namespaces { var ( @@ -333,13 +333,13 @@ func (m *flushManager) indexFlush( continue } - shardFlushes, err := ns.FlushIndex(indexFlush) + flushes, err := ns.FlushIndex(indexFlush) if err != nil { multiErr = multiErr.Add(err) } else { namespaceFlushes[ns.ID().String()] = namespaceFlush{ namespace: ns, - shardFlushes: shardFlushes, + shardFlushes: flushes, } } } diff --git a/src/dbnode/storage/flush_test.go b/src/dbnode/storage/flush_test.go index f321cedaf8..0529228aec 100644 --- a/src/dbnode/storage/flush_test.go +++ b/src/dbnode/storage/flush_test.go @@ -384,11 +384,11 @@ func TestFlushManagerNamespaceIndexingEnabled(t *testing.T) { ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any()).Return([]databaseShard{s1, s2}, nil).Times(blocks), ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes(), ns.EXPECT().FlushIndex(gomock.Any()).Return(mockFlushedShards, nil), + s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil), + s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil), + s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil), + s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil), ) - steps = append(steps, s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil)) - steps = append(steps, s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil)) - steps = append(steps, s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil)) - steps = append(steps, s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil)) gomock.InOrder(steps...) var ( From e80e38382acdf3aa8e86a94facaf35df529dfc71 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Wed, 18 Aug 2021 09:42:09 -0400 Subject: [PATCH 22/36] Add RelookupAndCheckIsEmpty --- .../storage/{series/lookup => }/entry.go | 19 +++- .../lookup => }/entry_blackbox_test.go | 12 +-- .../lookup => }/entry_whitebox_test.go | 2 +- src/dbnode/storage/index/mutable_segments.go | 13 ++- src/dbnode/storage/index/types.go | 2 +- .../{series/lookup => }/lookup_mock.go | 6 +- src/dbnode/storage/series_resolver.go | 5 +- src/dbnode/storage/series_resolver_test.go | 40 ++++----- src/dbnode/storage/shard.go | 87 ++++++++++--------- .../storage/shard_foreachentry_prop_test.go | 6 +- src/dbnode/storage/shard_index_test.go | 4 +- src/dbnode/storage/shard_insert_queue.go | 3 +- src/dbnode/storage/shard_insert_queue_test.go | 10 +-- src/dbnode/storage/shard_test.go | 40 ++++----- src/dbnode/storage/types.go | 3 + 15 files changed, 139 insertions(+), 113 deletions(-) rename src/dbnode/storage/{series/lookup => }/entry.go (95%) rename src/dbnode/storage/{series/lookup => }/entry_blackbox_test.go (91%) rename src/dbnode/storage/{series/lookup => }/entry_whitebox_test.go (99%) rename src/dbnode/storage/{series/lookup => }/lookup_mock.go (95%) diff --git a/src/dbnode/storage/series/lookup/entry.go b/src/dbnode/storage/entry.go similarity index 95% rename from src/dbnode/storage/series/lookup/entry.go rename to src/dbnode/storage/entry.go index 045fc86f43..b8e95dba56 100644 --- a/src/dbnode/storage/series/lookup/entry.go +++ b/src/dbnode/storage/entry.go @@ -18,7 +18,7 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -package lookup +package storage import ( "sync" @@ -58,6 +58,7 @@ type IndexWriter interface { // members to track lifecycle and minimize indexing overhead. // NB: users are expected to use `NewEntry` to construct these objects. type Entry struct { + Shard Shard Series series.DatabaseSeries Index uint64 indexWriter IndexWriter @@ -78,6 +79,7 @@ var _ bootstrap.SeriesRefResolver = &Entry{} // NewEntryOptions supplies options for a new entry. type NewEntryOptions struct { + Shard Shard Series series.DatabaseSeries Index uint64 IndexWriter IndexWriter @@ -91,6 +93,7 @@ func NewEntry(opts NewEntryOptions) *Entry { nowFn = opts.NowFn } entry := &Entry{ + Shard: opts.Shard, Series: opts.Series, Index: opts.Index, indexWriter: opts.IndexWriter, @@ -223,9 +226,17 @@ func (entry *Entry) IfAlreadyIndexedMarkIndexSuccessAndFinalize( return successAlready } -// IsEmpty returns true if the entry has no in-memory series data. -func (entry *Entry) IsEmpty() bool { - return entry.Series.IsEmpty() +// RelookupAndCheckIsEmpty looks up the series and checks if it is empty. +// The first result indicates if the series is empty. +// The second result indicates if the series can be looked up at all. +func (entry *Entry) RelookupAndCheckIsEmpty() (bool, bool) { + e, _, err := entry.Shard.TryRetrieveWritableSeries(entry.Series.ID()) + if err != nil || e == nil { + return false, false + } + defer entry.DecrementReaderWriterCount() + + return entry.Series.IsEmpty(), true } // Write writes a new value. diff --git a/src/dbnode/storage/series/lookup/entry_blackbox_test.go b/src/dbnode/storage/entry_blackbox_test.go similarity index 91% rename from src/dbnode/storage/series/lookup/entry_blackbox_test.go rename to src/dbnode/storage/entry_blackbox_test.go index e7a059ed93..b56e0ba698 100644 --- a/src/dbnode/storage/series/lookup/entry_blackbox_test.go +++ b/src/dbnode/storage/entry_blackbox_test.go @@ -18,14 +18,14 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -package lookup_test +package storage import ( "sync" "testing" "time" - "github.com/m3db/m3/src/dbnode/storage/series/lookup" + "github.com/m3db/m3/src/dbnode/storage" xtime "github.com/m3db/m3/src/x/time" "github.com/fortytw2/leaktest" @@ -43,7 +43,7 @@ func newTime(n int) xtime.UnixNano { } func TestEntryReaderWriterCount(t *testing.T) { - e := lookup.NewEntry(lookup.NewEntryOptions{}) + e := storage.NewEntry(storage.NewEntryOptions{}) require.Equal(t, int32(0), e.ReaderWriterCount()) e.IncrementReaderWriterCount() @@ -54,7 +54,7 @@ func TestEntryReaderWriterCount(t *testing.T) { } func TestEntryIndexSuccessPath(t *testing.T) { - e := lookup.NewEntry(lookup.NewEntryOptions{}) + e := storage.NewEntry(storage.NewEntryOptions{}) t0 := newTime(0) require.False(t, e.IndexedForBlockStart(t0)) @@ -69,7 +69,7 @@ func TestEntryIndexSuccessPath(t *testing.T) { } func TestEntryIndexFailPath(t *testing.T) { - e := lookup.NewEntry(lookup.NewEntryOptions{}) + e := storage.NewEntry(storage.NewEntryOptions{}) t0 := newTime(0) require.False(t, e.IndexedForBlockStart(t0)) @@ -85,7 +85,7 @@ func TestEntryIndexFailPath(t *testing.T) { func TestEntryMultipleGoroutinesRaceIndexUpdate(t *testing.T) { defer leaktest.CheckTimeout(t, time.Second)() - e := lookup.NewEntry(lookup.NewEntryOptions{}) + e := storage.NewEntry(storage.NewEntryOptions{}) t0 := newTime(0) require.False(t, e.IndexedForBlockStart(t0)) diff --git a/src/dbnode/storage/series/lookup/entry_whitebox_test.go b/src/dbnode/storage/entry_whitebox_test.go similarity index 99% rename from src/dbnode/storage/series/lookup/entry_whitebox_test.go rename to src/dbnode/storage/entry_whitebox_test.go index 52de0011b7..4dba1b89b9 100644 --- a/src/dbnode/storage/series/lookup/entry_whitebox_test.go +++ b/src/dbnode/storage/entry_whitebox_test.go @@ -18,7 +18,7 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -package lookup +package storage import ( "testing" diff --git a/src/dbnode/storage/index/mutable_segments.go b/src/dbnode/storage/index/mutable_segments.go index c92d64d188..7ee940547d 100644 --- a/src/dbnode/storage/index/mutable_segments.go +++ b/src/dbnode/storage/index/mutable_segments.go @@ -626,8 +626,19 @@ func (m *mutableSegments) backgroundCompactWithTask( return true } + isEmpty, ok := entry.RelookupAndCheckIsEmpty() + if !ok { + // Should not happen since shard will not expire until + // no more block starts are indexed. + // We do not GC this series if shard is missing since + // we open up a race condition where the entry is not + // in the shard yet and we GC it since we can't find it + // due to an asynchronous insert. + return true + } + // Keep if not yet empty (i.e. there is still in-memory data associated with the series). - return !entry.IsEmpty() + return !isEmpty }) } diff --git a/src/dbnode/storage/index/types.go b/src/dbnode/storage/index/types.go index c628fdaa03..94fee0b195 100644 --- a/src/dbnode/storage/index/types.go +++ b/src/dbnode/storage/index/types.go @@ -400,7 +400,7 @@ type OnIndexSeries interface { blockStart xtime.UnixNano, ) bool - IsEmpty() bool + RelookupAndCheckIsEmpty() (bool, bool) DecrementReaderWriterCount() diff --git a/src/dbnode/storage/series/lookup/lookup_mock.go b/src/dbnode/storage/lookup_mock.go similarity index 95% rename from src/dbnode/storage/series/lookup/lookup_mock.go rename to src/dbnode/storage/lookup_mock.go index 77e76f9073..57f9dd3a25 100644 --- a/src/dbnode/storage/series/lookup/lookup_mock.go +++ b/src/dbnode/storage/lookup_mock.go @@ -1,5 +1,5 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/m3db/m3/src/dbnode/storage/series/lookup (interfaces: IndexWriter) +// Source: github.com/m3db/m3/src/dbnode/storage (interfaces: IndexWriter) // Copyright (c) 2021 Uber Technologies, Inc. // @@ -21,8 +21,8 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -// Package lookup is a generated GoMock package. -package lookup +// Package storage is a generated GoMock package. +package storage import ( "reflect" diff --git a/src/dbnode/storage/series_resolver.go b/src/dbnode/storage/series_resolver.go index 8e3b2798c5..a425a2f180 100644 --- a/src/dbnode/storage/series_resolver.go +++ b/src/dbnode/storage/series_resolver.go @@ -25,12 +25,11 @@ import ( "sync" "github.com/m3db/m3/src/dbnode/storage/bootstrap" - "github.com/m3db/m3/src/dbnode/storage/series/lookup" "github.com/m3db/m3/src/x/ident" ) // retrieveWritableSeriesFn represents the function to retrieve series entry. -type retrieveWritableSeriesFn func(id ident.ID) (*lookup.Entry, error) +type retrieveWritableSeriesFn func(id ident.ID) (*Entry, error) type seriesResolver struct { sync.RWMutex @@ -41,7 +40,7 @@ type seriesResolver struct { resolved bool resolvedErr error - entry *lookup.Entry + entry *Entry } // NewSeriesResolver creates new series ref resolver. diff --git a/src/dbnode/storage/series_resolver_test.go b/src/dbnode/storage/series_resolver_test.go index b55e6581f6..d3d46013a1 100644 --- a/src/dbnode/storage/series_resolver_test.go +++ b/src/dbnode/storage/series_resolver_test.go @@ -27,14 +27,14 @@ import ( "github.com/stretchr/testify/require" - "github.com/m3db/m3/src/dbnode/storage/series/lookup" + "github.com/m3db/m3/src/dbnode/storage" "github.com/m3db/m3/src/x/ident" ) func TestResolveError(t *testing.T) { wg := sync.WaitGroup{} id := ident.StringID("foo") - sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*lookup.Entry, error) { + sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*storage.Entry, error) { return nil, fmt.Errorf("unable to resolve series") }) _, err := sut.SeriesRef() @@ -44,7 +44,7 @@ func TestResolveError(t *testing.T) { func TestResolveNilEntry(t *testing.T) { wg := sync.WaitGroup{} id := ident.StringID("foo") - sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*lookup.Entry, error) { + sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*storage.Entry, error) { return nil, nil }) _, err := sut.SeriesRef() @@ -54,53 +54,53 @@ func TestResolveNilEntry(t *testing.T) { func TestResolve(t *testing.T) { wg := sync.WaitGroup{} id := ident.StringID("foo") - sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*lookup.Entry, error) { - return lookup.NewEntry(lookup.NewEntryOptions{ + sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*storage.Entry, error) { + return storage.NewEntry(storage.NewEntryOptions{ Index: 11, }), nil }) seriesRef, err := sut.SeriesRef() require.NoError(t, err) - require.IsType(t, &lookup.Entry{}, seriesRef) - entry := seriesRef.(*lookup.Entry) + require.IsType(t, &storage.Entry{}, seriesRef) + entry := seriesRef.(*storage.Entry) require.Equal(t, uint64(11), entry.Index) } func TestSecondResolveWontWait(t *testing.T) { wg := sync.WaitGroup{} id := ident.StringID("foo") - sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*lookup.Entry, error) { - return lookup.NewEntry(lookup.NewEntryOptions{ + sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*storage.Entry, error) { + return storage.NewEntry(storage.NewEntryOptions{ Index: 11, }), nil }) seriesRef, err := sut.SeriesRef() require.NoError(t, err) - require.IsType(t, &lookup.Entry{}, seriesRef) - entry := seriesRef.(*lookup.Entry) + require.IsType(t, &storage.Entry{}, seriesRef) + entry := seriesRef.(*storage.Entry) require.Equal(t, uint64(11), entry.Index) wg.Add(1) seriesRef2, err := sut.SeriesRef() require.NoError(t, err) - require.IsType(t, &lookup.Entry{}, seriesRef2) - entry2 := seriesRef2.(*lookup.Entry) + require.IsType(t, &storage.Entry{}, seriesRef2) + entry2 := seriesRef2.(*storage.Entry) require.Equal(t, entry, entry2) } func TestReleaseRef(t *testing.T) { wg := sync.WaitGroup{} id := ident.StringID("foo") - sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*lookup.Entry, error) { - entry := lookup.NewEntry(lookup.NewEntryOptions{}) + sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*storage.Entry, error) { + entry := storage.NewEntry(storage.NewEntryOptions{}) entry.IncrementReaderWriterCount() return entry, nil }) seriesRef, err := sut.SeriesRef() require.NoError(t, err) - require.IsType(t, &lookup.Entry{}, seriesRef) + require.IsType(t, &storage.Entry{}, seriesRef) - entry := seriesRef.(*lookup.Entry) + entry := seriesRef.(*storage.Entry) require.Equal(t, int32(1), entry.ReaderWriterCount()) err = sut.ReleaseRef() require.NoError(t, err) @@ -110,7 +110,7 @@ func TestReleaseRef(t *testing.T) { func TestReleaseRefError(t *testing.T) { wg := sync.WaitGroup{} id := ident.StringID("foo") - sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*lookup.Entry, error) { + sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*storage.Entry, error) { return nil, fmt.Errorf("unable to resolve series") }) err := sut.ReleaseRef() @@ -120,8 +120,8 @@ func TestReleaseRefError(t *testing.T) { func TestReleaseRefWithoutSeriesRef(t *testing.T) { wg := sync.WaitGroup{} id := ident.StringID("foo") - sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*lookup.Entry, error) { - entry := lookup.NewEntry(lookup.NewEntryOptions{}) + sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*storage.Entry, error) { + entry := storage.NewEntry(storage.NewEntryOptions{}) entry.IncrementReaderWriterCount() return entry, nil }) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index dda3205ee7..4f247da4a7 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -43,7 +43,6 @@ import ( "github.com/m3db/m3/src/dbnode/storage/index/convert" "github.com/m3db/m3/src/dbnode/storage/repair" "github.com/m3db/m3/src/dbnode/storage/series" - "github.com/m3db/m3/src/dbnode/storage/series/lookup" "github.com/m3db/m3/src/dbnode/tracepoint" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/dbnode/ts/writes" @@ -223,9 +222,9 @@ func newDatabaseShardMetrics(shardID uint32, scope tally.Scope) dbShardMetrics { } } -type dbShardEntryWorkFn func(entry *lookup.Entry) bool +type dbShardEntryWorkFn func(entry *Entry) bool -type dbShardEntryBatchWorkFn func(entries []*lookup.Entry) bool +type dbShardEntryBatchWorkFn func(entries []*Entry) bool type shardListElement *list.Element @@ -525,7 +524,7 @@ func (s *dbShard) OnEvictedFromWiredList(id ident.ID, blockStart xtime.UnixNano) } func (s *dbShard) forEachShardEntry(entryFn dbShardEntryWorkFn) error { - return s.forEachShardEntryBatch(func(currEntries []*lookup.Entry) bool { + return s.forEachShardEntryBatch(func(currEntries []*Entry) bool { for _, entry := range currEntries { if continueForEach := entryFn(entry); !continueForEach { return false @@ -554,11 +553,11 @@ func (s *dbShard) forEachShardEntryBatch(entriesBatchFn dbShardEntryBatchWorkFn) if e == nil { return } - e.Value.(*lookup.Entry).DecrementReaderWriterCount() + e.Value.(*Entry).DecrementReaderWriterCount() } var ( - currEntries = make([]*lookup.Entry, 0, batchSize) + currEntries = make([]*Entry, 0, batchSize) first = true nextElem *list.Element ) @@ -578,7 +577,7 @@ func (s *dbShard) forEachShardEntryBatch(entriesBatchFn dbShardEntryBatchWorkFn) elem := nextElem for ticked := 0; ticked < batchSize && elem != nil; ticked++ { nextElem = elem.Next() - entry := elem.Value.(*lookup.Entry) + entry := elem.Value.(*Entry) entry.IncrementReaderWriterCount() currEntries = append(currEntries, entry) elem = nextElem @@ -587,7 +586,7 @@ func (s *dbShard) forEachShardEntryBatch(entriesBatchFn dbShardEntryBatchWorkFn) // NB(prateek): inc a reference to the next element while we have a lock, // to guarantee the element pointer cannot be changed from under us. if nextElem != nil { - nextElem.Value.(*lookup.Entry).IncrementReaderWriterCount() + nextElem.Value.(*Entry).IncrementReaderWriterCount() } s.RUnlock() @@ -707,7 +706,7 @@ func (s *dbShard) tickAndExpire( terminatedTickingDueToClosing bool i int slept time.Duration - expired []*lookup.Entry + expired []*Entry ) s.RLock() tickSleepBatch := s.currRuntimeOptions.tickSleepSeriesBatchSize @@ -718,7 +717,7 @@ func (s *dbShard) tickAndExpire( // future read lock attempts. blockStates := s.blockStatesSnapshotWithRLock() s.RUnlock() - s.forEachShardEntryBatch(func(currEntries []*lookup.Entry) bool { + s.forEachShardEntryBatch(func(currEntries []*Entry) bool { // re-using `expired` to amortize allocs, still need to reset it // to be safe for re-use. for i := range expired { @@ -802,7 +801,7 @@ func (s *dbShard) tickAndExpire( // Currently, this function is only called by the lambda inside `tickAndExpire`'s `forEachShardEntryBatch` // call. This satisfies the contract of all entries it operating upon being guaranteed to have a // readerWriterEntryCount of at least 1, by virtue of the implementation of `forEachShardEntryBatch`. -func (s *dbShard) purgeExpiredSeries(expiredEntries []*lookup.Entry) { +func (s *dbShard) purgeExpiredSeries(expiredEntries []*Entry) { // Remove all expired series from lookup and list. s.Lock() for _, entry := range expiredEntries { @@ -887,7 +886,7 @@ func (s *dbShard) writeAndIndex( shouldReverseIndex bool, ) (SeriesWrite, error) { // Prepare write - entry, opts, err := s.tryRetrieveWritableSeries(id) + entry, opts, err := s.TryRetrieveWritableSeries(id) if err != nil { return SeriesWrite{}, err } @@ -895,7 +894,7 @@ func (s *dbShard) writeAndIndex( writable := entry != nil // If no entry and we are not writing new series asynchronously. - if !writable && !opts.writeNewSeriesAsync { + if !writable && !opts.WriteNewSeriesAsync { // Avoid double lookup by enqueueing insert immediately. result, err := s.insertSeriesAsyncBatched(id, tagResolver, dbShardInsertAsyncOptions{ hasPendingIndexing: shouldReverseIndex, @@ -947,7 +946,7 @@ func (s *dbShard) writeAndIndex( commitLogSeriesUniqueIndex = entry.Index if err == nil && shouldReverseIndex { if entry.NeedsIndexUpdate(s.reverseIndex.BlockStartForWriteTime(timestamp)) { - if !opts.writeNewSeriesAsync { + if !opts.WriteNewSeriesAsync { return SeriesWrite{}, fmt.Errorf("to index async need write new series to be enabled") } needsIndex = true @@ -986,7 +985,7 @@ func (s *dbShard) writeAndIndex( } if shouldReverseIndex { - if !opts.writeNewSeriesAsync { + if !opts.WriteNewSeriesAsync { return SeriesWrite{}, fmt.Errorf("to index async need write new series to be enabled") } needsIndex = true @@ -1112,7 +1111,7 @@ func (s *dbShard) FetchWideEntry( } // lookupEntryWithLock returns the entry for a given id while holding a read lock or a write lock. -func (s *dbShard) lookupEntryWithLock(id ident.ID) (*lookup.Entry, *list.Element, error) { +func (s *dbShard) lookupEntryWithLock(id ident.ID) (*Entry, *list.Element, error) { if s.state != dbShardStateOpen { // NB(r): Return an invalid params error here so any upstream // callers will not retry this operation @@ -1122,10 +1121,10 @@ func (s *dbShard) lookupEntryWithLock(id ident.ID) (*lookup.Entry, *list.Element if !exists { return nil, nil, errShardEntryNotFound } - return elem.Value.(*lookup.Entry), elem, nil + return elem.Value.(*Entry), elem, nil } -func (s *dbShard) writableSeries(id ident.ID, tagResolver convert.TagMetadataResolver) (*lookup.Entry, error) { +func (s *dbShard) writableSeries(id ident.ID, tagResolver convert.TagMetadataResolver) (*Entry, error) { for { entry, err := s.retrieveWritableSeries(id) if entry != nil { @@ -1146,18 +1145,21 @@ func (s *dbShard) writableSeries(id ident.ID, tagResolver convert.TagMetadataRes } } -type writableSeriesOptions struct { - writeNewSeriesAsync bool +// WritableSeriesOptions defines writable series options. +type WritableSeriesOptions struct { + // WriteNewSeriesAsync specifies if the series should be async written. + WriteNewSeriesAsync bool } -func (s *dbShard) tryRetrieveWritableSeries(id ident.ID) ( - *lookup.Entry, - writableSeriesOptions, +// TryRetrieveWritableSeries attempts to retrieve a writable series. +func (s *dbShard) TryRetrieveWritableSeries(id ident.ID) ( + *Entry, + WritableSeriesOptions, error, ) { s.RLock() - opts := writableSeriesOptions{ - writeNewSeriesAsync: s.currRuntimeOptions.writeNewSeriesAsync, + opts := WritableSeriesOptions{ + WriteNewSeriesAsync: s.currRuntimeOptions.writeNewSeriesAsync, } if entry, _, err := s.lookupEntryWithLock(id); err == nil { entry.IncrementReaderWriterCount() @@ -1171,15 +1173,15 @@ func (s *dbShard) tryRetrieveWritableSeries(id ident.ID) ( return nil, opts, nil } -func (s *dbShard) retrieveWritableSeries(id ident.ID) (*lookup.Entry, error) { - entry, _, err := s.tryRetrieveWritableSeries(id) +func (s *dbShard) retrieveWritableSeries(id ident.ID) (*Entry, error) { + entry, _, err := s.TryRetrieveWritableSeries(id) return entry, err } func (s *dbShard) newShardEntry( id ident.ID, tagResolver convert.TagMetadataResolver, -) (*lookup.Entry, error) { +) (*Entry, error) { // NB(r): As documented in storage/series.DatabaseSeries the series IDs // and metadata are garbage collected, hence we cast the ID to a BytesID // that can't be finalized. @@ -1215,7 +1217,8 @@ func (s *dbShard) newShardEntry( OnEvictedFromWiredList: s, Options: s.seriesOpts, }) - return lookup.NewEntry(lookup.NewEntryOptions{ + return NewEntry(NewEntryOptions{ + Shard: s, Series: newSeries, Index: uniqueIndex, IndexWriter: s.reverseIndex, @@ -1229,11 +1232,11 @@ type insertAsyncResult struct { // entry is not guaranteed to be the final entry // inserted into the shard map in case there is already // an existing entry waiting in the insert queue - entry *lookup.Entry + entry *Entry } func (s *dbShard) pendingIndexInsert( - entry *lookup.Entry, + entry *Entry, timestamp xtime.UnixNano, ) writes.PendingIndexInsert { // inc a ref on the entry to ensure it's valid until the queue acts upon it. @@ -1249,7 +1252,7 @@ func (s *dbShard) pendingIndexInsert( } func (s *dbShard) insertSeriesForIndexingAsyncBatched( - entry *lookup.Entry, + entry *Entry, timestamp xtime.UnixNano, async bool, ) error { @@ -1333,7 +1336,7 @@ func (s *dbShard) insertSeriesSync( id ident.ID, tagResolver convert.TagMetadataResolver, opts insertSyncOptions, -) (*lookup.Entry, error) { +) (*Entry, error) { // NB(r): Create new shard entry outside of write lock to reduce // time using write lock. newEntry, err := s.newShardEntry(id, tagResolver) @@ -1398,7 +1401,7 @@ func (s *dbShard) insertSeriesSync( return newEntry, nil } -func (s *dbShard) insertNewShardEntryWithLock(entry *lookup.Entry) { +func (s *dbShard) insertNewShardEntryWithLock(entry *Entry) { // Set the lookup value, we use the copied ID and since it is GC'd // we explicitly set it with options to not copy the key and not to // finalize it. @@ -1639,7 +1642,7 @@ func (s *dbShard) fetchActiveBlocksMetadata( ) var loopErr error - s.forEachShardEntry(func(entry *lookup.Entry) bool { + s.forEachShardEntry(func(entry *Entry) bool { // Break out of the iteration loop once we've accumulated enough entries. if int64(len(res.Results())) >= limit { next := int64(entry.Index) @@ -2005,7 +2008,7 @@ func (s *dbShard) Bootstrap( } // Move any bootstrap buffers into position for reading. - s.forEachShardEntry(func(entry *lookup.Entry) bool { + s.forEachShardEntry(func(entry *Entry) bool { if err := entry.Series.Bootstrap(nsCtx); err != nil { multiErr = multiErr.Add(err) } @@ -2081,7 +2084,7 @@ func (s *dbShard) loadBlock( ) // First lookup if series already exists. - entry, shardOpts, err := s.tryRetrieveWritableSeries(id) + entry, shardOpts, err := s.TryRetrieveWritableSeries(id) if err != nil && err != errShardEntryNotFound { return result, err } @@ -2137,7 +2140,7 @@ func (s *dbShard) loadBlock( if s.reverseIndex != nil && entry.NeedsIndexUpdate(s.reverseIndex.BlockStartForWriteTime(timestamp)) { err = s.insertSeriesForIndexingAsyncBatched(entry, timestamp, - shardOpts.writeNewSeriesAsync) + shardOpts.WriteNewSeriesAsync) if err != nil { return result, err } @@ -2202,7 +2205,7 @@ func (s *dbShard) WarmFlush( flushCtx := s.contextPool.Get() // From pool so finalizers are from pool. flushResult := dbShardFlushResult{} - s.forEachShardEntry(func(entry *lookup.Entry) bool { + s.forEachShardEntry(func(entry *Entry) bool { curr := entry.Series // Use a temporary context here so the stream readers can be returned to // the pool after we finish fetching flushing the series. @@ -2268,7 +2271,7 @@ func (s *dbShard) ColdFlush( ) // 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 { + s.forEachShardEntry(func(entry *Entry) bool { curr := entry.Series seriesMetadata := curr.Metadata() blockStarts := curr.ColdFlushBlockStarts(blockStatesSnapshot) @@ -2378,7 +2381,7 @@ func (s *dbShard) Snapshot( var needsSnapshot bool checkNeedsSnapshotTimer := s.metrics.snapshotCheckNeedsSnapshotLatency.Start() - s.forEachShardEntry(func(entry *lookup.Entry) bool { + s.forEachShardEntry(func(entry *Entry) bool { if !entry.Series.IsBufferEmptyAtBlockStart(blockStart) { needsSnapshot = true return false @@ -2418,7 +2421,7 @@ func (s *dbShard) Snapshot( stats series.SnapshotResultStats multiErr xerrors.MultiError ) - s.forEachShardEntry(func(entry *lookup.Entry) bool { + s.forEachShardEntry(func(entry *Entry) bool { series := entry.Series // Use a temporary context here so the stream readers can be returned to // pool after we finish fetching flushing the series diff --git a/src/dbnode/storage/shard_foreachentry_prop_test.go b/src/dbnode/storage/shard_foreachentry_prop_test.go index b286eebb7e..ad9a965c73 100644 --- a/src/dbnode/storage/shard_foreachentry_prop_test.go +++ b/src/dbnode/storage/shard_foreachentry_prop_test.go @@ -31,7 +31,7 @@ import ( "time" "github.com/m3db/m3/src/dbnode/namespace" - "github.com/m3db/m3/src/dbnode/storage/series/lookup" + "github.com/m3db/m3/src/dbnode/storage" "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/ident" xtime "github.com/m3db/m3/src/x/time" @@ -205,7 +205,7 @@ func shardEntriesAreEqual(shard *dbShard, expectedEntries []shardEntryState) err return fmt.Errorf("expected to have %d idx, but did not see anything", idx) } nextElem := elem.Next() - entry := elem.Value.(*lookup.Entry) + entry := elem.Value.(*storage.Entry) if !entry.Series.ID().Equal(expectedEntry.id) { return fmt.Errorf("expected id: %s at %d, observed: %s", expectedEntry.id.String(), idx, entry.Series.ID().String()) @@ -253,7 +253,7 @@ func genBatchWorkFn() gopter.Gen { return gen.UInt8(). Map(func(n uint8) dbShardEntryBatchWorkFn { i := uint8(0) - return func([]*lookup.Entry) bool { + return func([]*storage.Entry) bool { i++ return i < n } diff --git a/src/dbnode/storage/shard_index_test.go b/src/dbnode/storage/shard_index_test.go index 4341f782ab..e7b9ee586d 100644 --- a/src/dbnode/storage/shard_index_test.go +++ b/src/dbnode/storage/shard_index_test.go @@ -140,7 +140,7 @@ func TestShardAsyncInsertMarkIndexedForBlockStart(t *testing.T) { start := time.Now() for time.Since(start) < 10*time.Second { - entry, _, err := shard.tryRetrieveWritableSeries(ident.StringID("foo")) + entry, _, err := shard.TryRetrieveWritableSeries(ident.StringID("foo")) require.NoError(t, err) if entry == nil { time.Sleep(10 * time.Millisecond) @@ -190,7 +190,7 @@ func TestShardAsyncIndexIfExpired(t *testing.T) { // make sure next block not marked as indexed start := time.Now() for time.Since(start) < 10*time.Second { - entry, _, err := shard.tryRetrieveWritableSeries(ident.StringID("foo")) + entry, _, err := shard.TryRetrieveWritableSeries(ident.StringID("foo")) require.NoError(t, err) if entry == nil { time.Sleep(10 * time.Millisecond) diff --git a/src/dbnode/storage/shard_insert_queue.go b/src/dbnode/storage/shard_insert_queue.go index 927f0868e8..6f93c05ade 100644 --- a/src/dbnode/storage/shard_insert_queue.go +++ b/src/dbnode/storage/shard_insert_queue.go @@ -29,7 +29,6 @@ import ( "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/storage/series" - "github.com/m3db/m3/src/dbnode/storage/series/lookup" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/x/checked" "github.com/m3db/m3/src/x/clock" @@ -335,7 +334,7 @@ type dbShardInsertsByCPUCore struct { } type dbShardInsert struct { - entry *lookup.Entry + entry *Entry opts dbShardInsertAsyncOptions } diff --git a/src/dbnode/storage/shard_insert_queue_test.go b/src/dbnode/storage/shard_insert_queue_test.go index 1190a80bb3..5aedbd9c70 100644 --- a/src/dbnode/storage/shard_insert_queue_test.go +++ b/src/dbnode/storage/shard_insert_queue_test.go @@ -26,7 +26,7 @@ import ( "testing" "time" - "github.com/m3db/m3/src/dbnode/storage/series/lookup" + "github.com/m3db/m3/src/dbnode/storage" "github.com/fortytw2/leaktest" "github.com/stretchr/testify/assert" @@ -90,16 +90,16 @@ func TestShardInsertQueueBatchBackoff(t *testing.T) { }() // first insert - _, err := q.Insert(dbShardInsert{entry: &lookup.Entry{Index: 0}}) + _, err := q.Insert(dbShardInsert{entry: &storage.Entry{Index: 0}}) require.NoError(t, err) // wait for first insert batch to complete insertWgs[0].Wait() // now next batch will need to wait as we haven't progressed time - _, err = q.Insert(dbShardInsert{entry: &lookup.Entry{Index: 1}}) + _, err = q.Insert(dbShardInsert{entry: &storage.Entry{Index: 1}}) require.NoError(t, err) - _, err = q.Insert(dbShardInsert{entry: &lookup.Entry{Index: 2}}) + _, err = q.Insert(dbShardInsert{entry: &storage.Entry{Index: 2}}) require.NoError(t, err) // allow first insert to finish @@ -112,7 +112,7 @@ func TestShardInsertQueueBatchBackoff(t *testing.T) { assert.Equal(t, 1, numSleeps) // insert third batch, will also need to wait - _, err = q.Insert(dbShardInsert{entry: &lookup.Entry{Index: 3}}) + _, err = q.Insert(dbShardInsert{entry: &storage.Entry{Index: 3}}) require.NoError(t, err) // allow second batch to finish diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index deb21e5d09..2d4274b522 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -38,11 +38,11 @@ import ( "github.com/m3db/m3/src/dbnode/persist/fs" "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/runtime" + "github.com/m3db/m3/src/dbnode/storage" "github.com/m3db/m3/src/dbnode/storage/block" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" "github.com/m3db/m3/src/dbnode/storage/index/convert" "github.com/m3db/m3/src/dbnode/storage/series" - "github.com/m3db/m3/src/dbnode/storage/series/lookup" "github.com/m3db/m3/src/dbnode/ts" xmetrics "github.com/m3db/m3/src/dbnode/x/metrics" "github.com/m3db/m3/src/dbnode/x/xio" @@ -100,7 +100,7 @@ func addMockSeries(ctrl *gomock.Controller, shard *dbShard, id ident.ID, tags id series.EXPECT().ID().Return(id).AnyTimes() series.EXPECT().IsEmpty().Return(false).AnyTimes() shard.Lock() - shard.insertNewShardEntryWithLock(lookup.NewEntry(lookup.NewEntryOptions{ + shard.insertNewShardEntryWithLock(storage.NewEntry(storage.NewEntryOptions{ Series: series, Index: index, })) @@ -215,7 +215,7 @@ func TestShardBootstrapWithFlushVersion(t *testing.T) { // Load the mock into the shard as an expected series so that we can assert // on the call to its Bootstrap() method below. - entry := lookup.NewEntry(lookup.NewEntryOptions{ + entry := storage.NewEntry(storage.NewEntryOptions{ Series: mockSeries, }) s.Lock() @@ -464,7 +464,7 @@ func TestShardFlushSeriesFlushError(t *testing.T) { flushed[i] = struct{}{} }). Return(series.FlushOutcomeErr, expectedErr) - s.list.PushBack(lookup.NewEntry(lookup.NewEntryOptions{ + s.list.PushBack(storage.NewEntry(storage.NewEntryOptions{ Series: curr, })) } @@ -551,7 +551,7 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { flushed[i] = struct{}{} }). Return(series.FlushOutcomeFlushedToDisk, nil) - s.list.PushBack(lookup.NewEntry(lookup.NewEntryOptions{ + s.list.PushBack(storage.NewEntry(storage.NewEntryOptions{ Series: curr, })) } @@ -662,7 +662,7 @@ func TestShardColdFlush(t *testing.T) { curr.EXPECT().Metadata().Return(doc.Metadata{ID: ds.id.Bytes()}).AnyTimes() curr.EXPECT().ColdFlushBlockStarts(gomock.Any()). Return(optimizedTimesFromTimes(ds.dirtyTimes)) - shard.list.PushBack(lookup.NewEntry(lookup.NewEntryOptions{ + shard.list.PushBack(storage.NewEntry(storage.NewEntryOptions{ Series: curr, })) } @@ -866,7 +866,7 @@ func TestShardSnapshotSeriesSnapshotSuccess(t *testing.T) { snapshotted[i] = struct{}{} }). Return(series.SnapshotResult{}, nil) - s.list.PushBack(lookup.NewEntry(lookup.NewEntryOptions{ + s.list.PushBack(storage.NewEntry(storage.NewEntryOptions{ Series: entry, })) } @@ -886,7 +886,7 @@ func addMockTestSeries(ctrl *gomock.Controller, shard *dbShard, id ident.ID) *se series := series.NewMockDatabaseSeries(ctrl) series.EXPECT().ID().AnyTimes().Return(id) shard.Lock() - shard.insertNewShardEntryWithLock(lookup.NewEntry(lookup.NewEntryOptions{ + shard.insertNewShardEntryWithLock(storage.NewEntry(storage.NewEntryOptions{ Series: series, })) shard.Unlock() @@ -904,7 +904,7 @@ func addTestSeriesWithCount(shard *dbShard, id ident.ID, count int32) series.Dat Options: shard.seriesOpts, }) shard.Lock() - entry := lookup.NewEntry(lookup.NewEntryOptions{ + entry := storage.NewEntry(storage.NewEntryOptions{ Series: seriesEntry, }) for i := int32(0); i < count; i++ { @@ -1244,7 +1244,7 @@ func TestShardTickRace(t *testing.T) { wg.Wait() shard.RLock() - shardlen := shard.lookup.Len() + shardlen := shard.storage.Len() shard.RUnlock() require.Equal(t, 0, shardlen) @@ -1265,7 +1265,7 @@ func TestShardTickCleanupSmallBatchSize(t *testing.T) { addTestSeries(shard, ident.StringID("foo")) _, err := shard.Tick(context.NewNoOpCanncellable(), xtime.Now(), namespace.Context{}) require.NoError(t, err) - require.Equal(t, 0, shard.lookup.Len()) + require.Equal(t, 0, shard.storage.Len()) } // This tests ensures the shard returns an error if two ticks are triggered concurrently. @@ -1409,7 +1409,7 @@ func TestPurgeExpiredSeriesEmptySeries(t *testing.T) { require.NoError(t, err) shard.RLock() - require.Equal(t, 0, shard.lookup.Len()) + require.Equal(t, 0, shard.storage.Len()) shard.RUnlock() } @@ -1466,7 +1466,7 @@ func TestPurgeExpiredSeriesWriteAfterTicking(t *testing.T) { require.NoError(t, err) require.Equal(t, 0, r.activeSeries) require.Equal(t, 1, r.expiredSeries) - require.Equal(t, 1, shard.lookup.Len()) + require.Equal(t, 1, shard.storage.Len()) } // This tests the scenario where tickForEachSeries finishes, and before purgeExpiredSeries @@ -1476,7 +1476,7 @@ func TestPurgeExpiredSeriesWriteAfterPurging(t *testing.T) { ctrl := xtest.NewController(t) defer ctrl.Finish() - var entry *lookup.Entry + var entry *storage.Entry opts := DefaultTestOptions() shard := testDatabaseShard(t, opts) @@ -1494,10 +1494,10 @@ func TestPurgeExpiredSeriesWriteAfterPurging(t *testing.T) { require.NoError(t, err) require.Equal(t, 0, r.activeSeries) require.Equal(t, 1, r.expiredSeries) - require.Equal(t, 1, shard.lookup.Len()) + require.Equal(t, 1, shard.storage.Len()) entry.DecrementReaderWriterCount() - require.Equal(t, 1, shard.lookup.Len()) + require.Equal(t, 1, shard.storage.Len()) } func TestForEachShardEntry(t *testing.T) { @@ -1509,7 +1509,7 @@ func TestForEachShardEntry(t *testing.T) { } count := 0 - entryFn := func(entry *lookup.Entry) bool { + entryFn := func(entry *storage.Entry) bool { if entry.Series.ID().String() == "foo.8" { return false } @@ -1529,7 +1529,7 @@ func TestForEachShardEntry(t *testing.T) { // Ensure that reader writer count gets reset shard.RLock() for elem := shard.list.Front(); elem != nil; elem = elem.Next() { - entry := elem.Value.(*lookup.Entry) + entry := elem.Value.(*storage.Entry) assert.Equal(t, int32(0), entry.ReaderWriterCount()) } shard.RUnlock() @@ -1887,7 +1887,7 @@ func TestShardNewEntryDoesNotAlterIDOrTags(t *testing.T) { shard.insertNewShardEntryWithLock(entry) shard.Unlock() - entry, _, err = shard.tryRetrieveWritableSeries(seriesID) + entry, _, err = shard.TryRetrieveWritableSeries(seriesID) require.NoError(t, err) entryIDBytes := entry.Series.ID().Bytes() @@ -2023,7 +2023,7 @@ func TestSeriesRefResolver(t *testing.T) { // should return already inserted entry as series. resolverEntry, err := shard.SeriesRefResolver(seriesID, iter) require.NoError(t, err) - require.IsType(t, &lookup.Entry{}, resolverEntry) + require.IsType(t, &storage.Entry{}, resolverEntry) refEntry, err := resolverEntry.SeriesRef() require.NoError(t, err) require.Equal(t, seriesRef, refEntry) diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 91ddbc8e34..c8412c09fe 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -502,6 +502,9 @@ type Shard interface { // OpenStreamingDataReader creates and opens a streaming fs.DataFileSetReader // on the latest volume of the given block. OpenStreamingReader(blockStart xtime.UnixNano) (fs.DataFileSetReader, error) + + // TryRetrieveWritableSeries attempts to retrieve a writable series. + TryRetrieveWritableSeries(id ident.ID) (*Entry, WritableSeriesOptions, error) } type databaseShard interface { From db170c55e929687f2c65056b08ff6d84038e39bc Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Wed, 18 Aug 2021 10:28:40 -0400 Subject: [PATCH 23/36] Mock --- src/dbnode/generated/mocks/generate.go | 2 +- src/dbnode/storage/index/index_mock.go | 29 +++++++++--------- src/dbnode/storage/index/mutable_segments.go | 3 ++ src/dbnode/storage/storage_mock.go | 32 ++++++++++++++++++++ 4 files changed, 51 insertions(+), 15 deletions(-) diff --git a/src/dbnode/generated/mocks/generate.go b/src/dbnode/generated/mocks/generate.go index 0c8ad0f3f5..83b5e48972 100644 --- a/src/dbnode/generated/mocks/generate.go +++ b/src/dbnode/generated/mocks/generate.go @@ -24,7 +24,7 @@ //go:generate sh -c "mockgen -package=xio $PACKAGE/src/dbnode/x/xio SegmentReader,SegmentReaderPool | genclean -pkg $PACKAGE/src/dbnode/x/xio -out $GOPATH/src/$PACKAGE/src/dbnode/x/xio/io_mock.go" //go:generate sh -c "mockgen -package=digest -destination=$GOPATH/src/$PACKAGE/src/dbnode/digest/digest_mock.go $PACKAGE/src/dbnode/digest ReaderWithDigest" //go:generate sh -c "mockgen -package=series $PACKAGE/src/dbnode/storage/series DatabaseSeries,QueryableBlockRetriever | genclean -pkg $PACKAGE/src/dbnode/storage/series -out $GOPATH/src/$PACKAGE/src/dbnode/storage/series/series_mock.go" -//go:generate sh -c "mockgen -package=lookup $PACKAGE/src/dbnode/storage/series/lookup IndexWriter | genclean -pkg $PACKAGE/src/dbnode/storage/series/lookup -out $GOPATH/src/$PACKAGE/src/dbnode/storage/series/lookup/lookup_mock.go" +//go:generate sh -c "mockgen -package=storage $PACKAGE/src/dbnode/storage IndexWriter | genclean -pkg $PACKAGE/src/dbnode/storage -out $GOPATH/src/$PACKAGE/src/dbnode/storage/lookup_mock.go" // mockgen rules for generating mocks for unexported interfaces (file mode) //go:generate sh -c "mockgen -package=encoding -destination=$GOPATH/src/$PACKAGE/src/dbnode/encoding/encoding_mock.go -source=$GOPATH/src/$PACKAGE/src/dbnode/encoding/types.go" diff --git a/src/dbnode/storage/index/index_mock.go b/src/dbnode/storage/index/index_mock.go index cca1c1cd86..24447d00d2 100644 --- a/src/dbnode/storage/index/index_mock.go +++ b/src/dbnode/storage/index/index_mock.go @@ -853,20 +853,6 @@ func (mr *MockOnIndexSeriesMockRecorder) IndexedForBlockStart(indexBlockStart in return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexedForBlockStart", reflect.TypeOf((*MockOnIndexSeries)(nil).IndexedForBlockStart), indexBlockStart) } -// IsEmpty mocks base method. -func (m *MockOnIndexSeries) IsEmpty() bool { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "IsEmpty") - ret0, _ := ret[0].(bool) - return ret0 -} - -// IsEmpty indicates an expected call of IsEmpty. -func (mr *MockOnIndexSeriesMockRecorder) IsEmpty() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IsEmpty", reflect.TypeOf((*MockOnIndexSeries)(nil).IsEmpty)) -} - // NeedsIndexUpdate mocks base method. func (m *MockOnIndexSeries) NeedsIndexUpdate(indexBlockStartForWrite time0.UnixNano) bool { m.ctrl.T.Helper() @@ -917,6 +903,21 @@ func (mr *MockOnIndexSeriesMockRecorder) OnIndexSuccess(blockStart interface{}) return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnIndexSuccess", reflect.TypeOf((*MockOnIndexSeries)(nil).OnIndexSuccess), blockStart) } +// RelookupAndCheckIsEmpty mocks base method. +func (m *MockOnIndexSeries) RelookupAndCheckIsEmpty() (bool, bool) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RelookupAndCheckIsEmpty") + ret0, _ := ret[0].(bool) + ret1, _ := ret[1].(bool) + return ret0, ret1 +} + +// RelookupAndCheckIsEmpty indicates an expected call of RelookupAndCheckIsEmpty. +func (mr *MockOnIndexSeriesMockRecorder) RelookupAndCheckIsEmpty() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RelookupAndCheckIsEmpty", reflect.TypeOf((*MockOnIndexSeries)(nil).RelookupAndCheckIsEmpty)) +} + // MockBlock is a mock of Block interface. type MockBlock struct { ctrl *gomock.Controller diff --git a/src/dbnode/storage/index/mutable_segments.go b/src/dbnode/storage/index/mutable_segments.go index 7ee940547d..18fdfdb0d3 100644 --- a/src/dbnode/storage/index/mutable_segments.go +++ b/src/dbnode/storage/index/mutable_segments.go @@ -634,6 +634,9 @@ func (m *mutableSegments) backgroundCompactWithTask( // we open up a race condition where the entry is not // in the shard yet and we GC it since we can't find it // due to an asynchronous insert. + instrument.EmitAndLogInvariantViolation(m.iopts, func(l *zap.Logger) { + l.Error("unexpected checking series entry does not exist") + }) return true } diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index 49c222ad97..0b110b25c4 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -1951,6 +1951,22 @@ func (mr *MockShardMockRecorder) OpenStreamingReader(blockStart interface{}) *go return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OpenStreamingReader", reflect.TypeOf((*MockShard)(nil).OpenStreamingReader), blockStart) } +// TryRetrieveWritableSeries mocks base method. +func (m *MockShard) TryRetrieveWritableSeries(id ident.ID) (*Entry, WritableSeriesOptions, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "TryRetrieveWritableSeries", id) + ret0, _ := ret[0].(*Entry) + ret1, _ := ret[1].(WritableSeriesOptions) + ret2, _ := ret[2].(error) + return ret0, ret1, ret2 +} + +// TryRetrieveWritableSeries indicates an expected call of TryRetrieveWritableSeries. +func (mr *MockShardMockRecorder) TryRetrieveWritableSeries(id interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TryRetrieveWritableSeries", reflect.TypeOf((*MockShard)(nil).TryRetrieveWritableSeries), id) +} + // MockdatabaseShard is a mock of databaseShard interface. type MockdatabaseShard struct { ctrl *gomock.Controller @@ -2365,6 +2381,22 @@ func (mr *MockdatabaseShardMockRecorder) Tick(c, startTime, nsCtx interface{}) * return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Tick", reflect.TypeOf((*MockdatabaseShard)(nil).Tick), c, startTime, nsCtx) } +// TryRetrieveWritableSeries mocks base method. +func (m *MockdatabaseShard) TryRetrieveWritableSeries(id ident.ID) (*Entry, WritableSeriesOptions, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "TryRetrieveWritableSeries", id) + ret0, _ := ret[0].(*Entry) + ret1, _ := ret[1].(WritableSeriesOptions) + ret2, _ := ret[2].(error) + return ret0, ret1, ret2 +} + +// TryRetrieveWritableSeries indicates an expected call of TryRetrieveWritableSeries. +func (mr *MockdatabaseShardMockRecorder) TryRetrieveWritableSeries(id interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TryRetrieveWritableSeries", reflect.TypeOf((*MockdatabaseShard)(nil).TryRetrieveWritableSeries), id) +} + // UpdateFlushStates mocks base method. func (m *MockdatabaseShard) UpdateFlushStates() { m.ctrl.T.Helper() From db0b9c1855326e8a7aca2df1f1b61b3c1420aed3 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Wed, 18 Aug 2021 10:46:14 -0400 Subject: [PATCH 24/36] Fix OnIndexSeries ref type --- src/dbnode/storage/entry.go | 5 ++- src/dbnode/storage/index/block.go | 28 ++++++------ src/dbnode/storage/index/mutable_segments.go | 16 ++----- src/dbnode/storage/index/types.go | 44 +------------------ src/m3ninx/doc/document.go | 6 +-- src/m3ninx/doc/types.go | 46 ++++++++++++++++++++ 6 files changed, 70 insertions(+), 75 deletions(-) diff --git a/src/dbnode/storage/entry.go b/src/dbnode/storage/entry.go index b8e95dba56..2042bf79ff 100644 --- a/src/dbnode/storage/entry.go +++ b/src/dbnode/storage/entry.go @@ -30,6 +30,7 @@ import ( "github.com/m3db/m3/src/dbnode/storage/index" "github.com/m3db/m3/src/dbnode/storage/series" "github.com/m3db/m3/src/dbnode/ts/writes" + "github.com/m3db/m3/src/m3ninx/doc" "github.com/m3db/m3/src/x/clock" "github.com/m3db/m3/src/x/context" xtime "github.com/m3db/m3/src/x/time" @@ -68,8 +69,8 @@ type Entry struct { pendingIndexBatchSizeOne []writes.PendingIndexInsert } -// ensure Entry satisfies the `index.OnIndexSeries` interface. -var _ index.OnIndexSeries = &Entry{} +// ensure Entry satisfies the `doc.OnIndexSeries` interface. +var _ doc.OnIndexSeries = &Entry{} // ensure Entry satisfies the `bootstrap.SeriesRef` interface. var _ bootstrap.SeriesRef = &Entry{} diff --git a/src/dbnode/storage/index/block.go b/src/dbnode/storage/index/block.go index ee8894b1ad..c5ff50a472 100644 --- a/src/dbnode/storage/index/block.go +++ b/src/dbnode/storage/index/block.go @@ -523,23 +523,21 @@ func (b *block) queryWithSpan( // Ensure that the block contains any of the relevant time segments for the query range. doc := iter.Current() - if md, ok := doc.Metadata(); ok { - if entry, ok := md.Ref.(OnIndexSeries); ok { - var ( - inBlock bool - currentBlock = opts.StartInclusive.Truncate(b.blockSize) - ) - for !inBlock { - inBlock = entry.IndexedForBlockStart(currentBlock) - currentBlock = currentBlock.Add(b.blockSize) - if !currentBlock.Before(opts.EndExclusive) { - break - } + if md, ok := doc.Metadata(); ok && md.OnIndexSeries != nil { + var ( + inBlock bool + currentBlock = opts.StartInclusive.Truncate(b.blockSize) + ) + for !inBlock { + inBlock = md.OnIndexSeries.IndexedForBlockStart(currentBlock) + currentBlock = currentBlock.Add(b.blockSize) + if !currentBlock.Before(opts.EndExclusive) { + break } + } - if !inBlock { - continue - } + if !inBlock { + continue } } diff --git a/src/dbnode/storage/index/mutable_segments.go b/src/dbnode/storage/index/mutable_segments.go index 18fdfdb0d3..c6868e4c57 100644 --- a/src/dbnode/storage/index/mutable_segments.go +++ b/src/dbnode/storage/index/mutable_segments.go @@ -196,7 +196,7 @@ func (m *mutableSegments) WriteBatch(inserts *WriteBatch) (MutableSegmentsStats, // Set the doc ref for later recall. for i := range entries { - docs[i].Ref = entries[i].OnIndexSeries + docs[i].OnIndexSeries = entries[i].OnIndexSeries } segmentBuilder.Reset() @@ -611,22 +611,14 @@ func (m *mutableSegments) backgroundCompactWithTask( documentsFilter = segment.DocumentsFilterFn(func(d doc.Metadata) bool { // Filter out any documents that only were indexed for // sealed blocks. - if d.Ref == nil { + if d.OnIndexSeries == nil { instrument.EmitAndLogInvariantViolation(m.iopts, func(l *zap.Logger) { - l.Error("unexpected nil for document ref for background compact") + l.Error("unexpected nil for document index entry for background compact") }) return true } - entry, ok := d.Ref.(OnIndexSeries) - if !ok { - instrument.EmitAndLogInvariantViolation(m.iopts, func(l *zap.Logger) { - l.Error("unexpected type for document ref for background compact") - }) - return true - } - - isEmpty, ok := entry.RelookupAndCheckIsEmpty() + isEmpty, ok := d.OnIndexSeries.RelookupAndCheckIsEmpty() if !ok { // Should not happen since shard will not expire until // no more block starts are indexed. diff --git a/src/dbnode/storage/index/types.go b/src/dbnode/storage/index/types.go index 94fee0b195..9371ded4f9 100644 --- a/src/dbnode/storage/index/types.go +++ b/src/dbnode/storage/index/types.go @@ -365,48 +365,6 @@ type AggregateResultsEntry struct { Terms []ident.ID } -// OnIndexSeries provides a set of callback hooks to allow the reverse index -// to do lifecycle management of any resources retained during indexing. -type OnIndexSeries interface { - // OnIndexSuccess is executed when an entry is successfully indexed. The - // provided value for `blockStart` is the blockStart for which the write - // was indexed. - OnIndexSuccess(blockStart xtime.UnixNano) - - // OnIndexFinalize is executed when the index no longer holds any references - // to the provided resources. It can be used to cleanup any resources held - // during the course of indexing. `blockStart` is the startTime of the index - // block for which the write was attempted. - OnIndexFinalize(blockStart xtime.UnixNano) - - // OnIndexPrepare prepares the Entry to be handed off to the indexing sub-system. - // NB(prateek): we retain the ref count on the entry while the indexing is pending, - // the callback executed on the entry once the indexing is completed releases this - // reference. - OnIndexPrepare(blockStart xtime.UnixNano) - - // NeedsIndexUpdate returns a bool to indicate if the Entry needs to be indexed - // for the provided blockStart. It only allows a single index attempt at a time - // for a single entry. - // NB(prateek): NeedsIndexUpdate is a CAS, i.e. when this method returns true, it - // also sets state on the entry to indicate that a write for the given blockStart - // is going to be sent to the index, and other go routines should not attempt the - // same write. Callers are expected to ensure they follow this guideline. - // Further, every call to NeedsIndexUpdate which returns true needs to have a corresponding - // OnIndexFinalze() call. This is required for correct lifecycle maintenance. - NeedsIndexUpdate(indexBlockStartForWrite xtime.UnixNano) bool - - IfAlreadyIndexedMarkIndexSuccessAndFinalize( - blockStart xtime.UnixNano, - ) bool - - RelookupAndCheckIsEmpty() (bool, bool) - - DecrementReaderWriterCount() - - IndexedForBlockStart(indexBlockStart xtime.UnixNano) bool -} - // RemoveIndexedForBlockStartsResult is the result from calling // RemoveIndexedForBlockStarts. type RemoveIndexedForBlockStartsResult struct { @@ -935,7 +893,7 @@ type WriteBatchEntry struct { Timestamp xtime.UnixNano // OnIndexSeries is a listener/callback for when this entry is marked done // it is set to nil when the entry is marked done - OnIndexSeries OnIndexSeries + OnIndexSeries doc.OnIndexSeries // EnqueuedAt is the timestamp that this entry was enqueued for indexing // so that we can calculate the latency it takes to index the entry EnqueuedAt time.Time diff --git a/src/m3ninx/doc/document.go b/src/m3ninx/doc/document.go index 8faefedf80..741c10efec 100644 --- a/src/m3ninx/doc/document.go +++ b/src/m3ninx/doc/document.go @@ -89,9 +89,9 @@ func (f Fields) shallowCopy() Fields { // Metadata represents a document to be indexed. type Metadata struct { - ID []byte - Fields []Field - Ref interface{} + ID []byte + Fields []Field + OnIndexSeries OnIndexSeries } // Get returns the value of the specified field name in the document if it exists. diff --git a/src/m3ninx/doc/types.go b/src/m3ninx/doc/types.go index 879172719f..3ed6cb3ffa 100644 --- a/src/m3ninx/doc/types.go +++ b/src/m3ninx/doc/types.go @@ -20,6 +20,10 @@ package doc +import ( + xtime "github.com/m3db/m3/src/x/time" +) + // MetadataIterator provides an iterator over a collection of document metadata. It is NOT // safe for multiple goroutines to invoke methods on an MetadataIterator simultaneously. type MetadataIterator interface { @@ -72,3 +76,45 @@ type QueryDocIterator interface { // worker. Done() bool } + +// OnIndexSeries provides a set of callback hooks to allow the reverse index +// to do lifecycle management of any resources retained during indexing. +type OnIndexSeries interface { + // OnIndexSuccess is executed when an entry is successfully indexed. The + // provided value for `blockStart` is the blockStart for which the write + // was indexed. + OnIndexSuccess(blockStart xtime.UnixNano) + + // OnIndexFinalize is executed when the index no longer holds any references + // to the provided resources. It can be used to cleanup any resources held + // during the course of indexing. `blockStart` is the startTime of the index + // block for which the write was attempted. + OnIndexFinalize(blockStart xtime.UnixNano) + + // OnIndexPrepare prepares the Entry to be handed off to the indexing sub-system. + // NB(prateek): we retain the ref count on the entry while the indexing is pending, + // the callback executed on the entry once the indexing is completed releases this + // reference. + OnIndexPrepare(blockStart xtime.UnixNano) + + // NeedsIndexUpdate returns a bool to indicate if the Entry needs to be indexed + // for the provided blockStart. It only allows a single index attempt at a time + // for a single entry. + // NB(prateek): NeedsIndexUpdate is a CAS, i.e. when this method returns true, it + // also sets state on the entry to indicate that a write for the given blockStart + // is going to be sent to the index, and other go routines should not attempt the + // same write. Callers are expected to ensure they follow this guideline. + // Further, every call to NeedsIndexUpdate which returns true needs to have a corresponding + // OnIndexFinalze() call. This is required for correct lifecycle maintenance. + NeedsIndexUpdate(indexBlockStartForWrite xtime.UnixNano) bool + + IfAlreadyIndexedMarkIndexSuccessAndFinalize( + blockStart xtime.UnixNano, + ) bool + + RelookupAndCheckIsEmpty() (bool, bool) + + DecrementReaderWriterCount() + + IndexedForBlockStart(indexBlockStart xtime.UnixNano) bool +} From b2a016c4fce842591521bcd538061c313aa6b75d Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Wed, 18 Aug 2021 10:56:41 -0400 Subject: [PATCH 25/36] Cleanup feedback --- src/dbnode/storage/flush.go | 45 +++++++++++++++++++------------------ src/dbnode/storage/index.go | 19 ++++++++-------- 2 files changed, 33 insertions(+), 31 deletions(-) diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index b7c1e6a8aa..12c06207e6 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -56,16 +56,16 @@ const ( type namespaceFlushes map[string]namespaceFlush -type shardFlushes map[shardFlush]bool - type namespaceFlush struct { namespace databaseNamespace - shardFlushes map[shardFlush]bool + shardFlushes shardFlushes } -type shardFlush struct { - time xtime.UnixNano - shard databaseShard +type shardFlushes map[shardFlushKey]databaseShard + +type shardFlushKey struct { + shardID uint32 + blockStart xtime.UnixNano } type flushManagerMetrics struct { @@ -186,24 +186,24 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { // If index is enabled, then a shard+blockStart is "flushed" if the data AND index has been flushed. for _, n := range namespaces { var ( - indexEnabled = n.Options().IndexOptions().Enabled() - flushedShards map[shardFlush]bool + indexEnabled = n.Options().IndexOptions().Enabled() + flushed shardFlushes ) if indexEnabled { flushesForNs, ok := indexFlushes[n.ID().String()] if !ok { continue } - flushedShards = flushesForNs.shardFlushes + flushed = flushesForNs.shardFlushes } else { flushesForNs, ok := dataFlushes[n.ID().String()] if !ok { continue } - flushedShards = flushesForNs.shardFlushes + flushed = flushesForNs.shardFlushes } - for s := range flushedShards { + for k, v := range flushed { // Block sizes for data and index can differ and so if we are driving the flushing by // the index blockStarts, we must expand them to mark all containing data blockStarts. // E.g. if blockSize == 2h and indexBlockSize == 4h and the flushed index time is 6:00pm, @@ -211,11 +211,11 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { if indexEnabled { blockSize := n.Options().RetentionOptions().BlockSize() indexBlockSize := n.Options().IndexOptions().BlockSize() - for start := s.time; start < s.time.Add(indexBlockSize); start = start.Add(blockSize) { - s.shard.MarkWarmFlushStateSuccessOrError(start, err) + for start := k.blockStart; start < k.blockStart.Add(indexBlockSize); start = start.Add(blockSize) { + v.MarkWarmFlushStateSuccessOrError(start, err) } } else { - s.shard.MarkWarmFlushStateSuccessOrError(s.time, err) + v.MarkWarmFlushStateSuccessOrError(k.blockStart, err) } } } @@ -431,7 +431,7 @@ func (m *flushManager) flushNamespaceWithTimes( times []xtime.UnixNano, flushPreparer persist.FlushPreparer, ) (namespaceFlush, error) { - flushes := make(map[shardFlush]bool, 0) + flushes := make(shardFlushes, 0) multiErr := xerrors.NewMultiError() for _, t := range times { // NB(xichen): we still want to proceed if a namespace fails to flush its data. @@ -441,13 +441,14 @@ func (m *flushManager) flushNamespaceWithTimes( detailedErr := fmt.Errorf("namespace %s failed to flush data: %v", ns.ID().String(), err) multiErr = multiErr.Add(detailedErr) - } else { - for _, s := range shards { - flushes[shardFlush{ - shard: s, - time: t, - }] = true - } + continue + } + + for _, s := range shards { + flushes[shardFlushKey{ + shardID: s.ID(), + blockStart: t, + }] = s } } return namespaceFlush{ diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 5c0c0cd155..34d949a605 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -1039,7 +1039,7 @@ func (i *nsIndex) WarmFlush( defer i.metrics.flushIndexingConcurrency.Update(0) var evicted int - shardFlushes := make(map[shardFlush]bool, 0) + flushes := make(shardFlushes, 0) for _, block := range flushable { immutableSegments, err := i.flushBlock(flush, block, shards, builder) if err != nil { @@ -1074,17 +1074,18 @@ func (i *nsIndex) WarmFlush( zap.Error(err), zap.Time("blockStart", block.StartTime().ToTime()), ) - } else { - for _, s := range shards { - shardFlushes[shardFlush{ - shard: s, - time: block.StartTime(), - }] = true - } + continue + } + + for _, s := range shards { + flushes[shardFlushKey{ + shardID: s.ID(), + blockStart: block.StartTime(), + }] = s } } i.metrics.blocksEvictedMutableSegments.Inc(int64(evicted)) - return shardFlushes, nil + return flushes, nil } func (i *nsIndex) ColdFlush(shards []databaseShard) (OnColdFlushDone, error) { From 5c5d6e08993cc0c2642259e9141f65f1f447dd23 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Wed, 18 Aug 2021 11:52:20 -0400 Subject: [PATCH 26/36] Fixing tests 1 --- src/dbnode/storage/entry.go | 8 ++++ .../{ => entry}/entry_blackbox_test.go | 2 +- .../{ => entry}/entry_whitebox_test.go | 21 +++------- src/dbnode/storage/flush_test.go | 12 +++--- src/dbnode/storage/index/block_bench_test.go | 6 +-- src/dbnode/storage/index_block_test.go | 2 +- .../storage/index_queue_forward_write_test.go | 2 +- src/dbnode/storage/index_test.go | 4 +- src/dbnode/storage/series_resolver_test.go | 39 +++++++++---------- .../series_wired_list_interaction_test.go | 3 +- .../storage/shard_foreachentry_prop_test.go | 5 +-- src/dbnode/storage/shard_insert_queue_test.go | 10 ++--- src/dbnode/storage/shard_test.go | 37 +++++++++--------- 13 files changed, 73 insertions(+), 78 deletions(-) rename src/dbnode/storage/{ => entry}/entry_blackbox_test.go (99%) rename src/dbnode/storage/{ => entry}/entry_whitebox_test.go (86%) diff --git a/src/dbnode/storage/entry.go b/src/dbnode/storage/entry.go index 2042bf79ff..49c6508863 100644 --- a/src/dbnode/storage/entry.go +++ b/src/dbnode/storage/entry.go @@ -120,6 +120,14 @@ func (entry *Entry) DecrementReaderWriterCount() { atomic.AddInt32(&entry.curReadWriters, -1) } +// IndexedBlockCount returns the count of indexed block states. +func (entry *Entry) IndexedBlockCount() int { + entry.reverseIndex.RLock() + count := len(entry.reverseIndex.states) + entry.reverseIndex.RUnlock() + return count +} + // IndexedForBlockStart returns a bool to indicate if the Entry has been successfully // indexed for the given index blockstart. func (entry *Entry) IndexedForBlockStart(indexBlockStart xtime.UnixNano) bool { diff --git a/src/dbnode/storage/entry_blackbox_test.go b/src/dbnode/storage/entry/entry_blackbox_test.go similarity index 99% rename from src/dbnode/storage/entry_blackbox_test.go rename to src/dbnode/storage/entry/entry_blackbox_test.go index b56e0ba698..2a3db5ce9f 100644 --- a/src/dbnode/storage/entry_blackbox_test.go +++ b/src/dbnode/storage/entry/entry_blackbox_test.go @@ -18,7 +18,7 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -package storage +package entry import ( "sync" diff --git a/src/dbnode/storage/entry_whitebox_test.go b/src/dbnode/storage/entry/entry_whitebox_test.go similarity index 86% rename from src/dbnode/storage/entry_whitebox_test.go rename to src/dbnode/storage/entry/entry_whitebox_test.go index 4dba1b89b9..3665e10afa 100644 --- a/src/dbnode/storage/entry_whitebox_test.go +++ b/src/dbnode/storage/entry/entry_whitebox_test.go @@ -18,13 +18,14 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -package storage +package entry import ( "testing" "time" "github.com/golang/mock/gomock" + "github.com/m3db/m3/src/dbnode/storage" "github.com/m3db/m3/src/dbnode/storage/index" "github.com/m3db/m3/src/dbnode/storage/series" "github.com/m3db/m3/src/dbnode/ts/writes" @@ -35,22 +36,12 @@ import ( "github.com/stretchr/testify/require" ) -var ( - initTime = time.Date(2018, time.May, 12, 15, 55, 0, 0, time.UTC) - testBlockSize = 24 * time.Hour -) - -func newTime(n int) xtime.UnixNano { - t := initTime.Truncate(testBlockSize).Add(time.Duration(n) * testBlockSize) - return xtime.ToUnixNano(t) -} - func TestEntryIndexAttemptRotatesSlice(t *testing.T) { - e := NewEntry(NewEntryOptions{}) + e := storage.NewEntry(storage.NewEntryOptions{}) for i := 0; i < 10; i++ { ti := newTime(i) require.True(t, e.NeedsIndexUpdate(ti)) - require.Equal(t, i+1, len(e.reverseIndex.states)) + require.Equal(t, i+1, e.IndexedBlockCount()) } // ensure only the latest ones are held on to @@ -64,7 +55,7 @@ func TestEntryIndexSeriesRef(t *testing.T) { ctrl := gomock.NewController(t) now := time.Now() blockStart := newTime(0) - mockIndexWriter := NewMockIndexWriter(ctrl) + mockIndexWriter := storage.NewMockIndexWriter(ctrl) mockIndexWriter.EXPECT().BlockStartForWriteTime(blockStart). Return(blockStart). Times(2) @@ -80,7 +71,7 @@ func TestEntryIndexSeriesRef(t *testing.T) { series.WriteOptions{}, ).Return(true, series.WarmWrite, nil) - e := NewEntry(NewEntryOptions{ + e := storage.NewEntry(storage.NewEntryOptions{ Series: mockSeries, IndexWriter: mockIndexWriter, NowFn: func() time.Time { diff --git a/src/dbnode/storage/flush_test.go b/src/dbnode/storage/flush_test.go index 0529228aec..2141eca8b5 100644 --- a/src/dbnode/storage/flush_test.go +++ b/src/dbnode/storage/flush_test.go @@ -323,6 +323,8 @@ func TestFlushManagerSkipNamespaceIndexingDisabled(t *testing.T) { ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(true, nil).AnyTimes() ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any()).Return([]databaseShard{s1, s2}, nil).AnyTimes() ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() + s1.EXPECT().ID().Return(uint32(1)).AnyTimes() + s2.EXPECT().ID().Return(uint32(2)).AnyTimes() s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil).AnyTimes() s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil).AnyTimes() @@ -373,11 +375,11 @@ func TestFlushManagerNamespaceIndexingEnabled(t *testing.T) { // Validate that the flush state is marked as successful only AFTER all prequisite steps have been run. // Order is important to avoid any edge case where data is GCed from memory without all flushing operations // being completed. - mockFlushedShards := map[shardFlush]bool{ - shardFlush{shard: s1, time: xtime.Now().Add(time.Minute * 1)}: true, - shardFlush{shard: s1, time: xtime.Now().Add(time.Minute * 2)}: true, - shardFlush{shard: s2, time: xtime.Now().Add(time.Minute * 1)}: true, - shardFlush{shard: s2, time: xtime.Now().Add(time.Minute * 2)}: true, + mockFlushedShards := shardFlushes{ + shardFlushKey{shardID: s1.ID(), blockStart: xtime.Now().Add(time.Minute * 1)}: s1, + shardFlushKey{shardID: s1.ID(), blockStart: xtime.Now().Add(time.Minute * 2)}: s1, + shardFlushKey{shardID: s2.ID(), blockStart: xtime.Now().Add(time.Minute * 1)}: s2, + shardFlushKey{shardID: s2.ID(), blockStart: xtime.Now().Add(time.Minute * 2)}: s2, } steps := make([]*gomock.Call, 0) steps = append(steps, diff --git a/src/dbnode/storage/index/block_bench_test.go b/src/dbnode/storage/index/block_bench_test.go index 3780d1bf01..9d3f209bce 100644 --- a/src/dbnode/storage/index/block_bench_test.go +++ b/src/dbnode/storage/index/block_bench_test.go @@ -115,7 +115,7 @@ func BenchmarkBlockWrite(b *testing.B) { // useless to use in benchmarks type mockOnIndexSeries struct{} -var _ OnIndexSeries = mockOnIndexSeries{} +var _ doc.OnIndexSeries = mockOnIndexSeries{} func (m mockOnIndexSeries) OnIndexSuccess(_ xtime.UnixNano) {} func (m mockOnIndexSeries) OnIndexFinalize(_ xtime.UnixNano) {} @@ -133,5 +133,5 @@ func (m mockOnIndexSeries) RemoveIndexedForBlockStarts( ) RemoveIndexedForBlockStartsResult { return RemoveIndexedForBlockStartsResult{} } -func (m mockOnIndexSeries) IndexedOrAttemptedAny() bool { return false } -func (m mockOnIndexSeries) IsEmpty() bool { return false } +func (m mockOnIndexSeries) IndexedOrAttemptedAny() bool { return false } +func (m mockOnIndexSeries) RelookupAndCheckIsEmpty() (bool, bool) { return false, false } diff --git a/src/dbnode/storage/index_block_test.go b/src/dbnode/storage/index_block_test.go index 3ace4a6210..1cf3d175fb 100644 --- a/src/dbnode/storage/index_block_test.go +++ b/src/dbnode/storage/index_block_test.go @@ -98,7 +98,7 @@ func testWriteBatchEntry( id ident.ID, tags ident.Tags, timestamp xtime.UnixNano, - fns index.OnIndexSeries, + fns doc.OnIndexSeries, ) (index.WriteBatchEntry, doc.Metadata) { d := doc.Metadata{ID: copyBytes(id.Bytes())} for _, tag := range tags.Values() { diff --git a/src/dbnode/storage/index_queue_forward_write_test.go b/src/dbnode/storage/index_queue_forward_write_test.go index c083128f9b..98cf0a5446 100644 --- a/src/dbnode/storage/index_queue_forward_write_test.go +++ b/src/dbnode/storage/index_queue_forward_write_test.go @@ -282,7 +282,7 @@ func setupMockBlock( ts xtime.UnixNano, id ident.ID, tag ident.Tag, - lifecycle index.OnIndexSeries, + lifecycle doc.OnIndexSeries, ) { bl.EXPECT(). WriteBatch(gomock.Any()). diff --git a/src/dbnode/storage/index_test.go b/src/dbnode/storage/index_test.go index 81c0da47a6..10733673f4 100644 --- a/src/dbnode/storage/index_test.go +++ b/src/dbnode/storage/index_test.go @@ -842,7 +842,7 @@ func verifyFlushForShards( persistCalledTimes int actualDocs = make([]doc.Metadata, 0) expectedDocs = make([]doc.Metadata, 0) - expectedShardFlushes = make(map[shardFlush]bool, 0) + expectedShardFlushes = make(shardFlushes, 0) ) // NB(bodu): Always align now w/ the index's view of now. idx.nowFn = func() time.Time { @@ -929,7 +929,7 @@ func verifyFlushForShards( mockShard.EXPECT().FetchBlocksMetadataV2(gomock.Any(), blockStart, blockStart.Add(idx.blockSize), gomock.Any(), gomock.Any(), block.FetchBlocksMetadataOptions{OnlyDisk: true}).Return(results, nil, nil) - expectedShardFlushes[shardFlush{shard: mockShard, time: blockStart}] = true + expectedShardFlushes[shardFlushKey{shardID: mockShard.ID(), blockStart: blockStart}] = mockShard } mockBlock.EXPECT().IsSealed().Return(true) diff --git a/src/dbnode/storage/series_resolver_test.go b/src/dbnode/storage/series_resolver_test.go index d3d46013a1..99b9198b7a 100644 --- a/src/dbnode/storage/series_resolver_test.go +++ b/src/dbnode/storage/series_resolver_test.go @@ -27,14 +27,13 @@ import ( "github.com/stretchr/testify/require" - "github.com/m3db/m3/src/dbnode/storage" "github.com/m3db/m3/src/x/ident" ) func TestResolveError(t *testing.T) { wg := sync.WaitGroup{} id := ident.StringID("foo") - sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*storage.Entry, error) { + sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*Entry, error) { return nil, fmt.Errorf("unable to resolve series") }) _, err := sut.SeriesRef() @@ -44,7 +43,7 @@ func TestResolveError(t *testing.T) { func TestResolveNilEntry(t *testing.T) { wg := sync.WaitGroup{} id := ident.StringID("foo") - sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*storage.Entry, error) { + sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*Entry, error) { return nil, nil }) _, err := sut.SeriesRef() @@ -54,53 +53,53 @@ func TestResolveNilEntry(t *testing.T) { func TestResolve(t *testing.T) { wg := sync.WaitGroup{} id := ident.StringID("foo") - sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*storage.Entry, error) { - return storage.NewEntry(storage.NewEntryOptions{ + sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*Entry, error) { + return NewEntry(NewEntryOptions{ Index: 11, }), nil }) seriesRef, err := sut.SeriesRef() require.NoError(t, err) - require.IsType(t, &storage.Entry{}, seriesRef) - entry := seriesRef.(*storage.Entry) + require.IsType(t, &Entry{}, seriesRef) + entry := seriesRef.(*Entry) require.Equal(t, uint64(11), entry.Index) } func TestSecondResolveWontWait(t *testing.T) { wg := sync.WaitGroup{} id := ident.StringID("foo") - sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*storage.Entry, error) { - return storage.NewEntry(storage.NewEntryOptions{ + sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*Entry, error) { + return NewEntry(NewEntryOptions{ Index: 11, }), nil }) seriesRef, err := sut.SeriesRef() require.NoError(t, err) - require.IsType(t, &storage.Entry{}, seriesRef) - entry := seriesRef.(*storage.Entry) + require.IsType(t, &Entry{}, seriesRef) + entry := seriesRef.(*Entry) require.Equal(t, uint64(11), entry.Index) wg.Add(1) seriesRef2, err := sut.SeriesRef() require.NoError(t, err) - require.IsType(t, &storage.Entry{}, seriesRef2) - entry2 := seriesRef2.(*storage.Entry) + require.IsType(t, &Entry{}, seriesRef2) + entry2 := seriesRef2.(*Entry) require.Equal(t, entry, entry2) } func TestReleaseRef(t *testing.T) { wg := sync.WaitGroup{} id := ident.StringID("foo") - sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*storage.Entry, error) { - entry := storage.NewEntry(storage.NewEntryOptions{}) + sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*Entry, error) { + entry := NewEntry(NewEntryOptions{}) entry.IncrementReaderWriterCount() return entry, nil }) seriesRef, err := sut.SeriesRef() require.NoError(t, err) - require.IsType(t, &storage.Entry{}, seriesRef) + require.IsType(t, &Entry{}, seriesRef) - entry := seriesRef.(*storage.Entry) + entry := seriesRef.(*Entry) require.Equal(t, int32(1), entry.ReaderWriterCount()) err = sut.ReleaseRef() require.NoError(t, err) @@ -110,7 +109,7 @@ func TestReleaseRef(t *testing.T) { func TestReleaseRefError(t *testing.T) { wg := sync.WaitGroup{} id := ident.StringID("foo") - sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*storage.Entry, error) { + sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*Entry, error) { return nil, fmt.Errorf("unable to resolve series") }) err := sut.ReleaseRef() @@ -120,8 +119,8 @@ func TestReleaseRefError(t *testing.T) { func TestReleaseRefWithoutSeriesRef(t *testing.T) { wg := sync.WaitGroup{} id := ident.StringID("foo") - sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*storage.Entry, error) { - entry := storage.NewEntry(storage.NewEntryOptions{}) + sut := NewSeriesResolver(&wg, id, func(id ident.ID) (*Entry, error) { + entry := NewEntry(NewEntryOptions{}) entry.IncrementReaderWriterCount() return entry, nil }) diff --git a/src/dbnode/storage/series_wired_list_interaction_test.go b/src/dbnode/storage/series_wired_list_interaction_test.go index 52a2b6ab04..ef242c7080 100644 --- a/src/dbnode/storage/series_wired_list_interaction_test.go +++ b/src/dbnode/storage/series_wired_list_interaction_test.go @@ -29,7 +29,6 @@ import ( "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/storage/block" "github.com/m3db/m3/src/dbnode/storage/series" - "github.com/m3db/m3/src/dbnode/storage/series/lookup" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/x/clock" "github.com/m3db/m3/src/x/context" @@ -116,7 +115,7 @@ func TestSeriesWiredListConcurrentInteractions(t *testing.T) { require.NoError(t, err) shard.Lock() - shard.insertNewShardEntryWithLock(lookup.NewEntry(lookup.NewEntryOptions{ + shard.insertNewShardEntryWithLock(NewEntry(NewEntryOptions{ Series: seriesEntry, })) shard.Unlock() diff --git a/src/dbnode/storage/shard_foreachentry_prop_test.go b/src/dbnode/storage/shard_foreachentry_prop_test.go index ad9a965c73..bcbcb7863f 100644 --- a/src/dbnode/storage/shard_foreachentry_prop_test.go +++ b/src/dbnode/storage/shard_foreachentry_prop_test.go @@ -31,7 +31,6 @@ import ( "time" "github.com/m3db/m3/src/dbnode/namespace" - "github.com/m3db/m3/src/dbnode/storage" "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/ident" xtime "github.com/m3db/m3/src/x/time" @@ -205,7 +204,7 @@ func shardEntriesAreEqual(shard *dbShard, expectedEntries []shardEntryState) err return fmt.Errorf("expected to have %d idx, but did not see anything", idx) } nextElem := elem.Next() - entry := elem.Value.(*storage.Entry) + entry := elem.Value.(*Entry) if !entry.Series.ID().Equal(expectedEntry.id) { return fmt.Errorf("expected id: %s at %d, observed: %s", expectedEntry.id.String(), idx, entry.Series.ID().String()) @@ -253,7 +252,7 @@ func genBatchWorkFn() gopter.Gen { return gen.UInt8(). Map(func(n uint8) dbShardEntryBatchWorkFn { i := uint8(0) - return func([]*storage.Entry) bool { + return func([]*Entry) bool { i++ return i < n } diff --git a/src/dbnode/storage/shard_insert_queue_test.go b/src/dbnode/storage/shard_insert_queue_test.go index 5aedbd9c70..b11464c0ce 100644 --- a/src/dbnode/storage/shard_insert_queue_test.go +++ b/src/dbnode/storage/shard_insert_queue_test.go @@ -26,8 +26,6 @@ import ( "testing" "time" - "github.com/m3db/m3/src/dbnode/storage" - "github.com/fortytw2/leaktest" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -90,16 +88,16 @@ func TestShardInsertQueueBatchBackoff(t *testing.T) { }() // first insert - _, err := q.Insert(dbShardInsert{entry: &storage.Entry{Index: 0}}) + _, err := q.Insert(dbShardInsert{entry: &Entry{Index: 0}}) require.NoError(t, err) // wait for first insert batch to complete insertWgs[0].Wait() // now next batch will need to wait as we haven't progressed time - _, err = q.Insert(dbShardInsert{entry: &storage.Entry{Index: 1}}) + _, err = q.Insert(dbShardInsert{entry: &Entry{Index: 1}}) require.NoError(t, err) - _, err = q.Insert(dbShardInsert{entry: &storage.Entry{Index: 2}}) + _, err = q.Insert(dbShardInsert{entry: &Entry{Index: 2}}) require.NoError(t, err) // allow first insert to finish @@ -112,7 +110,7 @@ func TestShardInsertQueueBatchBackoff(t *testing.T) { assert.Equal(t, 1, numSleeps) // insert third batch, will also need to wait - _, err = q.Insert(dbShardInsert{entry: &storage.Entry{Index: 3}}) + _, err = q.Insert(dbShardInsert{entry: &Entry{Index: 3}}) require.NoError(t, err) // allow second batch to finish diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index 2d4274b522..b071449d72 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -38,7 +38,6 @@ import ( "github.com/m3db/m3/src/dbnode/persist/fs" "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/runtime" - "github.com/m3db/m3/src/dbnode/storage" "github.com/m3db/m3/src/dbnode/storage/block" "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" "github.com/m3db/m3/src/dbnode/storage/index/convert" @@ -100,7 +99,7 @@ func addMockSeries(ctrl *gomock.Controller, shard *dbShard, id ident.ID, tags id series.EXPECT().ID().Return(id).AnyTimes() series.EXPECT().IsEmpty().Return(false).AnyTimes() shard.Lock() - shard.insertNewShardEntryWithLock(storage.NewEntry(storage.NewEntryOptions{ + shard.insertNewShardEntryWithLock(NewEntry(NewEntryOptions{ Series: series, Index: index, })) @@ -215,7 +214,7 @@ func TestShardBootstrapWithFlushVersion(t *testing.T) { // Load the mock into the shard as an expected series so that we can assert // on the call to its Bootstrap() method below. - entry := storage.NewEntry(storage.NewEntryOptions{ + entry := NewEntry(NewEntryOptions{ Series: mockSeries, }) s.Lock() @@ -464,7 +463,7 @@ func TestShardFlushSeriesFlushError(t *testing.T) { flushed[i] = struct{}{} }). Return(series.FlushOutcomeErr, expectedErr) - s.list.PushBack(storage.NewEntry(storage.NewEntryOptions{ + s.list.PushBack(NewEntry(NewEntryOptions{ Series: curr, })) } @@ -551,7 +550,7 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { flushed[i] = struct{}{} }). Return(series.FlushOutcomeFlushedToDisk, nil) - s.list.PushBack(storage.NewEntry(storage.NewEntryOptions{ + s.list.PushBack(NewEntry(NewEntryOptions{ Series: curr, })) } @@ -662,7 +661,7 @@ func TestShardColdFlush(t *testing.T) { curr.EXPECT().Metadata().Return(doc.Metadata{ID: ds.id.Bytes()}).AnyTimes() curr.EXPECT().ColdFlushBlockStarts(gomock.Any()). Return(optimizedTimesFromTimes(ds.dirtyTimes)) - shard.list.PushBack(storage.NewEntry(storage.NewEntryOptions{ + shard.list.PushBack(NewEntry(NewEntryOptions{ Series: curr, })) } @@ -866,7 +865,7 @@ func TestShardSnapshotSeriesSnapshotSuccess(t *testing.T) { snapshotted[i] = struct{}{} }). Return(series.SnapshotResult{}, nil) - s.list.PushBack(storage.NewEntry(storage.NewEntryOptions{ + s.list.PushBack(NewEntry(NewEntryOptions{ Series: entry, })) } @@ -886,7 +885,7 @@ func addMockTestSeries(ctrl *gomock.Controller, shard *dbShard, id ident.ID) *se series := series.NewMockDatabaseSeries(ctrl) series.EXPECT().ID().AnyTimes().Return(id) shard.Lock() - shard.insertNewShardEntryWithLock(storage.NewEntry(storage.NewEntryOptions{ + shard.insertNewShardEntryWithLock(NewEntry(NewEntryOptions{ Series: series, })) shard.Unlock() @@ -904,7 +903,7 @@ func addTestSeriesWithCount(shard *dbShard, id ident.ID, count int32) series.Dat Options: shard.seriesOpts, }) shard.Lock() - entry := storage.NewEntry(storage.NewEntryOptions{ + entry := NewEntry(NewEntryOptions{ Series: seriesEntry, }) for i := int32(0); i < count; i++ { @@ -1244,7 +1243,7 @@ func TestShardTickRace(t *testing.T) { wg.Wait() shard.RLock() - shardlen := shard.storage.Len() + shardlen := shard.lookup.Len() shard.RUnlock() require.Equal(t, 0, shardlen) @@ -1265,7 +1264,7 @@ func TestShardTickCleanupSmallBatchSize(t *testing.T) { addTestSeries(shard, ident.StringID("foo")) _, err := shard.Tick(context.NewNoOpCanncellable(), xtime.Now(), namespace.Context{}) require.NoError(t, err) - require.Equal(t, 0, shard.storage.Len()) + require.Equal(t, 0, shard.lookup.Len()) } // This tests ensures the shard returns an error if two ticks are triggered concurrently. @@ -1409,7 +1408,7 @@ func TestPurgeExpiredSeriesEmptySeries(t *testing.T) { require.NoError(t, err) shard.RLock() - require.Equal(t, 0, shard.storage.Len()) + require.Equal(t, 0, shard.lookup.Len()) shard.RUnlock() } @@ -1466,7 +1465,7 @@ func TestPurgeExpiredSeriesWriteAfterTicking(t *testing.T) { require.NoError(t, err) require.Equal(t, 0, r.activeSeries) require.Equal(t, 1, r.expiredSeries) - require.Equal(t, 1, shard.storage.Len()) + require.Equal(t, 1, shard.lookup.Len()) } // This tests the scenario where tickForEachSeries finishes, and before purgeExpiredSeries @@ -1476,7 +1475,7 @@ func TestPurgeExpiredSeriesWriteAfterPurging(t *testing.T) { ctrl := xtest.NewController(t) defer ctrl.Finish() - var entry *storage.Entry + var entry *Entry opts := DefaultTestOptions() shard := testDatabaseShard(t, opts) @@ -1494,10 +1493,10 @@ func TestPurgeExpiredSeriesWriteAfterPurging(t *testing.T) { require.NoError(t, err) require.Equal(t, 0, r.activeSeries) require.Equal(t, 1, r.expiredSeries) - require.Equal(t, 1, shard.storage.Len()) + require.Equal(t, 1, shard.lookup.Len()) entry.DecrementReaderWriterCount() - require.Equal(t, 1, shard.storage.Len()) + require.Equal(t, 1, shard.lookup.Len()) } func TestForEachShardEntry(t *testing.T) { @@ -1509,7 +1508,7 @@ func TestForEachShardEntry(t *testing.T) { } count := 0 - entryFn := func(entry *storage.Entry) bool { + entryFn := func(entry *Entry) bool { if entry.Series.ID().String() == "foo.8" { return false } @@ -1529,7 +1528,7 @@ func TestForEachShardEntry(t *testing.T) { // Ensure that reader writer count gets reset shard.RLock() for elem := shard.list.Front(); elem != nil; elem = elem.Next() { - entry := elem.Value.(*storage.Entry) + entry := elem.Value.(*Entry) assert.Equal(t, int32(0), entry.ReaderWriterCount()) } shard.RUnlock() @@ -2023,7 +2022,7 @@ func TestSeriesRefResolver(t *testing.T) { // should return already inserted entry as series. resolverEntry, err := shard.SeriesRefResolver(seriesID, iter) require.NoError(t, err) - require.IsType(t, &storage.Entry{}, resolverEntry) + require.IsType(t, &Entry{}, resolverEntry) refEntry, err := resolverEntry.SeriesRef() require.NoError(t, err) require.Equal(t, seriesRef, refEntry) From a4e1f0a229228029e97d8a3c0b98dc1fc7854867 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Wed, 18 Aug 2021 12:01:39 -0400 Subject: [PATCH 27/36] Fixing tests 2 --- src/dbnode/storage/flush_test.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/dbnode/storage/flush_test.go b/src/dbnode/storage/flush_test.go index 2141eca8b5..3e232e166d 100644 --- a/src/dbnode/storage/flush_test.go +++ b/src/dbnode/storage/flush_test.go @@ -371,15 +371,15 @@ func TestFlushManagerNamespaceIndexingEnabled(t *testing.T) { ns.EXPECT().Options().Return(nsOpts).AnyTimes() ns.EXPECT().ID().Return(defaultTestNs1ID).AnyTimes() ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(true, nil).AnyTimes() + s1.EXPECT().ID().Return(uint32(1)).AnyTimes() + s2.EXPECT().ID().Return(uint32(2)).AnyTimes() // Validate that the flush state is marked as successful only AFTER all prequisite steps have been run. // Order is important to avoid any edge case where data is GCed from memory without all flushing operations // being completed. mockFlushedShards := shardFlushes{ shardFlushKey{shardID: s1.ID(), blockStart: xtime.Now().Add(time.Minute * 1)}: s1, - shardFlushKey{shardID: s1.ID(), blockStart: xtime.Now().Add(time.Minute * 2)}: s1, shardFlushKey{shardID: s2.ID(), blockStart: xtime.Now().Add(time.Minute * 1)}: s2, - shardFlushKey{shardID: s2.ID(), blockStart: xtime.Now().Add(time.Minute * 2)}: s2, } steps := make([]*gomock.Call, 0) steps = append(steps, @@ -387,8 +387,6 @@ func TestFlushManagerNamespaceIndexingEnabled(t *testing.T) { ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes(), ns.EXPECT().FlushIndex(gomock.Any()).Return(mockFlushedShards, nil), s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil), - s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil), - s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil), s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil), ) gomock.InOrder(steps...) From b5b3713ef34b250748040fbe683c2c99bb26df1f Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Wed, 18 Aug 2021 12:12:01 -0400 Subject: [PATCH 28/36] Mock --- src/dbnode/storage/index/index_mock.go | 128 ------------------------ src/m3ninx/doc/doc_mock.go | 130 +++++++++++++++++++++++++ 2 files changed, 130 insertions(+), 128 deletions(-) diff --git a/src/dbnode/storage/index/index_mock.go b/src/dbnode/storage/index/index_mock.go index 24447d00d2..7c6840382c 100644 --- a/src/dbnode/storage/index/index_mock.go +++ b/src/dbnode/storage/index/index_mock.go @@ -790,134 +790,6 @@ func (mr *MockAggregateValuesPoolMockRecorder) Put(value interface{}) *gomock.Ca return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Put", reflect.TypeOf((*MockAggregateValuesPool)(nil).Put), value) } -// MockOnIndexSeries is a mock of OnIndexSeries interface. -type MockOnIndexSeries struct { - ctrl *gomock.Controller - recorder *MockOnIndexSeriesMockRecorder -} - -// MockOnIndexSeriesMockRecorder is the mock recorder for MockOnIndexSeries. -type MockOnIndexSeriesMockRecorder struct { - mock *MockOnIndexSeries -} - -// NewMockOnIndexSeries creates a new mock instance. -func NewMockOnIndexSeries(ctrl *gomock.Controller) *MockOnIndexSeries { - mock := &MockOnIndexSeries{ctrl: ctrl} - mock.recorder = &MockOnIndexSeriesMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use. -func (m *MockOnIndexSeries) EXPECT() *MockOnIndexSeriesMockRecorder { - return m.recorder -} - -// DecrementReaderWriterCount mocks base method. -func (m *MockOnIndexSeries) DecrementReaderWriterCount() { - m.ctrl.T.Helper() - m.ctrl.Call(m, "DecrementReaderWriterCount") -} - -// DecrementReaderWriterCount indicates an expected call of DecrementReaderWriterCount. -func (mr *MockOnIndexSeriesMockRecorder) DecrementReaderWriterCount() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DecrementReaderWriterCount", reflect.TypeOf((*MockOnIndexSeries)(nil).DecrementReaderWriterCount)) -} - -// IfAlreadyIndexedMarkIndexSuccessAndFinalize mocks base method. -func (m *MockOnIndexSeries) IfAlreadyIndexedMarkIndexSuccessAndFinalize(blockStart time0.UnixNano) bool { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "IfAlreadyIndexedMarkIndexSuccessAndFinalize", blockStart) - ret0, _ := ret[0].(bool) - return ret0 -} - -// IfAlreadyIndexedMarkIndexSuccessAndFinalize indicates an expected call of IfAlreadyIndexedMarkIndexSuccessAndFinalize. -func (mr *MockOnIndexSeriesMockRecorder) IfAlreadyIndexedMarkIndexSuccessAndFinalize(blockStart interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IfAlreadyIndexedMarkIndexSuccessAndFinalize", reflect.TypeOf((*MockOnIndexSeries)(nil).IfAlreadyIndexedMarkIndexSuccessAndFinalize), blockStart) -} - -// IndexedForBlockStart mocks base method. -func (m *MockOnIndexSeries) IndexedForBlockStart(indexBlockStart time0.UnixNano) bool { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "IndexedForBlockStart", indexBlockStart) - ret0, _ := ret[0].(bool) - return ret0 -} - -// IndexedForBlockStart indicates an expected call of IndexedForBlockStart. -func (mr *MockOnIndexSeriesMockRecorder) IndexedForBlockStart(indexBlockStart interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexedForBlockStart", reflect.TypeOf((*MockOnIndexSeries)(nil).IndexedForBlockStart), indexBlockStart) -} - -// NeedsIndexUpdate mocks base method. -func (m *MockOnIndexSeries) NeedsIndexUpdate(indexBlockStartForWrite time0.UnixNano) bool { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "NeedsIndexUpdate", indexBlockStartForWrite) - ret0, _ := ret[0].(bool) - return ret0 -} - -// NeedsIndexUpdate indicates an expected call of NeedsIndexUpdate. -func (mr *MockOnIndexSeriesMockRecorder) NeedsIndexUpdate(indexBlockStartForWrite interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NeedsIndexUpdate", reflect.TypeOf((*MockOnIndexSeries)(nil).NeedsIndexUpdate), indexBlockStartForWrite) -} - -// OnIndexFinalize mocks base method. -func (m *MockOnIndexSeries) OnIndexFinalize(blockStart time0.UnixNano) { - m.ctrl.T.Helper() - m.ctrl.Call(m, "OnIndexFinalize", blockStart) -} - -// OnIndexFinalize indicates an expected call of OnIndexFinalize. -func (mr *MockOnIndexSeriesMockRecorder) OnIndexFinalize(blockStart interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnIndexFinalize", reflect.TypeOf((*MockOnIndexSeries)(nil).OnIndexFinalize), blockStart) -} - -// OnIndexPrepare mocks base method. -func (m *MockOnIndexSeries) OnIndexPrepare(blockStart time0.UnixNano) { - m.ctrl.T.Helper() - m.ctrl.Call(m, "OnIndexPrepare", blockStart) -} - -// OnIndexPrepare indicates an expected call of OnIndexPrepare. -func (mr *MockOnIndexSeriesMockRecorder) OnIndexPrepare(blockStart interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnIndexPrepare", reflect.TypeOf((*MockOnIndexSeries)(nil).OnIndexPrepare), blockStart) -} - -// OnIndexSuccess mocks base method. -func (m *MockOnIndexSeries) OnIndexSuccess(blockStart time0.UnixNano) { - m.ctrl.T.Helper() - m.ctrl.Call(m, "OnIndexSuccess", blockStart) -} - -// OnIndexSuccess indicates an expected call of OnIndexSuccess. -func (mr *MockOnIndexSeriesMockRecorder) OnIndexSuccess(blockStart interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnIndexSuccess", reflect.TypeOf((*MockOnIndexSeries)(nil).OnIndexSuccess), blockStart) -} - -// RelookupAndCheckIsEmpty mocks base method. -func (m *MockOnIndexSeries) RelookupAndCheckIsEmpty() (bool, bool) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "RelookupAndCheckIsEmpty") - ret0, _ := ret[0].(bool) - ret1, _ := ret[1].(bool) - return ret0, ret1 -} - -// RelookupAndCheckIsEmpty indicates an expected call of RelookupAndCheckIsEmpty. -func (mr *MockOnIndexSeriesMockRecorder) RelookupAndCheckIsEmpty() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RelookupAndCheckIsEmpty", reflect.TypeOf((*MockOnIndexSeries)(nil).RelookupAndCheckIsEmpty)) -} - // MockBlock is a mock of Block interface. type MockBlock struct { ctrl *gomock.Controller diff --git a/src/m3ninx/doc/doc_mock.go b/src/m3ninx/doc/doc_mock.go index a44db060d3..897732adab 100644 --- a/src/m3ninx/doc/doc_mock.go +++ b/src/m3ninx/doc/doc_mock.go @@ -27,6 +27,8 @@ package doc import ( "reflect" + "github.com/m3db/m3/src/x/time" + "github.com/golang/mock/gomock" ) @@ -280,3 +282,131 @@ func (mr *MockQueryDocIteratorMockRecorder) Next() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Next", reflect.TypeOf((*MockQueryDocIterator)(nil).Next)) } + +// MockOnIndexSeries is a mock of OnIndexSeries interface. +type MockOnIndexSeries struct { + ctrl *gomock.Controller + recorder *MockOnIndexSeriesMockRecorder +} + +// MockOnIndexSeriesMockRecorder is the mock recorder for MockOnIndexSeries. +type MockOnIndexSeriesMockRecorder struct { + mock *MockOnIndexSeries +} + +// NewMockOnIndexSeries creates a new mock instance. +func NewMockOnIndexSeries(ctrl *gomock.Controller) *MockOnIndexSeries { + mock := &MockOnIndexSeries{ctrl: ctrl} + mock.recorder = &MockOnIndexSeriesMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockOnIndexSeries) EXPECT() *MockOnIndexSeriesMockRecorder { + return m.recorder +} + +// DecrementReaderWriterCount mocks base method. +func (m *MockOnIndexSeries) DecrementReaderWriterCount() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "DecrementReaderWriterCount") +} + +// DecrementReaderWriterCount indicates an expected call of DecrementReaderWriterCount. +func (mr *MockOnIndexSeriesMockRecorder) DecrementReaderWriterCount() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DecrementReaderWriterCount", reflect.TypeOf((*MockOnIndexSeries)(nil).DecrementReaderWriterCount)) +} + +// IfAlreadyIndexedMarkIndexSuccessAndFinalize mocks base method. +func (m *MockOnIndexSeries) IfAlreadyIndexedMarkIndexSuccessAndFinalize(blockStart time.UnixNano) bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "IfAlreadyIndexedMarkIndexSuccessAndFinalize", blockStart) + ret0, _ := ret[0].(bool) + return ret0 +} + +// IfAlreadyIndexedMarkIndexSuccessAndFinalize indicates an expected call of IfAlreadyIndexedMarkIndexSuccessAndFinalize. +func (mr *MockOnIndexSeriesMockRecorder) IfAlreadyIndexedMarkIndexSuccessAndFinalize(blockStart interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IfAlreadyIndexedMarkIndexSuccessAndFinalize", reflect.TypeOf((*MockOnIndexSeries)(nil).IfAlreadyIndexedMarkIndexSuccessAndFinalize), blockStart) +} + +// IndexedForBlockStart mocks base method. +func (m *MockOnIndexSeries) IndexedForBlockStart(indexBlockStart time.UnixNano) bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "IndexedForBlockStart", indexBlockStart) + ret0, _ := ret[0].(bool) + return ret0 +} + +// IndexedForBlockStart indicates an expected call of IndexedForBlockStart. +func (mr *MockOnIndexSeriesMockRecorder) IndexedForBlockStart(indexBlockStart interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexedForBlockStart", reflect.TypeOf((*MockOnIndexSeries)(nil).IndexedForBlockStart), indexBlockStart) +} + +// NeedsIndexUpdate mocks base method. +func (m *MockOnIndexSeries) NeedsIndexUpdate(indexBlockStartForWrite time.UnixNano) bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "NeedsIndexUpdate", indexBlockStartForWrite) + ret0, _ := ret[0].(bool) + return ret0 +} + +// NeedsIndexUpdate indicates an expected call of NeedsIndexUpdate. +func (mr *MockOnIndexSeriesMockRecorder) NeedsIndexUpdate(indexBlockStartForWrite interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NeedsIndexUpdate", reflect.TypeOf((*MockOnIndexSeries)(nil).NeedsIndexUpdate), indexBlockStartForWrite) +} + +// OnIndexFinalize mocks base method. +func (m *MockOnIndexSeries) OnIndexFinalize(blockStart time.UnixNano) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "OnIndexFinalize", blockStart) +} + +// OnIndexFinalize indicates an expected call of OnIndexFinalize. +func (mr *MockOnIndexSeriesMockRecorder) OnIndexFinalize(blockStart interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnIndexFinalize", reflect.TypeOf((*MockOnIndexSeries)(nil).OnIndexFinalize), blockStart) +} + +// OnIndexPrepare mocks base method. +func (m *MockOnIndexSeries) OnIndexPrepare(blockStart time.UnixNano) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "OnIndexPrepare", blockStart) +} + +// OnIndexPrepare indicates an expected call of OnIndexPrepare. +func (mr *MockOnIndexSeriesMockRecorder) OnIndexPrepare(blockStart interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnIndexPrepare", reflect.TypeOf((*MockOnIndexSeries)(nil).OnIndexPrepare), blockStart) +} + +// OnIndexSuccess mocks base method. +func (m *MockOnIndexSeries) OnIndexSuccess(blockStart time.UnixNano) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "OnIndexSuccess", blockStart) +} + +// OnIndexSuccess indicates an expected call of OnIndexSuccess. +func (mr *MockOnIndexSeriesMockRecorder) OnIndexSuccess(blockStart interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnIndexSuccess", reflect.TypeOf((*MockOnIndexSeries)(nil).OnIndexSuccess), blockStart) +} + +// RelookupAndCheckIsEmpty mocks base method. +func (m *MockOnIndexSeries) RelookupAndCheckIsEmpty() (bool, bool) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RelookupAndCheckIsEmpty") + ret0, _ := ret[0].(bool) + ret1, _ := ret[1].(bool) + return ret0, ret1 +} + +// RelookupAndCheckIsEmpty indicates an expected call of RelookupAndCheckIsEmpty. +func (mr *MockOnIndexSeriesMockRecorder) RelookupAndCheckIsEmpty() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RelookupAndCheckIsEmpty", reflect.TypeOf((*MockOnIndexSeries)(nil).RelookupAndCheckIsEmpty)) +} From cdcab192346bde3482df0c2b42e6b67df41264a5 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Wed, 18 Aug 2021 12:20:04 -0400 Subject: [PATCH 29/36] Lint --- src/dbnode/storage/flush.go | 2 +- src/dbnode/storage/index.go | 2 +- src/dbnode/storage/index_test.go | 2 +- src/dbnode/storage/shard.go | 36 ++++++++++++++++---------------- 4 files changed, 21 insertions(+), 21 deletions(-) diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index 12c06207e6..ce9ecdf1dd 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -431,7 +431,7 @@ func (m *flushManager) flushNamespaceWithTimes( times []xtime.UnixNano, flushPreparer persist.FlushPreparer, ) (namespaceFlush, error) { - flushes := make(shardFlushes, 0) + flushes := make(shardFlushes) multiErr := xerrors.NewMultiError() for _, t := range times { // NB(xichen): we still want to proceed if a namespace fails to flush its data. diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 34d949a605..44ab1aad89 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -1039,7 +1039,7 @@ func (i *nsIndex) WarmFlush( defer i.metrics.flushIndexingConcurrency.Update(0) var evicted int - flushes := make(shardFlushes, 0) + flushes := make(shardFlushes) for _, block := range flushable { immutableSegments, err := i.flushBlock(flush, block, shards, builder) if err != nil { diff --git a/src/dbnode/storage/index_test.go b/src/dbnode/storage/index_test.go index 10733673f4..b0d52faffc 100644 --- a/src/dbnode/storage/index_test.go +++ b/src/dbnode/storage/index_test.go @@ -842,7 +842,7 @@ func verifyFlushForShards( persistCalledTimes int actualDocs = make([]doc.Metadata, 0) expectedDocs = make([]doc.Metadata, 0) - expectedShardFlushes = make(shardFlushes, 0) + expectedShardFlushes = make(shardFlushes) ) // NB(bodu): Always align now w/ the index's view of now. idx.nowFn = func() time.Time { diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 4f247da4a7..0f70363720 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -445,7 +445,7 @@ func (s *dbShard) OnRetrieveBlock( nsCtx namespace.Context, ) { s.RLock() - entry, _, err := s.lookupEntryWithLock(id) + entry, err := s.lookupEntryWithLock(id) if entry != nil { entry.IncrementReaderWriterCount() defer entry.DecrementReaderWriterCount() @@ -499,7 +499,7 @@ func (s *dbShard) OnRetrieveBlock( func (s *dbShard) OnEvictedFromWiredList(id ident.ID, blockStart xtime.UnixNano) { s.RLock() - entry, _, err := s.lookupEntryWithLock(id) + entry, err := s.lookupEntryWithLock(id) s.RUnlock() if err != nil && err != errShardEntryNotFound { @@ -523,8 +523,8 @@ func (s *dbShard) OnEvictedFromWiredList(id ident.ID, blockStart xtime.UnixNano) entry.Series.OnEvictedFromWiredList(id, blockStart) } -func (s *dbShard) forEachShardEntry(entryFn dbShardEntryWorkFn) error { - return s.forEachShardEntryBatch(func(currEntries []*Entry) bool { +func (s *dbShard) forEachShardEntry(entryFn dbShardEntryWorkFn) { + s.forEachShardEntryBatch(func(currEntries []*Entry) bool { for _, entry := range currEntries { if continueForEach := entryFn(entry); !continueForEach { return false @@ -542,7 +542,7 @@ func iterateBatchSize(elemsLen int) int { return int(math.Max(shardIterateBatchMinSize, t)) } -func (s *dbShard) forEachShardEntryBatch(entriesBatchFn dbShardEntryBatchWorkFn) error { +func (s *dbShard) forEachShardEntryBatch(entriesBatchFn dbShardEntryBatchWorkFn) { // NB(r): consider using a lockless list for ticking. s.RLock() elemsLen := s.list.Len() @@ -598,11 +598,11 @@ func (s *dbShard) forEachShardEntryBatch(entriesBatchFn dbShardEntryBatchWorkFn) currEntries = currEntries[:0] if !continueExecution { decRefElem(nextElem) - return nil + return } } - return nil + return } func (s *dbShard) IsBootstrapped() bool { @@ -1066,7 +1066,7 @@ func (s *dbShard) ReadEncoded( nsCtx namespace.Context, ) (series.BlockReaderIter, error) { s.RLock() - entry, _, err := s.lookupEntryWithLock(id) + entry, err := s.lookupEntryWithLock(id) if entry != nil { // NB(r): Ensure readers have consistent view of this series, do // not expire the series while being read from. @@ -1111,17 +1111,17 @@ func (s *dbShard) FetchWideEntry( } // lookupEntryWithLock returns the entry for a given id while holding a read lock or a write lock. -func (s *dbShard) lookupEntryWithLock(id ident.ID) (*Entry, *list.Element, error) { +func (s *dbShard) lookupEntryWithLock(id ident.ID) (*Entry, error) { if s.state != dbShardStateOpen { // NB(r): Return an invalid params error here so any upstream // callers will not retry this operation - return nil, nil, xerrors.NewInvalidParamsError(errShardNotOpen) + return nil, xerrors.NewInvalidParamsError(errShardNotOpen) } elem, exists := s.lookup.Get(id) if !exists { - return nil, nil, errShardEntryNotFound + return nil, errShardEntryNotFound } - return elem.Value.(*Entry), elem, nil + return elem.Value.(*Entry), nil } func (s *dbShard) writableSeries(id ident.ID, tagResolver convert.TagMetadataResolver) (*Entry, error) { @@ -1161,7 +1161,7 @@ func (s *dbShard) TryRetrieveWritableSeries(id ident.ID) ( opts := WritableSeriesOptions{ WriteNewSeriesAsync: s.currRuntimeOptions.writeNewSeriesAsync, } - if entry, _, err := s.lookupEntryWithLock(id); err == nil { + if entry, err := s.lookupEntryWithLock(id); err == nil { entry.IncrementReaderWriterCount() s.RUnlock() return entry, opts, nil @@ -1359,7 +1359,7 @@ func (s *dbShard) insertSeriesSync( } }() - existingEntry, _, err := s.lookupEntryWithLock(id) + existingEntry, err := s.lookupEntryWithLock(id) if err != nil && err != errShardEntryNotFound { // Shard not taking inserts likely. return nil, err @@ -1444,7 +1444,7 @@ func (s *dbShard) insertSeriesBatch(inserts []dbShardInsert) error { // i.e. we don't have a ref on provided entry, so we check if between the operation being // enqueue in the shard insert queue, and this function executing, an entry was created // for the same ID. - entry, _, err := s.lookupEntryWithLock(inserts[i].entry.Series.ID()) + entry, err := s.lookupEntryWithLock(inserts[i].entry.Series.ID()) if entry != nil { // Already exists so update the entry we're pointed at for this insert. inserts[i].entry = entry @@ -1578,7 +1578,7 @@ func (s *dbShard) FetchBlocks( nsCtx namespace.Context, ) ([]block.FetchBlockResult, error) { s.RLock() - entry, _, err := s.lookupEntryWithLock(id) + entry, err := s.lookupEntryWithLock(id) if entry != nil { // NB(r): Ensure readers have consistent view of this series, do // not expire the series while being read from. @@ -1619,7 +1619,7 @@ func (s *dbShard) FetchBlocksForColdFlush( nsCtx namespace.Context, ) (block.FetchBlockResult, error) { s.RLock() - entry, _, err := s.lookupEntryWithLock(seriesID) + entry, err := s.lookupEntryWithLock(seriesID) s.RUnlock() if entry == nil || err != nil { return block.FetchBlockResult{}, err @@ -2646,7 +2646,7 @@ func (s *dbShard) DocRef(id ident.ID) (doc.Metadata, bool, error) { s.RLock() defer s.RUnlock() - entry, _, err := s.lookupEntryWithLock(id) + entry, err := s.lookupEntryWithLock(id) if err == nil { return entry.Series.Metadata(), true, nil } From cad7fcdfa0424135d54e2476181152c07892f8e6 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Wed, 18 Aug 2021 12:51:29 -0400 Subject: [PATCH 30/36] More fixing tests 1 --- src/dbnode/storage/index/block_test.go | 44 +++++++++---------- src/dbnode/storage/index/for_each_test.go | 4 +- src/dbnode/storage/index/write_batch_test.go | 7 +-- src/dbnode/storage/index_block_test.go | 6 +-- src/dbnode/storage/index_insert_queue_test.go | 5 ++- .../storage/index_query_concurrent_test.go | 2 +- .../storage/index_queue_forward_write_test.go | 14 +++--- src/dbnode/storage/index_queue_test.go | 10 ++--- src/dbnode/storage/shard_ref_count_test.go | 16 +++---- src/dbnode/storage/shard_test.go | 6 +-- 10 files changed, 58 insertions(+), 56 deletions(-) diff --git a/src/dbnode/storage/index/block_test.go b/src/dbnode/storage/index/block_test.go index 151c5bf309..d92ebd040d 100644 --- a/src/dbnode/storage/index/block_test.go +++ b/src/dbnode/storage/index/block_test.go @@ -111,7 +111,7 @@ func TestBlockWriteAfterClose(t *testing.T) { require.NoError(t, err) require.NoError(t, b.Close()) - lifecycle := NewMockOnIndexSeries(ctrl) + lifecycle := doc.NewMockOnIndexSeries(ctrl) lifecycle.EXPECT().OnIndexFinalize(blockStart) batch := NewWriteBatch(WriteBatchOptions{ @@ -160,7 +160,7 @@ func TestBlockWriteAfterSeal(t *testing.T) { require.NoError(t, err) require.NoError(t, b.Seal()) - lifecycle := NewMockOnIndexSeries(ctrl) + lifecycle := doc.NewMockOnIndexSeries(ctrl) lifecycle.EXPECT().OnIndexFinalize(blockStart) batch := NewWriteBatch(WriteBatchOptions{ @@ -214,11 +214,11 @@ func TestBlockWrite(t *testing.T) { b, ok := blk.(*block) require.True(t, ok) - h1 := NewMockOnIndexSeries(ctrl) + h1 := doc.NewMockOnIndexSeries(ctrl) h1.EXPECT().OnIndexFinalize(blockStart) h1.EXPECT().OnIndexSuccess(blockStart) - h2 := NewMockOnIndexSeries(ctrl) + h2 := doc.NewMockOnIndexSeries(ctrl) h2.EXPECT().OnIndexFinalize(blockStart) h2.EXPECT().OnIndexSuccess(blockStart) @@ -260,11 +260,11 @@ func TestBlockWriteActualSegmentPartialFailure(t *testing.T) { b, ok := blk.(*block) require.True(t, ok) - h1 := NewMockOnIndexSeries(ctrl) + h1 := doc.NewMockOnIndexSeries(ctrl) h1.EXPECT().OnIndexFinalize(blockStart) h1.EXPECT().OnIndexSuccess(blockStart) - h2 := NewMockOnIndexSeries(ctrl) + h2 := doc.NewMockOnIndexSeries(ctrl) h2.EXPECT().OnIndexFinalize(blockStart) batch := NewWriteBatch(WriteBatchOptions{ @@ -321,11 +321,11 @@ func TestBlockWritePartialFailure(t *testing.T) { b, ok := blk.(*block) require.True(t, ok) - h1 := NewMockOnIndexSeries(ctrl) + h1 := doc.NewMockOnIndexSeries(ctrl) h1.EXPECT().OnIndexFinalize(blockStart) h1.EXPECT().OnIndexSuccess(blockStart) - h2 := NewMockOnIndexSeries(ctrl) + h2 := doc.NewMockOnIndexSeries(ctrl) h2.EXPECT().OnIndexFinalize(blockStart) batch := NewWriteBatch(WriteBatchOptions{ @@ -1236,7 +1236,7 @@ func TestBlockNeedsMutableSegmentsEvicted(t *testing.T) { require.False(t, b.NeedsMutableSegmentsEvicted()) // perform write and ensure it says it needs eviction - h1 := NewMockOnIndexSeries(ctrl) + h1 := doc.NewMockOnIndexSeries(ctrl) h1.EXPECT().OnIndexFinalize(start) h1.EXPECT().OnIndexSuccess(start) batch := NewWriteBatch(WriteBatchOptions{ @@ -1372,11 +1372,11 @@ func TestBlockE2EInsertQuery(t *testing.T) { b, ok := blk.(*block) require.True(t, ok) - h1 := NewMockOnIndexSeries(ctrl) + h1 := doc.NewMockOnIndexSeries(ctrl) h1.EXPECT().OnIndexFinalize(blockStart) h1.EXPECT().OnIndexSuccess(blockStart) - h2 := NewMockOnIndexSeries(ctrl) + h2 := doc.NewMockOnIndexSeries(ctrl) h2.EXPECT().OnIndexFinalize(blockStart) h2.EXPECT().OnIndexSuccess(blockStart) @@ -1456,14 +1456,14 @@ func TestBlockE2EInsertQueryLimit(t *testing.T) { b, ok := blk.(*block) require.True(t, ok) - h1 := NewMockOnIndexSeries(ctrl) + h1 := doc.NewMockOnIndexSeries(ctrl) h1.EXPECT().OnIndexFinalize(blockStart) h1.EXPECT().OnIndexSuccess(blockStart) h1.EXPECT().IndexedForBlockStart(blockStart). Return(true). AnyTimes() - h2 := NewMockOnIndexSeries(ctrl) + h2 := doc.NewMockOnIndexSeries(ctrl) h2.EXPECT().OnIndexFinalize(blockStart) h2.EXPECT().OnIndexSuccess(blockStart) h1.EXPECT().IndexedForBlockStart(blockStart). @@ -1548,14 +1548,14 @@ func TestBlockE2EInsertAddResultsQuery(t *testing.T) { b, ok := blk.(*block) require.True(t, ok) - h1 := NewMockOnIndexSeries(ctrl) + h1 := doc.NewMockOnIndexSeries(ctrl) h1.EXPECT().OnIndexFinalize(blockStart) h1.EXPECT().OnIndexSuccess(blockStart) h1.EXPECT().IndexedForBlockStart(blockStart). Return(true). AnyTimes() - h2 := NewMockOnIndexSeries(ctrl) + h2 := doc.NewMockOnIndexSeries(ctrl) h2.EXPECT().OnIndexFinalize(blockStart) h2.EXPECT().OnIndexSuccess(blockStart) h2.EXPECT().IndexedForBlockStart(blockStart). @@ -1650,7 +1650,7 @@ func TestBlockE2EInsertAddResultsMergeQuery(t *testing.T) { b, ok := blk.(*block) require.True(t, ok) - h1 := NewMockOnIndexSeries(ctrl) + h1 := doc.NewMockOnIndexSeries(ctrl) h1.EXPECT().OnIndexFinalize(blockStart) h1.EXPECT().OnIndexSuccess(blockStart) h1.EXPECT().IndexedForBlockStart(blockStart). @@ -1754,11 +1754,11 @@ func TestBlockWriteBackgroundCompact(t *testing.T) { b.mutableSegments.compact.compactingBackgroundGarbageCollect = true // First write - h1 := NewMockOnIndexSeries(ctrl) + h1 := doc.NewMockOnIndexSeries(ctrl) h1.EXPECT().OnIndexFinalize(blockStart) h1.EXPECT().OnIndexSuccess(blockStart) - h2 := NewMockOnIndexSeries(ctrl) + h2 := doc.NewMockOnIndexSeries(ctrl) h2.EXPECT().OnIndexFinalize(blockStart) h2.EXPECT().OnIndexSuccess(blockStart) @@ -1787,7 +1787,7 @@ func TestBlockWriteBackgroundCompact(t *testing.T) { b.Unlock() // Second write - h1 = NewMockOnIndexSeries(ctrl) + h1 = doc.NewMockOnIndexSeries(ctrl) h1.EXPECT().OnIndexFinalize(blockStart) h1.EXPECT().OnIndexSuccess(blockStart) @@ -2184,15 +2184,15 @@ func TestBlockE2EInsertAggregate(t *testing.T) { b, ok := blk.(*block) require.True(t, ok) - h1 := NewMockOnIndexSeries(ctrl) + h1 := doc.NewMockOnIndexSeries(ctrl) h1.EXPECT().OnIndexFinalize(blockStart) h1.EXPECT().OnIndexSuccess(blockStart) - h2 := NewMockOnIndexSeries(ctrl) + h2 := doc.NewMockOnIndexSeries(ctrl) h2.EXPECT().OnIndexFinalize(blockStart) h2.EXPECT().OnIndexSuccess(blockStart) - h3 := NewMockOnIndexSeries(ctrl) + h3 := doc.NewMockOnIndexSeries(ctrl) h3.EXPECT().OnIndexFinalize(blockStart) h3.EXPECT().OnIndexSuccess(blockStart) diff --git a/src/dbnode/storage/index/for_each_test.go b/src/dbnode/storage/index/for_each_test.go index b2dc532922..94e042ab0f 100644 --- a/src/dbnode/storage/index/for_each_test.go +++ b/src/dbnode/storage/index/for_each_test.go @@ -53,7 +53,7 @@ func TestWriteBatchForEachUnmarkedBatchByBlockStart(t *testing.T) { for _, n := range []int64{2, 0, 1} { batch.Append(WriteBatchEntry{ Timestamp: tn(n), - OnIndexSeries: NewMockOnIndexSeries(ctrl), + OnIndexSeries: doc.NewMockOnIndexSeries(ctrl), }, d(n)) } @@ -109,7 +109,7 @@ func TestWriteBatchForEachUnmarkedBatchByBlockStartMore(t *testing.T) { } { batch.Append(WriteBatchEntry{ Timestamp: tn(v.nTime), - OnIndexSeries: NewMockOnIndexSeries(ctrl), + OnIndexSeries: doc.NewMockOnIndexSeries(ctrl), }, d(v.nDoc)) } diff --git a/src/dbnode/storage/index/write_batch_test.go b/src/dbnode/storage/index/write_batch_test.go index 1f7edef18a..323896e3f5 100644 --- a/src/dbnode/storage/index/write_batch_test.go +++ b/src/dbnode/storage/index/write_batch_test.go @@ -28,6 +28,7 @@ import ( "github.com/stretchr/testify/require" "github.com/golang/mock/gomock" + "github.com/m3db/m3/src/m3ninx/doc" xtime "github.com/m3db/m3/src/x/time" ) @@ -44,14 +45,14 @@ func TestWriteBatchSortByUnmarkedAndIndexBlockStart(t *testing.T) { Truncate(blockSize). Add(time.Minute) - h1 := NewMockOnIndexSeries(ctrl) + h1 := doc.NewMockOnIndexSeries(ctrl) h1.EXPECT().OnIndexFinalize(blockStart) h1.EXPECT().OnIndexSuccess(blockStart) - h2 := NewMockOnIndexSeries(ctrl) + h2 := doc.NewMockOnIndexSeries(ctrl) h2.EXPECT().OnIndexFinalize(blockStart) - h3 := NewMockOnIndexSeries(ctrl) + h3 := doc.NewMockOnIndexSeries(ctrl) h3.EXPECT().OnIndexFinalize(blockStart) h3.EXPECT().OnIndexSuccess(blockStart) diff --git a/src/dbnode/storage/index_block_test.go b/src/dbnode/storage/index_block_test.go index 1cf3d175fb..6b6029b410 100644 --- a/src/dbnode/storage/index_block_test.go +++ b/src/dbnode/storage/index_block_test.go @@ -256,7 +256,7 @@ func TestNamespaceIndexWrite(t *testing.T) { id := ident.StringID("foo") tag := ident.StringTag("name", "value") tags := ident.NewTags(tag) - lifecycle := index.NewMockOnIndexSeries(ctrl) + lifecycle := doc.NewMockOnIndexSeries(ctrl) mockWriteBatch(t, &now, lifecycle, mockBlock, &tag) lifecycle.EXPECT().IfAlreadyIndexedMarkIndexSuccessAndFinalize(gomock.Any()).Return(false) batch := index.NewWriteBatch(index.WriteBatchOptions{ @@ -325,7 +325,7 @@ func TestNamespaceIndexWriteCreatesBlock(t *testing.T) { id := ident.StringID("foo") tag := ident.StringTag("name", "value") tags := ident.NewTags(tag) - lifecycle := index.NewMockOnIndexSeries(ctrl) + lifecycle := doc.NewMockOnIndexSeries(ctrl) mockWriteBatch(t, &now, lifecycle, bActive, &tag) lifecycle.EXPECT().IfAlreadyIndexedMarkIndexSuccessAndFinalize(gomock.Any()). Return(false). @@ -1595,7 +1595,7 @@ func TestNamespaceIndexBlockAggregateQueryAggPath(t *testing.T) { func mockWriteBatch(t *testing.T, now *xtime.UnixNano, - lifecycle *index.MockOnIndexSeries, + lifecycle *doc.MockOnIndexSeries, block *index.MockBlock, tag *ident.Tag, ) { diff --git a/src/dbnode/storage/index_insert_queue_test.go b/src/dbnode/storage/index_insert_queue_test.go index b52aac8052..b69a3005c3 100644 --- a/src/dbnode/storage/index_insert_queue_test.go +++ b/src/dbnode/storage/index_insert_queue_test.go @@ -29,6 +29,7 @@ import ( "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/storage/index" + "github.com/m3db/m3/src/m3ninx/doc" "github.com/m3db/m3/src/x/ident" xtest "github.com/m3db/m3/src/x/test" xtime "github.com/m3db/m3/src/x/time" @@ -89,7 +90,7 @@ func TestIndexInsertQueueCallback(t *testing.T) { q = newTestIndexInsertQueue(newTestNamespaceMetadata(t)) insertLock sync.Mutex insertedBatches []*index.WriteBatch - callback = index.NewMockOnIndexSeries(ctrl) + callback = doc.NewMockOnIndexSeries(ctrl) ) q.indexBatchFn = func(inserts *index.WriteBatch) { insertLock.Lock() @@ -150,7 +151,7 @@ func TestIndexInsertQueueBatchBackoff(t *testing.T) { } q.indexBatchBackoff = backoff - callback := index.NewMockOnIndexSeries(ctrl) + callback := doc.NewMockOnIndexSeries(ctrl) var slept time.Duration var numSleeps int diff --git a/src/dbnode/storage/index_query_concurrent_test.go b/src/dbnode/storage/index_query_concurrent_test.go index fdae6c74f8..49048eabc6 100644 --- a/src/dbnode/storage/index_query_concurrent_test.go +++ b/src/dbnode/storage/index_query_concurrent_test.go @@ -162,7 +162,7 @@ func testNamespaceIndexHighConcurrentQueries( var onIndexWg sync.WaitGroup onIndexWg.Add(idsPerBlock) - onIndexSeries := index.NewMockOnIndexSeries(ctrl) + onIndexSeries := doc.NewMockOnIndexSeries(ctrl) onIndexSeries.EXPECT(). OnIndexSuccess(gomock.Any()). Times(idsPerBlock). diff --git a/src/dbnode/storage/index_queue_forward_write_test.go b/src/dbnode/storage/index_queue_forward_write_test.go index 98cf0a5446..ea1138fdca 100644 --- a/src/dbnode/storage/index_queue_forward_write_test.go +++ b/src/dbnode/storage/index_queue_forward_write_test.go @@ -107,7 +107,7 @@ func setupForwardIndex( tags = ident.NewTags( ident.StringTag("name", "value"), ) - lifecycle = index.NewMockOnIndexSeries(ctrl) + lifecycle = doc.NewMockOnIndexSeries(ctrl) ) gomock.InOrder( @@ -377,7 +377,7 @@ func TestNamespaceIndexForwardWrite(t *testing.T) { id := ident.StringID("foo") tag := ident.StringTag("name", "value") tags := ident.NewTags(tag) - lifecycle := index.NewMockOnIndexSeries(ctrl) + lifecycle := doc.NewMockOnIndexSeries(ctrl) var ( ts = idx.(*nsIndex).state.latestBlock.StartTime() @@ -420,7 +420,7 @@ func TestNamespaceIndexForwardWriteCreatesBlock(t *testing.T) { id := ident.StringID("foo") tag := ident.StringTag("name", "value") tags := ident.NewTags(tag) - lifecycle := index.NewMockOnIndexSeries(ctrl) + lifecycle := doc.NewMockOnIndexSeries(ctrl) var ( ts = idx.(*nsIndex).state.latestBlock.StartTime() @@ -598,7 +598,7 @@ func testShardForwardWriteTaggedSyncRefCount( // ensure all entries have no references left for _, id := range []string{"foo", "bar", "baz"} { shard.Lock() - entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) + entry, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() require.NoError(t, err) require.Equal(t, int32(0), entry.ReaderWriterCount(), id) @@ -614,7 +614,7 @@ func testShardForwardWriteTaggedSyncRefCount( // // ensure all entries have no references left for _, id := range []string{"foo", "bar", "baz"} { shard.Lock() - entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) + entry, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() require.NoError(t, err) require.Equal(t, int32(0), entry.ReaderWriterCount(), id) @@ -658,7 +658,7 @@ func testShardForwardWriteTaggedAsyncRefCount( // ensure all entries have no references left for _, id := range []string{"foo", "bar", "baz"} { shard.Lock() - entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) + entry, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() require.NoError(t, err) require.Equal(t, int32(0), entry.ReaderWriterCount(), id) @@ -673,7 +673,7 @@ func testShardForwardWriteTaggedAsyncRefCount( // ensure all entries have no references left for _, id := range []string{"foo", "bar", "baz"} { shard.Lock() - entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) + entry, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() require.NoError(t, err) require.Equal(t, int32(0), entry.ReaderWriterCount(), id) diff --git a/src/dbnode/storage/index_queue_test.go b/src/dbnode/storage/index_queue_test.go index ea181e61a6..cd8790ae20 100644 --- a/src/dbnode/storage/index_queue_test.go +++ b/src/dbnode/storage/index_queue_test.go @@ -145,7 +145,7 @@ func TestNamespaceIndexWriteAfterClose(t *testing.T) { now := xtime.Now() - lifecycle := index.NewMockOnIndexSeries(ctrl) + lifecycle := doc.NewMockOnIndexSeries(ctrl) lifecycle.EXPECT().OnIndexFinalize(now.Truncate(idx.blockSize)) lifecycle.EXPECT().IfAlreadyIndexedMarkIndexSuccessAndFinalize(gomock.Any()). Return(false). @@ -169,7 +169,7 @@ func TestNamespaceIndexWriteQueueError(t *testing.T) { ) n := xtime.Now() - lifecycle := index.NewMockOnIndexSeries(ctrl) + lifecycle := doc.NewMockOnIndexSeries(ctrl) lifecycle.EXPECT().OnIndexFinalize(n.Truncate(idx.blockSize)) lifecycle.EXPECT().IfAlreadyIndexedMarkIndexSuccessAndFinalize(gomock.Any()).Return(false) q.EXPECT(). @@ -213,7 +213,7 @@ func TestNamespaceIndexInsertOlderThanRetentionPeriod(t *testing.T) { tags = ident.NewTags( ident.StringTag("name", "value"), ) - lifecycle = index.NewMockOnIndexSeries(ctrl) + lifecycle = doc.NewMockOnIndexSeries(ctrl) ) tooOld := now.Add(-1 * idx.bufferPast).Add(-1 * time.Second) @@ -279,7 +279,7 @@ func TestNamespaceIndexInsertQueueInteraction(t *testing.T) { now := xtime.Now() var wg sync.WaitGroup - lifecycle := index.NewMockOnIndexSeries(ctrl) + lifecycle := doc.NewMockOnIndexSeries(ctrl) q.EXPECT().InsertBatch(gomock.Any()).Return(&wg, nil) lifecycle.EXPECT().IfAlreadyIndexedMarkIndexSuccessAndFinalize(gomock.Any()). Return(false). @@ -319,7 +319,7 @@ func setupIndex(t *testing.T, tags = ident.NewTags( ident.StringTag("name", "value"), ) - lifecycleFns = index.NewMockOnIndexSeries(ctrl) + lifecycleFns = doc.NewMockOnIndexSeries(ctrl) ) lifecycleFns.EXPECT().OnIndexFinalize(ts) diff --git a/src/dbnode/storage/shard_ref_count_test.go b/src/dbnode/storage/shard_ref_count_test.go index 74cbffac9d..f0bfb49edc 100644 --- a/src/dbnode/storage/shard_ref_count_test.go +++ b/src/dbnode/storage/shard_ref_count_test.go @@ -90,7 +90,7 @@ func testShardWriteSyncRefCount(t *testing.T, opts Options) { // ensure all entries have no references left for _, id := range []string{"foo", "bar", "baz"} { shard.Lock() - entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) + entry, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() assert.NoError(t, err) assert.Equal(t, int32(0), entry.ReaderWriterCount(), id) @@ -114,7 +114,7 @@ func testShardWriteSyncRefCount(t *testing.T, opts Options) { // ensure all entries have no references left for _, id := range []string{"foo", "bar", "baz"} { shard.Lock() - entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) + entry, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() assert.NoError(t, err) assert.Equal(t, int32(0), entry.ReaderWriterCount(), id) @@ -213,7 +213,7 @@ func testShardWriteTaggedSyncRefCount(t *testing.T, idx NamespaceIndex) { // ensure all entries have no references left for _, id := range []string{"foo", "bar", "baz"} { shard.Lock() - entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) + entry, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() assert.NoError(t, err) assert.Equal(t, int32(0), entry.ReaderWriterCount(), id) @@ -240,7 +240,7 @@ func testShardWriteTaggedSyncRefCount(t *testing.T, idx NamespaceIndex) { // ensure all entries have no references left for _, id := range []string{"foo", "bar", "baz"} { shard.Lock() - entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) + entry, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() assert.NoError(t, err) assert.Equal(t, int32(0), entry.ReaderWriterCount(), id) @@ -293,7 +293,7 @@ func TestShardWriteAsyncRefCount(t *testing.T) { // ensure all entries have no references left for _, id := range []string{"foo", "bar", "baz"} { shard.Lock() - entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) + entry, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() assert.NoError(t, err) assert.Equal(t, int32(0), entry.ReaderWriterCount(), id) @@ -317,7 +317,7 @@ func TestShardWriteAsyncRefCount(t *testing.T) { // ensure all entries have no references left for _, id := range []string{"foo", "bar", "baz"} { shard.Lock() - entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) + entry, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() assert.NoError(t, err) assert.Equal(t, int32(0), entry.ReaderWriterCount(), id) @@ -459,7 +459,7 @@ func testShardWriteTaggedAsyncRefCount(t *testing.T, idx NamespaceIndex, nowFn f // ensure all entries have no references left for _, id := range []string{"foo", "bar", "baz"} { shard.Lock() - entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) + entry, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() assert.NoError(t, err) assert.Equal(t, int32(0), entry.ReaderWriterCount(), id) @@ -486,7 +486,7 @@ func testShardWriteTaggedAsyncRefCount(t *testing.T, idx NamespaceIndex, nowFn f // ensure all entries have no references left for _, id := range []string{"foo", "bar", "baz"} { shard.Lock() - entry, _, err := shard.lookupEntryWithLock(ident.StringID(id)) + entry, err := shard.lookupEntryWithLock(ident.StringID(id)) shard.Unlock() assert.NoError(t, err) assert.Equal(t, int32(0), entry.ReaderWriterCount(), id) diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index b071449d72..e1756a1fcb 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -1698,7 +1698,7 @@ func TestShardFetchIndexChecksum(t *testing.T) { time.Sleep(time.Second) shard.RLock() - entry, _, err := shard.lookupEntryWithLock(ident.StringID("foo")) + entry, err := shard.lookupEntryWithLock(ident.StringID("foo")) shard.RUnlock() require.Equal(t, err, errShardEntryNotFound) @@ -1792,7 +1792,7 @@ func TestShardReadEncodedCachesSeriesWithRecentlyReadPolicy(t *testing.T) { begin := time.Now() for time.Since(begin) < 10*time.Second { shard.RLock() - entry, _, err := shard.lookupEntryWithLock(ident.StringID("foo")) + entry, err := shard.lookupEntryWithLock(ident.StringID("foo")) shard.RUnlock() if err == errShardEntryNotFound { time.Sleep(5 * time.Millisecond) @@ -1806,7 +1806,7 @@ func TestShardReadEncodedCachesSeriesWithRecentlyReadPolicy(t *testing.T) { } shard.RLock() - entry, _, err := shard.lookupEntryWithLock(ident.StringID("foo")) + entry, err := shard.lookupEntryWithLock(ident.StringID("foo")) shard.RUnlock() require.NoError(t, err) require.NotNil(t, entry) From 707027860331a21a99b3ceb40fa4a524db1f1a79 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Wed, 18 Aug 2021 15:10:50 -0400 Subject: [PATCH 31/36] Lint --- src/dbnode/storage/shard.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 0f70363720..85432294cf 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -601,8 +601,6 @@ func (s *dbShard) forEachShardEntryBatch(entriesBatchFn dbShardEntryBatchWorkFn) return } } - - return } func (s *dbShard) IsBootstrapped() bool { From 45e9a92a4e5de6c18fdf43ac5830a7c0c067bc08 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Wed, 18 Aug 2021 22:30:04 -0400 Subject: [PATCH 32/36] Split warm flush status into data and index --- src/dbnode/storage/flush.go | 116 +++++---------------------- src/dbnode/storage/flush_test.go | 16 ++-- src/dbnode/storage/fs.go | 7 +- src/dbnode/storage/index.go | 61 +++++++++----- src/dbnode/storage/index_test.go | 52 ++++++------ src/dbnode/storage/namespace.go | 33 +++----- src/dbnode/storage/namespace_test.go | 57 ++++++++----- src/dbnode/storage/shard.go | 98 ++++++++++++++++++---- src/dbnode/storage/shard_test.go | 78 +++++++++++------- src/dbnode/storage/storage_mock.go | 47 ++++++----- src/dbnode/storage/types.go | 14 ++-- 11 files changed, 311 insertions(+), 268 deletions(-) diff --git a/src/dbnode/storage/flush.go b/src/dbnode/storage/flush.go index ce9ecdf1dd..7e206ffa71 100644 --- a/src/dbnode/storage/flush.go +++ b/src/dbnode/storage/flush.go @@ -54,20 +54,6 @@ const ( flushManagerIndexFlushInProgress ) -type namespaceFlushes map[string]namespaceFlush - -type namespaceFlush struct { - namespace databaseNamespace - shardFlushes shardFlushes -} - -type shardFlushes map[shardFlushKey]databaseShard - -type shardFlushKey struct { - shardID uint32 - blockStart xtime.UnixNano -} - type flushManagerMetrics struct { isFlushing tally.Gauge isSnapshotting tally.Gauge @@ -158,8 +144,7 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { // will attempt to snapshot blocks w/ unflushed data which would be wasteful if // the block is already flushable. multiErr := xerrors.NewMultiError() - dataFlushes, err := m.dataWarmFlush(namespaces, startTime) - if err != nil { + if err := m.dataWarmFlush(namespaces, startTime); err != nil { multiErr = multiErr.Add(err) } @@ -174,69 +159,26 @@ func (m *flushManager) Flush(startTime xtime.UnixNano) error { multiErr = multiErr.Add(fmt.Errorf("error rotating commitlog in mediator tick: %v", err)) } - indexFlushes, err := m.indexFlush(namespaces) - if err != nil { + if err := m.indexFlush(namespaces); err != nil { multiErr = multiErr.Add(err) } - err = multiErr.FinalError() - - // Mark all flushed shards as such. - // If index is not enabled, then a shard+blockStart is "flushed" if the data has been flushed. - // If index is enabled, then a shard+blockStart is "flushed" if the data AND index has been flushed. - for _, n := range namespaces { - var ( - indexEnabled = n.Options().IndexOptions().Enabled() - flushed shardFlushes - ) - if indexEnabled { - flushesForNs, ok := indexFlushes[n.ID().String()] - if !ok { - continue - } - flushed = flushesForNs.shardFlushes - } else { - flushesForNs, ok := dataFlushes[n.ID().String()] - if !ok { - continue - } - flushed = flushesForNs.shardFlushes - } - - for k, v := range flushed { - // Block sizes for data and index can differ and so if we are driving the flushing by - // the index blockStarts, we must expand them to mark all containing data blockStarts. - // E.g. if blockSize == 2h and indexBlockSize == 4h and the flushed index time is 6:00pm, - // we should mark as flushed [6:00pm, 8:00pm]. - if indexEnabled { - blockSize := n.Options().RetentionOptions().BlockSize() - indexBlockSize := n.Options().IndexOptions().BlockSize() - for start := k.blockStart; start < k.blockStart.Add(indexBlockSize); start = start.Add(blockSize) { - v.MarkWarmFlushStateSuccessOrError(start, err) - } - } else { - v.MarkWarmFlushStateSuccessOrError(k.blockStart, err) - } - } - } - - return err + return multiErr.FinalError() } func (m *flushManager) dataWarmFlush( namespaces []databaseNamespace, startTime xtime.UnixNano, -) (namespaceFlushes, error) { +) error { flushPersist, err := m.pm.StartFlushPersist() if err != nil { - return nil, err + return err } m.setState(flushManagerFlushInProgress) var ( - start = m.nowFn() - multiErr = xerrors.NewMultiError() - allFlushes = make(map[string]namespaceFlush) + start = m.nowFn() + multiErr = xerrors.NewMultiError() ) for _, ns := range namespaces { // Flush first because we will only snapshot if there are no outstanding flushes. @@ -245,11 +187,9 @@ func (m *flushManager) dataWarmFlush( multiErr = multiErr.Add(err) continue } - flush, err := m.flushNamespaceWithTimes(ns, flushTimes, flushPersist) - if err != nil { + if err := m.flushNamespaceWithTimes(ns, flushTimes, flushPersist); err != nil { multiErr = multiErr.Add(err) } - allFlushes[ns.ID().String()] = flush } err = flushPersist.DoneFlush() @@ -258,7 +198,7 @@ func (m *flushManager) dataWarmFlush( } m.metrics.dataWarmFlushDuration.Record(m.nowFn().Sub(start)) - return allFlushes, multiErr.FinalError() + return multiErr.FinalError() } func (m *flushManager) dataSnapshot( @@ -312,17 +252,16 @@ func (m *flushManager) dataSnapshot( func (m *flushManager) indexFlush( namespaces []databaseNamespace, -) (namespaceFlushes, error) { +) error { indexFlush, err := m.pm.StartIndexPersist() if err != nil { - return nil, err + return err } m.setState(flushManagerIndexFlushInProgress) var ( - start = m.nowFn() - multiErr = xerrors.NewMultiError() - namespaceFlushes = make(map[string]namespaceFlush) + start = m.nowFn() + multiErr = xerrors.NewMultiError() ) for _, ns := range namespaces { var ( @@ -333,20 +272,14 @@ func (m *flushManager) indexFlush( continue } - flushes, err := ns.FlushIndex(indexFlush) - if err != nil { + if err := ns.FlushIndex(indexFlush); err != nil { multiErr = multiErr.Add(err) - } else { - namespaceFlushes[ns.ID().String()] = namespaceFlush{ - namespace: ns, - shardFlushes: flushes, - } } } multiErr = multiErr.Add(indexFlush.DoneIndex()) m.metrics.indexFlushDuration.Record(m.nowFn().Sub(start)) - return namespaceFlushes, multiErr.FinalError() + return multiErr.FinalError() } func (m *flushManager) Report() { @@ -430,31 +363,18 @@ func (m *flushManager) flushNamespaceWithTimes( ns databaseNamespace, times []xtime.UnixNano, flushPreparer persist.FlushPreparer, -) (namespaceFlush, error) { - flushes := make(shardFlushes) +) error { multiErr := xerrors.NewMultiError() for _, t := range times { // NB(xichen): we still want to proceed if a namespace fails to flush its data. // Probably want to emit a counter here, but for now just log it. - shards, err := ns.WarmFlush(t, flushPreparer) - if err != nil { + if err := ns.WarmFlush(t, flushPreparer); err != nil { detailedErr := fmt.Errorf("namespace %s failed to flush data: %v", ns.ID().String(), err) multiErr = multiErr.Add(detailedErr) - continue - } - - for _, s := range shards { - flushes[shardFlushKey{ - shardID: s.ID(), - blockStart: t, - }] = s } } - return namespaceFlush{ - namespace: ns, - shardFlushes: flushes, - }, multiErr.FinalError() + return multiErr.FinalError() } func (m *flushManager) LastSuccessfulSnapshotStartTime() (xtime.UnixNano, bool) { diff --git a/src/dbnode/storage/flush_test.go b/src/dbnode/storage/flush_test.go index 3e232e166d..a5249bad05 100644 --- a/src/dbnode/storage/flush_test.go +++ b/src/dbnode/storage/flush_test.go @@ -321,12 +321,10 @@ func TestFlushManagerSkipNamespaceIndexingDisabled(t *testing.T) { ns.EXPECT().Options().Return(nsOpts).AnyTimes() ns.EXPECT().ID().Return(defaultTestNs1ID).AnyTimes() ns.EXPECT().NeedsFlush(gomock.Any(), gomock.Any()).Return(true, nil).AnyTimes() - ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any()).Return([]databaseShard{s1, s2}, nil).AnyTimes() + ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any()).Return(nil).AnyTimes() ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() s1.EXPECT().ID().Return(uint32(1)).AnyTimes() s2.EXPECT().ID().Return(uint32(2)).AnyTimes() - s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil).AnyTimes() - s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil).AnyTimes() var ( mockFlushPersist = persist.NewMockFlushPreparer(ctrl) @@ -377,17 +375,13 @@ func TestFlushManagerNamespaceIndexingEnabled(t *testing.T) { // Validate that the flush state is marked as successful only AFTER all prequisite steps have been run. // Order is important to avoid any edge case where data is GCed from memory without all flushing operations // being completed. - mockFlushedShards := shardFlushes{ - shardFlushKey{shardID: s1.ID(), blockStart: xtime.Now().Add(time.Minute * 1)}: s1, - shardFlushKey{shardID: s2.ID(), blockStart: xtime.Now().Add(time.Minute * 1)}: s2, - } steps := make([]*gomock.Call, 0) steps = append(steps, - ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any()).Return([]databaseShard{s1, s2}, nil).Times(blocks), + ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any()).Return(nil).Times(blocks), ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes(), - ns.EXPECT().FlushIndex(gomock.Any()).Return(mockFlushedShards, nil), - s1.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil), - s2.EXPECT().MarkWarmFlushStateSuccessOrError(gomock.Any(), nil), + ns.EXPECT().FlushIndex(gomock.Any()).Return(nil), + s1.EXPECT().MarkWarmIndexFlushStateSuccessOrError(gomock.Any(), nil), + s2.EXPECT().MarkWarmIndexFlushStateSuccessOrError(gomock.Any(), nil), ) gomock.InOrder(steps...) diff --git a/src/dbnode/storage/fs.go b/src/dbnode/storage/fs.go index c6f99366df..7c247b0a65 100644 --- a/src/dbnode/storage/fs.go +++ b/src/dbnode/storage/fs.go @@ -39,12 +39,17 @@ const ( fileOpFailed ) +type warmStatus struct { + DataFlushed fileOpStatus + IndexFlushed fileOpStatus +} + type fileOpState struct { // WarmStatus is the status of data persistence for WarmWrites only. // Each block will only be warm-flushed once, so not keeping track of a // version here is okay. This is used in the buffer Tick to determine when // a warm bucket is evictable from memory. - WarmStatus fileOpStatus + WarmStatus warmStatus // ColdVersionRetrievable keeps track of data persistence for ColdWrites only. // Each block can be cold-flushed multiple times, so this tracks which // version of the flush completed successfully. This is ultimately used in diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 44ab1aad89..6cba8bf860 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -1007,15 +1007,15 @@ func (i *nsIndex) tickingBlocks( func (i *nsIndex) WarmFlush( flush persist.IndexFlush, shards []databaseShard, -) (shardFlushes, error) { +) error { if len(shards) == 0 { // No-op if no shards currently owned. - return nil, nil + return nil } flushable, err := i.flushableBlocks(shards, series.WarmWrite) if err != nil { - return nil, err + return err } // Determine the current flush indexing concurrency. @@ -1029,7 +1029,7 @@ func (i *nsIndex) WarmFlush( builder, err := builder.NewBuilderFromDocuments(builderOpts) if err != nil { - return nil, err + return err } defer builder.Close() @@ -1039,11 +1039,10 @@ func (i *nsIndex) WarmFlush( defer i.metrics.flushIndexingConcurrency.Update(0) var evicted int - flushes := make(shardFlushes) for _, block := range flushable { immutableSegments, err := i.flushBlock(flush, block, shards, builder) if err != nil { - return nil, err + return err } // Make a result that covers the entire time ranges for the // block for each shard @@ -1060,7 +1059,7 @@ func (i *nsIndex) WarmFlush( results := result.NewIndexBlockByVolumeType(block.StartTime()) results.SetBlock(idxpersist.DefaultIndexVolumeType, blockResult) if err := block.AddResults(results); err != nil { - return nil, err + return err } evicted++ @@ -1074,18 +1073,16 @@ func (i *nsIndex) WarmFlush( zap.Error(err), zap.Time("blockStart", block.StartTime().ToTime()), ) - continue } - for _, s := range shards { - flushes[shardFlushKey{ - shardID: s.ID(), - blockStart: block.StartTime(), - }] = s + for _, t := range i.blockStartsFromIndexBlockStart(block.StartTime()) { + for _, s := range shards { + s.MarkWarmIndexFlushStateSuccessOrError(t, err) + } } } i.metrics.blocksEvictedMutableSegments.Inc(int64(evicted)) - return flushes, nil + return nil } func (i *nsIndex) ColdFlush(shards []databaseShard) (OnColdFlushDone, error) { @@ -1115,6 +1112,18 @@ func (i *nsIndex) ColdFlush(shards []databaseShard) (OnColdFlushDone, error) { }, nil } +// WarmFlushedBlockStarts returns all index blockStarts which have been flushed to disk. +func (i *nsIndex) WarmFlushedBlockStarts() []xtime.UnixNano { + flushed := make([]xtime.UnixNano, 0) + infoFiles := i.readInfoFilesAsMap() + for blockStart := range infoFiles { + if i.hasIndexWarmFlushedToDisk(infoFiles, blockStart) { + flushed = append(flushed, blockStart) + } + } + return flushed +} + func (i *nsIndex) readInfoFilesAsMap() map[xtime.UnixNano]fs.ReadIndexInfoFileResult { fsOpts := i.opts.CommitLogOptions().FilesystemOptions() infoFiles := i.readIndexInfoFilesFn(fs.ReadIndexInfoFilesOptions{ @@ -1198,18 +1207,15 @@ func (i *nsIndex) canFlushBlockWithRLock( Debug("skipping index cold flush due to shard not bootstrapped yet") continue } - start := blockStart - end := blockStart.Add(i.blockSize) - dataBlockSize := i.nsMetadata.Options().RetentionOptions().BlockSize() - for t := start; t.Before(end); t = t.Add(dataBlockSize) { + + for _, t := range i.blockStartsFromIndexBlockStart(blockStart) { flushState, err := shard.FlushState(t) if err != nil { return false, err } - // Skip if the data flushing failed. We mark as "success" only once both - // data and index are flushed. - if flushState.WarmStatus == fileOpFailed { + // Skip if the data flushing failed. Data flushing precedes index flushing. + if flushState.WarmStatus.DataFlushed != fileOpSuccess { return false, nil } } @@ -1218,6 +1224,19 @@ func (i *nsIndex) canFlushBlockWithRLock( return true, nil } +// blockStartsFromIndexBlockStart returns the possibly many blocksStarts that exist within +// a given index block (since index block size >= data block size) +func (i *nsIndex) blockStartsFromIndexBlockStart(blockStart xtime.UnixNano) []xtime.UnixNano { + start := blockStart + end := blockStart.Add(i.blockSize) + dataBlockSize := i.nsMetadata.Options().RetentionOptions().BlockSize() + blockStarts := make([]xtime.UnixNano, 0) + for t := start; t.Before(end); t = t.Add(dataBlockSize) { + blockStarts = append(blockStarts, t) + } + return blockStarts +} + func (i *nsIndex) hasIndexWarmFlushedToDisk( infoFiles map[xtime.UnixNano]fs.ReadIndexInfoFileResult, blockStart xtime.UnixNano, diff --git a/src/dbnode/storage/index_test.go b/src/dbnode/storage/index_test.go index b0d52faffc..769466b2b7 100644 --- a/src/dbnode/storage/index_test.go +++ b/src/dbnode/storage/index_test.go @@ -644,14 +644,15 @@ func TestNamespaceIndexFlushShardStateNotSuccess(t *testing.T) { mockShard := NewMockdatabaseShard(ctrl) mockShard.EXPECT().IsBootstrapped().Return(true).AnyTimes() mockShard.EXPECT().ID().Return(uint32(0)).AnyTimes() - mockShard.EXPECT().FlushState(gomock.Any()).Return(fileOpState{WarmStatus: fileOpFailed}, nil).AnyTimes() + mockShard.EXPECT().FlushState(gomock.Any()).Return(fileOpState{WarmStatus: warmStatus{ + IndexFlushed: fileOpFailed, + }}, nil).AnyTimes() shards := []databaseShard{mockShard} mockFlush := persist.NewMockIndexFlush(ctrl) - flushed, err := idx.WarmFlush(mockFlush, shards) + err := idx.WarmFlush(mockFlush, shards) require.NoError(t, err) - require.Equal(t, shardFlushes{}, flushed) } func TestNamespaceIndexQueryNoMatchingBlocks(t *testing.T) { @@ -813,7 +814,9 @@ func TestNamespaceIndexFlushSkipBootstrappingShards(t *testing.T) { mockShard.EXPECT().IsBootstrapped().Return(shardInfo.isBootstrapped).AnyTimes() mockShard.EXPECT().ID().Return(shardInfo.id).AnyTimes() if shardInfo.isBootstrapped { - mockShard.EXPECT().FlushState(gomock.Any()).Return(fileOpState{WarmStatus: fileOpSuccess}, nil).AnyTimes() + mockShard.EXPECT().FlushState(gomock.Any()).Return(fileOpState{WarmStatus: warmStatus{ + IndexFlushed: fileOpSuccess, + }}, nil).AnyTimes() } shards = append(shards, mockShard) } @@ -831,18 +834,17 @@ func verifyFlushForShards( shards []uint32, ) { var ( - mockFlush = persist.NewMockIndexFlush(ctrl) - shardMap = make(map[uint32]struct{}) - now = xtime.Now() - warmBlockStart = now.Add(-idx.bufferPast).Truncate(idx.blockSize) - mockShards []*MockdatabaseShard - dbShards []databaseShard - numBlocks int - persistClosedTimes int - persistCalledTimes int - actualDocs = make([]doc.Metadata, 0) - expectedDocs = make([]doc.Metadata, 0) - expectedShardFlushes = make(shardFlushes) + mockFlush = persist.NewMockIndexFlush(ctrl) + shardMap = make(map[uint32]struct{}) + now = xtime.Now() + warmBlockStart = now.Add(-idx.bufferPast).Truncate(idx.blockSize) + mockShards []*MockdatabaseShard + dbShards []databaseShard + numBlocks int + persistClosedTimes int + persistCalledTimes int + actualDocs = make([]doc.Metadata, 0) + expectedDocs = make([]doc.Metadata, 0) ) // NB(bodu): Always align now w/ the index's view of now. idx.nowFn = func() time.Time { @@ -905,8 +907,12 @@ func verifyFlushForShards( for _, mockShard := range mockShards { mockShard.EXPECT().IsBootstrapped().Return(true) - mockShard.EXPECT().FlushState(blockStart).Return(fileOpState{WarmStatus: fileOpSuccess}, nil) - mockShard.EXPECT().FlushState(blockStart.Add(blockSize)).Return(fileOpState{WarmStatus: fileOpSuccess}, nil) + mockShard.EXPECT().FlushState(blockStart).Return(fileOpState{WarmStatus: warmStatus{ + IndexFlushed: fileOpSuccess, + }}, nil) + mockShard.EXPECT().FlushState(blockStart.Add(blockSize)).Return(fileOpState{WarmStatus: warmStatus{ + IndexFlushed: fileOpSuccess, + }}, nil) resultsTags1 := ident.NewTagsIterator(ident.NewTags()) resultsTags2 := ident.NewTagsIterator(ident.NewTags()) @@ -928,25 +934,17 @@ func verifyFlushForShards( mockShard.EXPECT().FetchBlocksMetadataV2(gomock.Any(), blockStart, blockStart.Add(idx.blockSize), gomock.Any(), gomock.Any(), block.FetchBlocksMetadataOptions{OnlyDisk: true}).Return(results, nil, nil) - - expectedShardFlushes[shardFlushKey{shardID: mockShard.ID(), blockStart: blockStart}] = mockShard } mockBlock.EXPECT().IsSealed().Return(true) mockBlock.EXPECT().AddResults(gomock.Any()).Return(nil) mockBlock.EXPECT().EvictMutableSegments().Return(nil) } - flushed, err := idx.WarmFlush(mockFlush, dbShards) + err := idx.WarmFlush(mockFlush, dbShards) require.NoError(t, err) - require.Equal(t, len(expectedShardFlushes), len(flushed)) require.Equal(t, numBlocks, persistClosedTimes) require.Equal(t, numBlocks, persistCalledTimes) require.Equal(t, expectedDocs, actualDocs) - - for flushedShard := range flushed { - _, ok := expectedShardFlushes[flushedShard] - require.Equal(t, true, ok) - } } func newReadIndexInfoFileResult( diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index 3920621219..3a57c4104f 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -1181,7 +1181,7 @@ func (n *dbNamespace) Bootstrap( func (n *dbNamespace) WarmFlush( blockStart xtime.UnixNano, flushPersist persist.FlushPreparer, -) ([]databaseShard, error) { +) error { // NB(rartoul): This value can be used for emitting metrics, but should not be used // for business logic. callStart := n.nowFn() @@ -1190,25 +1190,24 @@ func (n *dbNamespace) WarmFlush( if n.bootstrapState != Bootstrapped { n.RUnlock() n.metrics.flushWarmData.ReportError(n.nowFn().Sub(callStart)) - return nil, errNamespaceNotBootstrapped + return errNamespaceNotBootstrapped } nsCtx := n.nsContextWithRLock() n.RUnlock() if n.ReadOnly() || !n.nopts.FlushEnabled() { n.metrics.flushWarmData.ReportSuccess(n.nowFn().Sub(callStart)) - return nil, nil + return nil } // check if blockStart is aligned with the namespace's retention options bs := n.nopts.RetentionOptions().BlockSize() if t := blockStart.Truncate(bs); !blockStart.Equal(t) { - return nil, fmt.Errorf("failed to flush at time %v, not aligned to blockSize", blockStart.String()) + return fmt.Errorf("failed to flush at time %v, not aligned to blockSize", blockStart.String()) } multiErr := xerrors.NewMultiError() shards := n.OwnedShards() - flushedShards := make([]databaseShard, 0) for _, shard := range shards { if !shard.IsBootstrapped() { n.log. @@ -1219,10 +1218,10 @@ func (n *dbNamespace) WarmFlush( flushState, err := shard.FlushState(blockStart) if err != nil { - return nil, err + return err } // skip flushing if the shard has already flushed data for the `blockStart` - if flushState.WarmStatus == fileOpSuccess { + if flushState.WarmStatus.DataFlushed == fileOpSuccess { continue } @@ -1232,18 +1231,12 @@ func (n *dbNamespace) WarmFlush( detailedErr := fmt.Errorf("shard %d failed to flush data: %v", shard.ID(), err) multiErr = multiErr.Add(detailedErr) - - // Errors so mark as failed here. - // We mark as success later in time to ensure that happens after index flushing as well. - shard.MarkWarmFlushStateSuccessOrError(blockStart, err) - } else { - flushedShards = append(flushedShards, shard) } } res := multiErr.FinalError() n.metrics.flushWarmData.ReportSuccessOrError(res, n.nowFn().Sub(callStart)) - return flushedShards, res + return res } // idAndBlockStart is the composite key for the genny map used to keep track of @@ -1394,25 +1387,25 @@ func (n *dbNamespace) ColdFlush(flushPersist persist.FlushPreparer) error { return res } -func (n *dbNamespace) FlushIndex(flush persist.IndexFlush) (shardFlushes, error) { +func (n *dbNamespace) FlushIndex(flush persist.IndexFlush) error { callStart := n.nowFn() n.RLock() if n.bootstrapState != Bootstrapped { n.RUnlock() n.metrics.flushIndex.ReportError(n.nowFn().Sub(callStart)) - return nil, errNamespaceNotBootstrapped + return errNamespaceNotBootstrapped } n.RUnlock() if !n.nopts.FlushEnabled() || !n.nopts.IndexOptions().Enabled() { n.metrics.flushIndex.ReportSuccess(n.nowFn().Sub(callStart)) - return nil, nil + return nil } shards := n.OwnedShards() - flushes, err := n.reverseIndex.WarmFlush(flush, shards) + err := n.reverseIndex.WarmFlush(flush, shards) n.metrics.flushIndex.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) - return flushes, err + return err } func (n *dbNamespace) Snapshot( @@ -1506,7 +1499,7 @@ func (n *dbNamespace) needsFlushWithLock( if err != nil { return false, err } - if flushState.WarmStatus != fileOpSuccess { + if flushState.WarmStatus.DataFlushed != fileOpSuccess { return true, nil } } diff --git a/src/dbnode/storage/namespace_test.go b/src/dbnode/storage/namespace_test.go index 0709661577..435b7517c7 100644 --- a/src/dbnode/storage/namespace_test.go +++ b/src/dbnode/storage/namespace_test.go @@ -607,9 +607,8 @@ func testNamespaceBootstrapUnfulfilledShards( func TestNamespaceFlushNotBootstrapped(t *testing.T) { ns, closer := newTestNamespace(t) defer closer() - flushed, err := ns.WarmFlush(xtime.Now(), nil) + err := ns.WarmFlush(xtime.Now(), nil) require.Equal(t, errNamespaceNotBootstrapped, err) - require.Equal(t, 0, len(flushed)) require.Equal(t, errNamespaceNotBootstrapped, ns.ColdFlush(nil)) } @@ -619,9 +618,8 @@ func TestNamespaceFlushDontNeedFlush(t *testing.T) { defer close() ns.bootstrapState = Bootstrapped - flushed, err := ns.WarmFlush(xtime.Now(), nil) + err := ns.WarmFlush(xtime.Now(), nil) require.NoError(t, err) - require.Equal(t, 0, len(flushed)) require.NoError(t, ns.ColdFlush(nil)) } @@ -631,9 +629,8 @@ func TestNamespaceSkipFlushIfReadOnly(t *testing.T) { ns.bootstrapState = Bootstrapped ns.SetReadOnly(true) - flushed, err := ns.WarmFlush(xtime.Now(), nil) + err := ns.WarmFlush(xtime.Now(), nil) require.NoError(t, err) - require.Equal(t, 0, len(flushed)) require.NoError(t, ns.ColdFlush(nil)) } @@ -651,22 +648,21 @@ func TestNamespaceFlushSkipFlushed(t *testing.T) { blockStart := xtime.Now().Truncate(ns.Options().RetentionOptions().BlockSize()) states := []fileOpState{ - {WarmStatus: fileOpNotStarted}, - {WarmStatus: fileOpSuccess}, + {WarmStatus: warmStatus{DataFlushed: fileOpNotStarted}}, + {WarmStatus: warmStatus{DataFlushed: fileOpSuccess}}, } for i, s := range states { shard := NewMockdatabaseShard(ctrl) shard.EXPECT().IsBootstrapped().Return(true).AnyTimes() shard.EXPECT().FlushState(blockStart).Return(s, nil) - if s.WarmStatus != fileOpSuccess { + if s.WarmStatus.DataFlushed != fileOpSuccess { shard.EXPECT().WarmFlush(blockStart, gomock.Any(), gomock.Any()).Return(nil) } ns.shards[testShardIDs[i].ID()] = shard } - flushed, err := ns.WarmFlush(blockStart, nil) + err := ns.WarmFlush(blockStart, nil) require.NoError(t, err) - require.Equal(t, 1, len(flushed)) } func TestNamespaceFlushSkipShardNotBootstrapped(t *testing.T) { @@ -687,9 +683,8 @@ func TestNamespaceFlushSkipShardNotBootstrapped(t *testing.T) { shard.EXPECT().IsBootstrapped().Return(false) ns.shards[testShardIDs[0].ID()] = shard - flushed, err := ns.WarmFlush(blockStart, nil) + err := ns.WarmFlush(blockStart, nil) require.NoError(t, err) - require.Equal(t, 0, len(flushed)) require.NoError(t, ns.ColdFlush(nil)) } @@ -1017,11 +1012,15 @@ func setShardExpects(ns *dbNamespace, ctrl *gomock.Controller, cases []needsFlus for t, needFlush := range cs.needsFlush { if needFlush { shard.EXPECT().FlushState(t).Return(fileOpState{ - WarmStatus: fileOpNotStarted, + WarmStatus: warmStatus{ + DataFlushed: fileOpNotStarted, + }, }, nil).AnyTimes() } else { shard.EXPECT().FlushState(t).Return(fileOpState{ - WarmStatus: fileOpSuccess, + WarmStatus: warmStatus{ + DataFlushed: fileOpSuccess, + }, }, nil).AnyTimes() } } @@ -1150,7 +1149,9 @@ func TestNamespaceNeedsFlushAllSuccess(t *testing.T) { shard := NewMockdatabaseShard(ctrl) shard.EXPECT().ID().Return(s.ID()).AnyTimes() shard.EXPECT().FlushState(blockStart).Return(fileOpState{ - WarmStatus: fileOpSuccess, + WarmStatus: warmStatus{ + DataFlushed: fileOpSuccess, + }, }, nil).AnyTimes() ns.shards[s.ID()] = shard } @@ -1194,15 +1195,21 @@ func TestNamespaceNeedsFlushAnyFailed(t *testing.T) { switch shard.ID() { case shards[0].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ - WarmStatus: fileOpSuccess, + WarmStatus: warmStatus{ + DataFlushed: fileOpSuccess, + }, }, nil).AnyTimes() case shards[1].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ - WarmStatus: fileOpSuccess, + WarmStatus: warmStatus{ + DataFlushed: fileOpSuccess, + }, }, nil).AnyTimes() case shards[2].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ - WarmStatus: fileOpFailed, + WarmStatus: warmStatus{ + DataFlushed: fileOpFailed, + }, NumFailures: 999, }, nil).AnyTimes() } @@ -1248,15 +1255,21 @@ func TestNamespaceNeedsFlushAnyNotStarted(t *testing.T) { switch shard.ID() { case shards[0].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ - WarmStatus: fileOpSuccess, + WarmStatus: warmStatus{ + DataFlushed: fileOpSuccess, + }, }, nil).AnyTimes() case shards[1].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ - WarmStatus: fileOpNotStarted, + WarmStatus: warmStatus{ + DataFlushed: fileOpNotStarted, + }, }, nil).AnyTimes() case shards[2].ID(): shard.EXPECT().FlushState(blockStart).Return(fileOpState{ - WarmStatus: fileOpSuccess, + WarmStatus: warmStatus{ + DataFlushed: fileOpSuccess, + }, }, nil).AnyTimes() } ns.shards[s.ID()] = shard diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 85432294cf..d3d2fd7fe9 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -375,7 +375,21 @@ func (s *dbShard) hasWarmFlushed(blockStart xtime.UnixNano) (bool, error) { if err != nil { return false, err } - return statusIsRetrievable(flushState.WarmStatus), nil + return s.warmStatusIsRetrievable(flushState.WarmStatus), nil +} + +func (s *dbShard) warmStatusIsRetrievable(status warmStatus) bool { + if !statusIsRetrievable(status.DataFlushed) { + return false + } + + // If the index is disabled, then we only are tracking data flushing. + // Otherwise, warm status requires both data and index flushed. + if !s.namespace.Options().IndexOptions().Enabled() { + return true + } + + return statusIsRetrievable(status.IndexFlushed) } func statusIsRetrievable(status fileOpStatus) bool { @@ -424,7 +438,7 @@ func (s *dbShard) blockStatesSnapshotWithRLock() series.ShardBlockStateSnapshot snapshot := make(map[xtime.UnixNano]series.BlockState, len(s.flushState.statesByTime)) for time, state := range s.flushState.statesByTime { snapshot[time] = series.BlockState{ - WarmRetrievable: statusIsRetrievable(state.WarmStatus), + WarmRetrievable: s.warmStatusIsRetrievable(state.WarmStatus), // Use ColdVersionRetrievable instead of ColdVersionFlushed since the snapshot // will be used to make eviction decisions and we don't want to evict data before // it is retrievable. @@ -1937,6 +1951,23 @@ func (s *dbShard) UpdateFlushStates() { readInfoFilesResults := fs.ReadInfoFiles(fsOpts.FilePathPrefix(), s.namespace.ID(), s.shard, fsOpts.InfoReaderBufferSize(), fsOpts.DecodingOptions(), persist.FileSetFlushType) + // TODO: use for blockSize differences + blockSize := s.namespace.Options().RetentionOptions().BlockSize() + indexBlockSize := s.namespace.Options().IndexOptions().BlockSize() + + // expose for getting all info files + indexFlushedBlockStarts := s.reverseIndex.WarmFlushedBlockStarts() + for _, blockStart := range indexFlushedBlockStarts { + // Index block size is wider than data block size, so we want to set all data blockStarts + // within the range of a given index blockStart + for at := blockStart; at < blockStart.Add(indexBlockSize); at = at.Add(blockSize) { + currState := s.flushStateNoBootstrapCheck(at) + if currState.WarmStatus.DataFlushed != fileOpSuccess { + s.markWarmIndexFlushStateSuccess(at) + } + } + } + for _, result := range readInfoFilesResults { if err := result.Err.Error(); err != nil { s.logger.Error("unable to read info files in shard bootstrap", @@ -1950,10 +1981,12 @@ func (s *dbShard) UpdateFlushStates() { info := result.Info at := xtime.UnixNano(info.BlockStart) currState := s.flushStateNoBootstrapCheck(at) - if currState.WarmStatus != fileOpSuccess { - s.markWarmFlushStateSuccess(at) + if currState.WarmStatus.DataFlushed != fileOpSuccess { + s.markWarmDataFlushStateSuccess(at) } + // we need to init if index is flushed to pull into this granular state. + // Cold version needs to get bootstrapped so that the 1:1 relationship // between volume number and cold version is maintained and the volume // numbers / flush versions remain monotonically increasing. @@ -2230,7 +2263,7 @@ func (s *dbShard) WarmFlush( multiErr = multiErr.Add(err) } - return multiErr.FinalError() + return s.markWarmDataFlushStateSuccessOrError(blockStart, multiErr.FinalError()) } func (s *dbShard) ColdFlush( @@ -2487,33 +2520,64 @@ func (s *dbShard) flushStateNoBootstrapCheck(blockStart xtime.UnixNano) fileOpSt func (s *dbShard) flushStateWithRLock(blockStart xtime.UnixNano) fileOpState { state, ok := s.flushState.statesByTime[blockStart] if !ok { - return fileOpState{WarmStatus: fileOpNotStarted} + return fileOpState{WarmStatus: warmStatus{ + DataFlushed: fileOpNotStarted, + IndexFlushed: fileOpNotStarted, + }} } return state } -func (s *dbShard) MarkWarmFlushStateSuccessOrError(blockStart xtime.UnixNano, err error) { +func (s *dbShard) markWarmDataFlushStateSuccessOrError(blockStart xtime.UnixNano, err error) error { // Track flush state for block state if err == nil { - s.markWarmFlushStateSuccess(blockStart) + s.markWarmDataFlushStateSuccess(blockStart) } else { - s.markWarmFlushStateFail(blockStart) + s.markWarmDataFlushStateFail(blockStart) } + return err } -func (s *dbShard) markWarmFlushStateSuccess(blockStart xtime.UnixNano) { +func (s *dbShard) markWarmDataFlushStateSuccess(blockStart xtime.UnixNano) { s.flushState.Lock() - s.flushState.statesByTime[blockStart] = - fileOpState{ - WarmStatus: fileOpSuccess, - } + state := s.flushState.statesByTime[blockStart] + state.WarmStatus.DataFlushed = fileOpSuccess + s.flushState.statesByTime[blockStart] = state + s.flushState.Unlock() +} + +func (s *dbShard) markWarmDataFlushStateFail(blockStart xtime.UnixNano) { + s.flushState.Lock() + state := s.flushState.statesByTime[blockStart] + state.WarmStatus.DataFlushed = fileOpFailed + state.NumFailures++ + s.flushState.statesByTime[blockStart] = state + s.flushState.Unlock() +} + +// MarkWarmIndexFlushStateSuccessOrError marks the blockStart as +// success or fail based on the provided err. +func (s *dbShard) MarkWarmIndexFlushStateSuccessOrError(blockStart xtime.UnixNano, err error) { + // Track flush state for block state + if err == nil { + s.markWarmIndexFlushStateSuccess(blockStart) + } else { + s.markWarmIndexFlushStateFail(blockStart) + } +} + +func (s *dbShard) markWarmIndexFlushStateSuccess(blockStart xtime.UnixNano) { + s.flushState.Lock() + state := s.flushState.statesByTime[blockStart] + state.WarmStatus.IndexFlushed = fileOpSuccess + s.flushState.statesByTime[blockStart] = state s.flushState.Unlock() } -func (s *dbShard) markWarmFlushStateFail(blockStart xtime.UnixNano) { +func (s *dbShard) markWarmIndexFlushStateFail(blockStart xtime.UnixNano) { s.flushState.Lock() state := s.flushState.statesByTime[blockStart] - state.WarmStatus = fileOpFailed + state.WarmStatus.IndexFlushed = fileOpFailed state.NumFailures++ s.flushState.statesByTime[blockStart] = state s.flushState.Unlock() @@ -2700,7 +2764,7 @@ func (s *dbShard) finishWriting( markWarmFlushStateSuccess bool, ) error { if markWarmFlushStateSuccess { - s.markWarmFlushStateSuccess(blockStart) + s.markWarmDataFlushStateSuccess(blockStart) } // After writing the full block successfully update the ColdVersionFlushed number. This will diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index e1756a1fcb..4b2121bfc6 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -176,7 +176,9 @@ func TestShardFlushStateNotStarted(t *testing.T) { nsCtx := namespace.Context{ID: ident.StringID("foo")} s.Bootstrap(ctx, nsCtx) - notStarted := fileOpState{WarmStatus: fileOpNotStarted} + notStarted := fileOpState{WarmStatus: warmStatus{ + DataFlushed: fileOpNotStarted, + }} for st := earliest; !st.After(latest); st = st.Add(ropts.BlockSize()) { flushState, err := s.FlushState(earliest) require.NoError(t, err) @@ -430,7 +432,9 @@ func TestShardFlushSeriesFlushError(t *testing.T) { s.Bootstrap(ctx, nsCtx) s.flushState.statesByTime[blockStart] = fileOpState{ - WarmStatus: fileOpNotStarted, + WarmStatus: warmStatus{ + DataFlushed: fileOpNotStarted, + }, NumFailures: 0, } @@ -483,16 +487,20 @@ func TestShardFlushSeriesFlushError(t *testing.T) { flushState, err := s.FlushState(blockStart) require.NoError(t, err) require.Equal(t, fileOpState{ - WarmStatus: fileOpNotStarted, + WarmStatus: warmStatus{ + DataFlushed: fileOpNotStarted, + }, NumFailures: 0, }, flushState) - s.MarkWarmFlushStateSuccessOrError(blockStart, flushErr) + _ = s.markWarmDataFlushStateSuccessOrError(blockStart, flushErr) flushState, err = s.FlushState(blockStart) require.NoError(t, err) require.Equal(t, fileOpState{ - WarmStatus: fileOpFailed, + WarmStatus: warmStatus{ + DataFlushed: fileOpFailed, + }, ColdVersionRetrievable: 0, NumFailures: 1, }, flushState) @@ -520,7 +528,9 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { s.Bootstrap(ctx, nsCtx) s.flushState.statesByTime[blockStart] = fileOpState{ - WarmStatus: fileOpNotStarted, + WarmStatus: warmStatus{ + DataFlushed: fileOpNotStarted, + }, NumFailures: 0, } @@ -570,17 +580,21 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { flushState, err := s.FlushState(blockStart) require.NoError(t, err) require.Equal(t, fileOpState{ - WarmStatus: fileOpNotStarted, + WarmStatus: warmStatus{ + DataFlushed: fileOpNotStarted, + }, ColdVersionRetrievable: 0, NumFailures: 0, }, flushState) - s.MarkWarmFlushStateSuccessOrError(blockStart, nil) + _ = s.markWarmDataFlushStateSuccessOrError(blockStart, nil) flushState, err = s.FlushState(blockStart) require.NoError(t, err) require.Equal(t, fileOpState{ - WarmStatus: fileOpSuccess, + WarmStatus: warmStatus{ + DataFlushed: fileOpSuccess, + }, ColdVersionRetrievable: 0, NumFailures: 0, }, flushState) @@ -641,13 +655,13 @@ func TestShardColdFlush(t *testing.T) { // happen after a successful warm flush because warm flushes currently don't // have merging logic. This means that all blocks except t7 should // successfully cold flush. - shard.markWarmFlushStateSuccess(t0) - shard.markWarmFlushStateSuccess(t1) - shard.markWarmFlushStateSuccess(t2) - shard.markWarmFlushStateSuccess(t3) - shard.markWarmFlushStateSuccess(t4) - shard.markWarmFlushStateSuccess(t5) - shard.markWarmFlushStateSuccess(t6) + shard.markWarmDataFlushStateSuccess(t0) + shard.markWarmDataFlushStateSuccess(t1) + shard.markWarmDataFlushStateSuccess(t2) + shard.markWarmDataFlushStateSuccess(t3) + shard.markWarmDataFlushStateSuccess(t4) + shard.markWarmDataFlushStateSuccess(t5) + shard.markWarmDataFlushStateSuccess(t6) dirtyData := []testDirtySeries{ {id: ident.StringID("id0"), dirtyTimes: []xtime.UnixNano{t0, t2, t3, t4}}, @@ -722,10 +736,10 @@ func TestShardColdFlushNoMergeIfNothingDirty(t *testing.T) { t1 := t0.Add(1 * blockSize) t2 := t0.Add(2 * blockSize) t3 := t0.Add(3 * blockSize) - shard.markWarmFlushStateSuccess(t0) - shard.markWarmFlushStateSuccess(t1) - shard.markWarmFlushStateSuccess(t2) - shard.markWarmFlushStateSuccess(t3) + shard.markWarmDataFlushStateSuccess(t0) + shard.markWarmDataFlushStateSuccess(t1) + shard.markWarmDataFlushStateSuccess(t2) + shard.markWarmDataFlushStateSuccess(t3) preparer := persist.NewMockFlushPreparer(ctrl) fsReader := fs.NewMockDataFileSetReader(ctrl) @@ -991,10 +1005,14 @@ func TestShardTick(t *testing.T) { // Also check that it expires flush states by time shard.flushState.statesByTime[earliestFlush] = fileOpState{ - WarmStatus: fileOpSuccess, + WarmStatus: warmStatus{ + DataFlushed: fileOpSuccess, + }, } shard.flushState.statesByTime[beforeEarliestFlush] = fileOpState{ - WarmStatus: fileOpSuccess, + WarmStatus: warmStatus{ + DataFlushed: fileOpSuccess, + }, } assert.Equal(t, 2, len(shard.flushState.statesByTime)) @@ -1162,10 +1180,14 @@ func testShardWriteAsync(t *testing.T, writes []testWrite) { // Also check that it expires flush states by time shard.flushState.statesByTime[earliestFlush] = fileOpState{ - WarmStatus: fileOpSuccess, + WarmStatus: warmStatus{ + DataFlushed: fileOpSuccess, + }, } shard.flushState.statesByTime[beforeEarliestFlush] = fileOpState{ - WarmStatus: fileOpSuccess, + WarmStatus: warmStatus{ + DataFlushed: fileOpSuccess, + }, } assert.Equal(t, 2, len(shard.flushState.statesByTime)) @@ -1659,8 +1681,8 @@ func TestShardFetchIndexChecksum(t *testing.T) { ropts := shard.seriesOpts.RetentionOptions() end := xtime.ToUnixNano(opts.ClockOptions().NowFn()()).Truncate(ropts.BlockSize()) start := end.Add(-2 * ropts.BlockSize()) - shard.markWarmFlushStateSuccess(start) - shard.markWarmFlushStateSuccess(start.Add(ropts.BlockSize())) + shard.markWarmDataFlushStateSuccess(start) + shard.markWarmDataFlushStateSuccess(start.Add(ropts.BlockSize())) retriever := block.NewMockDatabaseBlockRetriever(ctrl) shard.setBlockRetriever(retriever) @@ -1733,8 +1755,8 @@ func TestShardReadEncodedCachesSeriesWithRecentlyReadPolicy(t *testing.T) { ropts := shard.seriesOpts.RetentionOptions() end := xtime.ToUnixNano(opts.ClockOptions().NowFn()()).Truncate(ropts.BlockSize()) start := end.Add(-2 * ropts.BlockSize()) - shard.markWarmFlushStateSuccess(start) - shard.markWarmFlushStateSuccess(start.Add(ropts.BlockSize())) + shard.markWarmDataFlushStateSuccess(start) + shard.markWarmDataFlushStateSuccess(start.Add(ropts.BlockSize())) retriever := block.NewMockDatabaseBlockRetriever(ctrl) shard.setBlockRetriever(retriever) diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index 0b110b25c4..32e2ebc2b4 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -1425,12 +1425,11 @@ func (mr *MockdatabaseNamespaceMockRecorder) FetchWideEntry(ctx, id, blockStart, } // FlushIndex mocks base method. -func (m *MockdatabaseNamespace) FlushIndex(flush persist.IndexFlush) (shardFlushes, error) { +func (m *MockdatabaseNamespace) FlushIndex(flush persist.IndexFlush) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "FlushIndex", flush) - ret0, _ := ret[0].(shardFlushes) - ret1, _ := ret[1].(error) - return ret0, ret1 + ret0, _ := ret[0].(error) + return ret0 } // FlushIndex indicates an expected call of FlushIndex. @@ -1785,12 +1784,11 @@ func (mr *MockdatabaseNamespaceMockRecorder) Truncate() *gomock.Call { } // WarmFlush mocks base method. -func (m *MockdatabaseNamespace) WarmFlush(blockStart time0.UnixNano, flush persist.FlushPreparer) ([]databaseShard, error) { +func (m *MockdatabaseNamespace) WarmFlush(blockStart time0.UnixNano, flush persist.FlushPreparer) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "WarmFlush", blockStart, flush) - ret0, _ := ret[0].([]databaseShard) - ret1, _ := ret[1].(error) - return ret0, ret1 + ret0, _ := ret[0].(error) + return ret0 } // WarmFlush indicates an expected call of WarmFlush. @@ -2239,16 +2237,16 @@ func (mr *MockdatabaseShardMockRecorder) LoadBlocks(series interface{}) *gomock. return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LoadBlocks", reflect.TypeOf((*MockdatabaseShard)(nil).LoadBlocks), series) } -// MarkWarmFlushStateSuccessOrError mocks base method. -func (m *MockdatabaseShard) MarkWarmFlushStateSuccessOrError(blockStart time0.UnixNano, err error) { +// MarkWarmIndexFlushStateSuccessOrError mocks base method. +func (m *MockdatabaseShard) MarkWarmIndexFlushStateSuccessOrError(blockStart time0.UnixNano, err error) { m.ctrl.T.Helper() - m.ctrl.Call(m, "MarkWarmFlushStateSuccessOrError", blockStart, err) + m.ctrl.Call(m, "MarkWarmIndexFlushStateSuccessOrError", blockStart, err) } -// MarkWarmFlushStateSuccessOrError indicates an expected call of MarkWarmFlushStateSuccessOrError. -func (mr *MockdatabaseShardMockRecorder) MarkWarmFlushStateSuccessOrError(blockStart, err interface{}) *gomock.Call { +// MarkWarmIndexFlushStateSuccessOrError indicates an expected call of MarkWarmIndexFlushStateSuccessOrError. +func (mr *MockdatabaseShardMockRecorder) MarkWarmIndexFlushStateSuccessOrError(blockStart, err interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "MarkWarmFlushStateSuccessOrError", reflect.TypeOf((*MockdatabaseShard)(nil).MarkWarmFlushStateSuccessOrError), blockStart, err) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "MarkWarmIndexFlushStateSuccessOrError", reflect.TypeOf((*MockdatabaseShard)(nil).MarkWarmIndexFlushStateSuccessOrError), blockStart, err) } // NumSeries mocks base method. @@ -2713,12 +2711,11 @@ func (mr *MockNamespaceIndexMockRecorder) Tick(c, startTime interface{}) *gomock } // WarmFlush mocks base method. -func (m *MockNamespaceIndex) WarmFlush(flush persist.IndexFlush, shards []databaseShard) (shardFlushes, error) { +func (m *MockNamespaceIndex) WarmFlush(flush persist.IndexFlush, shards []databaseShard) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "WarmFlush", flush, shards) - ret0, _ := ret[0].(shardFlushes) - ret1, _ := ret[1].(error) - return ret0, ret1 + ret0, _ := ret[0].(error) + return ret0 } // WarmFlush indicates an expected call of WarmFlush. @@ -2727,6 +2724,20 @@ func (mr *MockNamespaceIndexMockRecorder) WarmFlush(flush, shards interface{}) * return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WarmFlush", reflect.TypeOf((*MockNamespaceIndex)(nil).WarmFlush), flush, shards) } +// WarmFlushedBlockStarts mocks base method. +func (m *MockNamespaceIndex) WarmFlushedBlockStarts() []time0.UnixNano { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "WarmFlushedBlockStarts") + ret0, _ := ret[0].([]time0.UnixNano) + return ret0 +} + +// WarmFlushedBlockStarts indicates an expected call of WarmFlushedBlockStarts. +func (mr *MockNamespaceIndexMockRecorder) WarmFlushedBlockStarts() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WarmFlushedBlockStarts", reflect.TypeOf((*MockNamespaceIndex)(nil).WarmFlushedBlockStarts)) +} + // WideQuery mocks base method. func (m *MockNamespaceIndex) WideQuery(ctx context.Context, query index.Query, collector chan *ident.IDBatch, opts index.WideQueryOptions) error { m.ctrl.T.Helper() diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index c8412c09fe..4d295ef7be 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -425,12 +425,12 @@ type databaseNamespace interface { Bootstrap(ctx context.Context, bootstrapResult bootstrap.NamespaceResult) error // WarmFlush flushes in-memory WarmWrites. - WarmFlush(blockStart xtime.UnixNano, flush persist.FlushPreparer) ([]databaseShard, error) + WarmFlush(blockStart xtime.UnixNano, flush persist.FlushPreparer) error // FlushIndex flushes in-memory index data. FlushIndex( flush persist.IndexFlush, - ) (shardFlushes, error) + ) error // ColdFlush flushes unflushed in-memory ColdWrites. ColdFlush( @@ -614,8 +614,9 @@ type databaseShard interface { nsCtx namespace.Context, ) error - // MarkWarmFlushStateSuccessOrError marks the flush state as success or error. - MarkWarmFlushStateSuccessOrError(blockStart xtime.UnixNano, err error) + // MarkWarmIndexFlushStateSuccessOrError marks the blockStart as + // success or fail based on the provided err. + MarkWarmIndexFlushStateSuccessOrError(blockStart xtime.UnixNano, err error) // ColdFlush flushes the unflushed ColdWrites in this shard. ColdFlush( @@ -764,7 +765,10 @@ type NamespaceIndex interface { WarmFlush( flush persist.IndexFlush, shards []databaseShard, - ) (shardFlushes, error) + ) error + + // WarmFlushedBlockStarts returns all index blockStarts which have been flushed to disk. + WarmFlushedBlockStarts() []xtime.UnixNano // ColdFlush performs any cold flushes that the index has outstanding using // the owned shards of the database. Also returns a callback to be called when From 3e58b366d345f78670b615e5619d2e3b280688f6 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Thu, 19 Aug 2021 12:23:09 -0400 Subject: [PATCH 33/36] Fix tests --- src/dbnode/storage/flush_test.go | 6 +----- src/dbnode/storage/index_test.go | 11 ++++++++-- src/dbnode/storage/shard.go | 37 +++++++++++++++++--------------- src/dbnode/storage/shard_test.go | 26 +--------------------- 4 files changed, 31 insertions(+), 49 deletions(-) diff --git a/src/dbnode/storage/flush_test.go b/src/dbnode/storage/flush_test.go index a5249bad05..b072dd6e03 100644 --- a/src/dbnode/storage/flush_test.go +++ b/src/dbnode/storage/flush_test.go @@ -375,15 +375,11 @@ func TestFlushManagerNamespaceIndexingEnabled(t *testing.T) { // Validate that the flush state is marked as successful only AFTER all prequisite steps have been run. // Order is important to avoid any edge case where data is GCed from memory without all flushing operations // being completed. - steps := make([]*gomock.Call, 0) - steps = append(steps, + gomock.InOrder( ns.EXPECT().WarmFlush(gomock.Any(), gomock.Any()).Return(nil).Times(blocks), ns.EXPECT().Snapshot(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes(), ns.EXPECT().FlushIndex(gomock.Any()).Return(nil), - s1.EXPECT().MarkWarmIndexFlushStateSuccessOrError(gomock.Any(), nil), - s2.EXPECT().MarkWarmIndexFlushStateSuccessOrError(gomock.Any(), nil), ) - gomock.InOrder(steps...) var ( mockFlushPersist = persist.NewMockFlushPreparer(ctrl) diff --git a/src/dbnode/storage/index_test.go b/src/dbnode/storage/index_test.go index 769466b2b7..9802a54d82 100644 --- a/src/dbnode/storage/index_test.go +++ b/src/dbnode/storage/index_test.go @@ -908,10 +908,12 @@ func verifyFlushForShards( for _, mockShard := range mockShards { mockShard.EXPECT().IsBootstrapped().Return(true) mockShard.EXPECT().FlushState(blockStart).Return(fileOpState{WarmStatus: warmStatus{ - IndexFlushed: fileOpSuccess, + // Index flushing requires data flush already happened. + DataFlushed: fileOpSuccess, }}, nil) mockShard.EXPECT().FlushState(blockStart.Add(blockSize)).Return(fileOpState{WarmStatus: warmStatus{ - IndexFlushed: fileOpSuccess, + // Index flushing requires data flush already happened. + DataFlushed: fileOpSuccess, }}, nil) resultsTags1 := ident.NewTagsIterator(ident.NewTags()) @@ -934,6 +936,11 @@ func verifyFlushForShards( mockShard.EXPECT().FetchBlocksMetadataV2(gomock.Any(), blockStart, blockStart.Add(idx.blockSize), gomock.Any(), gomock.Any(), block.FetchBlocksMetadataOptions{OnlyDisk: true}).Return(results, nil, nil) + + // For a given index block, which in this test is 2x the size of a block, we expect that + // we mark as flushed 2 blockStarts that fall within the index block. + mockShard.EXPECT().MarkWarmIndexFlushStateSuccessOrError(blockStart, nil) + mockShard.EXPECT().MarkWarmIndexFlushStateSuccessOrError(blockStart.Add(blockSize), nil) } mockBlock.EXPECT().IsSealed().Return(true) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index d3d2fd7fe9..13d1d4743c 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -1951,23 +1951,6 @@ func (s *dbShard) UpdateFlushStates() { readInfoFilesResults := fs.ReadInfoFiles(fsOpts.FilePathPrefix(), s.namespace.ID(), s.shard, fsOpts.InfoReaderBufferSize(), fsOpts.DecodingOptions(), persist.FileSetFlushType) - // TODO: use for blockSize differences - blockSize := s.namespace.Options().RetentionOptions().BlockSize() - indexBlockSize := s.namespace.Options().IndexOptions().BlockSize() - - // expose for getting all info files - indexFlushedBlockStarts := s.reverseIndex.WarmFlushedBlockStarts() - for _, blockStart := range indexFlushedBlockStarts { - // Index block size is wider than data block size, so we want to set all data blockStarts - // within the range of a given index blockStart - for at := blockStart; at < blockStart.Add(indexBlockSize); at = at.Add(blockSize) { - currState := s.flushStateNoBootstrapCheck(at) - if currState.WarmStatus.DataFlushed != fileOpSuccess { - s.markWarmIndexFlushStateSuccess(at) - } - } - } - for _, result := range readInfoFilesResults { if err := result.Err.Error(); err != nil { s.logger.Error("unable to read info files in shard bootstrap", @@ -1999,6 +1982,26 @@ func (s *dbShard) UpdateFlushStates() { s.setFlushStateColdVersionFlushed(at, info.VolumeIndex) } } + + // Populate index flush state only if enabled. + if !s.namespace.Options().IndexOptions().Enabled() { + return + } + + blockSize := s.namespace.Options().RetentionOptions().BlockSize() + indexBlockSize := s.namespace.Options().IndexOptions().BlockSize() + + indexFlushedBlockStarts := s.reverseIndex.WarmFlushedBlockStarts() + for _, blockStart := range indexFlushedBlockStarts { + // Index block size is wider than data block size, so we want to set all data blockStarts + // within the range of a given index blockStart + for at := blockStart; at < blockStart.Add(indexBlockSize); at = at.Add(blockSize) { + currState := s.flushStateNoBootstrapCheck(at) + if currState.WarmStatus.DataFlushed != fileOpSuccess { + s.markWarmIndexFlushStateSuccess(at) + } + } + } } func (s *dbShard) Bootstrap( diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index 4b2121bfc6..af270ffbc3 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -486,23 +486,11 @@ func TestShardFlushSeriesFlushError(t *testing.T) { flushState, err := s.FlushState(blockStart) require.NoError(t, err) - require.Equal(t, fileOpState{ - WarmStatus: warmStatus{ - DataFlushed: fileOpNotStarted, - }, - NumFailures: 0, - }, flushState) - - _ = s.markWarmDataFlushStateSuccessOrError(blockStart, flushErr) - - flushState, err = s.FlushState(blockStart) - require.NoError(t, err) require.Equal(t, fileOpState{ WarmStatus: warmStatus{ DataFlushed: fileOpFailed, }, - ColdVersionRetrievable: 0, - NumFailures: 1, + NumFailures: 1, }, flushState) } @@ -579,18 +567,6 @@ func TestShardFlushSeriesFlushSuccess(t *testing.T) { // State not yet updated since an explicit call to MarkWarmFlushStateSuccessOrError is required. flushState, err := s.FlushState(blockStart) require.NoError(t, err) - require.Equal(t, fileOpState{ - WarmStatus: warmStatus{ - DataFlushed: fileOpNotStarted, - }, - ColdVersionRetrievable: 0, - NumFailures: 0, - }, flushState) - - _ = s.markWarmDataFlushStateSuccessOrError(blockStart, nil) - - flushState, err = s.FlushState(blockStart) - require.NoError(t, err) require.Equal(t, fileOpState{ WarmStatus: warmStatus{ DataFlushed: fileOpSuccess, From 548591f62d84146b41e7677549db100782df0a86 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Thu, 19 Aug 2021 13:35:33 -0400 Subject: [PATCH 34/36] Fixing tests --- src/dbnode/{storage/entry => }/entry_blackbox_test.go | 2 +- src/dbnode/{storage/entry => }/entry_whitebox_test.go | 2 +- src/dbnode/storage/shard.go | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) rename src/dbnode/{storage/entry => }/entry_blackbox_test.go (99%) rename src/dbnode/{storage/entry => }/entry_whitebox_test.go (99%) diff --git a/src/dbnode/storage/entry/entry_blackbox_test.go b/src/dbnode/entry_blackbox_test.go similarity index 99% rename from src/dbnode/storage/entry/entry_blackbox_test.go rename to src/dbnode/entry_blackbox_test.go index 2a3db5ce9f..1077238131 100644 --- a/src/dbnode/storage/entry/entry_blackbox_test.go +++ b/src/dbnode/entry_blackbox_test.go @@ -18,7 +18,7 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -package entry +package dbnode import ( "sync" diff --git a/src/dbnode/storage/entry/entry_whitebox_test.go b/src/dbnode/entry_whitebox_test.go similarity index 99% rename from src/dbnode/storage/entry/entry_whitebox_test.go rename to src/dbnode/entry_whitebox_test.go index 3665e10afa..b7e01f1b82 100644 --- a/src/dbnode/storage/entry/entry_whitebox_test.go +++ b/src/dbnode/entry_whitebox_test.go @@ -18,7 +18,7 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -package entry +package dbnode import ( "testing" diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 13d1d4743c..1655ef64ce 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -1997,7 +1997,7 @@ func (s *dbShard) UpdateFlushStates() { // within the range of a given index blockStart for at := blockStart; at < blockStart.Add(indexBlockSize); at = at.Add(blockSize) { currState := s.flushStateNoBootstrapCheck(at) - if currState.WarmStatus.DataFlushed != fileOpSuccess { + if currState.WarmStatus.IndexFlushed != fileOpSuccess { s.markWarmIndexFlushStateSuccess(at) } } From c68120becaca24e6441e8e5b43e35845bf122ddc Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Thu, 19 Aug 2021 13:54:31 -0400 Subject: [PATCH 35/36] Fixing tests 2 --- src/dbnode/{ => storage}/entry_blackbox_test.go | 11 +++++------ src/dbnode/{ => storage}/entry_whitebox_test.go | 9 ++++----- 2 files changed, 9 insertions(+), 11 deletions(-) rename src/dbnode/{ => storage}/entry_blackbox_test.go (91%) rename src/dbnode/{ => storage}/entry_whitebox_test.go (93%) diff --git a/src/dbnode/entry_blackbox_test.go b/src/dbnode/storage/entry_blackbox_test.go similarity index 91% rename from src/dbnode/entry_blackbox_test.go rename to src/dbnode/storage/entry_blackbox_test.go index 1077238131..97447e4fff 100644 --- a/src/dbnode/entry_blackbox_test.go +++ b/src/dbnode/storage/entry_blackbox_test.go @@ -18,14 +18,13 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -package dbnode +package storage import ( "sync" "testing" "time" - "github.com/m3db/m3/src/dbnode/storage" xtime "github.com/m3db/m3/src/x/time" "github.com/fortytw2/leaktest" @@ -43,7 +42,7 @@ func newTime(n int) xtime.UnixNano { } func TestEntryReaderWriterCount(t *testing.T) { - e := storage.NewEntry(storage.NewEntryOptions{}) + e := NewEntry(NewEntryOptions{}) require.Equal(t, int32(0), e.ReaderWriterCount()) e.IncrementReaderWriterCount() @@ -54,7 +53,7 @@ func TestEntryReaderWriterCount(t *testing.T) { } func TestEntryIndexSuccessPath(t *testing.T) { - e := storage.NewEntry(storage.NewEntryOptions{}) + e := NewEntry(NewEntryOptions{}) t0 := newTime(0) require.False(t, e.IndexedForBlockStart(t0)) @@ -69,7 +68,7 @@ func TestEntryIndexSuccessPath(t *testing.T) { } func TestEntryIndexFailPath(t *testing.T) { - e := storage.NewEntry(storage.NewEntryOptions{}) + e := NewEntry(NewEntryOptions{}) t0 := newTime(0) require.False(t, e.IndexedForBlockStart(t0)) @@ -85,7 +84,7 @@ func TestEntryIndexFailPath(t *testing.T) { func TestEntryMultipleGoroutinesRaceIndexUpdate(t *testing.T) { defer leaktest.CheckTimeout(t, time.Second)() - e := storage.NewEntry(storage.NewEntryOptions{}) + e := NewEntry(NewEntryOptions{}) t0 := newTime(0) require.False(t, e.IndexedForBlockStart(t0)) diff --git a/src/dbnode/entry_whitebox_test.go b/src/dbnode/storage/entry_whitebox_test.go similarity index 93% rename from src/dbnode/entry_whitebox_test.go rename to src/dbnode/storage/entry_whitebox_test.go index b7e01f1b82..34a552cfb0 100644 --- a/src/dbnode/entry_whitebox_test.go +++ b/src/dbnode/storage/entry_whitebox_test.go @@ -18,14 +18,13 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -package dbnode +package storage import ( "testing" "time" "github.com/golang/mock/gomock" - "github.com/m3db/m3/src/dbnode/storage" "github.com/m3db/m3/src/dbnode/storage/index" "github.com/m3db/m3/src/dbnode/storage/series" "github.com/m3db/m3/src/dbnode/ts/writes" @@ -37,7 +36,7 @@ import ( ) func TestEntryIndexAttemptRotatesSlice(t *testing.T) { - e := storage.NewEntry(storage.NewEntryOptions{}) + e := NewEntry(NewEntryOptions{}) for i := 0; i < 10; i++ { ti := newTime(i) require.True(t, e.NeedsIndexUpdate(ti)) @@ -55,7 +54,7 @@ func TestEntryIndexSeriesRef(t *testing.T) { ctrl := gomock.NewController(t) now := time.Now() blockStart := newTime(0) - mockIndexWriter := storage.NewMockIndexWriter(ctrl) + mockIndexWriter := NewMockIndexWriter(ctrl) mockIndexWriter.EXPECT().BlockStartForWriteTime(blockStart). Return(blockStart). Times(2) @@ -71,7 +70,7 @@ func TestEntryIndexSeriesRef(t *testing.T) { series.WriteOptions{}, ).Return(true, series.WarmWrite, nil) - e := storage.NewEntry(storage.NewEntryOptions{ + e := NewEntry(NewEntryOptions{ Series: mockSeries, IndexWriter: mockIndexWriter, NowFn: func() time.Time { From 6651096829690b1e8774440bae6d637ee77478d6 Mon Sep 17 00:00:00 2001 From: Ryan Allen Date: Thu, 19 Aug 2021 16:20:46 -0400 Subject: [PATCH 36/36] For bootstrapping just use presence of datafiles --- src/dbnode/storage/index.go | 12 ----------- src/dbnode/storage/shard.go | 32 ++++++++++-------------------- src/dbnode/storage/storage_mock.go | 14 ------------- src/dbnode/storage/types.go | 3 --- 4 files changed, 10 insertions(+), 51 deletions(-) diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 6cba8bf860..595bd0a3d8 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -1112,18 +1112,6 @@ func (i *nsIndex) ColdFlush(shards []databaseShard) (OnColdFlushDone, error) { }, nil } -// WarmFlushedBlockStarts returns all index blockStarts which have been flushed to disk. -func (i *nsIndex) WarmFlushedBlockStarts() []xtime.UnixNano { - flushed := make([]xtime.UnixNano, 0) - infoFiles := i.readInfoFilesAsMap() - for blockStart := range infoFiles { - if i.hasIndexWarmFlushedToDisk(infoFiles, blockStart) { - flushed = append(flushed, blockStart) - } - } - return flushed -} - func (i *nsIndex) readInfoFilesAsMap() map[xtime.UnixNano]fs.ReadIndexInfoFileResult { fsOpts := i.opts.CommitLogOptions().FilesystemOptions() infoFiles := i.readIndexInfoFilesFn(fs.ReadIndexInfoFilesOptions{ diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 1655ef64ce..a22825a2aa 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -1964,11 +1964,19 @@ func (s *dbShard) UpdateFlushStates() { info := result.Info at := xtime.UnixNano(info.BlockStart) currState := s.flushStateNoBootstrapCheck(at) + + // When initializing from disk, the data files being present are sufficient + // for considering the data+index are flushed because that distinction is only + // needed to account for the raciness surrounding GCing series based on when + // data + index flushes have occurred. For the purposes of just initializing + // the state of which blocks have been flushed when bootstrapping, we can + // just use the data being present as the indicator. if currState.WarmStatus.DataFlushed != fileOpSuccess { s.markWarmDataFlushStateSuccess(at) } - - // we need to init if index is flushed to pull into this granular state. + if currState.WarmStatus.IndexFlushed != fileOpSuccess { + s.markWarmIndexFlushStateSuccess(at) + } // Cold version needs to get bootstrapped so that the 1:1 relationship // between volume number and cold version is maintained and the volume @@ -1982,26 +1990,6 @@ func (s *dbShard) UpdateFlushStates() { s.setFlushStateColdVersionFlushed(at, info.VolumeIndex) } } - - // Populate index flush state only if enabled. - if !s.namespace.Options().IndexOptions().Enabled() { - return - } - - blockSize := s.namespace.Options().RetentionOptions().BlockSize() - indexBlockSize := s.namespace.Options().IndexOptions().BlockSize() - - indexFlushedBlockStarts := s.reverseIndex.WarmFlushedBlockStarts() - for _, blockStart := range indexFlushedBlockStarts { - // Index block size is wider than data block size, so we want to set all data blockStarts - // within the range of a given index blockStart - for at := blockStart; at < blockStart.Add(indexBlockSize); at = at.Add(blockSize) { - currState := s.flushStateNoBootstrapCheck(at) - if currState.WarmStatus.IndexFlushed != fileOpSuccess { - s.markWarmIndexFlushStateSuccess(at) - } - } - } } func (s *dbShard) Bootstrap( diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index 32e2ebc2b4..5e71c3c284 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -2724,20 +2724,6 @@ func (mr *MockNamespaceIndexMockRecorder) WarmFlush(flush, shards interface{}) * return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WarmFlush", reflect.TypeOf((*MockNamespaceIndex)(nil).WarmFlush), flush, shards) } -// WarmFlushedBlockStarts mocks base method. -func (m *MockNamespaceIndex) WarmFlushedBlockStarts() []time0.UnixNano { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "WarmFlushedBlockStarts") - ret0, _ := ret[0].([]time0.UnixNano) - return ret0 -} - -// WarmFlushedBlockStarts indicates an expected call of WarmFlushedBlockStarts. -func (mr *MockNamespaceIndexMockRecorder) WarmFlushedBlockStarts() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WarmFlushedBlockStarts", reflect.TypeOf((*MockNamespaceIndex)(nil).WarmFlushedBlockStarts)) -} - // WideQuery mocks base method. func (m *MockNamespaceIndex) WideQuery(ctx context.Context, query index.Query, collector chan *ident.IDBatch, opts index.WideQueryOptions) error { m.ctrl.T.Helper() diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 4d295ef7be..4c9a0bd9c1 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -767,9 +767,6 @@ type NamespaceIndex interface { shards []databaseShard, ) error - // WarmFlushedBlockStarts returns all index blockStarts which have been flushed to disk. - WarmFlushedBlockStarts() []xtime.UnixNano - // ColdFlush performs any cold flushes that the index has outstanding using // the owned shards of the database. Also returns a callback to be called when // cold flushing completes to perform houskeeping.