From f09377421f3f75553ac0d23909bf62d38a3c154b Mon Sep 17 00:00:00 2001 From: Linas Medziunas Date: Thu, 5 Nov 2020 18:26:56 +0200 Subject: [PATCH 01/12] [dbnode] Introduce Aggregator type --- src/dbnode/server/options.go | 1 + src/dbnode/server/server.go | 5 +++ src/dbnode/storage/options.go | 24 +++++++++++ src/dbnode/storage/storage_mock.go | 66 ++++++++++++++++++++++++++++++ src/dbnode/storage/types.go | 21 ++++++++++ 5 files changed, 117 insertions(+) diff --git a/src/dbnode/server/options.go b/src/dbnode/server/options.go index fa2cc58c2a..af175fc330 100644 --- a/src/dbnode/server/options.go +++ b/src/dbnode/server/options.go @@ -32,4 +32,5 @@ type StorageOptions struct { TChanNodeServerFn node.NewTChanNodeServerFn BackgroundProcessFns []storage.NewBackgroundProcessFn NamespaceHooks storage.NamespaceHooks + NewAggregatorFn storage.NewAggregatorFn } diff --git a/src/dbnode/server/server.go b/src/dbnode/server/server.go index 2adc9a52d1..5aa1576155 100644 --- a/src/dbnode/server/server.go +++ b/src/dbnode/server/server.go @@ -845,6 +845,11 @@ func Run(runOpts RunOptions) { opts = opts.SetNamespaceHooks(runOpts.StorageOptions.NamespaceHooks) } + if runOpts.StorageOptions.NewAggregatorFn != nil { + aggregator := runOpts.StorageOptions.NewAggregatorFn(iopts) + opts = opts.SetAggregator(aggregator) + } + // Set bootstrap options - We need to create a topology map provider from the // same topology that will be passed to the cluster so that when we make // bootstrapping decisions they are in sync with the clustered database diff --git a/src/dbnode/storage/options.go b/src/dbnode/storage/options.go index 3beba1a5a4..1ff0cfb5a6 100644 --- a/src/dbnode/storage/options.go +++ b/src/dbnode/storage/options.go @@ -174,6 +174,7 @@ type options struct { wideBatchSize int newBackgroundProcessFns []NewBackgroundProcessFn namespaceHooks NamespaceHooks + aggregator Aggregator } // NewOptions creates a new set of storage options with defaults @@ -249,6 +250,7 @@ func newOptions(poolOpts pool.ObjectPoolOptions) Options { mediatorTickInterval: defaultMediatorTickInterval, wideBatchSize: defaultWideBatchSize, namespaceHooks: &noopNamespaceHooks{}, + aggregator: &noopAggregator{}, } return o.SetEncodingM3TSZPooled() } @@ -863,6 +865,16 @@ func (o *options) NamespaceHooks() NamespaceHooks { return o.namespaceHooks } +func (o *options) SetAggregator(value Aggregator) Options { + opts := *o + opts.aggregator = value + return &opts +} + +func (o *options) Aggregator() Aggregator { + return o.aggregator +} + type noOpColdFlush struct{} func (n *noOpColdFlush) ColdFlushNamespace(Namespace) (OnColdFlushNamespace, error) { @@ -874,3 +886,15 @@ type noopNamespaceHooks struct{} func (h *noopNamespaceHooks) OnCreatedNamespace(Namespace, GetNamespaceFn) error { return nil } + +type noopAggregator struct{} + +func (a *noopAggregator) AggregateTiles( + opts AggregateTilesOptions, + ns Namespace, + shardID uint32, + readers []fs.DataFileSetReader, + writer fs.StreamingWriter, +) (int64, error) { + return 0, nil +} diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index feb8fbef7f..e6c44a2cbc 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -4992,6 +4992,34 @@ func (mr *MockOptionsMockRecorder) NamespaceHooks() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NamespaceHooks", reflect.TypeOf((*MockOptions)(nil).NamespaceHooks)) } +// SetAggregator mocks base method +func (m *MockOptions) SetAggregator(aggregator Aggregator) Options { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "SetAggregator", aggregator) + ret0, _ := ret[0].(Options) + return ret0 +} + +// SetAggregator indicates an expected call of SetAggregator +func (mr *MockOptionsMockRecorder) SetAggregator(aggregator interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetAggregator", reflect.TypeOf((*MockOptions)(nil).SetAggregator), aggregator) +} + +// Aggregator mocks base method +func (m *MockOptions) Aggregator() Aggregator { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Aggregator") + ret0, _ := ret[0].(Aggregator) + return ret0 +} + +// Aggregator indicates an expected call of Aggregator +func (mr *MockOptionsMockRecorder) Aggregator() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Aggregator", reflect.TypeOf((*MockOptions)(nil).Aggregator)) +} + // MockMemoryTracker is a mock of MemoryTracker interface type MockMemoryTracker struct { ctrl *gomock.Controller @@ -5079,6 +5107,44 @@ func (mr *MockMemoryTrackerMockRecorder) WaitForDec() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WaitForDec", reflect.TypeOf((*MockMemoryTracker)(nil).WaitForDec)) } +// MockAggregator is a mock of Aggregator interface +type MockAggregator struct { + ctrl *gomock.Controller + recorder *MockAggregatorMockRecorder +} + +// MockAggregatorMockRecorder is the mock recorder for MockAggregator +type MockAggregatorMockRecorder struct { + mock *MockAggregator +} + +// NewMockAggregator creates a new mock instance +func NewMockAggregator(ctrl *gomock.Controller) *MockAggregator { + mock := &MockAggregator{ctrl: ctrl} + mock.recorder = &MockAggregatorMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use +func (m *MockAggregator) EXPECT() *MockAggregatorMockRecorder { + return m.recorder +} + +// AggregateTiles mocks base method +func (m *MockAggregator) AggregateTiles(opts AggregateTilesOptions, namespaceID ident.ID, shardID uint32, readers []fs.DataFileSetReader, writer fs.StreamingWriter, targetSchemaDescr namespace.SchemaDescr) (int64, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "AggregateTiles", opts, namespaceID, shardID, readers, writer, targetSchemaDescr) + ret0, _ := ret[0].(int64) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// AggregateTiles indicates an expected call of AggregateTiles +func (mr *MockAggregatorMockRecorder) AggregateTiles(opts, namespaceID, shardID, readers, writer, targetSchemaDescr interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AggregateTiles", reflect.TypeOf((*MockAggregator)(nil).AggregateTiles), opts, namespaceID, shardID, readers, writer, targetSchemaDescr) +} + // MockNamespaceHooks is a mock of NamespaceHooks interface type MockNamespaceHooks struct { ctrl *gomock.Controller diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index b7753bc9f8..7381ba5c52 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -1323,6 +1323,12 @@ type Options interface { // NamespaceHooks returns the NamespaceHooks. NamespaceHooks() NamespaceHooks + + // SetAggregator sets the Aggregator. + SetAggregator(aggregator Aggregator) Options + + // Aggregator returns the Aggregator. + Aggregator() Aggregator } // MemoryTracker tracks memory. @@ -1394,6 +1400,21 @@ type AggregateTilesOptions struct { InsOptions instrument.Options } +// Aggregator is the interface for AggregateTiles. +type Aggregator interface { + // AggregateTiles does tile aggregation. + AggregateTiles( + opts AggregateTilesOptions, + ns Namespace, + shardID uint32, + readers []fs.DataFileSetReader, + writer fs.StreamingWriter, + ) (int64, error) +} + +// NewAggregatorFn creates a new Aggregator. +type NewAggregatorFn func(iOpts instrument.Options) Aggregator + // NamespaceHooks allows dynamic plugging into the namespace lifecycle. type NamespaceHooks interface { // OnCreatedNamespace gets invoked after each namespace is created. From f756658276cc789e0a02fe5d981d424569856501 Mon Sep 17 00:00:00 2001 From: Linas Medziunas Date: Fri, 6 Nov 2020 10:06:51 +0200 Subject: [PATCH 02/12] Lint --- src/dbnode/storage/options.go | 1 + 1 file changed, 1 insertion(+) diff --git a/src/dbnode/storage/options.go b/src/dbnode/storage/options.go index 1ff0cfb5a6..00f755ec35 100644 --- a/src/dbnode/storage/options.go +++ b/src/dbnode/storage/options.go @@ -868,6 +868,7 @@ func (o *options) NamespaceHooks() NamespaceHooks { func (o *options) SetAggregator(value Aggregator) Options { opts := *o opts.aggregator = value + return &opts } From 92ed0bf3ce96198d43eb10f0b999499a14eaf0bd Mon Sep 17 00:00:00 2001 From: Linas Medziunas Date: Fri, 6 Nov 2020 11:18:35 +0200 Subject: [PATCH 03/12] mock gen --- src/dbnode/storage/storage_mock.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index e6c44a2cbc..54698bb6d0 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -5131,18 +5131,18 @@ func (m *MockAggregator) EXPECT() *MockAggregatorMockRecorder { } // AggregateTiles mocks base method -func (m *MockAggregator) AggregateTiles(opts AggregateTilesOptions, namespaceID ident.ID, shardID uint32, readers []fs.DataFileSetReader, writer fs.StreamingWriter, targetSchemaDescr namespace.SchemaDescr) (int64, error) { +func (m *MockAggregator) AggregateTiles(opts AggregateTilesOptions, ns Namespace, shardID uint32, readers []fs.DataFileSetReader, writer fs.StreamingWriter) (int64, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "AggregateTiles", opts, namespaceID, shardID, readers, writer, targetSchemaDescr) + ret := m.ctrl.Call(m, "AggregateTiles", opts, ns, shardID, readers, writer) ret0, _ := ret[0].(int64) ret1, _ := ret[1].(error) return ret0, ret1 } // AggregateTiles indicates an expected call of AggregateTiles -func (mr *MockAggregatorMockRecorder) AggregateTiles(opts, namespaceID, shardID, readers, writer, targetSchemaDescr interface{}) *gomock.Call { +func (mr *MockAggregatorMockRecorder) AggregateTiles(opts, ns, shardID, readers, writer interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AggregateTiles", reflect.TypeOf((*MockAggregator)(nil).AggregateTiles), opts, namespaceID, shardID, readers, writer, targetSchemaDescr) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AggregateTiles", reflect.TypeOf((*MockAggregator)(nil).AggregateTiles), opts, ns, shardID, readers, writer) } // MockNamespaceHooks is a mock of NamespaceHooks interface From f276767456a70a08966c308385733cb12c7c607f Mon Sep 17 00:00:00 2001 From: Linas Medziunas Date: Fri, 6 Nov 2020 13:13:47 +0200 Subject: [PATCH 04/12] [dbnode] Refactoring dbShard --- src/dbnode/storage/namespace.go | 6 +- src/dbnode/storage/namespace_test.go | 22 ++- src/dbnode/storage/shard.go | 235 ++++----------------------- src/dbnode/storage/shard_test.go | 164 +++---------------- src/dbnode/storage/storage_mock.go | 8 +- src/dbnode/storage/types.go | 4 +- 6 files changed, 73 insertions(+), 366 deletions(-) diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index 6465df365d..e2771b233b 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -1747,10 +1747,6 @@ func (n *dbNamespace) aggregateTiles( return 0, errNamespaceNotBootstrapped } - n.RLock() - nsCtx := n.nsContextWithRLock() - n.RUnlock() - var ( processedShards = opts.InsOptions.MetricsScope().Counter("processed-shards") targetShards = n.OwnedShards() @@ -1793,7 +1789,7 @@ func (n *dbNamespace) aggregateTiles( } shardProcessedTileCount, err := targetShard.AggregateTiles( - sourceNs.ID(), sourceShard.ID(), blockReaders, writer, sourceBlockVolumes, opts, nsCtx.Schema) + sourceNs.ID(), n, sourceShard.ID(), blockReaders, writer, sourceBlockVolumes, opts) processedTileCount += shardProcessedTileCount processedShards.Inc(1) diff --git a/src/dbnode/storage/namespace_test.go b/src/dbnode/storage/namespace_test.go index 978ca641bb..ce73d99f6d 100644 --- a/src/dbnode/storage/namespace_test.go +++ b/src/dbnode/storage/namespace_test.go @@ -1415,8 +1415,8 @@ func TestNamespaceAggregateTiles(t *testing.T) { targetBlockSize = 2 * time.Hour start = time.Now().Truncate(targetBlockSize) secondSourceBlockStart = start.Add(sourceBlockSize) - sourceShard0ID uint32 = 10 - sourceShard1ID uint32 = 20 + shard0ID uint32 = 10 + shard1ID uint32 = 20 insOpts = instrument.NewOptions() ) @@ -1440,12 +1440,12 @@ func TestNamespaceAggregateTiles(t *testing.T) { sourceNs.shards[0] = sourceShard0 sourceNs.shards[1] = sourceShard1 - sourceShard0.EXPECT().ID().Return(sourceShard0ID) + sourceShard0.EXPECT().ID().Return(shard0ID) sourceShard0.EXPECT().IsBootstrapped().Return(true) sourceShard0.EXPECT().LatestVolume(start).Return(5, nil) sourceShard0.EXPECT().LatestVolume(start.Add(sourceBlockSize)).Return(15, nil) - sourceShard1.EXPECT().ID().Return(sourceShard1ID) + sourceShard1.EXPECT().ID().Return(shard1ID) sourceShard1.EXPECT().IsBootstrapped().Return(true) sourceShard1.EXPECT().LatestVolume(start).Return(7, nil) sourceShard1.EXPECT().LatestVolume(start.Add(sourceBlockSize)).Return(17, nil) @@ -1462,8 +1462,18 @@ func TestNamespaceAggregateTiles(t *testing.T) { sourceBlockVolumes1 := []shardBlockVolume{{start, 7}, {secondSourceBlockStart, 17}} sourceNsIDMatcher := ident.NewIDMatcher(sourceNsID.String()) - targetShard0.EXPECT().AggregateTiles(sourceNsIDMatcher, sourceShard0ID, gomock.Any(), gomock.Any(), sourceBlockVolumes0, opts, targetNs.Schema()).Return(int64(3), nil) - targetShard1.EXPECT().AggregateTiles(sourceNsIDMatcher, sourceShard1ID, gomock.Any(), gomock.Any(), sourceBlockVolumes1, opts, targetNs.Schema()).Return(int64(2), nil) + + targetShard0.EXPECT(). + AggregateTiles( + sourceNsIDMatcher, targetNs, shard0ID, gomock.Len(2), gomock.Any(), + sourceBlockVolumes0, opts). + Return(int64(3), nil) + + targetShard1.EXPECT(). + AggregateTiles( + sourceNsIDMatcher, targetNs, shard1ID, gomock.Len(2), gomock.Any(), + sourceBlockVolumes1, opts). + Return(int64(2), nil) processedTileCount, err := targetNs.AggregateTiles(sourceNs, opts) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 292a5dfb05..f32b002392 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -30,9 +30,6 @@ import ( "time" "github.com/m3db/m3/src/dbnode/clock" - "github.com/m3db/m3/src/dbnode/encoding" - "github.com/m3db/m3/src/dbnode/encoding/tile" - "github.com/m3db/m3/src/dbnode/generated/proto/annotation" "github.com/m3db/m3/src/dbnode/generated/proto/pagetoken" "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" @@ -49,7 +46,6 @@ import ( "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/downsample" "github.com/m3db/m3/src/dbnode/ts/writes" "github.com/m3db/m3/src/dbnode/x/xio" "github.com/m3db/m3/src/m3ninx/doc" @@ -73,18 +69,16 @@ const ( ) var ( - errShardEntryNotFound = errors.New("shard entry not found") - errShardNotOpen = errors.New("shard is not open") - errShardAlreadyTicking = errors.New("shard is already ticking") - errShardClosingTickTerminated = errors.New("shard is closing, terminating tick") - errShardInvalidPageToken = errors.New("shard could not unmarshal page token") - errNewShardEntryTagsTypeInvalid = errors.New("new shard entry options error: tags type invalid") - errNewShardEntryTagsIterNotAtIndexZero = errors.New("new shard entry options error: tags iter not at index zero") - errShardIsNotBootstrapped = errors.New("shard is not bootstrapped") - errShardAlreadyBootstrapped = errors.New("shard is already bootstrapped") - errFlushStateIsNotInitialized = errors.New("shard flush state is not initialized") - errFlushStateAlreadyInitialized = errors.New("shard flush state is already initialized") - errTriedToLoadNilSeries = errors.New("tried to load nil series into shard") + errShardEntryNotFound = errors.New("shard entry not found") + errShardNotOpen = errors.New("shard is not open") + errShardAlreadyTicking = errors.New("shard is already ticking") + errShardClosingTickTerminated = errors.New("shard is closing, terminating tick") + errShardInvalidPageToken = errors.New("shard could not unmarshal page token") + errNewShardEntryTagsTypeInvalid = errors.New("new shard entry options error: tags type invalid") + errShardIsNotBootstrapped = errors.New("shard is not bootstrapped") + errShardAlreadyBootstrapped = errors.New("shard is already bootstrapped") + errFlushStateIsNotInitialized = errors.New("shard flush state is not initialized") + errTriedToLoadNilSeries = errors.New("tried to load nil series into shard") // ErrDatabaseLoadLimitHit is the error returned when the database load limit // is hit or exceeded. @@ -189,6 +183,7 @@ type dbShard struct { currRuntimeOptions dbShardRuntimeOptions logger *zap.Logger metrics dbShardMetrics + aggregator Aggregator ticking bool shard uint32 coldWritesEnabled bool @@ -329,6 +324,7 @@ func newDatabaseShard( coldWritesEnabled: namespaceMetadata.Options().ColdWritesEnabled(), logger: opts.InstrumentOptions().Logger(), metrics: newDatabaseShardMetrics(shard, scope), + aggregator: opts.Aggregator(), } s.insertQueue = newDatabaseShardInsertQueue(s.insertSeriesBatch, s.nowFn, scope, opts.InstrumentOptions().Logger()) @@ -2688,21 +2684,24 @@ func (s *dbShard) Repair( func (s *dbShard) AggregateTiles( sourceNsID ident.ID, - sourceShardID uint32, + targetNs Namespace, + shardID uint32, blockReaders []fs.DataFileSetReader, writer fs.StreamingWriter, sourceBlockVolumes []shardBlockVolume, opts AggregateTilesOptions, - targetSchemaDescr namespace.SchemaDescr, ) (int64, error) { if len(blockReaders) != len(sourceBlockVolumes) { - return 0, fmt.Errorf("blockReaders and sourceBlockVolumes length mismatch (%d != %d)", len(blockReaders), len(sourceBlockVolumes)) + return 0, fmt.Errorf( + "blockReaders and sourceBlockVolumes length mismatch (%d != %d)", + len(blockReaders), + len(sourceBlockVolumes)) } openBlockReaders := make([]fs.DataFileSetReader, 0, len(blockReaders)) defer func() { for _, reader := range openBlockReaders { - reader.Close() + _ = reader.Close() } }() @@ -2712,7 +2711,7 @@ func (s *dbShard) AggregateTiles( openOpts := fs.DataReaderOpenOptions{ Identifier: fs.FileSetFileIdentifier{ Namespace: sourceNsID, - Shard: sourceShardID, + Shard: shardID, BlockStart: sourceBlockVolume.blockStart, VolumeIndex: sourceBlockVolume.latestVolume, }, @@ -2731,46 +2730,15 @@ func (s *dbShard) AggregateTiles( zap.Int("volumeIndex", sourceBlockVolume.latestVolume)) return 0, err } - if blockReader.Entries() > maxEntries { - maxEntries = blockReader.Entries() + + entries := blockReader.Entries() + if entries > maxEntries { + maxEntries = entries } openBlockReaders = append(openBlockReaders, blockReader) } - crossBlockReader, err := fs.NewCrossBlockReader(openBlockReaders, s.opts.InstrumentOptions()) - if err != nil { - s.logger.Error("NewCrossBlockReader", zap.Error(err)) - return 0, err - } - defer crossBlockReader.Close() - - tileOpts := tile.Options{ - FrameSize: opts.Step, - Start: xtime.ToUnixNano(opts.Start), - ReaderIteratorPool: s.opts.ReaderIteratorPool(), - } - - readerIter, err := tile.NewSeriesBlockIterator(crossBlockReader, tileOpts) - if err != nil { - s.logger.Error("error when creating new series block iterator", zap.Error(err)) - return 0, err - } - - closed := false - defer func() { - if !closed { - if err := readerIter.Close(); err != nil { - // NB: log the error on ungraceful exit. - s.logger.Error("could not close read iterator on error", zap.Error(err)) - } - } - }() - - encoder := s.opts.EncoderPool().Get() - defer encoder.Close() - encoder.Reset(opts.Start, 0, targetSchemaDescr) - latestTargetVolume, err := s.LatestVolume(opts.Start) if err != nil { return 0, err @@ -2789,54 +2757,11 @@ func (s *dbShard) AggregateTiles( return 0, err } - var ( - annotationPayload annotation.Payload - // NB: there is a maximum of 4 datapoints per frame for counters. - downsampledValues = make([]downsample.Value, 0, 4) - processedTileCount int64 - segmentCapacity int - writerData = make([][]byte, 2) - multiErr xerrors.MultiError - ) - - for readerIter.Next() { - seriesIter, id, encodedTags := readerIter.Current() - - seriesTileCount, err := encodeAggregatedSeries(seriesIter, annotationPayload, downsampledValues, encoder) - if err != nil { - s.metrics.largeTilesWriteErrors.Inc(1) - multiErr = multiErr.Add(err) - break - } - - if seriesTileCount == 0 { - break - } - - processedTileCount += seriesTileCount - segment := encoder.DiscardReset(opts.Start, segmentCapacity, targetSchemaDescr) - - segmentLen := segment.Len() - if segmentLen > segmentCapacity { - // Will use the same capacity for the next series. - segmentCapacity = segmentLen - } - - writerData[0] = segment.Head.Bytes() - writerData[1] = segment.Tail.Bytes() - checksum := segment.CalculateChecksum() - - if err := writer.WriteAll(id, encodedTags, writerData, checksum); err != nil { - s.metrics.largeTilesWriteErrors.Inc(1) - multiErr = multiErr.Add(err) - } else { - s.metrics.largeTilesWrites.Inc(1) - } - - segment.Finalize() - } + var multiErr xerrors.MultiError - if err := readerIter.Err(); err != nil { + processedTileCount, err := s.aggregator.AggregateTiles( + opts, targetNs, s.ID(), openBlockReaders, writer) + if err != nil { multiErr = multiErr.Add(err) } @@ -2848,18 +2773,12 @@ func (s *dbShard) AggregateTiles( multiErr = multiErr.Add(err) } else { // Notify all block leasers that a new volume for the namespace/shard/blockstart - // has been created. This will block until all leasers have relinquished their - // leases. + // has been created. This will block until all leasers have relinquished their leases. if err = s.finishWriting(opts.Start, nextVolume); err != nil { multiErr = multiErr.Add(err) } } - closed = true - if err := readerIter.Close(); err != nil { - multiErr = multiErr.Add(err) - } - if err := multiErr.FinalError(); err != nil { return 0, err } @@ -2871,102 +2790,6 @@ func (s *dbShard) AggregateTiles( return processedTileCount, nil } -func encodeAggregatedSeries( - seriesIter tile.SeriesFrameIterator, - annotationPayload annotation.Payload, - downsampledValues []downsample.Value, - encoder encoding.Encoder, -) (int64, error) { - var ( - prevFrameLastValue = math.NaN() - processedTileCount int64 - handleValueResets bool - firstUnit xtime.Unit - firstAnnotation ts.Annotation - err error - ) - - for seriesIter.Next() { - frame := seriesIter.Current() - - frameValues := frame.Values() - if len(frameValues) == 0 { - continue - } - - if processedTileCount == 0 { - firstUnit, err = frame.Units().Value(0) - if err != nil { - return 0, err - } - - firstAnnotation, err = frame.Annotations().Value(0) - if err != nil { - return 0, err - } - - annotationPayload.Reset() - if annotationPayload.Unmarshal(firstAnnotation) == nil { - // NB: unmarshall error might be a result of some historical annotation data - // which is not compatible with protobuf payload struct. This would generally mean - // that metrics type is unknown, so we should ignore the error here. - handleValueResets = annotationPayload.HandleValueResets - } - } - - downsampledValues = downsampledValues[:0] - lastIdx := len(frameValues) - 1 - - if handleValueResets { - // Last value plus possible few more datapoints to preserve counter semantics. - downsampledValues = downsample.DownsampleCounterResets(prevFrameLastValue, frameValues, downsampledValues) - } else { - // Plain last value per frame. - downsampledValue := downsample.Value{ - FrameIndex: lastIdx, - Value: frameValues[lastIdx], - } - downsampledValues = append(downsampledValues, downsampledValue) - } - - if err = encodeDownsampledValues(downsampledValues, frame, firstUnit, firstAnnotation, encoder); err != nil { - return 0, err - } - - prevFrameLastValue = frameValues[lastIdx] - processedTileCount++ - } - - if err := seriesIter.Err(); err != nil { - return 0, err - } - - return processedTileCount, nil -} - -func encodeDownsampledValues( - downsampledValues []downsample.Value, - frame tile.SeriesBlockFrame, - unit xtime.Unit, - annotation ts.Annotation, - encoder encoding.Encoder, -) error { - for _, downsampledValue := range downsampledValues { - timestamp := frame.Timestamps()[downsampledValue.FrameIndex] - dp := ts.Datapoint{ - Timestamp: timestamp, - TimestampNanos: xtime.ToUnixNano(timestamp), - Value: downsampledValue.Value, - } - - if err := encoder.Encode(dp, unit, annotation); err != nil { - return err - } - } - - return nil -} - func (s *dbShard) BootstrapState() BootstrapState { s.RLock() bs := s.bootstrapState diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index 5c5077d801..598ef80e84 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -21,10 +21,8 @@ package storage import ( - "bytes" "errors" "fmt" - "io" "io/ioutil" "os" "strconv" @@ -35,8 +33,6 @@ import ( "unsafe" "github.com/m3db/m3/src/dbnode/encoding" - "github.com/m3db/m3/src/dbnode/encoding/m3tsz" - "github.com/m3db/m3/src/dbnode/generated/proto/annotation" "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/persist/fs" @@ -200,9 +196,7 @@ func TestShardBootstrapWithFlushVersion(t *testing.T) { opts = DefaultTestOptions() fsOpts = opts.CommitLogOptions().FilesystemOptions(). SetFilePathPrefix(dir) - newClOpts = opts. - CommitLogOptions(). - SetFilesystemOptions(fsOpts) + newClOpts = opts.CommitLogOptions().SetFilesystemOptions(fsOpts) ) opts = opts. SetCommitLogOptions(newClOpts) @@ -278,11 +272,8 @@ func TestShardBootstrapWithFlushVersionNoCleanUp(t *testing.T) { var ( opts = DefaultTestOptions() - fsOpts = opts.CommitLogOptions().FilesystemOptions(). - SetFilePathPrefix(dir) - newClOpts = opts. - CommitLogOptions(). - SetFilesystemOptions(fsOpts) + fsOpts = opts.CommitLogOptions().FilesystemOptions().SetFilePathPrefix(dir) + newClOpts = opts.CommitLogOptions().SetFilesystemOptions(fsOpts) ) opts = opts. SetCommitLogOptions(newClOpts) @@ -336,11 +327,8 @@ func TestShardBootstrapWithCacheShardIndices(t *testing.T) { var ( opts = DefaultTestOptions() - fsOpts = opts.CommitLogOptions().FilesystemOptions(). - SetFilePathPrefix(dir) - newClOpts = opts. - CommitLogOptions(). - SetFilesystemOptions(fsOpts) + fsOpts = opts.CommitLogOptions().FilesystemOptions().SetFilePathPrefix(dir) + newClOpts = opts.CommitLogOptions().SetFilesystemOptions(fsOpts) mockRetriever = block.NewMockDatabaseBlockRetriever(ctrl) ) opts = opts.SetCommitLogOptions(newClOpts) @@ -1929,46 +1917,27 @@ func TestShardAggregateTiles(t *testing.T) { defer ctrl.Finish() var ( - testOpts = DefaultTestOptions() - err error - sourceBlockSize = time.Hour targetBlockSize = 2 * time.Hour start = time.Now().Truncate(targetBlockSize) opts = AggregateTilesOptions{Start: start, End: start.Add(targetBlockSize), Step: 10 * time.Minute} - - id1 = ident.BytesID("id1") - id2 = ident.BytesID("id2") - id3 = ident.BytesID("id3") - - tags1 = ts.EncodedTags("tags1") - tags2 = ts.EncodedTags("tags2") - tags3 = ts.EncodedTags("tags3") - - gaugePayload = &annotation.Payload{MetricType: annotation.MetricType_GAUGE} - counterPayload = &annotation.Payload{MetricType: annotation.MetricType_COUNTER, HandleValueResets: true} + err error ) + aggregator := NewMockAggregator(ctrl) + testOpts := DefaultTestOptions().SetAggregator(aggregator) + sourceShard := testDatabaseShard(t, testOpts) defer sourceShard.Close() - targetShard := testDatabaseShardWithIndexFn(t, testOpts, nil, true) - defer targetShard.Close() - sourceNsID := sourceShard.namespace.ID() reader0, volume0 := getMockReader(ctrl, t, sourceShard, start, true) - reader0.EXPECT().Entries().Return(2).AnyTimes() - reader0.EXPECT().StreamingRead().Return(id1, tags1, dataBytes(t, start, nil, 1, 5), uint32(11), nil) - reader0.EXPECT().StreamingRead().Return(id2, tags2, dataBytes(t, start, counterPayload, 0.5, 1, 2), uint32(22), nil) - reader0.EXPECT().StreamingRead().Return(nil, nil, nil, uint32(0), io.EOF) + reader0.EXPECT().Entries().Return(3) secondSourceBlockStart := start.Add(sourceBlockSize) reader1, volume1 := getMockReader(ctrl, t, sourceShard, secondSourceBlockStart, true) - reader1.EXPECT().Entries().Return(2).AnyTimes() - reader1.EXPECT().StreamingRead().Return(id2, tags2, dataBytes(t, secondSourceBlockStart, counterPayload, 5, 1, 3, 0, 9), uint32(33), nil) - reader1.EXPECT().StreamingRead().Return(id3, tags3, dataBytes(t, secondSourceBlockStart, gaugePayload, 4, 3), uint32(44), nil) - reader1.EXPECT().StreamingRead().Return(nil, nil, nil, uint32(0), io.EOF) + reader1.EXPECT().Entries().Return(2) thirdSourceBlockStart := secondSourceBlockStart.Add(sourceBlockSize) reader2, volume2 := getMockReader(ctrl, t, sourceShard, thirdSourceBlockStart, false) @@ -1980,21 +1949,8 @@ func TestShardAggregateTiles(t *testing.T) { {thirdSourceBlockStart, volume2}, } - write1 := newWrittenDataMatcher(t, []ts.Datapoint{ - dp(start.Add(time.Minute), 5), - }, xtime.Nanosecond, nil) - - write2 := newWrittenDataMatcher(t, []ts.Datapoint{ - dp(start, 0.5), - dp(start.Add(2*time.Minute), 2), - dp(secondSourceBlockStart.Add(2*time.Minute), 5+3), - dp(secondSourceBlockStart.Add(3*time.Minute), 0), - dp(secondSourceBlockStart.Add(4*time.Minute), 9), - }, xtime.Nanosecond, counterPayload) - - write3 := newWrittenDataMatcher(t, []ts.Datapoint{ - dp(secondSourceBlockStart.Add(time.Minute), 3), - }, xtime.Nanosecond, gaugePayload) + targetShard := testDatabaseShardWithIndexFn(t, testOpts, nil, true) + defer targetShard.Close() // nolint:errcheck writer := fs.NewMockStreamingWriter(ctrl) gomock.InOrder( @@ -2004,16 +1960,18 @@ func TestShardAggregateTiles(t *testing.T) { BlockStart: opts.Start, BlockSize: targetBlockSize, VolumeIndex: 1, - PlannedRecordsCount: 2, + PlannedRecordsCount: 3, }), - writer.EXPECT().WriteAll(id1, tags1, write1, gomock.Any()), - writer.EXPECT().WriteAll(id2, tags2, write2, gomock.Any()), - writer.EXPECT().WriteAll(id3, tags3, write3, gomock.Any()), writer.EXPECT().Close(), ) + targetNs := NewMockNamespace(ctrl) + aggregator.EXPECT(). + AggregateTiles(opts, targetNs, sourceShard.ID(), gomock.Len(2), writer). + Return(int64(4), nil) + processedTileCount, err := targetShard.AggregateTiles( - sourceNsID, sourceShard.ID(), blockReaders, writer, sourceBlockVolumes, opts, nil) + sourceNsID, targetNs, sourceShard.ID(), blockReaders, writer, sourceBlockVolumes, opts) require.NoError(t, err) assert.Equal(t, int64(4), processedTileCount) } @@ -2036,7 +1994,7 @@ func TestShardAggregateTilesVerifySliceLengths(t *testing.T) { writer := fs.NewMockStreamingWriter(ctrl) _, err := targetShard.AggregateTiles( - srcNsID, 1, blockReaders, writer, sourceBlockVolumes, AggregateTilesOptions{}, nil) + srcNsID, nil, 1, blockReaders, writer, sourceBlockVolumes, AggregateTilesOptions{}) require.EqualError(t, err, "blockReaders and sourceBlockVolumes length mismatch (0 != 1)") } @@ -2064,8 +2022,6 @@ func getMockReader( reader := fs.NewMockDataFileSetReader(ctrl) if dataFilesetFlushed { reader.EXPECT().Open(openOpts).Return(nil) - reader.EXPECT().StreamingEnabled().Return(true) - reader.EXPECT().Range().Return(xtime.Range{Start: blockStart}) reader.EXPECT().Close() } else { reader.EXPECT().Open(openOpts).Return(fs.ErrCheckpointFileNotFound) @@ -2073,81 +2029,3 @@ func getMockReader( return reader, latestSourceVolume } - -func dataBytes(t *testing.T, start time.Time, annotationPayload *annotation.Payload, values ...float64) []byte { - var ( - encoder = m3tsz.NewEncoder(start, nil, true, encoding.NewOptions()) - timestamp = start - annotationBytes ts.Annotation - err error - ) - - if annotationPayload != nil { - annotationBytes, err = annotationPayload.Marshal() - require.NoError(t, err) - } - - for _, value := range values { - err = encoder.Encode(dp(timestamp, value), xtime.Nanosecond, annotationBytes) - require.NoError(t, err) - timestamp = timestamp.Add(time.Minute) - } - - m3tszSegment := encoder.Discard() - encodedBytes := append(m3tszSegment.Head.Bytes(), m3tszSegment.Tail.Bytes()...) - m3tszSegment.Finalize() - - return encodedBytes -} - -func dp(timestamp time.Time, value float64) ts.Datapoint { - return ts.Datapoint{ - Timestamp: timestamp, - TimestampNanos: xtime.ToUnixNano(timestamp), - Value: value, - } -} - -type writtenDataMatcher struct { - *testing.T - expectedDPs []ts.Datapoint - expectedUnit xtime.Unit - expectedAnnotationPayload *annotation.Payload -} - -func newWrittenDataMatcher( - t *testing.T, - expectedDPs []ts.Datapoint, - expectedUnit xtime.Unit, - expectedAnnotationPayload *annotation.Payload) *writtenDataMatcher { - return &writtenDataMatcher{t, expectedDPs, expectedUnit, expectedAnnotationPayload} -} - -func (w writtenDataMatcher) Matches(x interface{}) bool { - actual, ok := x.([][]byte) - require.True(w, ok) - decoder := m3tsz.NewDecoder(true, encoding.NewOptions()) - iter := decoder.Decode(bytes.NewBuffer(append(actual[0], actual[1]...))) - for i, expectedDP := range w.expectedDPs { - require.True(w, iter.Next()) - dp, unit, annot := iter.Current() - assert.Equal(w, expectedDP, dp) - assert.Equal(w, w.expectedUnit, unit) - if i == 0 { - var annotPayload *annotation.Payload - if annot != nil { - annotPayload = &annotation.Payload{} - require.NoError(w, annotPayload.Unmarshal(annot)) - } - assert.Equal(w, w.expectedAnnotationPayload, annotPayload) - } - } - assert.False(w, iter.Next()) - require.NoError(w, iter.Err()) - - return true -} - -func (w writtenDataMatcher) String() string { - return fmt.Sprint(w.expectedDPs) -} diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index 54698bb6d0..4f520e78a1 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -2301,18 +2301,18 @@ func (mr *MockdatabaseShardMockRecorder) DocRef(id interface{}) *gomock.Call { } // AggregateTiles mocks base method -func (m *MockdatabaseShard) AggregateTiles(sourceNsID ident.ID, sourceShardID uint32, blockReaders []fs.DataFileSetReader, writer fs.StreamingWriter, sourceBlockVolumes []shardBlockVolume, opts AggregateTilesOptions, targetSchemaDesc namespace.SchemaDescr) (int64, error) { +func (m *MockdatabaseShard) AggregateTiles(sourceNsID ident.ID, targetNs Namespace, shardID uint32, blockReaders []fs.DataFileSetReader, writer fs.StreamingWriter, sourceBlockVolumes []shardBlockVolume, opts AggregateTilesOptions) (int64, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "AggregateTiles", sourceNsID, sourceShardID, blockReaders, writer, sourceBlockVolumes, opts, targetSchemaDesc) + ret := m.ctrl.Call(m, "AggregateTiles", sourceNsID, targetNs, shardID, blockReaders, writer, sourceBlockVolumes, opts) ret0, _ := ret[0].(int64) ret1, _ := ret[1].(error) return ret0, ret1 } // AggregateTiles indicates an expected call of AggregateTiles -func (mr *MockdatabaseShardMockRecorder) AggregateTiles(sourceNsID, sourceShardID, blockReaders, writer, sourceBlockVolumes, opts, targetSchemaDesc interface{}) *gomock.Call { +func (mr *MockdatabaseShardMockRecorder) AggregateTiles(sourceNsID, targetNs, shardID, blockReaders, writer, sourceBlockVolumes, opts interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AggregateTiles", reflect.TypeOf((*MockdatabaseShard)(nil).AggregateTiles), sourceNsID, sourceShardID, blockReaders, writer, sourceBlockVolumes, opts, targetSchemaDesc) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AggregateTiles", reflect.TypeOf((*MockdatabaseShard)(nil).AggregateTiles), sourceNsID, targetNs, shardID, blockReaders, writer, sourceBlockVolumes, opts) } // LatestVolume mocks base method diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 7381ba5c52..45c27b8a80 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -675,12 +675,12 @@ type databaseShard interface { // AggregateTiles does large tile aggregation from source shards into this shard. AggregateTiles( sourceNsID ident.ID, - sourceShardID uint32, + targetNs Namespace, + shardID uint32, blockReaders []fs.DataFileSetReader, writer fs.StreamingWriter, sourceBlockVolumes []shardBlockVolume, opts AggregateTilesOptions, - targetSchemaDesc namespace.SchemaDescr, ) (int64, error) // LatestVolume returns the latest volume for the combination of shard+blockStart. From 553771ed7d182632916f44d916b1c9a409b68889 Mon Sep 17 00:00:00 2001 From: arnikola Date: Wed, 11 Nov 2020 08:23:53 -0500 Subject: [PATCH 05/12] [dbnode] Refactor wide query path (#2826) --- .golangci.yml | 14 +- Makefile | 7 +- src/dbnode/encoding/types.go | 8 +- src/dbnode/generated/mocks/generate.go | 3 +- src/dbnode/integration/wide_query_test.go | 114 +--- src/dbnode/persist/fs/fs_mock.go | 405 +++++++------- src/dbnode/persist/fs/msgpack/decoder.go | 37 +- src/dbnode/persist/fs/msgpack/decoder_test.go | 26 +- .../persist/fs/msgpack/roundtrip_test.go | 85 +-- src/dbnode/persist/fs/retriever.go | 493 ++++++++++-------- src/dbnode/persist/fs/retriever_test.go | 4 - src/dbnode/persist/fs/seek.go | 135 +---- src/dbnode/persist/fs/types.go | 114 ++-- .../wide/entry_checksum_mismatch_checker.go | 280 ---------- ...try_checksum_mismatch_checker_prop_test.go | 463 ---------------- .../entry_checksum_mismatch_checker_test.go | 375 ------------- .../wide/index_checksum_block_batch_reader.go | 79 --- .../index_checksum_block_batch_reader_test.go | 49 -- src/dbnode/persist/fs/wide/options.go | 94 ---- src/dbnode/persist/fs/wide/options_test.go | 45 -- src/dbnode/persist/fs/wide/types.go | 117 ----- src/dbnode/persist/fs/wide/wide_mock.go | 147 ------ src/dbnode/persist/schema/types.go | 6 +- src/dbnode/storage/block/block_mock.go | 107 ++-- src/dbnode/storage/block/retriever_manager.go | 18 +- src/dbnode/storage/block/types.go | 63 +-- src/dbnode/storage/database.go | 123 +---- src/dbnode/storage/database_test.go | 68 +-- src/dbnode/storage/index/query_options.go | 13 +- .../storage/index/query_options_test.go | 8 +- src/dbnode/storage/index/types.go | 2 +- .../storage/index/wide_query_results_test.go | 14 +- .../storage/index_queue_forward_write_test.go | 9 +- src/dbnode/storage/index_queue_test.go | 14 +- src/dbnode/storage/namespace.go | 27 +- src/dbnode/storage/options.go | 11 + src/dbnode/storage/series/reader.go | 388 +++++++------- src/dbnode/storage/series/reader_test.go | 45 +- src/dbnode/storage/series/series.go | 27 +- src/dbnode/storage/series/series_mock.go | 59 +-- src/dbnode/storage/series/types.go | 16 +- src/dbnode/storage/shard.go | 39 +- src/dbnode/storage/shard_test.go | 96 +--- src/dbnode/storage/storage_mock.go | 97 +--- src/dbnode/storage/types.go | 47 +- src/dbnode/tracepoint/tracepoint.go | 4 +- .../xio/{index_checksum.go => wide_entry.go} | 33 +- src/x/checked/debug.go | 5 + 48 files changed, 1144 insertions(+), 3289 deletions(-) delete mode 100644 src/dbnode/persist/fs/wide/entry_checksum_mismatch_checker.go delete mode 100644 src/dbnode/persist/fs/wide/entry_checksum_mismatch_checker_prop_test.go delete mode 100644 src/dbnode/persist/fs/wide/entry_checksum_mismatch_checker_test.go delete mode 100644 src/dbnode/persist/fs/wide/index_checksum_block_batch_reader.go delete mode 100644 src/dbnode/persist/fs/wide/index_checksum_block_batch_reader_test.go delete mode 100644 src/dbnode/persist/fs/wide/options.go delete mode 100644 src/dbnode/persist/fs/wide/options_test.go delete mode 100644 src/dbnode/persist/fs/wide/types.go delete mode 100644 src/dbnode/persist/fs/wide/wide_mock.go rename src/dbnode/x/xio/{index_checksum.go => wide_entry.go} (72%) diff --git a/.golangci.yml b/.golangci.yml index 8ecd89e592..5c83931b61 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -108,6 +108,8 @@ linters-settings: locale: US ignore-words: - someword + exhaustive: + default-signifies-exhaustive: true lll: # max line length, lines longer will be reported. Default is 120. # '\t' is counted as 1 character by default, and can be changed with the tab-width option @@ -172,7 +174,6 @@ linters: - goconst - gocritic - gocyclo - - godox - goimports - golint - gosimple @@ -211,10 +212,17 @@ linters: - exhaustivestruct # We allow cuddling assignment following conditions because there are valid # logical groupings for this use-case (e.g. when evaluating config values). - - wsl + - wsl + # Wrapcheck can cause errors until all callsites checking explicit error + # types like io.EOF are converted to use errors.Is instead. Re-enable this + # linter once all error checks are upgraded. + - wrapcheck + # godox prevents using TODOs or FIXMEs which can be useful for demarkation + # of future work. + - godox # New line required before return would require a large fraction of the # code base to need updating, it's not worth the perceived benefit. - - nlreturn + - nlreturn disable-all: false presets: # bodyclose, errcheck, gosec, govet, scopelint, staticcheck, typecheck diff --git a/Makefile b/Makefile index 92943d6670..d54b60a32f 100644 --- a/Makefile +++ b/Makefile @@ -165,7 +165,7 @@ tools-linux-amd64: $(LINUX_AMD64_ENV) make tools .PHONY: all -all: lint test-ci-unit test-ci-integration services tools +all: test-ci-unit test-ci-integration services tools @echo Made all successfully .PHONY: install-tools @@ -256,7 +256,7 @@ SUBDIR_TARGETS := \ asset-gen \ genny-gen \ license-gen \ - all-gen \ + all-gen \ lint .PHONY: test-ci-unit @@ -384,6 +384,7 @@ endef # generate targets across SUBDIRS for each SUBDIR_TARGET. i.e. generate rules # which allow `make all-gen` to invoke `make all-gen-dbnode all-gen-coordinator ...` +# NB: we skip lint explicity as it runs as a separate CI step. $(foreach SUBDIR_TARGET, $(SUBDIR_TARGETS), $(eval $(SUBDIR_TARGET_RULE))) # Builds the single kube bundle from individual manifest files. @@ -401,7 +402,7 @@ go-mod-tidy: .PHONY: all-gen all-gen: \ install-tools \ - $(foreach SUBDIR_TARGET, $(SUBDIR_TARGETS), $(SUBDIR_TARGET)) \ + $(foreach SUBDIR_TARGET, $(filter-out lint all-gen,$(SUBDIR_TARGETS)), $(SUBDIR_TARGET)) \ kube-gen-all \ go-mod-tidy diff --git a/src/dbnode/encoding/types.go b/src/dbnode/encoding/types.go index 93af1b4746..230925b340 100644 --- a/src/dbnode/encoding/types.go +++ b/src/dbnode/encoding/types.go @@ -88,7 +88,7 @@ type Encoder interface { DiscardReset(t time.Time, capacity int, schema namespace.SchemaDescr) ts.Segment } -// NewEncoderFn creates a new encoder +// NewEncoderFn creates a new encoder. type NewEncoderFn func(start time.Time, bytes []byte) Encoder // Options represents different options for encoding time as well as markers. @@ -178,7 +178,7 @@ type Iterator interface { // object as it may get invalidated when the iterator calls Next(). Current() (ts.Datapoint, xtime.Unit, ts.Annotation) - // Err returns the error encountered + // Err returns the error encountered. Err() error // Close closes the iterator and if pooled will return to the pool. @@ -367,9 +367,9 @@ type IStream interface { // OStream encapsulates a writable stream. type OStream interface { - // Len returns the length of the OStream + // Len returns the length of the OStream. Len() int - // Empty returns whether the OStream is empty + // Empty returns whether the OStream is empty. Empty() bool // WriteBit writes the last bit of v. diff --git a/src/dbnode/generated/mocks/generate.go b/src/dbnode/generated/mocks/generate.go index 8da07ecdc5..3869d17577 100644 --- a/src/dbnode/generated/mocks/generate.go +++ b/src/dbnode/generated/mocks/generate.go @@ -20,8 +20,7 @@ // mockgen rules for generating mocks for exported interfaces (reflection mode) -//go:generate sh -c "mockgen -package=wide $PACKAGE/src/dbnode/persist/fs/wide EntryChecksumMismatchChecker,StreamedMismatch | genclean -pkg $PACKAGE/src/dbnode/persist/fs/wide -out $GOPATH/src/$PACKAGE/src/dbnode/persist/fs/wide/wide_mock.go" -//go:generate sh -c "mockgen -package=fs $PACKAGE/src/dbnode/persist/fs DataFileSetWriter,DataFileSetReader,DataFileSetSeeker,IndexFileSetWriter,IndexFileSetReader,IndexSegmentFileSetWriter,IndexSegmentFileSet,IndexSegmentFile,SnapshotMetadataFileWriter,DataFileSetSeekerManager,ConcurrentDataFileSetSeeker,MergeWith,CrossBlockReader,CrossBlockIterator,StreamingWriter | genclean -pkg $PACKAGE/src/dbnode/persist/fs -out $GOPATH/src/$PACKAGE/src/dbnode/persist/fs/fs_mock.go" +//go:generate sh -c "mockgen -package=fs $PACKAGE/src/dbnode/persist/fs CrossBlockReader,CrossBlockIterator,DataFileSetWriter,DataFileSetReader,DataFileSetSeeker,IndexFileSetWriter,IndexFileSetReader,IndexSegmentFileSetWriter,IndexSegmentFileSet,IndexSegmentFile,SnapshotMetadataFileWriter,DataFileSetSeekerManager,ConcurrentDataFileSetSeeker,MergeWith,StreamingWriter | genclean -pkg $PACKAGE/src/dbnode/persist/fs -out $GOPATH/src/$PACKAGE/src/dbnode/persist/fs/fs_mock.go" //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" diff --git a/src/dbnode/integration/wide_query_test.go b/src/dbnode/integration/wide_query_test.go index ff95fb918a..17192d9841 100644 --- a/src/dbnode/integration/wide_query_test.go +++ b/src/dbnode/integration/wide_query_test.go @@ -1,4 +1,4 @@ -// +build integration +// +build big // // Copyright (c) 2020 Uber Technologies, Inc. // @@ -23,7 +23,6 @@ package integration import ( - "bytes" "fmt" "io/ioutil" "runtime" @@ -32,10 +31,8 @@ import ( "testing" "time" - "github.com/m3db/m3/src/dbnode/encoding/m3tsz" "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist/fs" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/persist/schema" "github.com/m3db/m3/src/dbnode/sharding" "github.com/m3db/m3/src/dbnode/storage" @@ -60,9 +57,9 @@ const ( wideTagValFmt = "val-%05d" ) -type shardedIndexChecksum struct { - shard uint32 - checksums []schema.IndexChecksum +type shardedWideEntry struct { + shard uint32 + entries []schema.WideEntry } // buildExpectedChecksumsByShard sorts the given IDs into ascending shard order, @@ -72,10 +69,10 @@ func buildExpectedChecksumsByShard( allowedShards []uint32, shardSet sharding.ShardSet, batchSize int, -) []schema.IndexChecksum { - shardedChecksums := make([]shardedIndexChecksum, 0, len(ids)) +) []schema.WideEntry { + shardedEntries := make([]shardedWideEntry, 0, len(ids)) for i, id := range ids { - checksum := schema.IndexChecksum{ + entry := schema.WideEntry{ IndexEntry: schema.IndexEntry{ ID: []byte(id), }, @@ -98,13 +95,13 @@ func buildExpectedChecksumsByShard( } found := false - for idx, sharded := range shardedChecksums { + for idx, sharded := range shardedEntries { if shard != sharded.shard { continue } found = true - shardedChecksums[idx].checksums = append(sharded.checksums, checksum) + shardedEntries[idx].entries = append(sharded.entries, entry) break } @@ -112,36 +109,22 @@ func buildExpectedChecksumsByShard( continue } - shardedChecksums = append(shardedChecksums, shardedIndexChecksum{ - shard: shard, - checksums: []schema.IndexChecksum{checksum}, + shardedEntries = append(shardedEntries, shardedWideEntry{ + shard: shard, + entries: []schema.WideEntry{entry}, }) } - sort.Slice(shardedChecksums, func(i, j int) bool { - return shardedChecksums[i].shard < shardedChecksums[j].shard + sort.Slice(shardedEntries, func(i, j int) bool { + return shardedEntries[i].shard < shardedEntries[j].shard }) - var checksums []schema.IndexChecksum - for _, sharded := range shardedChecksums { - checksums = append(checksums, sharded.checksums...) + var entries []schema.WideEntry + for _, sharded := range shardedEntries { + entries = append(entries, sharded.entries...) } - // NB: IDs should only be included for documents that conclude a batch. - l := len(checksums) - if l == 0 { - return checksums - } - - // NB: only look at the last `l-1` elements, as the last element should - // always have its ID. - for i, checksum := range checksums[:l-1] { - if (i+1)%batchSize != 0 { - checksums[i].ID = checksum.ID[:0] - } - } - - return checksums + return entries } func assertTags( @@ -164,28 +147,6 @@ func assertTags( require.NoError(t, decoder.Err()) } -func assertData( - t *testing.T, - ex int64, - exTime time.Time, - mismatch wide.ReadMismatch, -) { - mismatch.Data.IncRef() - mismatchData := mismatch.Data.Bytes() - mismatch.Data.DecRef() - - decoder := m3tsz.NewDecoder(true, nil) - dataReader := bytes.NewBuffer(mismatchData) - it := decoder.Decode(dataReader) - assert.NoError(t, it.Err()) - assert.True(t, it.Next()) - ts, _, _ := it.Current() - assert.True(t, ts.Timestamp.Equal(exTime)) - assert.Equal(t, float64(ex), ts.Value) - assert.False(t, it.Next()) - assert.NoError(t, it.Err()) -} - func TestWideFetch(t *testing.T) { if testing.Short() { t.SkipNow() // Just skip if we're doing a short run @@ -315,10 +276,6 @@ func TestWideFetch(t *testing.T) { decoder := tagDecoderPool.Get() defer decoder.Close() - wideOpts := wide.NewOptions(). - SetDecodingOptions(decOpts). - SetBatchSize(batchSize) - for _, tt := range shardFilterTests { t.Run(tt.name, func(t *testing.T) { ctx := context.NewContext() @@ -333,32 +290,6 @@ func TestWideFetch(t *testing.T) { assert.Equal(t, expected[i].MetadataChecksum, checksum.MetadataChecksum) require.Equal(t, string(expected[i].ID), checksum.ID.String()) assertTags(t, checksum.EncodedTags, decoder, checksum.MetadataChecksum) - checksum.Finalize() - } - - ctx.Close() - }) - - t.Run(fmt.Sprintf("%s_checksum_mismatch", tt.name), func(t *testing.T) { - ctx := context.NewContext() - // NB: empty index checksum blocks. - inCh := make(chan wide.IndexChecksumBlockBatch) - batchReader := wide.NewIndexChecksumBlockBatchReader(inCh) - close(inCh) - - checker := wide.NewEntryChecksumMismatchChecker(batchReader, wideOpts) - mismatches, err := testSetup.DB().ReadMismatches(ctx, nsMetadata.ID(), query, - checker, now, tt.shards, iterOpts) - require.NoError(t, err) - - expected := buildExpectedChecksumsByShard(ids, tt.shards, - testSetup.ShardSet(), batchSize) - require.Equal(t, len(expected), len(mismatches)) - for i, mismatch := range mismatches { - assert.Equal(t, expected[i].MetadataChecksum, mismatch.MetadataChecksum) - assertTags(t, mismatch.EncodedTags, decoder, mismatch.MetadataChecksum) - assertData(t, expected[i].MetadataChecksum, now, mismatch) - mismatch.Finalize() } ctx.Close() @@ -394,15 +325,14 @@ func TestWideFetch(t *testing.T) { now, tt.shards, iterOpts) require.NoError(t, err) - if !tt.expected { - assert.Equal(t, 0, len(chk)) - } else { + if tt.expected { require.Equal(t, 1, len(chk)) checksum := chk[0] assert.Equal(t, int64(1), checksum.MetadataChecksum) assert.Equal(t, exactID, checksum.ID.String()) assertTags(t, checksum.EncodedTags, decoder, checksum.MetadataChecksum) - checksum.Finalize() + } else { + assert.Equal(t, 0, len(chk)) } ctx.Close() @@ -443,8 +373,6 @@ func TestWideFetch(t *testing.T) { expected[i].MetadataChecksum, checksum.MetadataChecksum) break } - - checksum.Finalize() } ctx.Close() diff --git a/src/dbnode/persist/fs/fs_mock.go b/src/dbnode/persist/fs/fs_mock.go index 11da9e4812..861c2659a1 100644 --- a/src/dbnode/persist/fs/fs_mock.go +++ b/src/dbnode/persist/fs/fs_mock.go @@ -1,5 +1,5 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/m3db/m3/src/dbnode/persist/fs (interfaces: DataFileSetWriter,DataFileSetReader,DataFileSetSeeker,IndexFileSetWriter,IndexFileSetReader,IndexSegmentFileSetWriter,IndexSegmentFileSet,IndexSegmentFile,SnapshotMetadataFileWriter,DataFileSetSeekerManager,ConcurrentDataFileSetSeeker,MergeWith,CrossBlockReader,CrossBlockIterator,StreamingWriter) +// Source: github.com/m3db/m3/src/dbnode/persist/fs (interfaces: CrossBlockReader,CrossBlockIterator,DataFileSetWriter,DataFileSetReader,DataFileSetSeeker,IndexFileSetWriter,IndexFileSetReader,IndexSegmentFileSetWriter,IndexSegmentFileSet,IndexSegmentFile,SnapshotMetadataFileWriter,DataFileSetSeekerManager,ConcurrentDataFileSetSeeker,MergeWith,StreamingWriter) // Copyright (c) 2020 Uber Technologies, Inc. // @@ -31,7 +31,6 @@ import ( "github.com/m3db/m3/src/dbnode/namespace" persist "github.com/m3db/m3/src/dbnode/persist" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/sharding" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/dbnode/x/xio" @@ -44,6 +43,178 @@ import ( "github.com/golang/mock/gomock" ) +// MockCrossBlockReader is a mock of CrossBlockReader interface +type MockCrossBlockReader struct { + ctrl *gomock.Controller + recorder *MockCrossBlockReaderMockRecorder +} + +// MockCrossBlockReaderMockRecorder is the mock recorder for MockCrossBlockReader +type MockCrossBlockReaderMockRecorder struct { + mock *MockCrossBlockReader +} + +// NewMockCrossBlockReader creates a new mock instance +func NewMockCrossBlockReader(ctrl *gomock.Controller) *MockCrossBlockReader { + mock := &MockCrossBlockReader{ctrl: ctrl} + mock.recorder = &MockCrossBlockReaderMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use +func (m *MockCrossBlockReader) EXPECT() *MockCrossBlockReaderMockRecorder { + return m.recorder +} + +// Close mocks base method +func (m *MockCrossBlockReader) Close() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Close") + ret0, _ := ret[0].(error) + return ret0 +} + +// Close indicates an expected call of Close +func (mr *MockCrossBlockReaderMockRecorder) Close() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockCrossBlockReader)(nil).Close)) +} + +// Current mocks base method +func (m *MockCrossBlockReader) Current() (ident.BytesID, ts.EncodedTags, []BlockRecord) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Current") + ret0, _ := ret[0].(ident.BytesID) + ret1, _ := ret[1].(ts.EncodedTags) + ret2, _ := ret[2].([]BlockRecord) + return ret0, ret1, ret2 +} + +// Current indicates an expected call of Current +func (mr *MockCrossBlockReaderMockRecorder) Current() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Current", reflect.TypeOf((*MockCrossBlockReader)(nil).Current)) +} + +// Err mocks base method +func (m *MockCrossBlockReader) Err() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Err") + ret0, _ := ret[0].(error) + return ret0 +} + +// Err indicates an expected call of Err +func (mr *MockCrossBlockReaderMockRecorder) Err() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Err", reflect.TypeOf((*MockCrossBlockReader)(nil).Err)) +} + +// Next mocks base method +func (m *MockCrossBlockReader) Next() bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Next") + ret0, _ := ret[0].(bool) + return ret0 +} + +// Next indicates an expected call of Next +func (mr *MockCrossBlockReaderMockRecorder) Next() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Next", reflect.TypeOf((*MockCrossBlockReader)(nil).Next)) +} + +// MockCrossBlockIterator is a mock of CrossBlockIterator interface +type MockCrossBlockIterator struct { + ctrl *gomock.Controller + recorder *MockCrossBlockIteratorMockRecorder +} + +// MockCrossBlockIteratorMockRecorder is the mock recorder for MockCrossBlockIterator +type MockCrossBlockIteratorMockRecorder struct { + mock *MockCrossBlockIterator +} + +// NewMockCrossBlockIterator creates a new mock instance +func NewMockCrossBlockIterator(ctrl *gomock.Controller) *MockCrossBlockIterator { + mock := &MockCrossBlockIterator{ctrl: ctrl} + mock.recorder = &MockCrossBlockIteratorMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use +func (m *MockCrossBlockIterator) EXPECT() *MockCrossBlockIteratorMockRecorder { + return m.recorder +} + +// Close mocks base method +func (m *MockCrossBlockIterator) Close() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Close") +} + +// Close indicates an expected call of Close +func (mr *MockCrossBlockIteratorMockRecorder) Close() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockCrossBlockIterator)(nil).Close)) +} + +// Current mocks base method +func (m *MockCrossBlockIterator) Current() (ts.Datapoint, time0.Unit, ts.Annotation) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Current") + ret0, _ := ret[0].(ts.Datapoint) + ret1, _ := ret[1].(time0.Unit) + ret2, _ := ret[2].(ts.Annotation) + return ret0, ret1, ret2 +} + +// Current indicates an expected call of Current +func (mr *MockCrossBlockIteratorMockRecorder) Current() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Current", reflect.TypeOf((*MockCrossBlockIterator)(nil).Current)) +} + +// Err mocks base method +func (m *MockCrossBlockIterator) Err() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Err") + ret0, _ := ret[0].(error) + return ret0 +} + +// Err indicates an expected call of Err +func (mr *MockCrossBlockIteratorMockRecorder) Err() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Err", reflect.TypeOf((*MockCrossBlockIterator)(nil).Err)) +} + +// Next mocks base method +func (m *MockCrossBlockIterator) Next() bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Next") + ret0, _ := ret[0].(bool) + return ret0 +} + +// Next indicates an expected call of Next +func (mr *MockCrossBlockIteratorMockRecorder) Next() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Next", reflect.TypeOf((*MockCrossBlockIterator)(nil).Next)) +} + +// Reset mocks base method +func (m *MockCrossBlockIterator) Reset(arg0 []BlockRecord) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Reset", arg0) +} + +// Reset indicates an expected call of Reset +func (mr *MockCrossBlockIteratorMockRecorder) Reset(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Reset", reflect.TypeOf((*MockCrossBlockIterator)(nil).Reset), arg0) +} + // MockDataFileSetWriter is a mock of DataFileSetWriter interface type MockDataFileSetWriter struct { ctrl *gomock.Controller @@ -523,34 +694,19 @@ func (mr *MockDataFileSetSeekerMockRecorder) SeekIndexEntry(arg0, arg1 interface return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SeekIndexEntry", reflect.TypeOf((*MockDataFileSetSeeker)(nil).SeekIndexEntry), arg0, arg1) } -// SeekIndexEntryToIndexChecksum mocks base method -func (m *MockDataFileSetSeeker) SeekIndexEntryToIndexChecksum(arg0 ident.ID, arg1 ReusableSeekerResources) (xio.IndexChecksum, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "SeekIndexEntryToIndexChecksum", arg0, arg1) - ret0, _ := ret[0].(xio.IndexChecksum) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// SeekIndexEntryToIndexChecksum indicates an expected call of SeekIndexEntryToIndexChecksum -func (mr *MockDataFileSetSeekerMockRecorder) SeekIndexEntryToIndexChecksum(arg0, arg1 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SeekIndexEntryToIndexChecksum", reflect.TypeOf((*MockDataFileSetSeeker)(nil).SeekIndexEntryToIndexChecksum), arg0, arg1) -} - -// SeekReadMismatchesByIndexChecksum mocks base method -func (m *MockDataFileSetSeeker) SeekReadMismatchesByIndexChecksum(arg0 xio.IndexChecksum, arg1 wide.EntryChecksumMismatchChecker, arg2 ReusableSeekerResources) (wide.ReadMismatch, error) { +// SeekWideEntry mocks base method +func (m *MockDataFileSetSeeker) SeekWideEntry(arg0 ident.ID, arg1 ReusableSeekerResources) (xio.WideEntry, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "SeekReadMismatchesByIndexChecksum", arg0, arg1, arg2) - ret0, _ := ret[0].(wide.ReadMismatch) + ret := m.ctrl.Call(m, "SeekWideEntry", arg0, arg1) + ret0, _ := ret[0].(xio.WideEntry) ret1, _ := ret[1].(error) return ret0, ret1 } -// SeekReadMismatchesByIndexChecksum indicates an expected call of SeekReadMismatchesByIndexChecksum -func (mr *MockDataFileSetSeekerMockRecorder) SeekReadMismatchesByIndexChecksum(arg0, arg1, arg2 interface{}) *gomock.Call { +// SeekWideEntry indicates an expected call of SeekWideEntry +func (mr *MockDataFileSetSeekerMockRecorder) SeekWideEntry(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SeekReadMismatchesByIndexChecksum", reflect.TypeOf((*MockDataFileSetSeeker)(nil).SeekReadMismatchesByIndexChecksum), arg0, arg1, arg2) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SeekWideEntry", reflect.TypeOf((*MockDataFileSetSeeker)(nil).SeekWideEntry), arg0, arg1) } // MockIndexFileSetWriter is a mock of IndexFileSetWriter interface @@ -1274,34 +1430,19 @@ func (mr *MockConcurrentDataFileSetSeekerMockRecorder) SeekIndexEntry(arg0, arg1 return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SeekIndexEntry", reflect.TypeOf((*MockConcurrentDataFileSetSeeker)(nil).SeekIndexEntry), arg0, arg1) } -// SeekIndexEntryToIndexChecksum mocks base method -func (m *MockConcurrentDataFileSetSeeker) SeekIndexEntryToIndexChecksum(arg0 ident.ID, arg1 ReusableSeekerResources) (xio.IndexChecksum, error) { +// SeekWideEntry mocks base method +func (m *MockConcurrentDataFileSetSeeker) SeekWideEntry(arg0 ident.ID, arg1 ReusableSeekerResources) (xio.WideEntry, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "SeekIndexEntryToIndexChecksum", arg0, arg1) - ret0, _ := ret[0].(xio.IndexChecksum) + ret := m.ctrl.Call(m, "SeekWideEntry", arg0, arg1) + ret0, _ := ret[0].(xio.WideEntry) ret1, _ := ret[1].(error) return ret0, ret1 } -// SeekIndexEntryToIndexChecksum indicates an expected call of SeekIndexEntryToIndexChecksum -func (mr *MockConcurrentDataFileSetSeekerMockRecorder) SeekIndexEntryToIndexChecksum(arg0, arg1 interface{}) *gomock.Call { +// SeekWideEntry indicates an expected call of SeekWideEntry +func (mr *MockConcurrentDataFileSetSeekerMockRecorder) SeekWideEntry(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SeekIndexEntryToIndexChecksum", reflect.TypeOf((*MockConcurrentDataFileSetSeeker)(nil).SeekIndexEntryToIndexChecksum), arg0, arg1) -} - -// SeekReadMismatchesByIndexChecksum mocks base method -func (m *MockConcurrentDataFileSetSeeker) SeekReadMismatchesByIndexChecksum(arg0 xio.IndexChecksum, arg1 wide.EntryChecksumMismatchChecker, arg2 ReusableSeekerResources) (wide.ReadMismatch, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "SeekReadMismatchesByIndexChecksum", arg0, arg1, arg2) - ret0, _ := ret[0].(wide.ReadMismatch) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// SeekReadMismatchesByIndexChecksum indicates an expected call of SeekReadMismatchesByIndexChecksum -func (mr *MockConcurrentDataFileSetSeekerMockRecorder) SeekReadMismatchesByIndexChecksum(arg0, arg1, arg2 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SeekReadMismatchesByIndexChecksum", reflect.TypeOf((*MockConcurrentDataFileSetSeeker)(nil).SeekReadMismatchesByIndexChecksum), arg0, arg1, arg2) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SeekWideEntry", reflect.TypeOf((*MockConcurrentDataFileSetSeeker)(nil).SeekWideEntry), arg0, arg1) } // MockMergeWith is a mock of MergeWith interface @@ -1357,178 +1498,6 @@ func (mr *MockMergeWithMockRecorder) Read(arg0, arg1, arg2, arg3 interface{}) *g return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Read", reflect.TypeOf((*MockMergeWith)(nil).Read), arg0, arg1, arg2, arg3) } -// MockCrossBlockReader is a mock of CrossBlockReader interface -type MockCrossBlockReader struct { - ctrl *gomock.Controller - recorder *MockCrossBlockReaderMockRecorder -} - -// MockCrossBlockReaderMockRecorder is the mock recorder for MockCrossBlockReader -type MockCrossBlockReaderMockRecorder struct { - mock *MockCrossBlockReader -} - -// NewMockCrossBlockReader creates a new mock instance -func NewMockCrossBlockReader(ctrl *gomock.Controller) *MockCrossBlockReader { - mock := &MockCrossBlockReader{ctrl: ctrl} - mock.recorder = &MockCrossBlockReaderMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *MockCrossBlockReader) EXPECT() *MockCrossBlockReaderMockRecorder { - return m.recorder -} - -// Close mocks base method -func (m *MockCrossBlockReader) Close() error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Close") - ret0, _ := ret[0].(error) - return ret0 -} - -// Close indicates an expected call of Close -func (mr *MockCrossBlockReaderMockRecorder) Close() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockCrossBlockReader)(nil).Close)) -} - -// Current mocks base method -func (m *MockCrossBlockReader) Current() (ident.BytesID, ts.EncodedTags, []BlockRecord) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Current") - ret0, _ := ret[0].(ident.BytesID) - ret1, _ := ret[1].(ts.EncodedTags) - ret2, _ := ret[2].([]BlockRecord) - return ret0, ret1, ret2 -} - -// Current indicates an expected call of Current -func (mr *MockCrossBlockReaderMockRecorder) Current() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Current", reflect.TypeOf((*MockCrossBlockReader)(nil).Current)) -} - -// Err mocks base method -func (m *MockCrossBlockReader) Err() error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Err") - ret0, _ := ret[0].(error) - return ret0 -} - -// Err indicates an expected call of Err -func (mr *MockCrossBlockReaderMockRecorder) Err() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Err", reflect.TypeOf((*MockCrossBlockReader)(nil).Err)) -} - -// Next mocks base method -func (m *MockCrossBlockReader) Next() bool { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Next") - ret0, _ := ret[0].(bool) - return ret0 -} - -// Next indicates an expected call of Next -func (mr *MockCrossBlockReaderMockRecorder) Next() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Next", reflect.TypeOf((*MockCrossBlockReader)(nil).Next)) -} - -// MockCrossBlockIterator is a mock of CrossBlockIterator interface -type MockCrossBlockIterator struct { - ctrl *gomock.Controller - recorder *MockCrossBlockIteratorMockRecorder -} - -// MockCrossBlockIteratorMockRecorder is the mock recorder for MockCrossBlockIterator -type MockCrossBlockIteratorMockRecorder struct { - mock *MockCrossBlockIterator -} - -// NewMockCrossBlockIterator creates a new mock instance -func NewMockCrossBlockIterator(ctrl *gomock.Controller) *MockCrossBlockIterator { - mock := &MockCrossBlockIterator{ctrl: ctrl} - mock.recorder = &MockCrossBlockIteratorMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *MockCrossBlockIterator) EXPECT() *MockCrossBlockIteratorMockRecorder { - return m.recorder -} - -// Close mocks base method -func (m *MockCrossBlockIterator) Close() { - m.ctrl.T.Helper() - m.ctrl.Call(m, "Close") -} - -// Close indicates an expected call of Close -func (mr *MockCrossBlockIteratorMockRecorder) Close() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockCrossBlockIterator)(nil).Close)) -} - -// Current mocks base method -func (m *MockCrossBlockIterator) Current() (ts.Datapoint, time0.Unit, ts.Annotation) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Current") - ret0, _ := ret[0].(ts.Datapoint) - ret1, _ := ret[1].(time0.Unit) - ret2, _ := ret[2].(ts.Annotation) - return ret0, ret1, ret2 -} - -// Current indicates an expected call of Current -func (mr *MockCrossBlockIteratorMockRecorder) Current() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Current", reflect.TypeOf((*MockCrossBlockIterator)(nil).Current)) -} - -// Err mocks base method -func (m *MockCrossBlockIterator) Err() error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Err") - ret0, _ := ret[0].(error) - return ret0 -} - -// Err indicates an expected call of Err -func (mr *MockCrossBlockIteratorMockRecorder) Err() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Err", reflect.TypeOf((*MockCrossBlockIterator)(nil).Err)) -} - -// Next mocks base method -func (m *MockCrossBlockIterator) Next() bool { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Next") - ret0, _ := ret[0].(bool) - return ret0 -} - -// Next indicates an expected call of Next -func (mr *MockCrossBlockIteratorMockRecorder) Next() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Next", reflect.TypeOf((*MockCrossBlockIterator)(nil).Next)) -} - -// Reset mocks base method -func (m *MockCrossBlockIterator) Reset(arg0 []BlockRecord) { - m.ctrl.T.Helper() - m.ctrl.Call(m, "Reset", arg0) -} - -// Reset indicates an expected call of Reset -func (mr *MockCrossBlockIteratorMockRecorder) Reset(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Reset", reflect.TypeOf((*MockCrossBlockIterator)(nil).Reset), arg0) -} - // MockStreamingWriter is a mock of StreamingWriter interface type MockStreamingWriter struct { ctrl *gomock.Controller diff --git a/src/dbnode/persist/fs/msgpack/decoder.go b/src/dbnode/persist/fs/msgpack/decoder.go index e2126a3873..532a3c66b3 100644 --- a/src/dbnode/persist/fs/msgpack/decoder.go +++ b/src/dbnode/persist/fs/msgpack/decoder.go @@ -38,6 +38,7 @@ var ( emptyIndexSummariesInfo schema.IndexSummariesInfo emptyIndexBloomFilterInfo schema.IndexBloomFilterInfo emptyIndexEntry schema.IndexEntry + emptyWideEntry schema.WideEntry emptyIndexSummary schema.IndexSummary emptyIndexSummaryToken IndexSummaryToken emptyLogInfo schema.LogInfo @@ -50,12 +51,12 @@ var ( errorIndexEntryChecksumMismatch = errors.New("decode index entry encountered checksum mismatch") ) -// IndexChecksumLookupStatus is the status for an index checksum lookup. -type IndexChecksumLookupStatus byte +// WideEntryLookupStatus is the status for a wide entry lookup. +type WideEntryLookupStatus byte const ( // ErrorLookupStatus indicates an error state. - ErrorLookupStatus IndexChecksumLookupStatus = iota + ErrorLookupStatus WideEntryLookupStatus = iota // MatchedLookupStatus indicates the current entry ID matches the requested ID. MatchedLookupStatus // MismatchLookupStatus indicates the current entry ID preceeds the requested ID. @@ -151,24 +152,24 @@ func (dec *Decoder) DecodeIndexEntry(bytesPool pool.BytesPool) (schema.IndexEntr return indexEntry, nil } -// DecodeIndexEntryToIndexChecksum decodes an index entry into a minimal index entry. -func (dec *Decoder) DecodeIndexEntryToIndexChecksum( +// DecodeToWideEntry decodes an index entry into a wide entry. +func (dec *Decoder) DecodeToWideEntry( compareID []byte, bytesPool pool.BytesPool, -) (schema.IndexChecksum, IndexChecksumLookupStatus, error) { +) (schema.WideEntry, WideEntryLookupStatus, error) { if dec.err != nil { - return schema.IndexChecksum{}, NotFoundLookupStatus, dec.err + return emptyWideEntry, NotFoundLookupStatus, dec.err } dec.readerWithDigest.setDigestReaderEnabled(true) _, numFieldsToSkip := dec.decodeRootObject(indexEntryVersion, indexEntryType) - indexWithMetaChecksum, status := dec.decodeIndexChecksum(compareID, bytesPool) + entry, status := dec.decodeWideEntry(compareID, bytesPool) dec.readerWithDigest.setDigestReaderEnabled(false) dec.skip(numFieldsToSkip) if status != MatchedLookupStatus || dec.err != nil { - return schema.IndexChecksum{}, status, dec.err + return emptyWideEntry, status, dec.err } - return indexWithMetaChecksum, status, nil + return entry, status, nil } // DecodeIndexSummary decodes index summary. @@ -482,13 +483,13 @@ func (dec *Decoder) decodeIndexEntry(bytesPool pool.BytesPool) schema.IndexEntry return indexEntry } -func (dec *Decoder) decodeIndexChecksum( +func (dec *Decoder) decodeWideEntry( compareID []byte, bytesPool pool.BytesPool, -) (schema.IndexChecksum, IndexChecksumLookupStatus) { +) (schema.WideEntry, WideEntryLookupStatus) { entry := dec.decodeIndexEntry(bytesPool) if dec.err != nil { - return schema.IndexChecksum{}, ErrorLookupStatus + return emptyWideEntry, ErrorLookupStatus } if entry.EncodedTags == nil { @@ -496,8 +497,8 @@ func (dec *Decoder) decodeIndexChecksum( bytesPool.Put(entry.ID) } - dec.err = fmt.Errorf("decode index checksum requires files V1+") - return schema.IndexChecksum{}, ErrorLookupStatus + dec.err = fmt.Errorf("decode wide index requires files V1+") + return emptyWideEntry, ErrorLookupStatus } compare := bytes.Compare(compareID, entry.ID) @@ -505,7 +506,7 @@ func (dec *Decoder) decodeIndexChecksum( if compare == 0 { // NB: need to compute hash before freeing entry bytes. checksum = dec.hasher.HashIndexEntry(entry) - return schema.IndexChecksum{ + return schema.WideEntry{ IndexEntry: entry, MetadataChecksum: checksum, }, MatchedLookupStatus @@ -518,12 +519,12 @@ func (dec *Decoder) decodeIndexChecksum( if compare > 0 { // compareID can still exist after the current entry.ID - return schema.IndexChecksum{}, MismatchLookupStatus + return emptyWideEntry, MismatchLookupStatus } // compareID must have been before the curret entry.ID, so this // ID will not be matched. - return schema.IndexChecksum{}, NotFoundLookupStatus + return emptyWideEntry, NotFoundLookupStatus } func (dec *Decoder) decodeIndexSummary() (schema.IndexSummary, IndexSummaryToken) { diff --git a/src/dbnode/persist/fs/msgpack/decoder_test.go b/src/dbnode/persist/fs/msgpack/decoder_test.go index 126f92bea5..56a40fcb04 100644 --- a/src/dbnode/persist/fs/msgpack/decoder_test.go +++ b/src/dbnode/persist/fs/msgpack/decoder_test.go @@ -269,7 +269,7 @@ func TestDecodeBytesAllocNew(t *testing.T) { require.Equal(t, []byte("testIndexEntry"), res.ID) } -func TestDecodeIndexEntryInvalidChecksum(t *testing.T) { +func TestDecodeIndexEntryInvalidWideEntry(t *testing.T) { var ( enc = NewEncoder() dec = NewDecoder(nil) @@ -299,9 +299,9 @@ func TestDecodeIndexEntryIncompleteFile(t *testing.T) { require.EqualError(t, err, "decode index entry encountered error: EOF") } -var decodeIndexChecksumTests = []struct { +var decodeWideEntryTests = []struct { id string - exStatus IndexChecksumLookupStatus + exStatus WideEntryLookupStatus exChecksum int64 }{ {id: "aaa", exStatus: NotFoundLookupStatus}, @@ -309,19 +309,19 @@ var decodeIndexChecksumTests = []struct { {id: "zzz", exStatus: MismatchLookupStatus}, } -func TestDecodeIndexEntryToIndexChecksum(t *testing.T) { +func TestDecodeToWideEntry(t *testing.T) { var ( enc = NewEncoder() dec = NewDecoder(NewDecodingOptions().SetIndexEntryHasher(xhash.NewParsedIndexHasher(t))) ) - require.NoError(t, enc.EncodeIndexEntry(testIndexCheksumEntry.IndexEntry)) + require.NoError(t, enc.EncodeIndexEntry(testWideEntry.IndexEntry)) data := enc.Bytes() - for _, tt := range decodeIndexChecksumTests { + for _, tt := range decodeWideEntryTests { t.Run(tt.id, func(t *testing.T) { dec.Reset(NewByteDecoderStream(data)) - res, status, err := dec.DecodeIndexEntryToIndexChecksum([]byte(tt.id), nil) + res, status, err := dec.DecodeToWideEntry([]byte(tt.id), nil) require.NoError(t, err) require.Equal(t, tt.exStatus, status) if tt.exStatus == MatchedLookupStatus { @@ -331,7 +331,7 @@ func TestDecodeIndexEntryToIndexChecksum(t *testing.T) { } } -func TestDecodeIndexEntryToIndexChecksumPooled(t *testing.T) { +func TestDecodeToWideEntryPooled(t *testing.T) { ctrl := xtest.NewController(t) defer ctrl.Finish() @@ -340,19 +340,19 @@ func TestDecodeIndexEntryToIndexChecksumPooled(t *testing.T) { dec = NewDecoder(NewDecodingOptions().SetIndexEntryHasher(xhash.NewParsedIndexHasher(t))) ) - require.NoError(t, enc.EncodeIndexEntry(testIndexCheksumEntry.IndexEntry)) + require.NoError(t, enc.EncodeIndexEntry(testWideEntry.IndexEntry)) data := enc.Bytes() - for _, tt := range decodeIndexChecksumTests { + for _, tt := range decodeWideEntryTests { t.Run(tt.id+"_pooled", func(t *testing.T) { dec.Reset(NewByteDecoderStream(data)) bytePool := pool.NewMockBytesPool(ctrl) - idLength := len(testIndexCheksumEntry.ID) + idLength := len(testWideEntry.ID) idBytes := make([]byte, idLength) bytePool.EXPECT().Get(idLength).Return(idBytes) - tagLength := len(testIndexCheksumEntry.EncodedTags) + tagLength := len(testWideEntry.EncodedTags) tagBytes := make([]byte, tagLength) bytePool.EXPECT().Get(tagLength).Return(tagBytes) @@ -361,7 +361,7 @@ func TestDecodeIndexEntryToIndexChecksumPooled(t *testing.T) { bytePool.EXPECT().Put(tagBytes) } - res, status, err := dec.DecodeIndexEntryToIndexChecksum([]byte(tt.id), bytePool) + res, status, err := dec.DecodeToWideEntry([]byte(tt.id), bytePool) require.NoError(t, err) require.Equal(t, tt.exStatus, status) if tt.exStatus == MatchedLookupStatus { diff --git a/src/dbnode/persist/fs/msgpack/roundtrip_test.go b/src/dbnode/persist/fs/msgpack/roundtrip_test.go index 46c98c20d0..251d2f476e 100644 --- a/src/dbnode/persist/fs/msgpack/roundtrip_test.go +++ b/src/dbnode/persist/fs/msgpack/roundtrip_test.go @@ -92,10 +92,10 @@ var ( EncodedTags: []byte("testLogMetadataTags"), } - // NB: 100 is the expected hash for checksums of `testIndexCheksumEntry` + // NB: 100 is the expected hash for checksums of `testWideEntry` testMetadataChecksum = int64(100) - testIndexCheksumEntry = schema.IndexChecksum{ + testWideEntry = schema.WideEntry{ IndexEntry: schema.IndexEntry{ Index: 234, ID: []byte("test100"), @@ -403,7 +403,7 @@ func TestIndexEntryRoundtrip(t *testing.T) { require.Equal(t, testIndexEntry, res) } -func TestIndexEntryIntoIndexChecksumRoundtripWithBytesPool(t *testing.T) { +func TestIndexEntryIntoWideEntryRoundtripWithBytesPool(t *testing.T) { var ( pool = pool.NewBytesPool(nil, nil) enc = NewEncoder() @@ -411,26 +411,26 @@ func TestIndexEntryIntoIndexChecksumRoundtripWithBytesPool(t *testing.T) { ) pool.Init() - require.NoError(t, enc.EncodeIndexEntry(testIndexCheksumEntry.IndexEntry)) + require.NoError(t, enc.EncodeIndexEntry(testWideEntry.IndexEntry)) dec.Reset(NewByteDecoderStream(enc.Bytes())) - checksum, _, err := dec.DecodeIndexEntryToIndexChecksum(testIndexCheksumEntry.ID, pool) + checksum, _, err := dec.DecodeToWideEntry(testWideEntry.ID, pool) require.NoError(t, err) - require.Equal(t, testIndexCheksumEntry.IndexEntry, checksum.IndexEntry) - require.Equal(t, testIndexCheksumEntry.MetadataChecksum, checksum.MetadataChecksum) + require.Equal(t, testWideEntry.IndexEntry, checksum.IndexEntry) + require.Equal(t, testWideEntry.MetadataChecksum, checksum.MetadataChecksum) } -func TestIndexEntryIntoIndexChecksumRoundtripWithoutBytesPool(t *testing.T) { +func TestIndexEntryIntoWideEntryRoundtripWithoutBytesPool(t *testing.T) { var ( enc = NewEncoder() dec = NewDecoder(NewDecodingOptions().SetIndexEntryHasher(xhash.NewParsedIndexHasher(t))) ) - require.NoError(t, enc.EncodeIndexEntry(testIndexCheksumEntry.IndexEntry)) + require.NoError(t, enc.EncodeIndexEntry(testWideEntry.IndexEntry)) dec.Reset(NewByteDecoderStream(enc.Bytes())) - checksum, _, err := dec.DecodeIndexEntryToIndexChecksum(testIndexCheksumEntry.ID, nil) + checksum, _, err := dec.DecodeToWideEntry(testWideEntry.ID, nil) require.NoError(t, err) - require.Equal(t, testIndexCheksumEntry.IndexEntry, checksum.IndexEntry) - require.Equal(t, testIndexCheksumEntry.MetadataChecksum, checksum.MetadataChecksum) + require.Equal(t, testWideEntry.IndexEntry, checksum.IndexEntry) + require.Equal(t, testWideEntry.MetadataChecksum, checksum.MetadataChecksum) } // Make sure the V3 decoding code can handle the V1 file format. @@ -450,32 +450,34 @@ func TestIndexEntryRoundTripBackwardsCompatibilityV1(t *testing.T) { // and then restore them at the end of the test - This is required // because the new decoder won't try and read the new fields from // the old file format. - currEncodedTags := testIndexCheksumEntry.EncodedTags + currEncodedTags := testWideEntry.EncodedTags - testIndexCheksumEntry.EncodedTags = nil + testWideEntry.EncodedTags = nil defer func() { - testIndexCheksumEntry.EncodedTags = currEncodedTags + testWideEntry.EncodedTags = currEncodedTags }() - enc.EncodeIndexEntry(testIndexCheksumEntry.IndexEntry) + err := enc.EncodeIndexEntry(testWideEntry.IndexEntry) + require.NoError(t, err) + bytes := enc.Bytes() cloned := append(make([]byte, 0, len(bytes)), bytes...) dec.Reset(NewByteDecoderStream(bytes)) res, err := dec.DecodeIndexEntry(nil) require.NoError(t, err) - expected := testIndexCheksumEntry.IndexEntry + expected := testWideEntry.IndexEntry expected.IndexChecksum = 0 require.Equal(t, expected, res) - // Index Checksum decoding should fail since it requires tags for now. + // Wide Entry decoding should fail since it requires tags for now. dec.Reset(NewByteDecoderStream(cloned)) pool := pool.NewMockBytesPool(ctrl) - idLength := len(testIndexCheksumEntry.ID) + idLength := len(testWideEntry.ID) idBytes := make([]byte, idLength) pool.EXPECT().Get(idLength).Return(idBytes) pool.EXPECT().Put(idBytes) - _, status, err := dec.DecodeIndexEntryToIndexChecksum(testIndexCheksumEntry.ID, pool) + _, status, err := dec.DecodeToWideEntry(testWideEntry.ID, pool) require.Error(t, err) assert.Equal(t, ErrorLookupStatus, status) } @@ -495,13 +497,14 @@ func TestIndexEntryRoundTripForwardsCompatibilityV1(t *testing.T) { // Set the default values on the fields that did not exist in V1 // and then restore them at the end of the test - This is required // because the old decoder won't read the new fields. - currEncodedTags := testIndexCheksumEntry.EncodedTags + currEncodedTags := testWideEntry.EncodedTags - enc.EncodeIndexEntry(testIndexCheksumEntry.IndexEntry) + err := enc.EncodeIndexEntry(testWideEntry.IndexEntry) + require.NoError(t, err) // Make sure to zero them before we compare, but after we have // encoded the data. - expected := testIndexCheksumEntry.IndexEntry + expected := testWideEntry.IndexEntry expected.EncodedTags = nil defer func() { expected.EncodedTags = currEncodedTags @@ -516,11 +519,11 @@ func TestIndexEntryRoundTripForwardsCompatibilityV1(t *testing.T) { dec.Reset(NewByteDecoderStream(enc.Bytes())) pool := pool.NewMockBytesPool(ctrl) - idLength := len(testIndexCheksumEntry.ID) + idLength := len(testWideEntry.ID) idBytes := make([]byte, idLength) pool.EXPECT().Get(idLength).Return(idBytes) pool.EXPECT().Put(idBytes) - _, status, err := dec.DecodeIndexEntryToIndexChecksum(testIndexCheksumEntry.ID, pool) + _, status, err := dec.DecodeToWideEntry(testWideEntry.ID, pool) require.Error(t, err) assert.Equal(t, ErrorLookupStatus, status) } @@ -538,23 +541,24 @@ func TestIndexEntryRoundTripBackwardsCompatibilityV2(t *testing.T) { // and decoder and is never set on the IndexEntry struct. Therefore, no need to zero out any field in the struct // to make a comparison. - enc.EncodeIndexEntry(testIndexCheksumEntry.IndexEntry) + err := enc.EncodeIndexEntry(testWideEntry.IndexEntry) + require.NoError(t, err) dec.Reset(NewByteDecoderStream(enc.Bytes())) res, err := dec.DecodeIndexEntry(nil) require.NoError(t, err) - expected := testIndexCheksumEntry.IndexEntry + expected := testWideEntry.IndexEntry expected.IndexChecksum = 0 require.Equal(t, expected, res) dec.Reset(NewByteDecoderStream(enc.Bytes())) - chk, status, err := dec.DecodeIndexEntryToIndexChecksum(testIndexCheksumEntry.ID, nil) + chk, status, err := dec.DecodeToWideEntry(testWideEntry.ID, nil) require.NoError(t, err) assert.Equal(t, MatchedLookupStatus, status) - ex := testIndexCheksumEntry.IndexEntry + ex := testWideEntry.IndexEntry // This file version does not have an IndexChecksum field. ex.IndexChecksum = 0 require.Equal(t, ex, chk.IndexEntry) - require.Equal(t, testIndexCheksumEntry.MetadataChecksum, chk.MetadataChecksum) + require.Equal(t, testWideEntry.MetadataChecksum, chk.MetadataChecksum) } // Make sure the V2 decoder code can handle the V3 file format. @@ -569,23 +573,24 @@ func TestIndexEntryRoundTripForwardsCompatibilityV2(t *testing.T) { // and decoder and is never set on the IndexEntry struct. Therefore, no need to zero out any field in the struct // to make a comparison. - enc.EncodeIndexEntry(testIndexCheksumEntry.IndexEntry) + err := enc.EncodeIndexEntry(testWideEntry.IndexEntry) + require.NoError(t, err) dec.Reset(NewByteDecoderStream(enc.Bytes())) res, err := dec.DecodeIndexEntry(nil) require.NoError(t, err) - expected := testIndexCheksumEntry.IndexEntry + expected := testWideEntry.IndexEntry expected.IndexChecksum = 0 require.Equal(t, expected, res) dec.Reset(NewByteDecoderStream(enc.Bytes())) - chk, status, err := dec.DecodeIndexEntryToIndexChecksum(testIndexCheksumEntry.ID, nil) + chk, status, err := dec.DecodeToWideEntry(testWideEntry.ID, nil) require.NoError(t, err) assert.Equal(t, MatchedLookupStatus, status) - ex := testIndexCheksumEntry.IndexEntry + ex := testWideEntry.IndexEntry // This file version does not have an IndexChecksum field. ex.IndexChecksum = 0 require.Equal(t, ex, chk.IndexEntry) - require.Equal(t, testIndexCheksumEntry.MetadataChecksum, chk.MetadataChecksum) + require.Equal(t, testWideEntry.MetadataChecksum, chk.MetadataChecksum) } func TestIndexSummaryRoundtrip(t *testing.T) { @@ -682,7 +687,7 @@ func TestMultiTypeRoundtripStress(t *testing.T) { require.NoError(t, enc.EncodeLogMetadata(testLogMetadata)) expected = append(expected, testLogMetadata) case 5: - require.NoError(t, enc.EncodeIndexEntry(testIndexCheksumEntry.IndexEntry)) + require.NoError(t, enc.EncodeIndexEntry(testWideEntry.IndexEntry)) expected = append(expected, testMetadataChecksum) } } @@ -702,12 +707,12 @@ func TestMultiTypeRoundtripStress(t *testing.T) { res, err = dec.DecodeLogMetadata() case 5: var ( - r schema.IndexChecksum - s IndexChecksumLookupStatus + e schema.WideEntry + s WideEntryLookupStatus ) - r, s, err = dec.DecodeIndexEntryToIndexChecksum(testIndexCheksumEntry.ID, nil) + e, s, err = dec.DecodeToWideEntry(testWideEntry.ID, nil) assert.Equal(t, s, MatchedLookupStatus) - res = r.MetadataChecksum + res = e.MetadataChecksum } require.NoError(t, err) output = append(output, res) diff --git a/src/dbnode/persist/fs/retriever.go b/src/dbnode/persist/fs/retriever.go index 0d96ceae25..7d9eca227e 100644 --- a/src/dbnode/persist/fs/retriever.go +++ b/src/dbnode/persist/fs/retriever.go @@ -33,19 +33,18 @@ package fs import ( "errors" + "fmt" "sort" "sync" "sync/atomic" "time" "github.com/m3db/m3/src/dbnode/namespace" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/sharding" "github.com/m3db/m3/src/dbnode/storage/block" "github.com/m3db/m3/src/dbnode/storage/limits" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/dbnode/x/xio" - "github.com/m3db/m3/src/x/checked" "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/ident" "github.com/m3db/m3/src/x/pool" @@ -68,8 +67,7 @@ const ( streamInvalidReq streamReqType = iota streamDataReq - streamIdxChecksumReq - streamReadMismatchReq + streamWideEntryReq ) type blockRetrieverStatus int @@ -201,9 +199,10 @@ func (r *blockRetriever) AssignShardSet(shardSet sharding.ShardSet) { func (r *blockRetriever) fetchLoop(seekerMgr DataFileSetSeekerManager) { var ( - seekerResources = NewReusableSeekerResources(r.fsOpts) - inFlight []*retrieveRequest - currBatchReqs []*retrieveRequest + seekerResources = NewReusableSeekerResources(r.fsOpts) + retrieverResources = newReuseableRetrieverResources() + inFlight []*retrieveRequest + currBatchReqs []*retrieveRequest ) for { // Free references to the inflight requests @@ -266,8 +265,8 @@ func (r *blockRetriever) fetchLoop(seekerMgr DataFileSetSeekerManager) { req.shard != currBatchShard { // Fetch any outstanding in the current batch if len(currBatchReqs) > 0 { - r.fetchBatch( - seekerMgr, currBatchShard, currBatchStart, currBatchReqs, seekerResources) + r.fetchBatch(seekerMgr, currBatchShard, currBatchStart, + currBatchReqs, seekerResources, retrieverResources) for i := range currBatchReqs { currBatchReqs[i] = nil } @@ -285,8 +284,8 @@ func (r *blockRetriever) fetchLoop(seekerMgr DataFileSetSeekerManager) { // Fetch any finally outstanding in the current batch if len(currBatchReqs) > 0 { - r.fetchBatch( - seekerMgr, currBatchShard, currBatchStart, currBatchReqs, seekerResources) + r.fetchBatch(seekerMgr, currBatchShard, currBatchStart, + currBatchReqs, seekerResources, retrieverResources) for i := range currBatchReqs { currBatchReqs[i] = nil } @@ -297,73 +296,69 @@ func (r *blockRetriever) fetchLoop(seekerMgr DataFileSetSeekerManager) { r.fetchLoopsHaveShutdownCh <- struct{}{} } -func (r *blockRetriever) processIndexChecksumRequest( - req *retrieveRequest, - seeker ConcurrentDataFileSetSeeker, - seekerResources ReusableSeekerResources, -) { - checksum, err := seeker.SeekIndexEntryToIndexChecksum(req.id, seekerResources) - if err != nil { - req.onError(err) - return - } - - req.onIndexChecksumCompleted(checksum) - req.onCallerOrRetrieverDone() -} - -func (r *blockRetriever) processReadMismatchRequest( - req *retrieveRequest, - seeker ConcurrentDataFileSetSeeker, - seekerResources ReusableSeekerResources, -) { - checksum, err := seeker.SeekIndexEntryToIndexChecksum(req.id, seekerResources) - if err != nil { - req.onError(err) - return - } - - mismatch, err := seeker.SeekReadMismatchesByIndexChecksum( - checksum, req.mismatchChecker, seekerResources) - - if err != nil && err != errSeekIDNotFound { - req.onError(err) - return - } - - if err == errSeekIDNotFound { - req.onIndexMismatchCompleted(wide.ReadMismatch{}) - return - } - - req.onIndexMismatchCompleted(mismatch) - req.onCallerOrRetrieverDone() -} - // filterAndCompleteWideReqs completes all wide operation retrieve requests, // returning a list of requests that need to be processed by other means. func (r *blockRetriever) filterAndCompleteWideReqs( reqs []*retrieveRequest, seeker ConcurrentDataFileSetSeeker, seekerResources ReusableSeekerResources, + retrieverResources *reuseableRetrieverResources, ) []*retrieveRequest { - filteredStreamRequests := reqs[:0] + retrieverResources.resetDataReqs() + retrieverResources.resetWideEntryReqs() for _, req := range reqs { switch req.streamReqType { case streamDataReq: // NB: filter out stream requests; these are handled outside of // wide logic functions. - filteredStreamRequests = append(filteredStreamRequests, req) - case streamIdxChecksumReq: - r.processIndexChecksumRequest(req, seeker, seekerResources) - case streamReadMismatchReq: - r.processReadMismatchRequest(req, seeker, seekerResources) + retrieverResources.dataReqs = append(retrieverResources.dataReqs, req) + + case streamWideEntryReq: + entry, err := seeker.SeekWideEntry(req.id, seekerResources) + if err != nil { + if errors.Is(err, errSeekIDNotFound) { + // Missing, return empty result, successful lookup. + req.wideEntry = xio.WideEntry{} + req.success = true + } else { + req.err = err + } + + continue + } + + // Enqueue for fetch in batch in offset ascending order. + req.wideEntry = entry + entry.Shard = req.shard + retrieverResources.appendWideEntryReq(req) + default: - req.onError(errUnsetRequestType) + req.err = errUnsetRequestType } } - return filteredStreamRequests + // Fulfill the wide entry data fetches in batch offset ascending. + sortByOffsetAsc := retrieveRequestByWideEntryOffsetAsc(retrieverResources.wideEntryReqs) + sort.Sort(sortByOffsetAsc) + for _, req := range retrieverResources.wideEntryReqs { + entry := IndexEntry{ + Size: uint32(req.wideEntry.Size), + DataChecksum: uint32(req.wideEntry.DataChecksum), + Offset: req.wideEntry.Offset, + } + data, err := seeker.SeekByIndexEntry(entry, seekerResources) + if err != nil { + req.err = err + continue + } + + // Success, inc ref so on finalize can decref and finalize. + req.wideEntry.Data = data + req.wideEntry.Data.IncRef() + req.success = true + } + + return retrieverResources.dataReqs } func (r *blockRetriever) fetchBatch( @@ -372,18 +367,43 @@ func (r *blockRetriever) fetchBatch( blockStart time.Time, allReqs []*retrieveRequest, seekerResources ReusableSeekerResources, + retrieverResources *reuseableRetrieverResources, ) { - // Resolve the seeker from the seeker mgr - seeker, err := seekerMgr.Borrow(shard, blockStart) - if err != nil { + var ( + seeker ConcurrentDataFileSetSeeker + callbackWg sync.WaitGroup + ) + + defer func() { + filteredReqs := allReqs[:0] + // Make sure requests are always fulfilled so if there's a code bug + // then errSeekNotCompleted is returned because req.success is not set + // rather than we have dangling goroutines stacking up. for _, req := range allReqs { - req.onError(err) + if !req.waitingForCallback { + req.onDone() + continue + } + + filteredReqs = append(filteredReqs, req) } - return - } - defer func() { - err = seekerMgr.Return(shard, blockStart, seeker) + callbackWg.Wait() + for _, req := range filteredReqs { + req.onDone() + } + + // Reset resources to free any pointers in the slices still pointing + // to requests that are now completed and returned to pools. + retrieverResources.resetAll() + + if seeker == nil { + // No borrowed seeker to return. + return + } + + // Return borrowed seeker. + err := seekerMgr.Return(shard, blockStart, seeker) if err != nil { r.logger.Error("err returning seeker for shard", zap.Uint32("shard", shard), @@ -393,86 +413,106 @@ func (r *blockRetriever) fetchBatch( } }() + var err error + seeker, err = seekerMgr.Borrow(shard, blockStart) + if err != nil { + for _, req := range allReqs { + req.err = err + } + return + } + // NB: filterAndCompleteWideReqs will complete any wide requests, returning // a filtered list of requests that should be processed below. These wide // requests must not take query limits into account. - reqs := r.filterAndCompleteWideReqs(allReqs, seeker, seekerResources) + reqs := r.filterAndCompleteWideReqs(allReqs, seeker, seekerResources, + retrieverResources) var limitErr error if err := r.queryLimits.AnyExceeded(); err != nil { for _, req := range reqs { - req.onError(err) + req.err = err } return } for _, req := range reqs { if limitErr != nil { - req.onError(limitErr) + req.err = limitErr continue } entry, err := seeker.SeekIndexEntry(req.id, seekerResources) - if err != nil && err != errSeekIDNotFound { - req.onError(err) + if err != nil && !errors.Is(err, errSeekIDNotFound) { + req.err = err continue } if err := r.bytesReadLimit.Inc(int(entry.Size)); err != nil { - req.onError(err) + req.err = err limitErr = err continue } - if err == errSeekIDNotFound { + if errors.Is(err, errSeekIDNotFound) { req.notFound = true } req.indexEntry = entry } - sort.Sort(retrieveRequestByOffsetAsc(reqs)) + sort.Sort(retrieveRequestByIndexEntryOffsetAsc(reqs)) tagDecoderPool := r.fsOpts.TagDecoderPool() blockCachingEnabled := r.opts.CacheBlocksOnRetrieve() && r.nsCacheBlocksOnRetrieve // Seek and execute all requests for _, req := range reqs { - var ( - data checked.Bytes - err error - ) + // Should always be a data request by this point. + if req.streamReqType != streamDataReq { + req.err = fmt.Errorf("wrong stream req type: expect=%d, actual=%d", + streamDataReq, req.streamReqType) + continue + } - // Only try to seek the ID if it exists and there haven't been any errors so - // far, otherwise we'll get a checksum mismatch error because the default - // offset value for indexEntry is zero. - if req.foundAndHasNoError() { - data, err = seeker.SeekByIndexEntry(req.indexEntry, seekerResources) - if err != nil && err != errSeekIDNotFound { - req.onError(err) - continue - } + if req.err != nil { + // Skip requests with error, will already get appropriate callback. + continue + } + + if req.notFound { + // Only try to seek the ID if it exists and there haven't been any errors so + // far, otherwise we'll get a checksum mismatch error because the default + // offset value for indexEntry is zero. + req.success = true + req.onCallerOrRetrieverDone() + continue + } + + data, err := seeker.SeekByIndexEntry(req.indexEntry, seekerResources) + if err != nil { + // If not found error is returned here, that's still an error since + // it's expected to be found if it was found in the index file. + req.err = err + continue } var ( seg, onRetrieveSeg ts.Segment checksum = req.indexEntry.DataChecksum ) - if data != nil { - seg = ts.NewSegment(data, nil, checksum, ts.FinalizeHead) - } + seg = ts.NewSegment(data, nil, checksum, ts.FinalizeHead) // We don't need to call onRetrieve.OnRetrieveBlock if the ID was not found. - callOnRetrieve := blockCachingEnabled && req.onRetrieve != nil && req.foundAndHasNoError() + callOnRetrieve := blockCachingEnabled && req.onRetrieve != nil if callOnRetrieve { // NB(r): Need to also trigger callback with a copy of the data. // This is used by the database to cache the in memory data for // consequent fetches. - if data != nil { - dataCopy := r.bytesPool.Get(data.Len()) - onRetrieveSeg = ts.NewSegment(dataCopy, nil, checksum, ts.FinalizeHead) - dataCopy.AppendAll(data.Bytes()) - } + dataCopy := r.bytesPool.Get(data.Len()) + onRetrieveSeg = ts.NewSegment(dataCopy, nil, checksum, ts.FinalizeHead) + dataCopy.AppendAll(data.Bytes()) + if tags := req.indexEntry.EncodedTags; tags != nil && tags.Len() > 0 { decoder := tagDecoderPool.Get() // DecRef because we're transferring ownership from the index entry to @@ -492,17 +532,24 @@ func (r *blockRetriever) fetchBatch( // Complete request. req.onRetrieved(seg, req.nsCtx) + req.success = true if !callOnRetrieve { - // No need to call the onRetrieve callback. + // No need to call the onRetrieve callback, but do need to call + // onCallerOrRetrieverDone since data requests do not get finalized + // when req.onDone is called since sometimes they need deferred + // finalization (when callOnRetrieve is true). req.onCallerOrRetrieverDone() continue } + callbackWg.Add(1) + req.waitingForCallback = true go func(r *retrieveRequest) { // Call the onRetrieve callback and finalize. r.onRetrieve.OnRetrieveBlock(r.id, r.tags, r.start, onRetrieveSeg, r.nsCtx) r.onCallerOrRetrieverDone() + callbackWg.Done() }(req) } } @@ -586,11 +633,14 @@ func (r *blockRetriever) Stream( found, err := r.streamRequest(ctx, req, shard, id, startTime, nsCtx) if err != nil { + req.resultWg.Done() return xio.EmptyBlockReader, err } if !found { req.onRetrieved(ts.Segment{}, namespace.Context{}) + req.success = true + req.onDone() } // The request may not have completed yet, but it has an internal @@ -601,52 +651,26 @@ func (r *blockRetriever) Stream( return req.toBlock(), nil } -func (r *blockRetriever) StreamIndexChecksum( - ctx context.Context, - shard uint32, - id ident.ID, - startTime time.Time, - nsCtx namespace.Context, -) (block.StreamedChecksum, error) { - req := r.reqPool.Get() - req.streamReqType = streamIdxChecksumReq - - found, err := r.streamRequest(ctx, req, shard, id, startTime, nsCtx) - if err != nil { - return block.EmptyStreamedChecksum, err - } - - if !found { - req.onIndexChecksumCompleted(xio.IndexChecksum{}) - } - - // The request may not have completed yet, but it has an internal - // waitgroup which the caller will have to wait for before retrieving - // the data. This means that even though we're returning nil for error - // here, the caller may still encounter an error when they attempt to - // read the data. - return req, nil -} - -func (r *blockRetriever) StreamReadMismatches( +func (r *blockRetriever) StreamWideEntry( ctx context.Context, shard uint32, - mismatchChecker wide.EntryChecksumMismatchChecker, id ident.ID, startTime time.Time, nsCtx namespace.Context, -) (wide.StreamedMismatch, error) { +) (block.StreamedWideEntry, error) { req := r.reqPool.Get() - req.mismatchChecker = mismatchChecker - req.streamReqType = streamReadMismatchReq + req.streamReqType = streamWideEntryReq found, err := r.streamRequest(ctx, req, shard, id, startTime, nsCtx) if err != nil { - return wide.EmptyStreamedMismatch, err + req.resultWg.Done() + return block.EmptyStreamedWideEntry, err } if !found { - req.onIndexMismatchCompleted(wide.ReadMismatch{}) + req.wideEntry = xio.WideEntry{} + req.success = true + req.onDone() } // The request may not have completed yet, but it has an internal @@ -740,7 +764,9 @@ func (reqs *shardRetrieveRequests) resetQueued() { // Don't forget to update the resetForReuse method when adding a new field type retrieveRequest struct { - resultWg sync.WaitGroup + finalized bool + waitingForCallback bool + resultWg sync.WaitGroup pool *reqPool @@ -753,12 +779,9 @@ type retrieveRequest struct { streamReqType streamReqType indexEntry IndexEntry - indexChecksum xio.IndexChecksum - mismatchBatch wide.ReadMismatch + wideEntry xio.WideEntry reader xio.SegmentReader - mismatchChecker wide.EntryChecksumMismatchChecker - err error // Finalize requires two calls to finalize (once both the user of the @@ -768,38 +791,16 @@ type retrieveRequest struct { shard uint32 notFound bool + success bool } -func (req *retrieveRequest) onIndexChecksumCompleted(indexChecksum xio.IndexChecksum) { - if req.err == nil { - req.indexChecksum = indexChecksum - // If there was an error, we've already called done. - req.resultWg.Done() - } -} - -func (req *retrieveRequest) RetrieveIndexChecksum() (xio.IndexChecksum, error) { +func (req *retrieveRequest) RetrieveWideEntry() (xio.WideEntry, error) { req.resultWg.Wait() if req.err != nil { - return xio.IndexChecksum{}, req.err - } - return req.indexChecksum, nil -} - -func (req *retrieveRequest) onIndexMismatchCompleted(batch wide.ReadMismatch) { - if req.err == nil { - req.mismatchBatch = batch - // If there was an error, we've already called done. - req.resultWg.Done() + return xio.WideEntry{}, req.err } -} -func (req *retrieveRequest) RetrieveMismatch() (wide.ReadMismatch, error) { - req.resultWg.Wait() - if req.err != nil { - return wide.ReadMismatch{}, req.err - } - return req.mismatchBatch, nil + return req.wideEntry, nil } func (req *retrieveRequest) toBlock() xio.BlockReader { @@ -810,49 +811,84 @@ func (req *retrieveRequest) toBlock() xio.BlockReader { } } -func (req *retrieveRequest) onError(err error) { - if req.err == nil { - req.err = err - req.resultWg.Done() - } -} - func (req *retrieveRequest) onRetrieved(segment ts.Segment, nsCtx namespace.Context) { - req.Reset(segment) req.nsCtx = nsCtx + req.Reset(segment) } -func (req *retrieveRequest) onCallerOrRetrieverDone() { - if atomic.AddUint32(&req.finalizes, 1) != 2 { - return - } - // NB: streamIdxChecksumReq ids are used to sort the resultant list, - // so they should not be finalized here. - if req.streamReqType == streamDataReq { - req.id.Finalize() +func (req *retrieveRequest) onDone() { + var ( + err = req.err + success = req.success + streamReqType = req.streamReqType + ) + + if err == nil && !success { + // Require explicit success, otherwise this request + // was never completed. + // This helps catch code bugs where this element wasn't explicitly + // handled as completed during a fetch batch call instead of + // returning but with no actual result set properly. + req.err = errSeekNotCompleted } - req.id = nil - if req.tags != nil { - req.tags.Close() - req.tags = ident.EmptyTagIterator + + req.resultWg.Done() + + switch streamReqType { + case streamDataReq: + // Do not call onCallerOrRetrieverDone since the OnRetrieveCallback + // code path will call req.onCallerOrRetrieverDone() when it's done. + // If encountered an error though, should call it since not waiting for + // callback to finish or even if not waiting for callback to finish + // the happy path that calls this pre-emptively has not executed either. + // That is if-and-only-if request is data request and is successful and + // will req.onCallerOrRetrieverDone() be called in a deferred manner. + if !success { + req.onCallerOrRetrieverDone() + } + default: + // All other requests will use this to increment the finalize count by + // one and the actual req.Finalize() by the final one to make count of + // two and actually return the request to the pool. + req.onCallerOrRetrieverDone() } - req.reader.Finalize() - req.reader = nil - req.pool.Put(req) } func (req *retrieveRequest) Reset(segment ts.Segment) { req.reader.Reset(segment) - if req.err == nil { - // If there was an error, we've already called done. - req.resultWg.Done() - } } func (req *retrieveRequest) ResetWindowed(segment ts.Segment, start time.Time, blockSize time.Duration) { - req.Reset(segment) req.start = start req.blockSize = blockSize + req.Reset(segment) +} + +func (req *retrieveRequest) onCallerOrRetrieverDone() { + if atomic.AddUint32(&req.finalizes, 1) != 2 { + return + } + + switch req.streamReqType { + case streamWideEntryReq: + // All pooled elements are set on the wideEntry. + req.wideEntry.Finalize() + default: + if req.id != nil { + req.id.Finalize() + req.id = nil + } + if req.tags != nil { + req.tags.Close() + req.tags = ident.EmptyTagIterator + } + if req.reader != nil { + req.reader.Finalize() + req.reader = nil + } + } + + req.pool.Put(req) } func (req *retrieveRequest) SegmentReader() (xio.SegmentReader, error) { @@ -898,11 +934,18 @@ func (req *retrieveRequest) Segment() (ts.Segment, error) { func (req *retrieveRequest) Finalize() { // May not actually finalize the request, depending on if // retriever is done too + if req.finalized { + return + } + + req.resultWg.Wait() + req.finalized = true req.onCallerOrRetrieverDone() } func (req *retrieveRequest) resetForReuse() { req.resultWg = sync.WaitGroup{} + req.finalized = false req.finalizes = 0 req.shard = 0 req.id = nil @@ -912,16 +955,11 @@ func (req *retrieveRequest) resetForReuse() { req.onRetrieve = nil req.streamReqType = streamInvalidReq req.indexEntry = IndexEntry{} - req.indexChecksum = xio.IndexChecksum{} - req.mismatchBatch = wide.ReadMismatch{} - req.mismatchChecker = nil + req.wideEntry = xio.WideEntry{} req.reader = nil req.err = nil req.notFound = false -} - -func (req *retrieveRequest) foundAndHasNoError() bool { - return !req.notFound && req.err == nil + req.success = false } type retrieveRequestByStartAscShardAsc []*retrieveRequest @@ -935,18 +973,29 @@ func (r retrieveRequestByStartAscShardAsc) Less(i, j int) bool { return r[i].shard < r[j].shard } -type retrieveRequestByOffsetAsc []*retrieveRequest +type retrieveRequestByIndexEntryOffsetAsc []*retrieveRequest -func (r retrieveRequestByOffsetAsc) Len() int { return len(r) } -func (r retrieveRequestByOffsetAsc) Swap(i, j int) { r[i], r[j] = r[j], r[i] } -func (r retrieveRequestByOffsetAsc) Less(i, j int) bool { +func (r retrieveRequestByIndexEntryOffsetAsc) Len() int { return len(r) } +func (r retrieveRequestByIndexEntryOffsetAsc) Swap(i, j int) { r[i], r[j] = r[j], r[i] } +func (r retrieveRequestByIndexEntryOffsetAsc) Less(i, j int) bool { return r[i].indexEntry.Offset < r[j].indexEntry.Offset } +type retrieveRequestByWideEntryOffsetAsc []*retrieveRequest + +func (r retrieveRequestByWideEntryOffsetAsc) Len() int { return len(r) } +func (r retrieveRequestByWideEntryOffsetAsc) Swap(i, j int) { r[i], r[j] = r[j], r[i] } +func (r retrieveRequestByWideEntryOffsetAsc) Less(i, j int) bool { + return r[i].wideEntry.Offset < r[j].wideEntry.Offset +} + // RetrieveRequestPool is the retrieve request pool. type RetrieveRequestPool interface { + // Init initializes the request pool. Init() + // Get gets a retrieve request. Get() *retrieveRequest + // Put returns a retrieve request to the pool. Put(req *retrieveRequest) } @@ -986,3 +1035,37 @@ func (p *reqPool) Put(req *retrieveRequest) { req.resetForReuse() p.pool.Put(req) } + +type reuseableRetrieverResources struct { + dataReqs []*retrieveRequest + wideEntryReqs []*retrieveRequest +} + +func newReuseableRetrieverResources() *reuseableRetrieverResources { + return &reuseableRetrieverResources{} +} + +func (r *reuseableRetrieverResources) resetAll() { + r.resetDataReqs() + r.resetWideEntryReqs() +} + +func (r *reuseableRetrieverResources) resetDataReqs() { + for i := range r.dataReqs { + r.dataReqs[i] = nil + } + r.dataReqs = r.dataReqs[:0] +} + +func (r *reuseableRetrieverResources) resetWideEntryReqs() { + for i := range r.wideEntryReqs { + r.wideEntryReqs[i] = nil + } + r.wideEntryReqs = r.wideEntryReqs[:0] +} + +func (r *reuseableRetrieverResources) appendWideEntryReq( + req *retrieveRequest, +) { + r.wideEntryReqs = append(r.wideEntryReqs, req) +} diff --git a/src/dbnode/persist/fs/retriever_test.go b/src/dbnode/persist/fs/retriever_test.go index 824e012c96..07754dc90e 100644 --- a/src/dbnode/persist/fs/retriever_test.go +++ b/src/dbnode/persist/fs/retriever_test.go @@ -1,5 +1,3 @@ -// +build big -// // Copyright (c) 2016 Uber Technologies, Inc. // // Permission is hereby granted, free of charge, to any person obtaining a copy @@ -772,8 +770,6 @@ func testBlockRetrieverOnRetrieve(t *testing.T, globalFlag bool, nsFlag bool) { } else { require.False(t, onRetrieveCalled) } - - require.NoError(t, err) } // TestBlockRetrieverHandlesErrors verifies the behavior of the Stream() method diff --git a/src/dbnode/persist/fs/seek.go b/src/dbnode/persist/fs/seek.go index 5fc175fcf1..956884134b 100644 --- a/src/dbnode/persist/fs/seek.go +++ b/src/dbnode/persist/fs/seek.go @@ -31,7 +31,6 @@ import ( "github.com/m3db/m3/src/dbnode/digest" xmsgpack "github.com/m3db/m3/src/dbnode/persist/fs/msgpack" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/persist/schema" "github.com/m3db/m3/src/dbnode/x/xio" "github.com/m3db/m3/src/x/checked" @@ -52,6 +51,9 @@ var ( // errSeekChecksumMismatch returned when data checksum does not match the expected checksum errSeekChecksumMismatch = errors.New("checksum does not match expected checksum") + // errSeekNotCompleted returned when no error but seek did not complete. + errSeekNotCompleted = errors.New("seek not completed") + // errClonesShouldNotBeOpened returned when Open() is called on a clone errClonesShouldNotBeOpened = errors.New("clone should not be opened") ) @@ -374,87 +376,6 @@ func (s *seeker) SeekByIndexEntry( return buffer, nil } -func (s *seeker) SeekReadMismatchesByIndexChecksum( - checksum xio.IndexChecksum, - mismatchChecker wide.EntryChecksumMismatchChecker, - resources ReusableSeekerResources, -) (wide.ReadMismatch, error) { - completed := false - defer func() { - // NB: if this fails to complete, finalize the checksum. - if !completed { - checksum.Finalize() - } - }() - - mismatchChecker.Lock() - // NB: first, apply the reader. - allMismatches, err := mismatchChecker.ComputeMismatchesForEntry(checksum) - if err != nil { - // NB: free checksum resources - return wide.ReadMismatch{}, err - } - - // NB: only filter out reader side mismatches. TODO: remove index checksum - // mismatches, since they are not necessary in the updated model. - mismatches := allMismatches[:0] - for _, m := range allMismatches { - if m.IsReaderMismatch() { - mismatches = append(mismatches, m) - } - } - - if len(mismatches) == 0 { - // This entry matches; no need to retrieve data. - return wide.ReadMismatch{}, nil - } - - if len(mismatches) > 1 { - return wide.ReadMismatch{}, fmt.Errorf("multiple reader mismatches") - } - - mismatchChecker.Unlock() - resources.offsetFileReader.reset(s.dataFd, checksum.Offset) - - // Obtain an appropriately sized buffer. - var buffer checked.Bytes - if s.opts.bytesPool != nil { - buffer = s.opts.bytesPool.Get(int(checksum.Size)) - buffer.IncRef() - defer buffer.DecRef() - buffer.Resize(int(checksum.Size)) - } else { - buffer = checked.NewBytes(make([]byte, checksum.Size), nil) - buffer.IncRef() - defer buffer.DecRef() - } - - // Copy the actual data into the underlying buffer. - underlyingBuf := buffer.Bytes() - n, err := io.ReadFull(resources.offsetFileReader, underlyingBuf) - if err != nil { - return wide.ReadMismatch{}, err - } - if n != int(checksum.Size) { - // This check is redundant because io.ReadFull will return an error if - // its not able to read the specified number of bytes, but we keep it - // in for posterity. - return wide.ReadMismatch{}, fmt.Errorf("tried to read: %d bytes but read: %d", checksum.Size, n) - } - - // NB(r): _must_ check the checksum against known checksum as the data - // file might not have been verified if we haven't read through the file yet. - if checksum.DataChecksum != int64(digest.Checksum(underlyingBuf)) { - return wide.ReadMismatch{}, errSeekChecksumMismatch - } - - completed = true - return wide.ReadMismatch{ - IndexChecksum: checksum, - Data: buffer, - }, nil -} - // SeekIndexEntry performs the following steps: // // 1. Go to the indexLookup and it will give us an offset that is a good starting @@ -544,25 +465,25 @@ func (s *seeker) SeekIndexEntry( } } -// SeekIndexEntryToIndexChecksum performs the following steps: +// SeekWideEntry performs the following steps: // // 1. Go to the indexLookup and it will give us an offset that is a good starting // point for scanning the index file. // 2. Reset an offsetFileReader with the index fd and an offset (so that calls to Read() will // begin at the offset provided by the offset lookup). // 3. Reset a decoder with fileDecoderStream (offsetFileReader wrapped in a bufio.Reader). -// 4. Call DecodeIndexEntry in a tight loop (which will advance our position in the +// 4. Call DecodeToWideEntry in a tight loop (which will advance our position in the // offsetFileReader internally) until we've either found the entry we're looking for or gone so // far we know it does not exist. -func (s *seeker) SeekIndexEntryToIndexChecksum( +func (s *seeker) SeekWideEntry( id ident.ID, resources ReusableSeekerResources, -) (xio.IndexChecksum, error) { +) (xio.WideEntry, error) { offset, err := s.indexLookup.getNearestIndexFileOffset(id, resources) // Should never happen, either something is really wrong with the code or // the file on disk was corrupted. if err != nil { - return xio.IndexChecksum{}, err + return xio.WideEntry{}, err } resources.offsetFileReader.reset(s.indexFd, offset) @@ -571,39 +492,39 @@ func (s *seeker) SeekIndexEntryToIndexChecksum( idBytes := id.Bytes() for { - checksum, status, err := resources.xmsgpackDecoder. - DecodeIndexEntryToIndexChecksum(idBytes, resources.decodeIndexEntryBytesPool) + entry, status, err := resources.xmsgpackDecoder. + DecodeToWideEntry(idBytes, resources.decodeIndexEntryBytesPool) if err != nil { // No longer being used so we can return to the pool. - resources.decodeIndexEntryBytesPool.Put(checksum.ID) - resources.decodeIndexEntryBytesPool.Put(checksum.EncodedTags) + resources.decodeIndexEntryBytesPool.Put(entry.ID) + resources.decodeIndexEntryBytesPool.Put(entry.EncodedTags) if err == io.EOF { // Reached the end of the file without finding the ID. - return xio.IndexChecksum{}, errSeekIDNotFound + return xio.WideEntry{}, errSeekIDNotFound } // Should never happen, either something is really wrong with the code or // the file on disk was corrupted. - return xio.IndexChecksum{}, instrument.InvariantErrorf(err.Error()) + return xio.WideEntry{}, instrument.InvariantErrorf(err.Error()) } if status != xmsgpack.MatchedLookupStatus { // No longer being used so we can return to the pool. - resources.decodeIndexEntryBytesPool.Put(checksum.ID) - resources.decodeIndexEntryBytesPool.Put(checksum.EncodedTags) + resources.decodeIndexEntryBytesPool.Put(entry.ID) + resources.decodeIndexEntryBytesPool.Put(entry.EncodedTags) if status == xmsgpack.NotFoundLookupStatus { - // a `NotFound` status for the index checksum decode indicates that the + // a `NotFound` status for the wide entry decode indicates that the // current seek has passed the point in the file where this ID could have // appeared; short-circuit here as the ID does not exist in the file. - return xio.IndexChecksum{}, errSeekIDNotFound + return xio.WideEntry{}, errSeekIDNotFound } else if status == xmsgpack.MismatchLookupStatus { - // a `Mismatch` status for the index checksum decode indicates that the + // a `Mismatch` status for the wide entry decode indicates that the // current seek does not match the ID, but that it may still appear in // the file. continue } else if status == xmsgpack.ErrorLookupStatus { - return xio.IndexChecksum{}, errors.New("unknown index lookup error") + return xio.WideEntry{}, errors.New("unknown index lookup error") } } @@ -611,23 +532,23 @@ func (s *seeker) SeekIndexEntryToIndexChecksum( // so they can be passed along. We use the "real" bytes pool here // because we're passing ownership of the bytes to the entry / caller. var checkedEncodedTags checked.Bytes - if tags := checksum.EncodedTags; len(tags) > 0 { + if tags := entry.EncodedTags; len(tags) > 0 { checkedEncodedTags = s.opts.bytesPool.Get(len(tags)) checkedEncodedTags.IncRef() checkedEncodedTags.AppendAll(tags) } // No longer being used so we can return to the pool. - resources.decodeIndexEntryBytesPool.Put(checksum.ID) - resources.decodeIndexEntryBytesPool.Put(checksum.EncodedTags) + resources.decodeIndexEntryBytesPool.Put(entry.ID) + resources.decodeIndexEntryBytesPool.Put(entry.EncodedTags) - return xio.IndexChecksum{ + return xio.WideEntry{ ID: id, - Size: checksum.Size, - Offset: checksum.Offset, - DataChecksum: checksum.DataChecksum, + Size: entry.Size, + Offset: entry.Offset, + DataChecksum: entry.DataChecksum, EncodedTags: checkedEncodedTags, - MetadataChecksum: checksum.MetadataChecksum, + MetadataChecksum: entry.MetadataChecksum, }, nil } } diff --git a/src/dbnode/persist/fs/types.go b/src/dbnode/persist/fs/types.go index 55a3e70874..4db0c0c477 100644 --- a/src/dbnode/persist/fs/types.go +++ b/src/dbnode/persist/fs/types.go @@ -29,7 +29,6 @@ import ( "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/persist/fs/msgpack" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/sharding" "github.com/m3db/m3/src/dbnode/storage/block" @@ -51,7 +50,7 @@ import ( xtime "github.com/m3db/m3/src/x/time" ) -// FileSetFileIdentifier contains all the information required to identify a FileSetFile +// FileSetFileIdentifier contains all the information required to identify a FileSetFile. type FileSetFileIdentifier struct { FileSetContentType persist.FileSetContentType Namespace ident.ID @@ -62,7 +61,7 @@ type FileSetFileIdentifier struct { VolumeIndex int } -// DataWriterOpenOptions is the options struct for the Open method on the DataFileSetWriter +// DataWriterOpenOptions is the options struct for the Open method on the DataFileSetWriter. type DataWriterOpenOptions struct { FileSetType persist.FileSetType FileSetContentType persist.FileSetContentType @@ -73,13 +72,13 @@ type DataWriterOpenOptions struct { } // DataWriterSnapshotOptions is the options struct for Open method on the DataFileSetWriter -// that contains information specific to writing snapshot files +// that contains information specific to writing snapshot files. type DataWriterSnapshotOptions struct { SnapshotTime time.Time SnapshotID []byte } -// DataFileSetWriter provides an unsynchronized writer for a TSDB file set +// DataFileSetWriter provides an unsynchronized writer for a TSDB file set. type DataFileSetWriter interface { io.Closer @@ -111,7 +110,7 @@ type SnapshotMetadataFileReader interface { Read(id SnapshotMetadataIdentifier) (SnapshotMetadata, error) } -// DataFileSetReaderStatus describes the status of a file set reader +// DataFileSetReaderStatus describes the status of a file set reader. type DataFileSetReaderStatus struct { Namespace ident.ID BlockStart time.Time @@ -135,23 +134,7 @@ type DataReaderOpenOptions struct { OptimizedReadMetadataOnly bool } -// StreamedChecksum yields a schema.IndexChecksum value asynchronously, -// and any errors encountered during execution. -type StreamedChecksum interface { - // RetrieveIndexChecksum retrieves the index checksum. - RetrieveIndexChecksum() (xio.IndexChecksum, error) -} - -type emptyStreamedChecksum struct{} - -func (emptyStreamedChecksum) RetrieveIndexChecksum() (xio.IndexChecksum, error) { - return xio.IndexChecksum{}, nil -} - -// EmptyStreamedChecksum is an empty streamed checksum. -var EmptyStreamedChecksum StreamedChecksum = emptyStreamedChecksum{} - -// DataFileSetReader provides an unsynchronized reader for a TSDB file set +// DataFileSetReader provides an unsynchronized reader for a TSDB file set. type DataFileSetReader interface { io.Closer @@ -167,8 +150,8 @@ type DataFileSetReader interface { // them so they can be returned to their respective pools. Read() (id ident.ID, tags ident.TagIterator, data checked.Bytes, checksum uint32, err error) - // StreamingRead returns the next unpooled id, encodedTags, data, checksum values ordered by id, - // or error, will return io.EOF at end of volume. + // StreamingRead returns the next unpooled id, encodedTags, data, checksum + // values ordered by id, or error, will return io.EOF at end of volume. // Can only by used when DataReaderOpenOptions.StreamingEnabled is enabled. // Note: the returned id, encodedTags and data get invalidated on the next call to StreamingRead. StreamingRead() (id ident.BytesID, encodedTags ts.EncodedTags, data []byte, checksum uint32, err error) @@ -183,36 +166,36 @@ type DataFileSetReader interface { // for concurrent use and has a Close() method for releasing resources when done. ReadBloomFilter() (*ManagedConcurrentBloomFilter, error) - // Validate validates both the metadata and data and returns an error if either is corrupted + // Validate validates both the metadata and data and returns an error if either is corrupted. Validate() error - // ValidateMetadata validates the data and returns an error if the data is corrupted + // ValidateMetadata validates the data and returns an error if the data is corrupted. ValidateMetadata() error - // ValidateData validates the data and returns an error if the data is corrupted + // ValidateData validates the data and returns an error if the data is corrupted. ValidateData() error - // Range returns the time range associated with data in the volume + // Range returns the time range associated with data in the volume. Range() xtime.Range - // Entries returns the count of entries in the volume + // Entries returns the count of entries in the volume. Entries() int - // EntriesRead returns the position read into the volume + // EntriesRead returns the position read into the volume. EntriesRead() int - // MetadataRead returns the position of metadata read into the volume + // MetadataRead returns the position of metadata read into the volume. MetadataRead() int - // StreamingEnabled returns true if the reader is opened in streaming mode + // StreamingEnabled returns true if the reader is opened in streaming mode. StreamingEnabled() bool } -// DataFileSetSeeker provides an out of order reader for a TSDB file set +// DataFileSetSeeker provides an out of order reader for a TSDB file set. type DataFileSetSeeker interface { io.Closer - // Open opens the files for the given shard and version for reading + // Open opens the files for the given shard and version for reading. Open( namespace ident.ID, shard uint32, @@ -230,23 +213,15 @@ type DataFileSetSeeker interface { // entry and don't want to waste resources looking it up again. SeekByIndexEntry(entry IndexEntry, resources ReusableSeekerResources) (checked.Bytes, error) - // SeekReadMismatchesByIndexChecksum seeks in a manner similar to - // SeekIndexByEntry, checking against a set of streamed index checksums. - SeekReadMismatchesByIndexChecksum( - checksum xio.IndexChecksum, - mismatchChecker wide.EntryChecksumMismatchChecker, - resources ReusableSeekerResources, - ) (wide.ReadMismatch, error) - // SeekIndexEntry returns the IndexEntry for the specified ID. This can be useful // ahead of issuing a number of seek requests so that the seek requests can be // made in order. The returned IndexEntry can also be passed to SeekUsingIndexEntry // to prevent duplicate index lookups. SeekIndexEntry(id ident.ID, resources ReusableSeekerResources) (IndexEntry, error) - // SeekIndexEntryToIndexChecksum seeks in a manner similar to SeekIndexEntry, but - // instead yields a minimal structure describing a checksum of the series. - SeekIndexEntryToIndexChecksum(id ident.ID, resources ReusableSeekerResources) (xio.IndexChecksum, error) + // SeekWideEntry seeks in a manner similar to SeekIndexEntry, but + // instead yields a wide entry checksum of the series. + SeekWideEntry(id ident.ID, resources ReusableSeekerResources) (xio.WideEntry, error) // Range returns the time range associated with data in the volume Range() xtime.Range @@ -280,18 +255,11 @@ type ConcurrentDataFileSetSeeker interface { // SeekByIndexEntry is the same as in DataFileSetSeeker. SeekByIndexEntry(entry IndexEntry, resources ReusableSeekerResources) (checked.Bytes, error) - // SeekReadMismatchesByIndexChecksum is the same as in DataFileSetSeeker. - SeekReadMismatchesByIndexChecksum( - checksum xio.IndexChecksum, - mismatchChecker wide.EntryChecksumMismatchChecker, - resources ReusableSeekerResources, - ) (wide.ReadMismatch, error) - // SeekIndexEntry is the same as in DataFileSetSeeker. SeekIndexEntry(id ident.ID, resources ReusableSeekerResources) (IndexEntry, error) - // SeekIndexEntryToIndexChecksum is the same as in DataFileSetSeeker. - SeekIndexEntryToIndexChecksum(id ident.ID, resources ReusableSeekerResources) (xio.IndexChecksum, error) + // SeekWideEntry is the same as in DataFileSetSeeker. + SeekWideEntry(id ident.ID, resources ReusableSeekerResources) (xio.WideEntry, error) // ConcurrentIDBloomFilter is the same as in DataFileSetSeeker. ConcurrentIDBloomFilter() *ManagedConcurrentBloomFilter @@ -327,19 +295,19 @@ type DataFileSetSeekerManager interface { Test(id ident.ID, shard uint32, start time.Time) (bool, error) } -// DataBlockRetriever provides a block retriever for TSDB file sets +// DataBlockRetriever provides a block retriever for TSDB file sets. type DataBlockRetriever interface { io.Closer block.DatabaseBlockRetriever - // Open the block retriever to retrieve from a namespace + // Open the block retriever to retrieve from a namespace. Open( md namespace.Metadata, shardSet sharding.ShardSet, ) error } -// RetrievableDataBlockSegmentReader is a retrievable block reader +// RetrievableDataBlockSegmentReader is a retrievable block reader. type RetrievableDataBlockSegmentReader interface { xio.SegmentReader } @@ -464,7 +432,7 @@ type Options interface { IndexSummariesPercent() float64 // SetIndexBloomFilterFalsePositivePercent size sets the percent of false positive - // rate to use for the index bloom filter size and k hashes estimation + // rate to use for the index bloom filter size and k hashes estimation. SetIndexBloomFilterFalsePositivePercent(value float64) Options // IndexBloomFilterFalsePositivePercent size returns the percent of false positive @@ -493,10 +461,12 @@ type Options interface { // WriterBufferSize returns the buffer size for writing TSDB files. WriterBufferSize() int - // SetInfoReaderBufferSize sets the buffer size for reading TSDB info, digest and checkpoint files. + // SetInfoReaderBufferSize sets the buffer size for reading TSDB info, + // digest and checkpoint files. SetInfoReaderBufferSize(value int) Options - // InfoReaderBufferSize returns the buffer size for reading TSDB info, digest and checkpoint files. + // InfoReaderBufferSize returns the buffer size for reading TSDB info, + // digest and checkpoint files. InfoReaderBufferSize() int // SetDataReaderBufferSize sets the buffer size for reading TSDB data and index files. @@ -568,7 +538,7 @@ type Options interface { EncodingOptions() msgpack.LegacyEncodingOptions } -// BlockRetrieverOptions represents the options for block retrieval +// BlockRetrieverOptions represents the options for block retrieval. type BlockRetrieverOptions interface { // Validate validates the options. Validate() error @@ -653,9 +623,10 @@ type Merger interface { onFlush persist.OnFlushSeries, ) (persist.DataCloser, error) - // MergeAndCleanup merges the specified fileset file with a merge target and removes the previous version of the - // fileset. This should only be called within the bootstrapper. Any other file deletions outside of the bootstrapper - // should be handled by the CleanupManager. + // MergeAndCleanup merges the specified fileset file with a merge target and + // removes the previous version of the fileset. This should only be called + // within the bootstrapper. Any other file deletions outside of the + // bootstrapper should be handled by the CleanupManager. MergeAndCleanup( fileID FileSetFileIdentifier, mergeWith MergeWith, @@ -695,20 +666,21 @@ type BlockRecord struct { DataChecksum uint32 } -// CrossBlockReader allows reading data (encoded bytes) from multiple DataFileSetReaders of the same shard, -// ordered by series id first, and block start time next. +// CrossBlockReader allows reading data (encoded bytes) from multiple +// DataFileSetReaders of the same shard, ordered lexographically by series ID, +// then by block time. type CrossBlockReader interface { io.Closer - // Next advances to the next data record and returns true, or returns false if no more data exists. + // Next advances to the next data record, returning true if it exists. Next() bool // Err returns the last error encountered (if any). Err() error - // Current returns distinct series id and encodedTags, plus a slice with data and checksums from all - // blocks corresponding to that series (in temporal order). - // id, encodedTags, records slice and underlying data are being invalidated on each call to Next(). + // Current returns distinct series id and encodedTags, plus a slice with data + // and checksums from all blocks corresponding to that series (in temporal order). + // ID, encodedTags, records, and underlying data are invalidated on each call to Next(). Current() (id ident.BytesID, encodedTags ts.EncodedTags, records []BlockRecord) } diff --git a/src/dbnode/persist/fs/wide/entry_checksum_mismatch_checker.go b/src/dbnode/persist/fs/wide/entry_checksum_mismatch_checker.go deleted file mode 100644 index 1b5d04324f..0000000000 --- a/src/dbnode/persist/fs/wide/entry_checksum_mismatch_checker.go +++ /dev/null @@ -1,280 +0,0 @@ -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE - -package wide - -import ( - "bytes" - "fmt" - "sync" - - "github.com/m3db/m3/src/dbnode/persist/fs/msgpack" - "github.com/m3db/m3/src/dbnode/persist/schema" - "github.com/m3db/m3/src/dbnode/x/xio" - "github.com/m3db/m3/src/x/instrument" - - "go.uber.org/zap" -) - -type entryWithChecksum struct { - idChecksum int64 - entry schema.IndexEntry -} - -type entryChecksumMismatchChecker struct { - mu sync.Mutex - - blockReader IndexChecksumBlockBatchReader - mismatches []ReadMismatch - strictLastID []byte - - decodeOpts msgpack.DecodingOptions - iOpts instrument.Options - - batchIdx int - exhausted bool - started bool -} - -// FIXME: remove once this is changed to single output. -func (c *entryChecksumMismatchChecker) Lock() { - c.mu.Lock() -} - -// FIXME: remove once this is changed to single output. -func (c *entryChecksumMismatchChecker) Unlock() { - c.mu.Unlock() -} - -// NewEntryChecksumMismatchChecker creates a new entry checksum mismatch -// checker, backed by the given block reader. -// NB: index entries MUST be checked in lexicographical order by ID. -func NewEntryChecksumMismatchChecker( - blockReader IndexChecksumBlockBatchReader, - opts Options, -) EntryChecksumMismatchChecker { - return &entryChecksumMismatchChecker{ - blockReader: blockReader, - mismatches: make([]ReadMismatch, 0, opts.BatchSize()), - decodeOpts: opts.DecodingOptions(), - iOpts: opts.InstrumentOptions(), - } -} - -func checksumMismatch(checksum xio.IndexChecksum) ReadMismatch { - return ReadMismatch{ - IndexChecksum: checksum, - } -} - -func (c *entryChecksumMismatchChecker) checksumMismatches( - checksums ...xio.IndexChecksum, -) []ReadMismatch { - for _, checksum := range checksums { - c.mismatches = append(c.mismatches, checksumMismatch(checksum)) - } - - return c.mismatches -} - -func (c *entryChecksumMismatchChecker) recordIndexMismatches(checksums ...int64) { - for _, checksum := range checksums { - c.mismatches = append(c.mismatches, ReadMismatch{ - IndexChecksum: xio.IndexChecksum{ - MetadataChecksum: checksum, - }, - }) - } -} - -func (c *entryChecksumMismatchChecker) emitInvariantViolation( - marker []byte, - checksum int64, - entry xio.IndexChecksum, -) error { - // Checksums match but IDs do not. Treat as an invariant violation. - err := fmt.Errorf("checksum collision") - instrument.EmitAndLogInvariantViolation(c.iOpts, func(l *zap.Logger) { - l.Error( - err.Error(), - zap.Int64("checksum", checksum), - zap.Binary("marker", marker), - zap.Any("entry", entry), - ) - }) - return err -} - -func (c *entryChecksumMismatchChecker) readNextBatch() IndexChecksumBlockBatch { - if !c.blockReader.Next() { - c.exhausted = true - // NB: set exhausted to true and return an empty since there are no - // more available checksum blocks. - return IndexChecksumBlockBatch{} - } - - c.batchIdx = 0 - batch := c.blockReader.Current() - return batch -} - -func (c *entryChecksumMismatchChecker) ComputeMismatchesForEntry( - entry xio.IndexChecksum, -) ([]ReadMismatch, error) { - c.mismatches = c.mismatches[:0] - if c.exhausted { - // NB: no remaining batches in the index checksum block; any further - // elements are mismatches (missing from primary). - return c.checksumMismatches(entry), nil - } - - if !c.started { - c.started = true - if !c.blockReader.Next() { - // NB: no index checksum blocks available; any further - // elements are mismatches (missing from primary). - c.exhausted = true - return c.checksumMismatches(entry), nil - } - - c.batchIdx = 0 - } - - batch := c.blockReader.Current() - for { - markerIdx := len(batch.Checksums) - 1 - - // NB: If the incoming checksum block is empty, move to the next one. - if len(batch.Checksums) == 0 { - batch = c.readNextBatch() - if c.exhausted { - return c.mismatches, nil - } - - continue - } - - checksum := batch.Checksums[c.batchIdx] - markerCompare := bytes.Compare(batch.EndMarker, entry.ID.Bytes()) - if c.batchIdx < markerIdx { - if checksum == entry.MetadataChecksum { - // Matches: increment batch index and return any gathered mismatches. - c.batchIdx++ - return c.mismatches, nil - } - - for nextBatchIdx := c.batchIdx + 1; nextBatchIdx < markerIdx; nextBatchIdx++ { - // NB: read next hashes, checking for index checksum matches. - nextChecksum := batch.Checksums[nextBatchIdx] - if entry.MetadataChecksum != nextChecksum { - continue - } - - // Checksum match. Add previous checksums as mismatches. - c.recordIndexMismatches(batch.Checksums[c.batchIdx:nextBatchIdx]...) - c.batchIdx = nextBatchIdx + 1 - return c.mismatches, nil - } - - checksum = batch.Checksums[markerIdx] - // NB: this is the last element in the batch. Check ID against MARKER. - if entry.MetadataChecksum == checksum { - if markerCompare != 0 { - // Checksums match but IDs do not. Treat as emitInvariantViolation violation. - return nil, c.emitInvariantViolation(batch.EndMarker, checksum, entry) - } - - c.recordIndexMismatches(batch.Checksums[c.batchIdx:markerIdx]...) - // ID and checksum match. Advance the block iter and return empty. - batch = c.readNextBatch() - return c.mismatches, nil - } - - // Checksums do not match. - if markerCompare > 0 { - // This is a mismatch on primary that appears before the - // marker element. Return mismatch but do not advance iter. - return c.checksumMismatches(entry), nil - } - - // Current value is past the end of this batch. Mark all in batch as - // mismatches, and receive next batch. - c.recordIndexMismatches(batch.Checksums[c.batchIdx:]...) - batch = c.readNextBatch() - if c.exhausted { - // If no further values, add the current entry as a mismatch and return. - return c.checksumMismatches(entry), nil - } - - // All mismatches marked for the current batch, check entry against next - // batch. - continue - } - - // NB: this is the last element in the batch. Check ID against MARKER. - if entry.MetadataChecksum == checksum { - if markerCompare != 0 { - // Checksums match but IDs do not. Treat as emitInvariantViolation violation. - return nil, c.emitInvariantViolation(batch.EndMarker, checksum, entry) - } - - // ID and checksum match. Advance the block iter and return gathered mismatches. - batch = c.readNextBatch() - return c.mismatches, nil - } - - // Checksum mismatch. - if markerCompare == 0 { - // IDs match but checksums do not. Advance the block iter and return - // mismatch. - batch = c.readNextBatch() - return c.checksumMismatches(entry), nil - } else if markerCompare > 0 { - // This is a mismatch on primary that appears before the - // marker element. Return mismatch but do not advance iter. - return c.checksumMismatches(entry), nil - } - - // The current batch here is exceeded. Emit the current batch marker as - // a mismatch on primary, and advance the block iter. - c.recordIndexMismatches(checksum) - batch = c.readNextBatch() - if c.exhausted { - // If no further values, add the current entry as a mismatch and return. - return c.checksumMismatches(entry), nil - } - } -} - -func (c *entryChecksumMismatchChecker) Drain() []ReadMismatch { - if c.exhausted { - return nil - } - - c.mismatches = c.mismatches[:0] - curr := c.blockReader.Current() - c.recordIndexMismatches(curr.Checksums[c.batchIdx:]...) - for c.blockReader.Next() { - curr := c.blockReader.Current() - c.recordIndexMismatches(curr.Checksums...) - } - - return c.mismatches -} diff --git a/src/dbnode/persist/fs/wide/entry_checksum_mismatch_checker_prop_test.go b/src/dbnode/persist/fs/wide/entry_checksum_mismatch_checker_prop_test.go deleted file mode 100644 index ef213871ce..0000000000 --- a/src/dbnode/persist/fs/wide/entry_checksum_mismatch_checker_prop_test.go +++ /dev/null @@ -1,463 +0,0 @@ -// +build big -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package wide - -import ( - "fmt" - "os" - "sort" - "testing" - "time" - - "github.com/m3db/m3/src/dbnode/persist/fs/msgpack" - "github.com/m3db/m3/src/dbnode/persist/schema" - "github.com/m3db/m3/src/dbnode/x/xio" - "github.com/m3db/m3/src/x/checked" - "github.com/m3db/m3/src/x/ident" - xhash "github.com/m3db/m3/src/x/test/hash" - - "github.com/leanovate/gopter" - "github.com/leanovate/gopter/gen" - "github.com/leanovate/gopter/prop" -) - -func generateRawChecksums(size int, opts Options) []xio.IndexChecksum { - checksums := make([]xio.IndexChecksum, size) - indexHasher := opts.DecodingOptions().IndexEntryHasher() - for i := range checksums { - idStr := fmt.Sprintf("id-%03d", i) - tags := []byte(fmt.Sprintf("tags-%03d", i)) - - entry := schema.IndexEntry{ - ID: []byte(idStr), - EncodedTags: tags, - } - - checksums[i] = xio.IndexChecksum{ - ID: ident.StringID(idStr), - EncodedTags: checked.NewBytes(tags, checked.NewBytesOptions()), - MetadataChecksum: indexHasher.HashIndexEntry(entry), - } - } - - return checksums -} - -type generatedEntries struct { - taking []bool - entries []xio.IndexChecksum -} - -// genEntryTestInput creates a list of indexChecksums, -// dropping a certain percentage. -func genEntryTestInput(size int, opts Options) gopter.Gen { - entries := generateRawChecksums(size, opts) - - return gopter.CombineGens( - // NB: This generator controls if the element should be removed - gen.SliceOfN(len(entries), gen.IntRange(0, 100)), - ).Map(func(val []interface{}) generatedEntries { - var ( - dropChancePercent = val[0].([]int) - - taking []bool - takenEntries []xio.IndexChecksum - ) - - for i, chance := range dropChancePercent { - shouldKeep := chance <= 80 - taking = append(taking, shouldKeep) - if shouldKeep { - takenEntries = append(takenEntries, entries[i]) - } - } - - return generatedEntries{taking: taking, entries: takenEntries} - }) -} - -type generatedChecksums struct { - taking []bool - blockBatch []IndexChecksumBlockBatch -} - -// genChecksumTestInput creates index checksum blockBatch of randomized sizes, -// dropping a certain percentage of index checksums. -func genChecksumTestInput(size int, opts Options) gopter.Gen { - entries := generateRawChecksums(size, opts) - - return gopter.CombineGens( - // NB: This generator controls if the element should be removed - gen.SliceOfN(len(entries), gen.IntRange(0, 100)), - // NB: This generator controls how large each batch will be - gen.SliceOfN(len(entries), gen.IntRange(1, len(entries))), - ).Map(func(val []interface{}) generatedChecksums { - var ( - dropChancePercent = val[0].([]int) - blockSizes = val[1].([]int) - - taking []bool - takenChecksums []xio.IndexChecksum - checksumBlocks []IndexChecksumBlockBatch - ) - - for i, chance := range dropChancePercent { - shouldKeep := chance <= 80 - taking = append(taking, shouldKeep) - if shouldKeep { - takenChecksums = append(takenChecksums, xio.IndexChecksum{ - ID: entries[i].ID, - MetadataChecksum: entries[i].MetadataChecksum, - }) - } - } - - for _, blockSize := range blockSizes { - remaining := len(takenChecksums) - if remaining == 0 { - break - } - - take := blockSize - if remaining < take { - take = remaining - } - - block := IndexChecksumBlockBatch{ - Checksums: make([]int64, 0, take), - } - - for i := 0; i < take; i++ { - block.Checksums = append(block.Checksums, takenChecksums[i].MetadataChecksum) - block.EndMarker = takenChecksums[i].ID.Bytes() - } - - takenChecksums = takenChecksums[take:] - checksumBlocks = append(checksumBlocks, block) - } - - return generatedChecksums{taking: taking, blockBatch: checksumBlocks} - }) -} - -type mismatchChecksumBatch struct { - lastElementMarker bool - mismatches []mismatchChecksum -} - -func (b *mismatchChecksumBatch) gatherContiguousMismatchValues() { - var ( - checksumSet bool - hasEntryMismatch bool - hasChecksumMismatch bool - contiguousCount int - nextContiguous int64 - ) - - for idx, mismatchChecksum := range b.mismatches { - var ( - lastIsContiguous bool - - checksum = mismatchChecksum.checksum - isLast = idx == len(b.mismatches)-1 - ) - - // NB: gather the number of contiguous mismatches. Mismatches are contiguous - // if they appear one after another, with no matching entries between them. - if !checksumSet || checksum == nextContiguous { - checksumSet = true - - if mismatchChecksum.entryMismatch { - hasEntryMismatch = true - } else { - hasChecksumMismatch = true - } - - contiguousCount++ - if !isLast { - // If this is not the last mismatch, increase the contiguous length. - nextContiguous = checksum + 1 - continue - } else { - lastIsContiguous = true - } - } - - // A contiguous set of mismatches should be sorted IFF: - // - at least 2 values - // - contiguous set contains both entry and checksum mismatches - // After sorting, all entry mismatches should appear first, in - // increasing order, followed by index mismatches in increasing order. - // NB: if the last element of a batch is a mismatch, it is fixed and should - // not be sorted. - if contiguousCount > 1 && hasEntryMismatch && hasChecksumMismatch { - firstContiguous := idx - contiguousCount - lastContiguous := idx - if lastIsContiguous { - firstContiguous++ - if !b.lastElementMarker { - lastContiguous++ - } - } - - contiguousSlice := b.mismatches[firstContiguous:lastContiguous] - sort.Slice(contiguousSlice, func(i, j int) bool { - iEntry, jEntry := contiguousSlice[i], contiguousSlice[j] - if iEntry.entryMismatch { - if !jEntry.entryMismatch { - // entry mismatches always come before checksum mismatches. - return true - } - - // these should be sorted by lex order - return iEntry.checksum < jEntry.checksum - } - - if jEntry.entryMismatch { - // checksum mismatches always come after entry mismatches. - return false - } - - // these should be sorted by lex order - return iEntry.checksum < jEntry.checksum - }) - } - - // clear - contiguousCount = 1 - hasChecksumMismatch = false - hasEntryMismatch = false - if mismatchChecksum.entryMismatch { - hasEntryMismatch = true - } else { - hasChecksumMismatch = true - } - - nextContiguous = checksum + 1 - } -} - -func allMismatchChecksumsToMismatchesByBatch( - checksums generatedChecksums, - allMismatchChecksums []mismatchChecksum, -) []mismatchChecksumBatch { - allMismatchIdx := 0 - var mismatchBatch []mismatchChecksumBatch - for _, batch := range checksums.blockBatch { - l := len(batch.Checksums) - if l == 0 { - continue - } - - lastChecksum := batch.Checksums[l-1] - lastElementMarker := false - var mismatches []mismatchChecksum - for _, mismatch := range allMismatchChecksums[allMismatchIdx:] { - if mismatch.checksum > lastChecksum { - // mismatch past last checksum in batch; append current batch and - // start a new one. - break - } - - mismatches = append(mismatches, mismatch) - allMismatchIdx++ - if mismatch.checksum == lastChecksum { - // mismatch is last checksum in batch; append current batch and - // start a new one. - lastElementMarker = true - break - } - } - - if len(mismatches) == 0 { - continue - } - - // add a mismatch batch; imporant to note if the last element is a mismatch, - // since if it is, it should always remain the last element, regardless of - // if it forms a contiguous group or not. - mismatchBatch = append(mismatchBatch, mismatchChecksumBatch{ - lastElementMarker: lastElementMarker, - mismatches: mismatches, - }) - } - - // add any remaining mismatch checksums as a separate batch. This is ok - // since they will all be entry mismatches, so no additional sorting will be - // performed on this batch. - if allMismatchIdx < len(allMismatchChecksums) { - mismatchBatch = append(mismatchBatch, mismatchChecksumBatch{ - lastElementMarker: false, - mismatches: allMismatchChecksums[allMismatchIdx:], - }) - } - - return mismatchBatch -} - -type mismatchChecksum struct { - missingOnBoth bool - checksum int64 - entryMismatch bool -} - -func buildExpectedMismatchChecksums( - checksums generatedChecksums, - takeEntries []bool, -) []mismatchChecksum { - var allMismatchChecksums []mismatchChecksum - takeChecksums := checksums.taking - // Collect only elements that don't match. - for idx, takeEntry := range takeEntries { - if takeEntry != takeChecksums[idx] { - allMismatchChecksums = append(allMismatchChecksums, mismatchChecksum{ - checksum: int64(idx), - entryMismatch: takeEntry, - }) - } else if !takeEntry && !takeChecksums[idx] { - // Note checksums missing from both sets; this will be necessary when - // checking for congiuous series in gatherContiguousMismatchValues. - allMismatchChecksums = append(allMismatchChecksums, mismatchChecksum{ - missingOnBoth: true, - checksum: int64(idx), - }) - } - } - - var gatheredMismatchChecksums []mismatchChecksum - // Gather mismatches to match incoming batches. - mismatchesByBatch := allMismatchChecksumsToMismatchesByBatch(checksums, allMismatchChecksums) - for _, batchMismatches := range mismatchesByBatch { - // Sort each batch as will be expected in output. - batchMismatches.gatherContiguousMismatchValues() - - // Filter out series which do not appear in either checksum source. - filteredMismatches := batchMismatches.mismatches[:0] - for _, mismatch := range batchMismatches.mismatches { - if !mismatch.missingOnBoth { - filteredMismatches = append(filteredMismatches, mismatch) - } - } - - gatheredMismatchChecksums = append(gatheredMismatchChecksums, filteredMismatches...) - } - - return gatheredMismatchChecksums -} - -func TestIndexEntryWideBatchMismatchChecker(t *testing.T) { - var ( - parameters = gopter.DefaultTestParameters() - seed = time.Now().UnixNano() - props = gopter.NewProperties(parameters) - reporter = gopter.NewFormatedReporter(true, 80, os.Stdout) - - hasher = xhash.NewParsedIndexHasher(t) - decodingOpts = msgpack.NewDecodingOptions().SetIndexEntryHasher(hasher) - opts = NewOptions().SetDecodingOptions(decodingOpts) - - size = 100 - numTests = 1000 - ) - - parameters.MinSuccessfulTests = numTests - parameters.Rng.Seed(seed) - - // NB: capture seed to be able to replicate failed runs. - fmt.Println("Running test with seed", seed) - props.Property("Checksum mismatcher detects correctly", - prop.ForAll( - func( - genChecksums generatedChecksums, - genEntries generatedEntries, - ) (bool, error) { - inputBlockCh := make(chan IndexChecksumBlockBatch) - inputBlockReader := NewIndexChecksumBlockBatchReader(inputBlockCh) - - go func() { - for _, bl := range genChecksums.blockBatch { - inputBlockCh <- bl - } - - close(inputBlockCh) - }() - - checker := NewEntryChecksumMismatchChecker(inputBlockReader, opts) - var readMismatches []ReadMismatch - for _, entry := range genEntries.entries { - entryMismatches, err := checker.ComputeMismatchesForEntry(entry) - if err != nil { - return false, fmt.Errorf("failed to compute index entry: %v", err) - } - - readMismatches = append(readMismatches, entryMismatches...) - } - - readMismatches = append(readMismatches, checker.Drain()...) - expectedMismatches := buildExpectedMismatchChecksums( - genChecksums, genEntries.taking) - - if len(expectedMismatches) != len(readMismatches) { - return false, fmt.Errorf("expected %d expectedMismatches, got %d", - len(expectedMismatches), len(readMismatches)) - } - - for i, expected := range expectedMismatches { - actual := readMismatches[i] - if actual.MetadataChecksum != expected.checksum { - return false, fmt.Errorf("expected checksum %d, got %d at %d", - actual.MetadataChecksum, expected.checksum, i) - } - - if expected.entryMismatch { - expectedTags := fmt.Sprintf("tags-%03d", actual.MetadataChecksum) - actual.EncodedTags.IncRef() - acTags := string(actual.EncodedTags.Bytes()) - actual.EncodedTags.DecRef() - if acTags != expectedTags { - return false, fmt.Errorf("expected tags %s, got %s", - expectedTags, acTags) - } - - expectedID := fmt.Sprintf("id-%03d", actual.MetadataChecksum) - if acID := actual.ID.String(); acID != expectedID { - return false, fmt.Errorf("expected tags %s, got %s", - expectedID, acID) - } - } else { - if actual.EncodedTags != nil { - return false, fmt.Errorf("index mismatch should not have tags") - } - if actual.ID != nil { - return false, fmt.Errorf("index mismatch should not have id") - } - } - } - - return true, nil - }, genChecksumTestInput(size, opts), genEntryTestInput(size, opts))) - - if !props.Run(reporter) { - t.Errorf("failed with initial seed: %d", seed) - } -} diff --git a/src/dbnode/persist/fs/wide/entry_checksum_mismatch_checker_test.go b/src/dbnode/persist/fs/wide/entry_checksum_mismatch_checker_test.go deleted file mode 100644 index 18a41aadc9..0000000000 --- a/src/dbnode/persist/fs/wide/entry_checksum_mismatch_checker_test.go +++ /dev/null @@ -1,375 +0,0 @@ -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package wide - -import ( - "testing" - "time" - - "github.com/m3db/m3/src/dbnode/persist/fs/msgpack" - "github.com/m3db/m3/src/dbnode/x/xio" - "github.com/m3db/m3/src/x/checked" - "github.com/m3db/m3/src/x/ident" - "github.com/m3db/m3/src/x/instrument" - xtest "github.com/m3db/m3/src/x/test" - xhash "github.com/m3db/m3/src/x/test/hash" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" -) - -func buildTestReader(bls ...IndexChecksumBlockBatch) IndexChecksumBlockBatchReader { - ch := make(chan IndexChecksumBlockBatch) - reader := NewIndexChecksumBlockBatchReader(ch) - go func() { - for _, bl := range bls { - ch <- bl - } - - close(ch) - }() - return reader -} - -// buildOpts builds default test options. The NewParsedIndexHasher sets -// the hash value for a xio.IndexEntry as any string represented integer -// values in the entry ID + entry tags. -func buildOpts(t *testing.T) Options { - decodingOpts := msgpack.NewDecodingOptions(). - SetIndexEntryHasher(xhash.NewParsedIndexHasher(t)) - opts := NewOptions(). - SetBatchSize(2). - SetDecodingOptions(decodingOpts). - SetInstrumentOptions(instrument.NewOptions()) - require.NoError(t, opts.Validate()) - return opts -} - -func toChecksum(id, tags string, checksum int64) xio.IndexChecksum { - return xio.IndexChecksum{ - ID: ident.StringID(id), - EncodedTags: checked.NewBytes([]byte(tags), checked.NewBytesOptions()), - MetadataChecksum: checksum, - } -} - -func testIdxMismatch(checksum int64) ReadMismatch { - return ReadMismatch{ - IndexChecksum: xio.IndexChecksum{ - MetadataChecksum: checksum, - }, - } -} - -func testEntryMismatch(id, tags string, checksum int64) ReadMismatch { - return ReadMismatch{ - IndexChecksum: toChecksum(id, tags, checksum), - } -} - -func testMismatches(t *testing.T, expected, actual []ReadMismatch) { - require.Equal(t, len(expected), len(actual)) - for i, ex := range expected { - assert.Equal(t, ex.ID, actual[i].ID) - assert.Equal(t, ex.Size, actual[i].Size) - assert.Equal(t, ex.Offset, actual[i].Offset) - assert.Equal(t, ex.DataChecksum, actual[i].DataChecksum) - assert.Equal(t, ex.MetadataChecksum, actual[i].MetadataChecksum) - } -} - -func TestEmitMismatches(t *testing.T) { - ctrl := xtest.NewController(t) - defer ctrl.Finish() - reader := buildTestReader() - chk := NewEntryChecksumMismatchChecker(reader, buildOpts(t)) - checker, ok := chk.(*entryChecksumMismatchChecker) - require.True(t, ok) - - id1, tags1 := "foo1", "encoded-tags1" - id2, tags2 := "foo2", "encoded-tags2" - id3, tags3 := "foo3", "encoded-tags3" - checker.checksumMismatches(toChecksum(id1, tags1, 0), toChecksum(id2, tags2, 1)) - checker.recordIndexMismatches(100, 200) - checker.checksumMismatches(toChecksum(id3, tags3, 2)) - checker.recordIndexMismatches(300) - - expected := []ReadMismatch{ - testEntryMismatch(id1, tags1, 0), - testEntryMismatch(id2, tags2, 1), - testIdxMismatch(100), - testIdxMismatch(200), - testEntryMismatch(id3, tags3, 2), - testIdxMismatch(300), - } - - testMismatches(t, expected, checker.mismatches) -} - -func TestComputeMismatchInvariant(t *testing.T) { - reader := buildTestReader(IndexChecksumBlockBatch{ - Checksums: []int64{1}, - EndMarker: []byte("foo1"), - }) - - chk := NewEntryChecksumMismatchChecker(reader, buildOpts(t)) - _, err := chk.ComputeMismatchesForEntry(toChecksum("bar1", "bar", 1)) - require.Error(t, err) -} - -func TestComputeMismatchInvariantEndOfBlock(t *testing.T) { - reader := buildTestReader(IndexChecksumBlockBatch{ - Checksums: []int64{1, 2}, - EndMarker: []byte("foo2"), - }) - - chk := NewEntryChecksumMismatchChecker(reader, buildOpts(t)) - _, err := chk.ComputeMismatchesForEntry(toChecksum("bar2", "bar", 2)) - require.Error(t, err) -} - -func assertNoMismatch( - t *testing.T, - chk EntryChecksumMismatchChecker, - checksum xio.IndexChecksum, -) { - mismatch, err := chk.ComputeMismatchesForEntry(checksum) - require.NoError(t, err) - assert.Equal(t, 0, len(mismatch)) -} - -func TestComputeMismatchWithDelayedReader(t *testing.T) { - ch := make(chan IndexChecksumBlockBatch) - reader := NewIndexChecksumBlockBatchReader(ch) - chk := NewEntryChecksumMismatchChecker(reader, buildOpts(t)) - - go func() { - time.Sleep(time.Millisecond * 100) - ch <- IndexChecksumBlockBatch{ - Checksums: []int64{1}, - EndMarker: []byte("foo1"), - } - time.Sleep(time.Millisecond * 200) - ch <- IndexChecksumBlockBatch{ - Checksums: []int64{10}, - EndMarker: []byte("qux10"), - } - close(ch) - }() - - assertNoMismatch(t, chk, toChecksum("foo1", "bar", 1)) - assertNoMismatch(t, chk, toChecksum("qux10", "baz", 10)) - assert.Equal(t, 0, len(chk.Drain())) -} - -func TestComputeMismatchNoMismatch(t *testing.T) { - reader := buildTestReader(IndexChecksumBlockBatch{ - Checksums: []int64{1, 2, 3}, - EndMarker: []byte("foo3"), - }, IndexChecksumBlockBatch{ - Checksums: []int64{100, 5}, - EndMarker: []byte("zoo5"), - }) - - chk := NewEntryChecksumMismatchChecker(reader, buildOpts(t)) - assertNoMismatch(t, chk, toChecksum("abc1", "aaa", 1)) - assertNoMismatch(t, chk, toChecksum("def2", "bbb", 2)) - assertNoMismatch(t, chk, toChecksum("foo3", "ccc", 3)) - assertNoMismatch(t, chk, toChecksum("qux100", "ddd", 100)) - assertNoMismatch(t, chk, toChecksum("zoo5", "eee", 5)) - assert.Equal(t, 0, len(chk.Drain())) -} - -func TestComputeMismatchMismatchesIndexMismatch(t *testing.T) { - reader := buildTestReader(IndexChecksumBlockBatch{ - Checksums: []int64{1, 2, 3}, - EndMarker: []byte("foo3"), - }, IndexChecksumBlockBatch{ - Checksums: []int64{4, 5}, - EndMarker: []byte("moo5"), - }, IndexChecksumBlockBatch{ - Checksums: []int64{6, 7, 8}, - EndMarker: []byte("qux8"), - }, IndexChecksumBlockBatch{ - Checksums: []int64{9, 10}, - EndMarker: []byte("zzz9"), - }) - - chk := NewEntryChecksumMismatchChecker(reader, buildOpts(t)) - - expected := []ReadMismatch{ - testIdxMismatch(1), - testIdxMismatch(2), - } - - mismatches, err := chk.ComputeMismatchesForEntry(toChecksum("foo3", "ccc", 3)) - require.NoError(t, err) - testMismatches(t, expected, mismatches) - - expected = []ReadMismatch{ - testIdxMismatch(4), - testIdxMismatch(5), - testIdxMismatch(6), - } - - mismatches, err = chk.ComputeMismatchesForEntry(toChecksum("qux7", "ddd", 7)) - require.NoError(t, err) - testMismatches(t, expected, mismatches) - - expected = []ReadMismatch{ - testIdxMismatch(8), - testIdxMismatch(9), - testIdxMismatch(10), - } - testMismatches(t, expected, chk.Drain()) -} - -func TestComputeMismatchMismatchesEntryMismatches(t *testing.T) { - reader := buildTestReader(IndexChecksumBlockBatch{ - Checksums: []int64{4}, - EndMarker: []byte("foo3"), - }, IndexChecksumBlockBatch{ - Checksums: []int64{5}, - EndMarker: []byte("goo5"), - }, IndexChecksumBlockBatch{ - Checksums: []int64{6}, - EndMarker: []byte("moo6"), - }, IndexChecksumBlockBatch{ - Checksums: []int64{7}, - EndMarker: []byte("qux7"), - }) - - chk := NewEntryChecksumMismatchChecker(reader, buildOpts(t)) - expected := []ReadMismatch{ - testEntryMismatch("abc1", "ccc", 1), - } - - mismatches, err := chk.ComputeMismatchesForEntry(toChecksum("abc1", "ccc", 1)) - require.NoError(t, err) - testMismatches(t, expected, mismatches) - - expected = []ReadMismatch{ - testEntryMismatch("def2", "ddd", 2), - } - - mismatches, err = chk.ComputeMismatchesForEntry(toChecksum("def2", "ddd", 2)) - require.NoError(t, err) - testMismatches(t, expected, mismatches) - - expected = []ReadMismatch{ - testEntryMismatch("foo3", "f1", 3), - } - - mismatches, err = chk.ComputeMismatchesForEntry(toChecksum("foo3", "f1", 3)) - require.NoError(t, err) - testMismatches(t, expected, mismatches) - - expected = []ReadMismatch{ - testIdxMismatch(5), - } - - mismatches, err = chk.ComputeMismatchesForEntry(toChecksum("moo6", "a", 6)) - require.NoError(t, err) - testMismatches(t, expected, mismatches) - - expected = []ReadMismatch{ - testIdxMismatch(7), - testEntryMismatch("zoo10", "z", 10), - } - - mismatches, err = chk.ComputeMismatchesForEntry(toChecksum("zoo10", "z", 10)) - require.NoError(t, err) - testMismatches(t, expected, mismatches) - - assert.Equal(t, 0, len(chk.Drain())) -} - -func TestComputeMismatchMismatchesOvershoot(t *testing.T) { - reader := buildTestReader(IndexChecksumBlockBatch{ - Checksums: []int64{1, 2, 3}, - EndMarker: []byte("foo3"), - }, IndexChecksumBlockBatch{ - Checksums: []int64{4, 5, 10}, - EndMarker: []byte("goo10"), - }) - - chk := NewEntryChecksumMismatchChecker(reader, buildOpts(t)) - expected := []ReadMismatch{ - testEntryMismatch("abc10", "ccc", 10), - } - - mismatches, err := chk.ComputeMismatchesForEntry(toChecksum("abc10", "ccc", 10)) - require.NoError(t, err) - testMismatches(t, expected, mismatches) - - expected = []ReadMismatch{ - testIdxMismatch(1), - testIdxMismatch(2), - testIdxMismatch(3), - testIdxMismatch(4), - testIdxMismatch(5), - testIdxMismatch(10), - testEntryMismatch("zzz20", "ccc", 20), - } - - mismatches, err = chk.ComputeMismatchesForEntry(toChecksum("zzz20", "ccc", 20)) - require.NoError(t, err) - testMismatches(t, expected, mismatches) - - assert.Equal(t, 0, len(chk.Drain())) -} - -func TestComputeMismatchMismatchesEntryMismatchSkipsFirst(t *testing.T) { - reader := buildTestReader(IndexChecksumBlockBatch{ - Checksums: []int64{4}, - EndMarker: []byte("foo3"), - }) - - chk := NewEntryChecksumMismatchChecker(reader, buildOpts(t)) - expected := []ReadMismatch{ - testEntryMismatch("foo3", "abc", 3), - } - - mismatches, err := chk.ComputeMismatchesForEntry(toChecksum("foo3", "abc", 3)) - require.NoError(t, err) - testMismatches(t, expected, mismatches) - - assert.Equal(t, 0, len(chk.Drain())) -} - -func TestComputeMismatchMismatchesEntryMismatchMatchesLast(t *testing.T) { - reader := buildTestReader(IndexChecksumBlockBatch{ - Checksums: []int64{1, 2, 3}, - EndMarker: []byte("foo3"), - }) - - chk := NewEntryChecksumMismatchChecker(reader, buildOpts(t)) - expected := []ReadMismatch{ - testIdxMismatch(1), - testIdxMismatch(2), - } - - mismatches, err := chk.ComputeMismatchesForEntry(toChecksum("foo3", "abc", 3)) - require.NoError(t, err) - testMismatches(t, expected, mismatches) - - assert.Equal(t, 0, len(chk.Drain())) -} diff --git a/src/dbnode/persist/fs/wide/index_checksum_block_batch_reader.go b/src/dbnode/persist/fs/wide/index_checksum_block_batch_reader.go deleted file mode 100644 index 1a02a3fb53..0000000000 --- a/src/dbnode/persist/fs/wide/index_checksum_block_batch_reader.go +++ /dev/null @@ -1,79 +0,0 @@ -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE - -package wide - -import ( - "sync" -) - -type indexChecksumBlockReader struct { - mu sync.Mutex - closed bool - - currentBlock IndexChecksumBlockBatch - blocks chan IndexChecksumBlockBatch -} - -// NewIndexChecksumBlockBatchReader creates a new IndexChecksumBlockBatchReader. -func NewIndexChecksumBlockBatchReader( - blockInput chan IndexChecksumBlockBatch, -) IndexChecksumBlockBatchReader { - return &indexChecksumBlockReader{ - blocks: blockInput, - } -} - -func (b *indexChecksumBlockReader) Current() IndexChecksumBlockBatch { - return b.currentBlock -} - -func (b *indexChecksumBlockReader) Next() bool { - b.mu.Lock() - defer b.mu.Unlock() - - if b.closed { - return false - } - - if bl, ok := <-b.blocks; ok { - b.currentBlock = bl - return true - } - - b.closed = true - return false -} - -func (b *indexChecksumBlockReader) Close() { - b.mu.Lock() - defer b.mu.Unlock() - - if b.closed { - return - } - - // NB: drain block channel. - for range b.blocks { - } - - b.closed = true - return -} diff --git a/src/dbnode/persist/fs/wide/index_checksum_block_batch_reader_test.go b/src/dbnode/persist/fs/wide/index_checksum_block_batch_reader_test.go deleted file mode 100644 index 12e8ed6171..0000000000 --- a/src/dbnode/persist/fs/wide/index_checksum_block_batch_reader_test.go +++ /dev/null @@ -1,49 +0,0 @@ -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE - -package wide - -import ( - "testing" - - "github.com/stretchr/testify/assert" -) - -func TestIndexChecksumBlockBatchReader(t *testing.T) { - ch := make(chan IndexChecksumBlockBatch) - buf := NewIndexChecksumBlockBatchReader(ch) - bl := IndexChecksumBlockBatch{EndMarker: []byte("foo")} - bl2 := IndexChecksumBlockBatch{ - Checksums: []int64{1, 2, 3}, - EndMarker: []byte("bar"), - } - - go func() { - ch <- bl - ch <- bl2 - close(ch) - }() - - assert.True(t, buf.Next()) - assert.Equal(t, bl, buf.Current()) - assert.True(t, buf.Next()) - assert.Equal(t, bl2, buf.Current()) - assert.False(t, buf.Next()) -} diff --git a/src/dbnode/persist/fs/wide/options.go b/src/dbnode/persist/fs/wide/options.go deleted file mode 100644 index ae6a7d5ee3..0000000000 --- a/src/dbnode/persist/fs/wide/options.go +++ /dev/null @@ -1,94 +0,0 @@ -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package wide - -import ( - "errors" - "fmt" - - "github.com/m3db/m3/src/dbnode/persist/fs/msgpack" - "github.com/m3db/m3/src/x/instrument" - "github.com/m3db/m3/src/x/pool" -) - -const ( - defaultbatchSize = 1024 -) - -var ( - errDecodingOptionsUnset = errors.New("decoding options unset") - invalidBatchSizeTemplate = "batch size %d must be greater than 0" -) - -type options struct { - batchSize int - bytesPool pool.BytesPool - decodingOptions msgpack.DecodingOptions - instrumentOpts instrument.Options -} - -// NewOptions creates a new set of wide query options. -func NewOptions() Options { - return &options{ - batchSize: defaultbatchSize, - instrumentOpts: instrument.NewOptions(), - } -} - -func (o *options) Validate() error { - if o.decodingOptions == nil { - return errDecodingOptionsUnset - } - if o.batchSize < 1 { - return fmt.Errorf(invalidBatchSizeTemplate, o.batchSize) - } - return nil -} - -func (o *options) SetBatchSize(value int) Options { - opts := *o - opts.batchSize = value - return &opts -} - -func (o *options) BatchSize() int { - return o.batchSize -} - -func (o *options) SetDecodingOptions(value msgpack.DecodingOptions) Options { - opts := *o - opts.decodingOptions = value - return &opts -} - -func (o *options) DecodingOptions() msgpack.DecodingOptions { - return o.decodingOptions -} - -func (o *options) SetInstrumentOptions(value instrument.Options) Options { - opts := *o - opts.instrumentOpts = value - return &opts -} - -func (o *options) InstrumentOptions() instrument.Options { - return o.instrumentOpts -} diff --git a/src/dbnode/persist/fs/wide/options_test.go b/src/dbnode/persist/fs/wide/options_test.go deleted file mode 100644 index 65d5442fed..0000000000 --- a/src/dbnode/persist/fs/wide/options_test.go +++ /dev/null @@ -1,45 +0,0 @@ -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package wide - -import ( - "testing" - - "github.com/m3db/m3/src/dbnode/persist/fs/msgpack" - - "github.com/stretchr/testify/assert" -) - -func TestOptions(t *testing.T) { - opts := NewOptions() - assert.Error(t, opts.Validate()) - - decOpts := msgpack.NewDecodingOptions() - opts = opts.SetDecodingOptions(decOpts) - assert.Equal(t, decOpts, opts.DecodingOptions()) - assert.NoError(t, opts.Validate()) - - opts = opts.SetBatchSize(-1) - assert.Error(t, opts.Validate()) - - opts = opts.SetBatchSize(100) - assert.Equal(t, 100, opts.BatchSize()) -} diff --git a/src/dbnode/persist/fs/wide/types.go b/src/dbnode/persist/fs/wide/types.go deleted file mode 100644 index 4b11230c5b..0000000000 --- a/src/dbnode/persist/fs/wide/types.go +++ /dev/null @@ -1,117 +0,0 @@ -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE - -package wide - -import ( - "github.com/m3db/m3/src/dbnode/persist/fs/msgpack" - "github.com/m3db/m3/src/dbnode/x/xio" - "github.com/m3db/m3/src/x/checked" - "github.com/m3db/m3/src/x/instrument" -) - -// Options represents the options for mismatch calculation. -type Options interface { - // Validate will validate the options and return an error if not valid. - Validate() error - - // SetBatchSize sets the batch size. - SetBatchSize(value int) Options - - // BatchSize returns the batch size. - BatchSize() int - - // SetDecodingOptions sets the decoding options. - SetDecodingOptions(value msgpack.DecodingOptions) Options - - // DecodingOptions returns the decoding options. - DecodingOptions() msgpack.DecodingOptions - - // SetInstrumentOptions sets the instrumentation options. - SetInstrumentOptions(value instrument.Options) Options - - // InstrumentOptions returns the instrumentation options. - InstrumentOptions() instrument.Options -} - -// ReadMismatch describes a series that does not match the expected wide index -// checksum, with a descriptor of the mismatch. This can indicate both scenarios -// where the expected checksum was not found, and when there is a mismatch. -type ReadMismatch struct { - // ReadMismatch extends IndexChecksum with additional mismatch fields. - xio.IndexChecksum - // Data is the data for the read mismatch. Set only on reader mismatches. - Data checked.Bytes -} - -// IsReaderMismatch is true if this mismatch is this mismatch is on the reader -// side. -func (r ReadMismatch) IsReaderMismatch() bool { - return r.IndexChecksum.ID != nil || - r.IndexChecksum.EncodedTags != nil -} - -// IndexChecksumBlockBatchReader is a reader across IndexChecksumBlockBatches. -type IndexChecksumBlockBatchReader interface { - // Next moves to the next IndexChecksumBlockBatch element. - Next() bool - // Current yields the current IndexChecksumBlockBatch. - Current() IndexChecksumBlockBatch - // Close closes the reader, draining any incoming reads without using them. - Close() -} - -// EntryChecksumMismatchChecker checks if a given entry should yield a mismatch. -type EntryChecksumMismatchChecker interface { - // ComputeMismatchesForEntry determines if the given index entry is a mismatch. - ComputeMismatchesForEntry(entry xio.IndexChecksum) ([]ReadMismatch, error) - // Drain returns any unconsumed IndexChecksumBlockBatches as mismatches. - Drain() []ReadMismatch - // Lock sets a mutex on this mismatch checker. - Lock() - // Unlock unlocks the mutex on the mismatch checker. - Unlock() -} - -// StreamedMismatch yields a ReadMismatch value asynchronously, -// and any errors encountered during execution. -type StreamedMismatch interface { - // RetrieveMismatch retrieves the mismatch. - RetrieveMismatch() (ReadMismatch, error) -} - -type emptyStreamedMismatch struct{} - -func (emptyStreamedMismatch) RetrieveMismatch() (ReadMismatch, error) { - return ReadMismatch{}, nil -} - -// EmptyStreamedMismatch is an empty streamed mismatch batch. -var EmptyStreamedMismatch StreamedMismatch = emptyStreamedMismatch{} - -// IndexChecksumBlockBatch represents a batch of index checksums originating -// from a single series block. -type IndexChecksumBlockBatch struct { - // Checksums is the list of index checksums. - Checksums []int64 - // EndMarker is a batch marker, signifying the ID of the - // last element in the batch. - EndMarker []byte -} diff --git a/src/dbnode/persist/fs/wide/wide_mock.go b/src/dbnode/persist/fs/wide/wide_mock.go deleted file mode 100644 index 4650401438..0000000000 --- a/src/dbnode/persist/fs/wide/wide_mock.go +++ /dev/null @@ -1,147 +0,0 @@ -// Code generated by MockGen. DO NOT EDIT. -// Source: github.com/m3db/m3/src/dbnode/persist/fs/wide (interfaces: EntryChecksumMismatchChecker,StreamedMismatch) - -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -// Package wide is a generated GoMock package. -package wide - -import ( - "reflect" - - "github.com/m3db/m3/src/dbnode/x/xio" - - "github.com/golang/mock/gomock" -) - -// MockEntryChecksumMismatchChecker is a mock of EntryChecksumMismatchChecker interface -type MockEntryChecksumMismatchChecker struct { - ctrl *gomock.Controller - recorder *MockEntryChecksumMismatchCheckerMockRecorder -} - -// MockEntryChecksumMismatchCheckerMockRecorder is the mock recorder for MockEntryChecksumMismatchChecker -type MockEntryChecksumMismatchCheckerMockRecorder struct { - mock *MockEntryChecksumMismatchChecker -} - -// NewMockEntryChecksumMismatchChecker creates a new mock instance -func NewMockEntryChecksumMismatchChecker(ctrl *gomock.Controller) *MockEntryChecksumMismatchChecker { - mock := &MockEntryChecksumMismatchChecker{ctrl: ctrl} - mock.recorder = &MockEntryChecksumMismatchCheckerMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *MockEntryChecksumMismatchChecker) EXPECT() *MockEntryChecksumMismatchCheckerMockRecorder { - return m.recorder -} - -// ComputeMismatchesForEntry mocks base method -func (m *MockEntryChecksumMismatchChecker) ComputeMismatchesForEntry(arg0 xio.IndexChecksum) ([]ReadMismatch, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ComputeMismatchesForEntry", arg0) - ret0, _ := ret[0].([]ReadMismatch) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// ComputeMismatchesForEntry indicates an expected call of ComputeMismatchesForEntry -func (mr *MockEntryChecksumMismatchCheckerMockRecorder) ComputeMismatchesForEntry(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ComputeMismatchesForEntry", reflect.TypeOf((*MockEntryChecksumMismatchChecker)(nil).ComputeMismatchesForEntry), arg0) -} - -// Drain mocks base method -func (m *MockEntryChecksumMismatchChecker) Drain() []ReadMismatch { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Drain") - ret0, _ := ret[0].([]ReadMismatch) - return ret0 -} - -// Drain indicates an expected call of Drain -func (mr *MockEntryChecksumMismatchCheckerMockRecorder) Drain() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Drain", reflect.TypeOf((*MockEntryChecksumMismatchChecker)(nil).Drain)) -} - -// Lock mocks base method -func (m *MockEntryChecksumMismatchChecker) Lock() { - m.ctrl.T.Helper() - m.ctrl.Call(m, "Lock") -} - -// Lock indicates an expected call of Lock -func (mr *MockEntryChecksumMismatchCheckerMockRecorder) Lock() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Lock", reflect.TypeOf((*MockEntryChecksumMismatchChecker)(nil).Lock)) -} - -// Unlock mocks base method -func (m *MockEntryChecksumMismatchChecker) Unlock() { - m.ctrl.T.Helper() - m.ctrl.Call(m, "Unlock") -} - -// Unlock indicates an expected call of Unlock -func (mr *MockEntryChecksumMismatchCheckerMockRecorder) Unlock() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Unlock", reflect.TypeOf((*MockEntryChecksumMismatchChecker)(nil).Unlock)) -} - -// MockStreamedMismatch is a mock of StreamedMismatch interface -type MockStreamedMismatch struct { - ctrl *gomock.Controller - recorder *MockStreamedMismatchMockRecorder -} - -// MockStreamedMismatchMockRecorder is the mock recorder for MockStreamedMismatch -type MockStreamedMismatchMockRecorder struct { - mock *MockStreamedMismatch -} - -// NewMockStreamedMismatch creates a new mock instance -func NewMockStreamedMismatch(ctrl *gomock.Controller) *MockStreamedMismatch { - mock := &MockStreamedMismatch{ctrl: ctrl} - mock.recorder = &MockStreamedMismatchMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *MockStreamedMismatch) EXPECT() *MockStreamedMismatchMockRecorder { - return m.recorder -} - -// RetrieveMismatch mocks base method -func (m *MockStreamedMismatch) RetrieveMismatch() (ReadMismatch, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "RetrieveMismatch") - ret0, _ := ret[0].(ReadMismatch) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// RetrieveMismatch indicates an expected call of RetrieveMismatch -func (mr *MockStreamedMismatchMockRecorder) RetrieveMismatch() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RetrieveMismatch", reflect.TypeOf((*MockStreamedMismatch)(nil).RetrieveMismatch)) -} diff --git a/src/dbnode/persist/schema/types.go b/src/dbnode/persist/schema/types.go index 1c2a3f7d3c..52ab6b233d 100644 --- a/src/dbnode/persist/schema/types.go +++ b/src/dbnode/persist/schema/types.go @@ -75,10 +75,10 @@ type IndexEntry struct { IndexChecksum int64 } -// IndexChecksum extends IndexEntry for use with queries, by providing +// WideEntry extends IndexEntry for use with queries, by providing // an additional metadata checksum field. -type IndexChecksum struct { - // IndexChecksum embeds IndexEntry. +type WideEntry struct { + // WideEntry embeds IndexEntry. IndexEntry // MetadataChecksum is the computed index metadata checksum. // NB: built from ID, DataChecksum, and tags. diff --git a/src/dbnode/storage/block/block_mock.go b/src/dbnode/storage/block/block_mock.go index 544b9c4998..a3462a1224 100644 --- a/src/dbnode/storage/block/block_mock.go +++ b/src/dbnode/storage/block/block_mock.go @@ -30,7 +30,6 @@ import ( "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/dbnode/namespace" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/sharding" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/dbnode/x/xio" @@ -833,42 +832,54 @@ func (mr *MockOnReadBlockMockRecorder) OnReadBlock(b interface{}) *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OnReadBlock", reflect.TypeOf((*MockOnReadBlock)(nil).OnReadBlock), b) } -// MockStreamedChecksum is a mock of StreamedChecksum interface -type MockStreamedChecksum struct { +// MockStreamedWideEntry is a mock of StreamedWideEntry interface +type MockStreamedWideEntry struct { ctrl *gomock.Controller - recorder *MockStreamedChecksumMockRecorder + recorder *MockStreamedWideEntryMockRecorder } -// MockStreamedChecksumMockRecorder is the mock recorder for MockStreamedChecksum -type MockStreamedChecksumMockRecorder struct { - mock *MockStreamedChecksum +// MockStreamedWideEntryMockRecorder is the mock recorder for MockStreamedWideEntry +type MockStreamedWideEntryMockRecorder struct { + mock *MockStreamedWideEntry } -// NewMockStreamedChecksum creates a new mock instance -func NewMockStreamedChecksum(ctrl *gomock.Controller) *MockStreamedChecksum { - mock := &MockStreamedChecksum{ctrl: ctrl} - mock.recorder = &MockStreamedChecksumMockRecorder{mock} +// NewMockStreamedWideEntry creates a new mock instance +func NewMockStreamedWideEntry(ctrl *gomock.Controller) *MockStreamedWideEntry { + mock := &MockStreamedWideEntry{ctrl: ctrl} + mock.recorder = &MockStreamedWideEntryMockRecorder{mock} return mock } // EXPECT returns an object that allows the caller to indicate expected use -func (m *MockStreamedChecksum) EXPECT() *MockStreamedChecksumMockRecorder { +func (m *MockStreamedWideEntry) EXPECT() *MockStreamedWideEntryMockRecorder { return m.recorder } -// RetrieveIndexChecksum mocks base method -func (m *MockStreamedChecksum) RetrieveIndexChecksum() (xio.IndexChecksum, error) { +// Finalize mocks base method +func (m *MockStreamedWideEntry) Finalize() { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "RetrieveIndexChecksum") - ret0, _ := ret[0].(xio.IndexChecksum) + m.ctrl.Call(m, "Finalize") +} + +// Finalize indicates an expected call of Finalize +func (mr *MockStreamedWideEntryMockRecorder) Finalize() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Finalize", reflect.TypeOf((*MockStreamedWideEntry)(nil).Finalize)) +} + +// RetrieveWideEntry mocks base method +func (m *MockStreamedWideEntry) RetrieveWideEntry() (xio.WideEntry, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RetrieveWideEntry") + ret0, _ := ret[0].(xio.WideEntry) ret1, _ := ret[1].(error) return ret0, ret1 } -// RetrieveIndexChecksum indicates an expected call of RetrieveIndexChecksum -func (mr *MockStreamedChecksumMockRecorder) RetrieveIndexChecksum() *gomock.Call { +// RetrieveWideEntry indicates an expected call of RetrieveWideEntry +func (mr *MockStreamedWideEntryMockRecorder) RetrieveWideEntry() *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RetrieveIndexChecksum", reflect.TypeOf((*MockStreamedChecksum)(nil).RetrieveIndexChecksum)) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RetrieveWideEntry", reflect.TypeOf((*MockStreamedWideEntry)(nil).RetrieveWideEntry)) } // MockDatabaseBlockRetriever is a mock of DatabaseBlockRetriever interface @@ -923,34 +934,19 @@ func (mr *MockDatabaseBlockRetrieverMockRecorder) Stream(ctx, shard, id, blockSt return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Stream", reflect.TypeOf((*MockDatabaseBlockRetriever)(nil).Stream), ctx, shard, id, blockStart, onRetrieve, nsCtx) } -// StreamIndexChecksum mocks base method -func (m *MockDatabaseBlockRetriever) StreamIndexChecksum(ctx context.Context, shard uint32, id ident.ID, blockStart time.Time, nsCtx namespace.Context) (StreamedChecksum, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "StreamIndexChecksum", ctx, shard, id, blockStart, nsCtx) - ret0, _ := ret[0].(StreamedChecksum) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// StreamIndexChecksum indicates an expected call of StreamIndexChecksum -func (mr *MockDatabaseBlockRetrieverMockRecorder) StreamIndexChecksum(ctx, shard, id, blockStart, nsCtx interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StreamIndexChecksum", reflect.TypeOf((*MockDatabaseBlockRetriever)(nil).StreamIndexChecksum), ctx, shard, id, blockStart, nsCtx) -} - -// StreamReadMismatches mocks base method -func (m *MockDatabaseBlockRetriever) StreamReadMismatches(ctx context.Context, shard uint32, mismatchChecker wide.EntryChecksumMismatchChecker, id ident.ID, blockStart time.Time, nsCtx namespace.Context) (wide.StreamedMismatch, error) { +// StreamWideEntry mocks base method +func (m *MockDatabaseBlockRetriever) StreamWideEntry(ctx context.Context, shard uint32, id ident.ID, blockStart time.Time, nsCtx namespace.Context) (StreamedWideEntry, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "StreamReadMismatches", ctx, shard, mismatchChecker, id, blockStart, nsCtx) - ret0, _ := ret[0].(wide.StreamedMismatch) + ret := m.ctrl.Call(m, "StreamWideEntry", ctx, shard, id, blockStart, nsCtx) + ret0, _ := ret[0].(StreamedWideEntry) ret1, _ := ret[1].(error) return ret0, ret1 } -// StreamReadMismatches indicates an expected call of StreamReadMismatches -func (mr *MockDatabaseBlockRetrieverMockRecorder) StreamReadMismatches(ctx, shard, mismatchChecker, id, blockStart, nsCtx interface{}) *gomock.Call { +// StreamWideEntry indicates an expected call of StreamWideEntry +func (mr *MockDatabaseBlockRetrieverMockRecorder) StreamWideEntry(ctx, shard, id, blockStart, nsCtx interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StreamReadMismatches", reflect.TypeOf((*MockDatabaseBlockRetriever)(nil).StreamReadMismatches), ctx, shard, mismatchChecker, id, blockStart, nsCtx) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StreamWideEntry", reflect.TypeOf((*MockDatabaseBlockRetriever)(nil).StreamWideEntry), ctx, shard, id, blockStart, nsCtx) } // AssignShardSet mocks base method @@ -1003,34 +999,19 @@ func (mr *MockDatabaseShardBlockRetrieverMockRecorder) Stream(ctx, id, blockStar return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Stream", reflect.TypeOf((*MockDatabaseShardBlockRetriever)(nil).Stream), ctx, id, blockStart, onRetrieve, nsCtx) } -// StreamIndexChecksum mocks base method -func (m *MockDatabaseShardBlockRetriever) StreamIndexChecksum(ctx context.Context, id ident.ID, blockStart time.Time, nsCtx namespace.Context) (StreamedChecksum, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "StreamIndexChecksum", ctx, id, blockStart, nsCtx) - ret0, _ := ret[0].(StreamedChecksum) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// StreamIndexChecksum indicates an expected call of StreamIndexChecksum -func (mr *MockDatabaseShardBlockRetrieverMockRecorder) StreamIndexChecksum(ctx, id, blockStart, nsCtx interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StreamIndexChecksum", reflect.TypeOf((*MockDatabaseShardBlockRetriever)(nil).StreamIndexChecksum), ctx, id, blockStart, nsCtx) -} - -// StreamReadMismatches mocks base method -func (m *MockDatabaseShardBlockRetriever) StreamReadMismatches(ctx context.Context, mismatchChecker wide.EntryChecksumMismatchChecker, id ident.ID, blockStart time.Time, nsCtx namespace.Context) (wide.StreamedMismatch, error) { +// StreamWideEntry mocks base method +func (m *MockDatabaseShardBlockRetriever) StreamWideEntry(ctx context.Context, id ident.ID, blockStart time.Time, nsCtx namespace.Context) (StreamedWideEntry, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "StreamReadMismatches", ctx, mismatchChecker, id, blockStart, nsCtx) - ret0, _ := ret[0].(wide.StreamedMismatch) + ret := m.ctrl.Call(m, "StreamWideEntry", ctx, id, blockStart, nsCtx) + ret0, _ := ret[0].(StreamedWideEntry) ret1, _ := ret[1].(error) return ret0, ret1 } -// StreamReadMismatches indicates an expected call of StreamReadMismatches -func (mr *MockDatabaseShardBlockRetrieverMockRecorder) StreamReadMismatches(ctx, mismatchChecker, id, blockStart, nsCtx interface{}) *gomock.Call { +// StreamWideEntry indicates an expected call of StreamWideEntry +func (mr *MockDatabaseShardBlockRetrieverMockRecorder) StreamWideEntry(ctx, id, blockStart, nsCtx interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StreamReadMismatches", reflect.TypeOf((*MockDatabaseShardBlockRetriever)(nil).StreamReadMismatches), ctx, mismatchChecker, id, blockStart, nsCtx) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StreamWideEntry", reflect.TypeOf((*MockDatabaseShardBlockRetriever)(nil).StreamWideEntry), ctx, id, blockStart, nsCtx) } // MockDatabaseBlockRetrieverManager is a mock of DatabaseBlockRetrieverManager interface diff --git a/src/dbnode/storage/block/retriever_manager.go b/src/dbnode/storage/block/retriever_manager.go index aae5943fc5..979735bf78 100644 --- a/src/dbnode/storage/block/retriever_manager.go +++ b/src/dbnode/storage/block/retriever_manager.go @@ -25,7 +25,6 @@ import ( "time" "github.com/m3db/m3/src/dbnode/namespace" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/sharding" "github.com/m3db/m3/src/dbnode/x/xio" "github.com/m3db/m3/src/x/context" @@ -113,27 +112,16 @@ func (r *shardBlockRetriever) Stream( blockStart, onRetrieve, nsCtx) } -func (r *shardBlockRetriever) StreamIndexChecksum( +func (r *shardBlockRetriever) StreamWideEntry( ctx context.Context, id ident.ID, blockStart time.Time, nsCtx namespace.Context, -) (StreamedChecksum, error) { - return r.DatabaseBlockRetriever.StreamIndexChecksum(ctx, r.shard, id, +) (StreamedWideEntry, error) { + return r.DatabaseBlockRetriever.StreamWideEntry(ctx, r.shard, id, blockStart, nsCtx) } -func (r *shardBlockRetriever) StreamReadMismatches( - ctx context.Context, - mismatchChecker wide.EntryChecksumMismatchChecker, - id ident.ID, - blockStart time.Time, - nsCtx namespace.Context, -) (wide.StreamedMismatch, error) { - return r.DatabaseBlockRetriever.StreamReadMismatches(ctx, r.shard, - mismatchChecker, id, blockStart, nsCtx) -} - type shardBlockRetrieverManager struct { sync.RWMutex retriever DatabaseBlockRetriever diff --git a/src/dbnode/storage/block/types.go b/src/dbnode/storage/block/types.go index 6bf349633f..34a2161ab8 100644 --- a/src/dbnode/storage/block/types.go +++ b/src/dbnode/storage/block/types.go @@ -25,7 +25,6 @@ import ( "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/dbnode/namespace" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/sharding" "github.com/m3db/m3/src/dbnode/topology" "github.com/m3db/m3/src/dbnode/ts" @@ -34,6 +33,7 @@ import ( "github.com/m3db/m3/src/x/context" "github.com/m3db/m3/src/x/ident" "github.com/m3db/m3/src/x/pool" + "github.com/m3db/m3/src/x/resource" xsync "github.com/m3db/m3/src/x/sync" xtime "github.com/m3db/m3/src/x/time" ) @@ -268,21 +268,26 @@ type RetrievableBlockMetadata struct { Checksum uint32 } -// StreamedChecksum yields a xio.IndexChecksum value asynchronously, +// StreamedWideEntry yields a xio.WideEntry value asynchronously, // and any errors encountered during execution. -type StreamedChecksum interface { - // RetrieveIndexChecksum retrieves the index checksum. - RetrieveIndexChecksum() (xio.IndexChecksum, error) +type StreamedWideEntry interface { + resource.Finalizer + + // RetrieveWideEntry retrieves the collected wide entry. + RetrieveWideEntry() (xio.WideEntry, error) } -type emptyStreamedChecksum struct{} +type emptyWideEntry struct{} + +func (emptyWideEntry) RetrieveWideEntry() (xio.WideEntry, error) { + return xio.WideEntry{}, nil +} -func (emptyStreamedChecksum) RetrieveIndexChecksum() (xio.IndexChecksum, error) { - return xio.IndexChecksum{}, nil +func (emptyWideEntry) Finalize() { } -// EmptyStreamedChecksum is an empty streamed checksum. -var EmptyStreamedChecksum StreamedChecksum = emptyStreamedChecksum{} +// EmptyStreamedWideEntry is an empty streamed wide entry. +var EmptyStreamedWideEntry StreamedWideEntry = emptyWideEntry{} // DatabaseBlockRetriever is a block retriever. type DatabaseBlockRetriever interface { @@ -300,27 +305,17 @@ type DatabaseBlockRetriever interface { nsCtx namespace.Context, ) (xio.BlockReader, error) - // StreamIndexChecksum will stream the index checksum for a given id within - // a block, yielding an index checksum if it is available in the shard. - StreamIndexChecksum( - ctx context.Context, - shard uint32, - id ident.ID, - blockStart time.Time, - nsCtx namespace.Context, - ) (StreamedChecksum, error) - - // StreamReadMismatches will stream reader mismatches for a given id within - // a block, yielding any streamed checksums within the shard. - StreamReadMismatches( + // StreamWideEntry will stream the wide entry for a given ID within + // a block, yielding a wide entry if it is available in the shard. + StreamWideEntry( ctx context.Context, shard uint32, - mismatchChecker wide.EntryChecksumMismatchChecker, id ident.ID, blockStart time.Time, nsCtx namespace.Context, - ) (wide.StreamedMismatch, error) + ) (StreamedWideEntry, error) + // AssignShardSet assigns the given shard set to this retriever. AssignShardSet(shardSet sharding.ShardSet) } @@ -335,24 +330,14 @@ type DatabaseShardBlockRetriever interface { nsCtx namespace.Context, ) (xio.BlockReader, error) - // StreamIndexChecksum will stream the index checksum for a given id within - // a block, yielding an index checksum if available. - StreamIndexChecksum( - ctx context.Context, - id ident.ID, - blockStart time.Time, - nsCtx namespace.Context, - ) (StreamedChecksum, error) - - // StreamReadMismatches will stream read index mismatches for a given id - // within a block, yielding any read mismatches. - StreamReadMismatches( + // StreamWideEntry will stream the wide entry for a given ID within + // a block, yielding a wide entry if available. + StreamWideEntry( ctx context.Context, - mismatchChecker wide.EntryChecksumMismatchChecker, id ident.ID, blockStart time.Time, nsCtx namespace.Context, - ) (wide.StreamedMismatch, error) + ) (StreamedWideEntry, error) } // DatabaseBlockRetrieverManager creates and holds block retrievers diff --git a/src/dbnode/storage/database.go b/src/dbnode/storage/database.go index 77e7fa4236..5f6ae35ea2 100644 --- a/src/dbnode/storage/database.go +++ b/src/dbnode/storage/database.go @@ -30,7 +30,6 @@ import ( "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/sharding" "github.com/m3db/m3/src/dbnode/storage/block" dberrors "github.com/m3db/m3/src/dbnode/storage/errors" @@ -994,7 +993,7 @@ func (d *db) WideQuery( queryStart time.Time, shards []uint32, iterOpts index.IterationOptions, -) ([]xio.IndexChecksum, error) { // FIXME: change when exact type known. +) ([]xio.WideEntry, error) { // nolint FIXME: change when exact type known. n, err := d.namespaceFor(namespace) if err != nil { d.metrics.unknownNamespaceRead.Inc(1) @@ -1005,7 +1004,7 @@ func (d *db) WideQuery( batchSize = d.opts.WideBatchSize() blockSize = n.Options().IndexOptions().BlockSize() - collectedChecksums = make([]xio.IndexChecksum, 0, 10) + collectedChecksums = make([]xio.WideEntry, 0, 10) ) opts, err := index.NewWideQueryOptions(queryStart, batchSize, blockSize, shards, iterOpts) @@ -1027,30 +1026,24 @@ func (d *db) WideQuery( defer sp.Finish() - streamedChecksums := make([]block.StreamedChecksum, 0, batchSize) + streamedWideEntries := make([]block.StreamedWideEntry, 0, batchSize) indexChecksumProcessor := func(batch *ident.IDBatch) error { - streamedChecksums = streamedChecksums[:0] + streamedWideEntries = streamedWideEntries[:0] for _, id := range batch.IDs { - streamedChecksum, err := d.fetchIndexChecksum(ctx, n, id, start) + streamedWideEntry, err := d.fetchWideEntries(ctx, n, id, start) if err != nil { return err } - streamedChecksums = append(streamedChecksums, streamedChecksum) + streamedWideEntries = append(streamedWideEntries, streamedWideEntry) } - for i, streamedChecksum := range streamedChecksums { - checksum, err := streamedChecksum.RetrieveIndexChecksum() + for _, streamedWideEntry := range streamedWideEntries { + checksum, err := streamedWideEntry.RetrieveWideEntry() if err != nil { return err } - // TODO: use index checksum value to call downstreams. - useID := i == len(batch.IDs)-1 - if !useID { - checksum.ID.Finalize() - } - collectedChecksums = append(collectedChecksums, checksum) } @@ -1065,13 +1058,13 @@ func (d *db) WideQuery( return collectedChecksums, nil } -func (d *db) fetchIndexChecksum( +func (d *db) fetchWideEntries( ctx context.Context, ns databaseNamespace, id ident.ID, start time.Time, -) (block.StreamedChecksum, error) { - ctx, sp, sampled := ctx.StartSampledTraceSpan(tracepoint.DBIndexChecksum) +) (block.StreamedWideEntry, error) { + ctx, sp, sampled := ctx.StartSampledTraceSpan(tracepoint.DBWideEntry) if sampled { sp.LogFields( opentracinglog.String("namespace", ns.ID().String()), @@ -1081,100 +1074,8 @@ func (d *db) fetchIndexChecksum( } defer sp.Finish() - return ns.FetchIndexChecksum(ctx, id, start) -} - -func (d *db) ReadMismatches( - ctx context.Context, - namespace ident.ID, - query index.Query, - mismatchChecker wide.EntryChecksumMismatchChecker, - queryStart time.Time, - shards []uint32, - iterOpts index.IterationOptions, -) ([]wide.ReadMismatch, error) { // TODO: update this type when reader hooked up - n, err := d.namespaceFor(namespace) - if err != nil { - d.metrics.unknownNamespaceRead.Inc(1) - return nil, err - } - - var ( - batchSize = d.opts.WideBatchSize() - blockSize = n.Options().IndexOptions().BlockSize() - collectedMismatches = make([]wide.ReadMismatch, 0, 10) - ) - - opts, err := index.NewWideQueryOptions(queryStart, batchSize, blockSize, shards, iterOpts) - if err != nil { - return nil, err - } - - start, end := opts.StartInclusive, opts.EndExclusive - ctx, sp, sampled := ctx.StartSampledTraceSpan(tracepoint.DBReadMismatches) - if sampled { - sp.LogFields( - opentracinglog.String("readMismatches", query.String()), - opentracinglog.String("namespace", namespace.String()), - opentracinglog.Int("batchSize", batchSize), - xopentracing.Time("start", start), - xopentracing.Time("end", end), - ) - } - - defer sp.Finish() - - streamedMismatches := make([]wide.StreamedMismatch, 0, batchSize) - streamMismatchProcessor := func(batch *ident.IDBatch) error { - streamedMismatches = streamedMismatches[:0] - for _, id := range batch.IDs { - streamedMismatch, err := d.fetchReadMismatch(ctx, n, mismatchChecker, id, start) - if err != nil { - return err - } - - streamedMismatches = append(streamedMismatches, streamedMismatch) - } - - for _, streamedMismatch := range streamedMismatches { - mismatch, err := streamedMismatch.RetrieveMismatch() - if err != nil { - return err - } - - collectedMismatches = append(collectedMismatches, mismatch) - } - - return nil - } - - err = d.batchProcessWideQuery(ctx, n, query, streamMismatchProcessor, opts) - if err != nil { - return nil, err - } - - return collectedMismatches, nil -} - -func (d *db) fetchReadMismatch( - ctx context.Context, - ns databaseNamespace, - mismatchChecker wide.EntryChecksumMismatchChecker, - id ident.ID, - start time.Time, -) (wide.StreamedMismatch, error) { - ctx, sp, sampled := ctx.StartSampledTraceSpan(tracepoint.DBFetchMismatch) - if sampled { - sp.LogFields( - opentracinglog.String("namespace", ns.ID().String()), - opentracinglog.String("id", id.String()), - xopentracing.Time("start", start), - ) - } - - defer sp.Finish() - return ns.FetchReadMismatch(ctx, mismatchChecker, id, start) + return ns.FetchWideEntry(ctx, id, start) } func (d *db) FetchBlocks( diff --git a/src/dbnode/storage/database_test.go b/src/dbnode/storage/database_test.go index b4406a7e72..88d3af5540 100644 --- a/src/dbnode/storage/database_test.go +++ b/src/dbnode/storage/database_test.go @@ -33,7 +33,6 @@ import ( "github.com/m3db/m3/src/dbnode/client" "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/sharding" "github.com/m3db/m3/src/dbnode/storage/block" @@ -311,7 +310,7 @@ func TestDatabaseWideQueryNamespaceNonExistent(t *testing.T) { require.True(t, dberrors.IsUnknownNamespaceError(err)) } -func TestDatabaseIndexChecksum(t *testing.T) { +func TestDatabaseWideEntry(t *testing.T) { ctrl := xtest.NewController(t) defer ctrl.Finish() @@ -328,33 +327,34 @@ func TestDatabaseIndexChecksum(t *testing.T) { end := time.Now() start := end.Add(-time.Hour) - indexChecksumWithID := block.NewMockStreamedChecksum(ctrl) - indexChecksumWithID.EXPECT().RetrieveIndexChecksum(). + indexChecksumWithID := block.NewMockStreamedWideEntry(ctrl) + indexChecksumWithID.EXPECT().RetrieveWideEntry(). Return( - xio.IndexChecksum{ + xio.WideEntry{ ID: ident.StringID("foo"), MetadataChecksum: 5, }, nil) mockNamespace := NewMockdatabaseNamespace(ctrl) - mockNamespace.EXPECT().FetchIndexChecksum(ctx, seriesID, start). + mockNamespace.EXPECT().FetchWideEntry(ctx, seriesID, start). Return(indexChecksumWithID, nil) - indexChecksumWithoutID := block.NewMockStreamedChecksum(ctrl) - indexChecksumWithoutID.EXPECT().RetrieveIndexChecksum(). - Return(xio.IndexChecksum{MetadataChecksum: 7}, nil) - mockNamespace.EXPECT().FetchIndexChecksum(ctx, seriesID, start). + indexChecksumWithoutID := block.NewMockStreamedWideEntry(ctrl) + indexChecksumWithoutID.EXPECT().RetrieveWideEntry(). + Return(xio.WideEntry{MetadataChecksum: 7}, nil) + mockNamespace.EXPECT().FetchWideEntry(ctx, seriesID, start). Return(indexChecksumWithoutID, nil) d.namespaces.Set(nsID, mockNamespace) - res, err := d.fetchIndexChecksum(ctx, mockNamespace, seriesID, start) + res, err := d.fetchWideEntries(ctx, mockNamespace, seriesID, start) require.NoError(t, err) - checksum, err := res.RetrieveIndexChecksum() + checksum, err := res.RetrieveWideEntry() require.NoError(t, err) assert.Equal(t, "foo", checksum.ID.String()) assert.Equal(t, 5, int(checksum.MetadataChecksum)) - res, err = d.fetchIndexChecksum(ctx, mockNamespace, seriesID, start) - checksum, err = res.RetrieveIndexChecksum() + res, err = d.fetchWideEntries(ctx, mockNamespace, seriesID, start) + require.NoError(t, err) + checksum, err = res.RetrieveWideEntry() require.NoError(t, err) require.NoError(t, err) assert.Nil(t, checksum.ID) @@ -953,9 +953,9 @@ func TestWideQuery(t *testing.T) { ctx context.Context, t *testing.T, ctrl *gomock.Controller, ns *MockdatabaseNamespace, d *db, q index.Query, now time.Time, shards []uint32, iterOpts index.IterationOptions) { - ns.EXPECT().FetchIndexChecksum(gomock.Any(), + ns.EXPECT().FetchWideEntry(gomock.Any(), ident.StringID("foo"), gomock.Any()). - Return(block.EmptyStreamedChecksum, nil) + Return(block.EmptyStreamedWideEntry, nil) _, err := d.WideQuery(ctx, ident.StringID("testns"), q, now, shards, iterOpts) require.NoError(t, err) @@ -965,7 +965,7 @@ func TestWideQuery(t *testing.T) { } exSpans := []string{ - tracepoint.DBIndexChecksum, + tracepoint.DBWideEntry, tracepoint.DBWideQuery, tracepoint.DBWideQuery, "root", @@ -974,35 +974,8 @@ func TestWideQuery(t *testing.T) { testWideFunction(t, readMismatchTest, exSpans) } -func TestReadMismatches(t *testing.T) { - readMismatchTest := func( - ctx context.Context, t *testing.T, ctrl *gomock.Controller, - ns *MockdatabaseNamespace, d *db, q index.Query, - now time.Time, shards []uint32, iterOpts index.IterationOptions) { - checker := wide.NewMockEntryChecksumMismatchChecker(ctrl) - ns.EXPECT().FetchReadMismatch(gomock.Any(), checker, - ident.StringID("foo"), gomock.Any()). - Return(wide.EmptyStreamedMismatch, nil) - - _, err := d.ReadMismatches(ctx, ident.StringID("testns"), q, checker, now, shards, iterOpts) - require.NoError(t, err) - - _, err = d.ReadMismatches(ctx, ident.StringID("testns"), q, checker, now, nil, iterOpts) - require.Error(t, err) - } - - exSpans := []string{ - tracepoint.DBFetchMismatch, - tracepoint.DBReadMismatches, - tracepoint.DBReadMismatches, - "root", - } - - testWideFunction(t, readMismatchTest, exSpans) -} - func testWideFunction(t *testing.T, testFn wideQueryTestFn, exSpans []string) { - ctrl := xtest.NewController(t) + ctrl := gomock.NewController(t) defer ctrl.Finish() d, mapCh, _ := defaultTestDatabase(t, ctrl, BootstrapNotStarted) @@ -1029,9 +1002,10 @@ func testWideFunction(t *testing.T, testFn wideQueryTestFn, exSpans []string) { } now = time.Now() + start = now.Truncate(2 * time.Hour) iterOpts = index.IterationOptions{} wideOpts = index.WideQueryOptions{ - StartInclusive: now.Truncate(2 * time.Hour), + StartInclusive: start, EndExclusive: now.Truncate(2 * time.Hour).Add(2 * time.Hour), IterationOptions: iterOpts, BatchSize: 1024, @@ -1064,7 +1038,7 @@ func testWideFunction(t *testing.T, testFn wideQueryTestFn, exSpans []string) { ns.EXPECT().WideQueryIDs(gomock.Any(), q, gomock.Any(), gomock.Any()). Return(fmt.Errorf("random err")) - testFn(ctx, t, ctrl, ns, d, q, now, shards, iterOpts) + testFn(ctx, t, ctrl, ns, d, q, start, shards, iterOpts) ns.EXPECT().Close().Return(nil) // Ensure commitlog is set before closing because this will call commitlog.Close() d.commitLog = commitLog diff --git a/src/dbnode/storage/index/query_options.go b/src/dbnode/storage/index/query_options.go index 1355e70232..ff7ffb5b4a 100644 --- a/src/dbnode/storage/index/query_options.go +++ b/src/dbnode/storage/index/query_options.go @@ -54,7 +54,7 @@ var ( // NewWideQueryOptions creates a new wide query options, snapped to block start. func NewWideQueryOptions( - queryStart time.Time, + blockStart time.Time, batchSize int, blockSize time.Duration, shards []uint32, @@ -68,8 +68,11 @@ func NewWideQueryOptions( return WideQueryOptions{}, fmt.Errorf(errInvalidBlockSize, blockSize) } - start := queryStart.Truncate(blockSize) - end := start.Add(blockSize) + if !blockStart.Equal(blockStart.Truncate(blockSize)) { + return WideQueryOptions{}, + fmt.Errorf("block start not divisible by block size: start=%v, size=%s", + blockStart.String(), blockSize.String()) + } // NB: shards queried must be sorted. sort.Slice(shards, func(i, j int) bool { @@ -77,8 +80,8 @@ func NewWideQueryOptions( }) return WideQueryOptions{ - StartInclusive: start, - EndExclusive: end, + StartInclusive: blockStart, + EndExclusive: blockStart.Add(blockSize), BatchSize: batchSize, IterationOptions: iterOpts, ShardsQueried: shards, diff --git a/src/dbnode/storage/index/query_options_test.go b/src/dbnode/storage/index/query_options_test.go index 47e2585da3..01add7e973 100644 --- a/src/dbnode/storage/index/query_options_test.go +++ b/src/dbnode/storage/index/query_options_test.go @@ -52,7 +52,7 @@ func TestQueryOptions(t *testing.T) { func TestInvalidWideQueryOptions(t *testing.T) { var ( - now = time.Now() + now = time.Now().Truncate(time.Hour).Add(1) iterOpts = IterationOptions{} batchSize int @@ -68,14 +68,18 @@ func TestInvalidWideQueryOptions(t *testing.T) { blockSize = time.Minute _, err = NewWideQueryOptions(now, batchSize, blockSize, nil, iterOpts) + require.Error(t, err) + + now = now.Truncate(blockSize) + _, err = NewWideQueryOptions(now, batchSize, blockSize, nil, iterOpts) require.NoError(t, err) } func TestWideQueryOptions(t *testing.T) { var ( - now = time.Now() batchSize = 100 blockSize = time.Hour * 2 + now = time.Now().Truncate(blockSize) iterOpts = IterationOptions{} shards = []uint32{100, 23, 1} ) diff --git a/src/dbnode/storage/index/types.go b/src/dbnode/storage/index/types.go index 78b2cbc1ea..ebceb8f8e4 100644 --- a/src/dbnode/storage/index/types.go +++ b/src/dbnode/storage/index/types.go @@ -101,7 +101,7 @@ type WideQueryOptions struct { StartInclusive time.Time // EndExclusive is the exclusive end for the query. EndExclusive time.Time - // BatchSize controls IndexChecksumQuery batch size. + // BatchSize controls wide query batch size. BatchSize int // ShardsQueried are the shards to query. These must be in ascending order. // If empty, all shards are queried. diff --git a/src/dbnode/storage/index/wide_query_results_test.go b/src/dbnode/storage/index/wide_query_results_test.go index b0ce72dd04..b0279e8bd5 100644 --- a/src/dbnode/storage/index/wide_query_results_test.go +++ b/src/dbnode/storage/index/wide_query_results_test.go @@ -120,8 +120,13 @@ func drainAndCheckBatches( } func TestWideSeriesResults(t *testing.T) { + var ( + max = 31 + blockSize = time.Hour * 2 + now = time.Now().Truncate(blockSize) + ) + // Test many different permutations of element count and batch sizes. - max := 31 for documentCount := 0; documentCount < max; documentCount++ { for docBatchSize := 1; docBatchSize < max; docBatchSize++ { for batchSize := 1; batchSize < max; batchSize++ { @@ -136,7 +141,7 @@ func TestWideSeriesResults(t *testing.T) { drainAndCheckBatches(t, expected, batchCh, doneCh) wideQueryOptions, err := NewWideQueryOptions( - time.Now(), batchSize, time.Hour*2, nil, IterationOptions{}) + now, batchSize, blockSize, nil, IterationOptions{}) require.NoError(t, err) wideRes := NewWideQueryResults(testNs, testIDPool, nil, batchCh, wideQueryOptions) @@ -180,6 +185,9 @@ func TestWideSeriesResultsWithShardFilter(t *testing.T) { batchCh = make(chan *ident.IDBatch) doneCh = make(chan struct{}) + + blockSize = time.Hour * 2 + now = time.Now().Truncate(blockSize) ) docs := buildDocs(documentCount, docBatchSize) @@ -189,7 +197,7 @@ func TestWideSeriesResultsWithShardFilter(t *testing.T) { drainAndCheckBatches(t, expected, batchCh, doneCh) wideQueryOptions, err := NewWideQueryOptions( - time.Now(), batchSize, time.Hour*2, shards, IterationOptions{}) + now, batchSize, blockSize, shards, IterationOptions{}) require.NoError(t, err) filter := func(id ident.ID) (uint32, bool) { i, err := strconv.Atoi(strings.TrimPrefix(id.String(), "foo")) diff --git a/src/dbnode/storage/index_queue_forward_write_test.go b/src/dbnode/storage/index_queue_forward_write_test.go index 15804cf2b4..8b72db6771 100644 --- a/src/dbnode/storage/index_queue_forward_write_test.go +++ b/src/dbnode/storage/index_queue_forward_write_test.go @@ -207,9 +207,9 @@ func TestNamespaceForwardIndexAggregateQuery(t *testing.T) { } func TestNamespaceForwardIndexWideQuery(t *testing.T) { - ctrl := gomock.NewController(t) + ctrl := xtest.NewController(t) defer ctrl.Finish() - defer leaktest.CheckTimeout(t, 2*time.Second)() + defer leaktest.CheckTimeout(t, 5*time.Second)() ctx := context.NewContext() defer ctx.Close() @@ -223,7 +223,10 @@ func TestNamespaceForwardIndexWideQuery(t *testing.T) { // NB: query both the current and the next index block to ensure that the // write was correctly indexed to both. nextBlockTime := now.Add(blockSize) - queryTimes := []time.Time{now, nextBlockTime} + queryTimes := []time.Time{ + now.Truncate(blockSize), + nextBlockTime.Truncate(blockSize), + } for _, ts := range queryTimes { collector := make(chan *ident.IDBatch) doneCh := make(chan struct{}) diff --git a/src/dbnode/storage/index_queue_test.go b/src/dbnode/storage/index_queue_test.go index 1b1c3e2e4c..9960da5beb 100644 --- a/src/dbnode/storage/index_queue_test.go +++ b/src/dbnode/storage/index_queue_test.go @@ -395,7 +395,7 @@ func TestNamespaceIndexInsertAggregateQuery(t *testing.T) { func TestNamespaceIndexInsertWideQuery(t *testing.T) { ctrl := xtest.NewController(t) defer ctrl.Finish() - defer leaktest.CheckTimeout(t, 2*time.Second)() + defer leaktest.CheckTimeout(t, 5*time.Second)() ctx := context.NewContext() defer ctx.Close() @@ -408,8 +408,9 @@ func TestNamespaceIndexInsertWideQuery(t *testing.T) { assert.NoError(t, err) doneCh := make(chan struct{}) collector := make(chan *ident.IDBatch) - queryOpts, err := index.NewWideQueryOptions(time.Now(), 5, - time.Hour*2, nil, index.IterationOptions{}) + blockSize := 2 * time.Hour + queryOpts, err := index.NewWideQueryOptions(time.Now().Truncate(blockSize), + 5, blockSize, nil, index.IterationOptions{}) require.NoError(t, err) expectedBatchIDs := [][]string{{"foo"}} @@ -441,7 +442,7 @@ func TestNamespaceIndexInsertWideQuery(t *testing.T) { func TestNamespaceIndexInsertWideQueryFilteredByShard(t *testing.T) { ctrl := xtest.NewController(t) defer ctrl.Finish() - defer leaktest.CheckTimeout(t, 2*time.Second)() + defer leaktest.CheckTimeout(t, 5*time.Second)() ctx := context.NewContext() defer ctx.Close() @@ -456,8 +457,9 @@ func TestNamespaceIndexInsertWideQueryFilteredByShard(t *testing.T) { collector := make(chan *ident.IDBatch) shard := testShardSet.Lookup(ident.StringID("foo")) offShard := shard + 1 - queryOpts, err := index.NewWideQueryOptions(time.Now(), 5, time.Hour*2, - []uint32{offShard}, index.IterationOptions{}) + blockSize := 2 * time.Hour + queryOpts, err := index.NewWideQueryOptions(time.Now().Truncate(blockSize), + 5, blockSize, []uint32{offShard}, index.IterationOptions{}) require.NoError(t, err) go func() { diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index 5367b17574..bc5fc27311 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -31,7 +31,6 @@ import ( "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/persist/fs" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/sharding" "github.com/m3db/m3/src/dbnode/storage/block" "github.com/m3db/m3/src/dbnode/storage/bootstrap" @@ -918,36 +917,22 @@ func (n *dbNamespace) ReadEncoded( return res, err } -func (n *dbNamespace) FetchIndexChecksum( +func (n *dbNamespace) FetchWideEntry( ctx context.Context, id ident.ID, blockStart time.Time, -) (block.StreamedChecksum, error) { +) (block.StreamedWideEntry, error) { callStart := n.nowFn() shard, nsCtx, err := n.readableShardFor(id) if err != nil { n.metrics.read.ReportError(n.nowFn().Sub(callStart)) - return block.EmptyStreamedChecksum, err - } - res, err := shard.FetchIndexChecksum(ctx, id, blockStart, nsCtx) - n.metrics.read.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) - return res, err -} -func (n *dbNamespace) FetchReadMismatch( - ctx context.Context, - mismatchChecker wide.EntryChecksumMismatchChecker, - id ident.ID, - blockStart time.Time, -) (wide.StreamedMismatch, error) { - callStart := n.nowFn() - shard, nsCtx, err := n.readableShardFor(id) - if err != nil { - n.metrics.read.ReportError(n.nowFn().Sub(callStart)) - return wide.EmptyStreamedMismatch, err + return block.EmptyStreamedWideEntry, err } - res, err := shard.FetchReadMismatch(ctx, mismatchChecker, id, blockStart, nsCtx) + + res, err := shard.FetchWideEntry(ctx, id, blockStart, nsCtx) n.metrics.read.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) + return res, err } diff --git a/src/dbnode/storage/options.go b/src/dbnode/storage/options.go index 999de9a99e..f5c79e9890 100644 --- a/src/dbnode/storage/options.go +++ b/src/dbnode/storage/options.go @@ -167,6 +167,7 @@ type options struct { schemaReg namespace.SchemaRegistry blockLeaseManager block.LeaseManager onColdFlush OnColdFlush + iterationOptions index.IterationOptions memoryTracker MemoryTracker mmapReporter mmap.Reporter doNotIndexWithFieldsMap map[string]string @@ -788,6 +789,16 @@ func (o *options) OnColdFlush() OnColdFlush { return o.onColdFlush } +func (o *options) SetIterationOptions(value index.IterationOptions) Options { + opts := *o + opts.iterationOptions = value + return &opts +} + +func (o *options) IterationOptions() index.IterationOptions { + return o.iterationOptions +} + func (o *options) SetMemoryTracker(memTracker MemoryTracker) Options { opts := *o opts.memoryTracker = memTracker diff --git a/src/dbnode/storage/series/reader.go b/src/dbnode/storage/series/reader.go index 32bd6e3d17..0fc9ccb44a 100644 --- a/src/dbnode/storage/series/reader.go +++ b/src/dbnode/storage/series/reader.go @@ -26,7 +26,6 @@ import ( "time" "github.com/m3db/m3/src/dbnode/namespace" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/storage/block" "github.com/m3db/m3/src/dbnode/x/xio" @@ -72,7 +71,7 @@ func NewReaderUsingRetriever( } // ReadEncoded reads encoded blocks using just a block retriever. -func (r Reader) ReadEncoded( +func (r *Reader) ReadEncoded( ctx context.Context, start, end time.Time, nsCtx namespace.Context, @@ -80,24 +79,13 @@ func (r Reader) ReadEncoded( return r.readersWithBlocksMapAndBuffer(ctx, start, end, nil, nil, nsCtx) } -func (r Reader) readersWithBlocksMapAndBuffer( +func (r *Reader) readersWithBlocksMapAndBuffer( ctx context.Context, start, end time.Time, seriesBlocks block.DatabaseSeriesBlocks, seriesBuffer databaseBuffer, nsCtx namespace.Context, ) ([][]xio.BlockReader, error) { - // Two-dimensional slice such that the first dimension is unique by blockstart - // and the second dimension is blocks of data for that blockstart (not necessarily - // in chronological order). - // - // ex. (querying 2P.M -> 6P.M with a 2-hour blocksize): - // [][]xio.BlockReader{ - // {block0, block1, block2}, // <- 2P.M - // {block0, block1}, // <-4P.M - // } - var results [][]xio.BlockReader - if end.Before(start) { return nil, xerrors.NewInvalidParamsError(errSeriesReadInvalidRange) } @@ -105,7 +93,6 @@ func (r Reader) readersWithBlocksMapAndBuffer( var ( nowFn = r.opts.ClockOptions().NowFn() now = nowFn() - cachePolicy = r.opts.CachePolicy() ropts = r.opts.RetentionOptions() size = ropts.BlockSize() alignedStart = start.Truncate(size) @@ -127,8 +114,41 @@ func (r Reader) readersWithBlocksMapAndBuffer( alignedEnd = latest } - first, last := alignedStart, alignedEnd - for blockAt := first; !blockAt.After(last); blockAt = blockAt.Add(size) { + return r.readersWithBlocksMapAndBufferAligned(ctx, alignedStart, alignedEnd, + seriesBlocks, seriesBuffer, nsCtx) +} + +// nolint: gocyclo +func (r *Reader) readersWithBlocksMapAndBufferAligned( + ctx context.Context, + start, end time.Time, + seriesBlocks block.DatabaseSeriesBlocks, + seriesBuffer databaseBuffer, + nsCtx namespace.Context, +) ([][]xio.BlockReader, error) { + var ( + nowFn = r.opts.ClockOptions().NowFn() + now = nowFn() + ropts = r.opts.RetentionOptions() + blockSize = ropts.BlockSize() + readerCount = end.Sub(start) / blockSize + ) + + if readerCount < 0 { + readerCount = 0 + } + + // Two-dimensional slice such that the first dimension is unique by blockstart + // and the second dimension is blocks of data for that blockstart (not necessarily + // in chronological order). + // + // ex. (querying 2P.M -> 6P.M with a 2-hour blocksize): + // [][]xio.BlockReader{ + // {block0, block1, block2}, // <- 2P.M + // {block0, block1}, // <-4P.M + // } + results := make([][]xio.BlockReader, 0, readerCount) + for blockAt := start; !blockAt.After(end); blockAt = blockAt.Add(blockSize) { // resultsBlock holds the results from one block. The flow is: // 1) Look in the cache for metrics for a block. // 2) If there is nothing in the cache, try getting metrics from disk. @@ -140,52 +160,30 @@ func (r Reader) readersWithBlocksMapAndBuffer( // in an out of order error in the MultiReaderIterator on query. var resultsBlock []xio.BlockReader - retrievedFromDiskCache := false - if seriesBlocks != nil { - if block, ok := seriesBlocks.BlockAt(blockAt); ok { - // Block served from in-memory or in-memory metadata - // will defer to disk read - streamedBlock, err := block.Stream(ctx) - if err != nil { - return nil, err - } - if streamedBlock.IsNotEmpty() { - resultsBlock = append(resultsBlock, streamedBlock) - // NB(r): Mark this block as read now - block.SetLastReadTime(now) - if r.onRead != nil { - r.onRead.OnReadBlock(block) - } - } - retrievedFromDiskCache = true - } + blockReader, block, found, err := retrieveCached(ctx, blockAt, seriesBlocks) + if err != nil { + return nil, err } - // Avoid going to disk if data was already in the cache. - if !retrievedFromDiskCache { - switch { - case cachePolicy == CacheAll: - // No-op, block metadata should have been in-memory - case r.retriever != nil: - // Try to stream from disk - isRetrievable, err := r.retriever.IsBlockRetrievable(blockAt) - if err != nil { - return nil, err - } - if isRetrievable { - streamedBlock, err := r.retriever.Stream(ctx, r.id, blockAt, r.onRetrieve, nsCtx) - if err != nil { - return nil, err - } - if streamedBlock.IsNotEmpty() { - resultsBlock = append(resultsBlock, streamedBlock) - } - } + if found { + // NB(r): Mark this block as read now + block.SetLastReadTime(now) + if r.onRead != nil { + r.onRead.OnReadBlock(block) + } + } else { + blockReader, found, err = r.streamBlock(ctx, blockAt, r.onRetrieve, nsCtx) + if err != nil { + return nil, err } } + if found { + resultsBlock = append(resultsBlock, blockReader) + } + if seriesBuffer != nil { - bufferResults, err := seriesBuffer.ReadEncoded(ctx, blockAt, blockAt.Add(size), nsCtx) + bufferResults, err := seriesBuffer.ReadEncoded(ctx, blockAt, blockAt.Add(blockSize), nsCtx) if err != nil { return nil, err } @@ -204,12 +202,12 @@ func (r Reader) readersWithBlocksMapAndBuffer( return results, nil } -// FetchIndexChecksum reads index checksum blocks using just a block retriever. -func (r Reader) FetchIndexChecksum( +// FetchWideEntry reads wide entries using just a block retriever. +func (r *Reader) FetchWideEntry( ctx context.Context, blockStart time.Time, nsCtx namespace.Context, -) (block.StreamedChecksum, error) { +) (block.StreamedWideEntry, error) { var ( nowFn = r.opts.ClockOptions().NowFn() now = nowFn() @@ -220,71 +218,31 @@ func (r Reader) FetchIndexChecksum( if blockStart.Before(earliest) { // NB: this block is falling out of retention; return empty result rather // than iterating over it. - return block.EmptyStreamedChecksum, nil + return block.EmptyStreamedWideEntry, nil } if r.retriever == nil { - return block.EmptyStreamedChecksum, nil + return block.EmptyStreamedWideEntry, nil } // Try to stream from disk isRetrievable, err := r.retriever.IsBlockRetrievable(blockStart) if err != nil { - return block.EmptyStreamedChecksum, err + return block.EmptyStreamedWideEntry, err } else if !isRetrievable { - return block.EmptyStreamedChecksum, nil + return block.EmptyStreamedWideEntry, nil } - streamedBlock, err := r.retriever.StreamIndexChecksum(ctx, + streamedEntry, err := r.retriever.StreamWideEntry(ctx, r.id, blockStart, nsCtx) if err != nil { - return block.EmptyStreamedChecksum, err - } - - return streamedBlock, nil -} - -// FetchReadMismatch compiles read mismatches using a block retriever and -// an incoming batchReader. -func (r Reader) FetchReadMismatch( - ctx context.Context, - mismatchChecker wide.EntryChecksumMismatchChecker, - blockStart time.Time, - nsCtx namespace.Context, -) (wide.StreamedMismatch, error) { - var ( - nowFn = r.opts.ClockOptions().NowFn() - now = nowFn() - ropts = r.opts.RetentionOptions() - ) - - earliest := retention.FlushTimeStart(ropts, now) - if blockStart.Before(earliest) { - // NB: this block is falling out of retention; return empty result rather - // than iterating over it. - return wide.EmptyStreamedMismatch, nil - } - - if r.retriever == nil { - return wide.EmptyStreamedMismatch, nil - } - // Try to stream from disk - isRetrievable, err := r.retriever.IsBlockRetrievable(blockStart) - if err != nil { - return wide.EmptyStreamedMismatch, err - } else if !isRetrievable { - return wide.EmptyStreamedMismatch, nil - } - streamedMismatches, err := r.retriever.StreamReadMismatches(ctx, - mismatchChecker, r.id, blockStart, nsCtx) - if err != nil { - return wide.EmptyStreamedMismatch, err + return block.EmptyStreamedWideEntry, err } - return streamedMismatches, nil + return streamedEntry, nil } // FetchBlocks returns data blocks given a list of block start times using // just a block retriever. -func (r Reader) FetchBlocks( +func (r *Reader) FetchBlocks( ctx context.Context, starts []time.Time, nsCtx namespace.Context, @@ -292,102 +250,14 @@ func (r Reader) FetchBlocks( return r.fetchBlocksWithBlocksMapAndBuffer(ctx, starts, nil, nil, nsCtx) } -func (r Reader) fetchBlocksWithBlocksMapAndBuffer( +func (r *Reader) fetchBlocksWithBlocksMapAndBuffer( ctx context.Context, starts []time.Time, seriesBlocks block.DatabaseSeriesBlocks, seriesBuffer databaseBuffer, nsCtx namespace.Context, ) ([]block.FetchBlockResult, error) { - var ( - // Two-dimensional slice (each block.FetchBlockResult has a []xio.BlockReader internally) - // such that the first dimension is unique by blockstart and the second dimension is blocks - // of data for that blockstart (not necessarily in chronological order). - // - // ex. (querying 2P.M -> 6P.M with a 2-hour blocksize): - // []block.FetchBlockResult{ - // block.FetchBlockResult{ - // Start: 2P.M, - // Blocks: []xio.BlockReader{block0, block1, block2}, - // }, - // block.FetchBlockResult{ - // Start: 4P.M, - // Blocks: []xio.BlockReader{block0}, - // }, - // } - res = make([]block.FetchBlockResult, 0, len(starts)) - cachePolicy = r.opts.CachePolicy() - // NB(r): Always use nil for OnRetrieveBlock so we don't cache the - // series after fetching it from disk, the fetch blocks API is called - // during streaming so to cache it in memory would mean we would - // eventually cache all series in memory when we stream results to a - // peer. - onRetrieve block.OnRetrieveBlock - ) - for _, start := range starts { - // Slice of xio.BlockReader such that all data belong to the same blockstart. - var blockReaders []xio.BlockReader - - retrievedFromDiskCache := false - if seriesBlocks != nil { - if b, exists := seriesBlocks.BlockAt(start); exists { - streamedBlock, err := b.Stream(ctx) - if err != nil { - // Short-circuit this entire blockstart if an error was encountered. - r := block.NewFetchBlockResult(start, nil, - fmt.Errorf("unable to retrieve block stream for series %s time %v: %v", - r.id.String(), start, err)) - res = append(res, r) - continue - } - - if streamedBlock.IsNotEmpty() { - blockReaders = append(blockReaders, streamedBlock) - } - retrievedFromDiskCache = true - } - } - - // Avoid going to disk if data was already in the cache. - if !retrievedFromDiskCache { - switch { - case cachePolicy == CacheAll: - // No-op, block metadata should have been in-memory - case r.retriever != nil: - // Try to stream from disk - isRetrievable, err := r.retriever.IsBlockRetrievable(start) - if err != nil { - // Short-circuit this entire blockstart if an error was encountered. - r := block.NewFetchBlockResult(start, nil, - fmt.Errorf("unable to retrieve block stream for series %s time %v: %v", - r.id.String(), start, err)) - res = append(res, r) - continue - } - - if isRetrievable { - streamedBlock, err := r.retriever.Stream(ctx, r.id, start, onRetrieve, nsCtx) - if err != nil { - // Short-circuit this entire blockstart if an error was encountered. - r := block.NewFetchBlockResult(start, nil, - fmt.Errorf("unable to retrieve block stream for series %s time %v: %v", - r.id.String(), start, err)) - res = append(res, r) - continue - } - - if streamedBlock.IsNotEmpty() { - blockReaders = append(blockReaders, streamedBlock) - } - } - } - } - - if len(blockReaders) > 0 { - res = append(res, block.NewFetchBlockResult(start, blockReaders, nil)) - } - } - + res := r.resolveBlockResults(ctx, starts, seriesBlocks, nsCtx) if seriesBuffer != nil && !seriesBuffer.IsEmpty() { bufferResults := seriesBuffer.FetchBlocks(ctx, starts, nsCtx) @@ -395,7 +265,8 @@ func (r Reader) fetchBlocksWithBlocksMapAndBuffer( block.SortFetchBlockResultByTimeAscending(res) block.SortFetchBlockResultByTimeAscending(bufferResults) bufferIdx := 0 - for i, blockResult := range res { + for i := range res { + blockResult := res[i] if !(bufferIdx < len(bufferResults)) { break } @@ -423,3 +294,122 @@ func (r Reader) fetchBlocksWithBlocksMapAndBuffer( block.SortFetchBlockResultByTimeAscending(res) return res, nil } + +func (r *Reader) resolveBlockResults( + ctx context.Context, + starts []time.Time, + seriesBlocks block.DatabaseSeriesBlocks, + nsCtx namespace.Context, +) []block.FetchBlockResult { + // Two-dimensional slice (each block.FetchBlockResult has a []xio.BlockReader internally) + // such that the first dimension is unique by blockstart and the second dimension is blocks + // of data for that blockstart (not necessarily in chronological order). + // + // ex. (querying 2P.M -> 6P.M with a 2-hour blocksize): + // []block.FetchBlockResult{ + // block.FetchBlockResult{ + // Start: 2P.M, + // Blocks: []xio.BlockReader{block0, block1, block2}, + // }, + // block.FetchBlockResult{ + // Start: 4P.M, + // Blocks: []xio.BlockReader{block0}, + // }, + // } + res := make([]block.FetchBlockResult, 0, len(starts)) + for _, start := range starts { + // Slice of xio.BlockReader such that all data belong to the same blockstart. + var blockReaders []xio.BlockReader + + blockReader, _, found, err := retrieveCached(ctx, start, seriesBlocks) + if err != nil { + // Short-circuit this entire blockstart if an error was encountered. + r := block.NewFetchBlockResult(start, nil, + fmt.Errorf("unable to retrieve block stream for series %s time %v: %w", + r.id.String(), start, err)) + res = append(res, r) + continue + } + + if !found { + // NB(r): Always use nil for OnRetrieveBlock so we don't cache the + // series after fetching it from disk, the fetch blocks API is called + // during streaming so to cache it in memory would mean we would + // eventually cache all series in memory when we stream results to a + // peer. + blockReader, found, err = r.streamBlock(ctx, start, nil, nsCtx) + if err != nil { + // Short-circuit this entire blockstart if an error was encountered. + r := block.NewFetchBlockResult(start, nil, + fmt.Errorf("unable to retrieve block stream for series %s time %v: %w", + r.id.String(), start, err)) + res = append(res, r) + continue + } + } + + if found { + blockReaders = append(blockReaders, blockReader) + } + + if len(blockReaders) > 0 { + res = append(res, block.NewFetchBlockResult(start, blockReaders, nil)) + } + } + + return res +} + +func retrieveCached( + ctx context.Context, + start time.Time, + seriesBlocks block.DatabaseSeriesBlocks, +) (xio.BlockReader, block.DatabaseBlock, bool, error) { + if seriesBlocks != nil { + if b, exists := seriesBlocks.BlockAt(start); exists { + streamedBlock, err := b.Stream(ctx) + if err != nil { + return xio.BlockReader{}, b, false, err + } + + if streamedBlock.IsNotEmpty() { + return streamedBlock, b, true, nil + } + } + } + + return xio.BlockReader{}, nil, false, nil +} + +func (r *Reader) streamBlock( + ctx context.Context, + start time.Time, + onRetrieve block.OnRetrieveBlock, + nsCtx namespace.Context, +) (xio.BlockReader, bool, error) { + cachePolicy := r.opts.CachePolicy() + switch { + case cachePolicy == CacheAll: + // No-op, block metadata should have been in-memory + case r.retriever != nil: + // Try to stream from disk + isRetrievable, err := r.retriever.IsBlockRetrievable(start) + if err != nil { + return xio.BlockReader{}, false, err + } + + if isRetrievable { + streamedBlock, err := r.retriever.Stream(ctx, r.id, start, onRetrieve, nsCtx) + if err != nil { + // Short-circuit this entire blockstart if an error was encountered. + return xio.BlockReader{}, false, err + } + + if streamedBlock.IsNotEmpty() { + return streamedBlock, true, nil + } + } + } + + return xio.BlockReader{}, false, nil +} diff --git a/src/dbnode/storage/series/reader_test.go b/src/dbnode/storage/series/reader_test.go index aeb0522e9e..70090c785c 100644 --- a/src/dbnode/storage/series/reader_test.go +++ b/src/dbnode/storage/series/reader_test.go @@ -88,7 +88,7 @@ func TestReaderUsingRetrieverReadEncoded(t *testing.T) { } } -func TestReaderUsingRetrieverIndexChecksumsBlockInvalid(t *testing.T) { +func TestReaderUsingRetrieverWideEntrysBlockInvalid(t *testing.T) { ctrl := xtest.NewController(t) defer ctrl.Finish() @@ -102,20 +102,20 @@ func TestReaderUsingRetrieverIndexChecksumsBlockInvalid(t *testing.T) { retriever.EXPECT().IsBlockRetrievable(gomock.Any()). Return(false, errors.New("err")) - _, err := reader.FetchIndexChecksum(ctx, time.Now(), namespace.Context{}) + _, err := reader.FetchWideEntry(ctx, time.Now(), namespace.Context{}) assert.EqualError(t, err, "err") retriever.EXPECT().IsBlockRetrievable(gomock.Any()).Return(false, nil) - c, err := reader.FetchIndexChecksum(ctx, time.Now(), namespace.Context{}) + e, err := reader.FetchWideEntry(ctx, time.Now(), namespace.Context{}) assert.NoError(t, err) - checksum, err := c.RetrieveIndexChecksum() + entry, err := e.RetrieveWideEntry() require.NoError(t, err) - assert.Equal(t, int64(0), checksum.MetadataChecksum) - assert.Nil(t, checksum.ID) + assert.Equal(t, int64(0), entry.MetadataChecksum) + assert.Nil(t, entry.ID) } -func TestReaderUsingRetrieverIndexChecksums(t *testing.T) { +func TestReaderUsingRetrieverWideEntrys(t *testing.T) { ctrl := xtest.NewController(t) defer ctrl.Finish() @@ -128,37 +128,36 @@ func TestReaderUsingRetrieverIndexChecksums(t *testing.T) { retriever := NewMockQueryableBlockRetriever(ctrl) retriever.EXPECT().IsBlockRetrievable(alignedStart).Return(true, nil).Times(2) - checksum := xio.IndexChecksum{ - MetadataChecksum: 5, - ID: ident.StringID("foo"), - } - - indexChecksum := block.NewMockStreamedChecksum(ctrl) - + streamedEntry := block.NewMockStreamedWideEntry(ctrl) ctx := opts.ContextPool().Get() defer ctx.Close() retriever.EXPECT(). - StreamIndexChecksum(ctx, ident.NewIDMatcher("foo"), + StreamWideEntry(ctx, ident.NewIDMatcher("foo"), alignedStart, gomock.Any()). - Return(indexChecksum, nil).Times(2) + Return(streamedEntry, nil).Times(2) reader := NewReaderUsingRetriever( ident.StringID("foo"), retriever, nil, nil, opts) - indexChecksum.EXPECT().RetrieveIndexChecksum().Return(xio.IndexChecksum{}, errors.New("err")) - streamed, err := reader.FetchIndexChecksum(ctx, alignedStart, namespace.Context{}) + streamedEntry.EXPECT().RetrieveWideEntry().Return(xio.WideEntry{}, errors.New("err")) + streamed, err := reader.FetchWideEntry(ctx, alignedStart, namespace.Context{}) require.NoError(t, err) - _, err = streamed.RetrieveIndexChecksum() + _, err = streamed.RetrieveWideEntry() assert.EqualError(t, err, "err") // Check reads as expected - indexChecksum.EXPECT().RetrieveIndexChecksum().Return(checksum, nil) - streamed, err = reader.FetchIndexChecksum(ctx, alignedStart, namespace.Context{}) + entry := xio.WideEntry{ + MetadataChecksum: 5, + ID: ident.StringID("foo"), + } + + streamedEntry.EXPECT().RetrieveWideEntry().Return(entry, nil) + streamed, err = reader.FetchWideEntry(ctx, alignedStart, namespace.Context{}) require.NoError(t, err) - actual, err := streamed.RetrieveIndexChecksum() + actual, err := streamed.RetrieveWideEntry() require.NoError(t, err) - assert.Equal(t, checksum, actual) + assert.Equal(t, entry, actual) } type readTestCase struct { diff --git a/src/dbnode/storage/series/series.go b/src/dbnode/storage/series/series.go index a6ea48a893..4c008db977 100644 --- a/src/dbnode/storage/series/series.go +++ b/src/dbnode/storage/series/series.go @@ -27,7 +27,6 @@ import ( "time" "github.com/m3db/m3/src/dbnode/persist" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/storage/block" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/dbnode/x/xio" @@ -403,29 +402,17 @@ func (s *dbSeries) ReadEncoded( return r, err } -func (s *dbSeries) FetchIndexChecksum( +func (s *dbSeries) FetchWideEntry( ctx context.Context, blockStart time.Time, nsCtx namespace.Context, -) (block.StreamedChecksum, error) { +) (block.StreamedWideEntry, error) { s.RLock() reader := NewReaderUsingRetriever(s.id, s.blockRetriever, s.onRetrieveBlock, s, s.opts) - r, err := reader.FetchIndexChecksum(ctx, blockStart, nsCtx) + e, err := reader.FetchWideEntry(ctx, blockStart, nsCtx) s.RUnlock() - return r, err -} -func (s *dbSeries) FetchReadMismatch( - ctx context.Context, - mismatchChecker wide.EntryChecksumMismatchChecker, - blockStart time.Time, - nsCtx namespace.Context, -) (wide.StreamedMismatch, error) { - s.RLock() - reader := NewReaderUsingRetriever(s.id, s.blockRetriever, s.onRetrieveBlock, s, s.opts) - r, err := reader.FetchReadMismatch(ctx, mismatchChecker, blockStart, nsCtx) - s.RUnlock() - return r, err + return e, err } func (s *dbSeries) FetchBlocksForColdFlush( @@ -449,12 +436,14 @@ func (s *dbSeries) FetchBlocks( nsCtx namespace.Context, ) ([]block.FetchBlockResult, error) { s.RLock() - r, err := Reader{ + reader := &Reader{ opts: s.opts, id: s.id, retriever: s.blockRetriever, onRetrieve: s.onRetrieveBlock, - }.fetchBlocksWithBlocksMapAndBuffer(ctx, starts, s.cachedBlocks, s.buffer, nsCtx) + } + + r, err := reader.fetchBlocksWithBlocksMapAndBuffer(ctx, starts, s.cachedBlocks, s.buffer, nsCtx) s.RUnlock() return r, err } diff --git a/src/dbnode/storage/series/series_mock.go b/src/dbnode/storage/series/series_mock.go index 3a8fdee444..cc20ef70b4 100644 --- a/src/dbnode/storage/series/series_mock.go +++ b/src/dbnode/storage/series/series_mock.go @@ -30,7 +30,6 @@ import ( "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/storage/block" "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/dbnode/x/xio" @@ -150,34 +149,19 @@ func (mr *MockDatabaseSeriesMockRecorder) FetchBlocksMetadata(arg0, arg1, arg2, return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchBlocksMetadata", reflect.TypeOf((*MockDatabaseSeries)(nil).FetchBlocksMetadata), arg0, arg1, arg2, arg3) } -// FetchIndexChecksum mocks base method -func (m *MockDatabaseSeries) FetchIndexChecksum(arg0 context.Context, arg1 time.Time, arg2 namespace.Context) (block.StreamedChecksum, error) { +// FetchWideEntry mocks base method +func (m *MockDatabaseSeries) FetchWideEntry(arg0 context.Context, arg1 time.Time, arg2 namespace.Context) (block.StreamedWideEntry, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "FetchIndexChecksum", arg0, arg1, arg2) - ret0, _ := ret[0].(block.StreamedChecksum) + ret := m.ctrl.Call(m, "FetchWideEntry", arg0, arg1, arg2) + ret0, _ := ret[0].(block.StreamedWideEntry) ret1, _ := ret[1].(error) return ret0, ret1 } -// FetchIndexChecksum indicates an expected call of FetchIndexChecksum -func (mr *MockDatabaseSeriesMockRecorder) FetchIndexChecksum(arg0, arg1, arg2 interface{}) *gomock.Call { +// FetchWideEntry indicates an expected call of FetchWideEntry +func (mr *MockDatabaseSeriesMockRecorder) FetchWideEntry(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchIndexChecksum", reflect.TypeOf((*MockDatabaseSeries)(nil).FetchIndexChecksum), arg0, arg1, arg2) -} - -// FetchReadMismatch mocks base method -func (m *MockDatabaseSeries) FetchReadMismatch(arg0 context.Context, arg1 wide.EntryChecksumMismatchChecker, arg2 time.Time, arg3 namespace.Context) (wide.StreamedMismatch, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "FetchReadMismatch", arg0, arg1, arg2, arg3) - ret0, _ := ret[0].(wide.StreamedMismatch) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// FetchReadMismatch indicates an expected call of FetchReadMismatch -func (mr *MockDatabaseSeriesMockRecorder) FetchReadMismatch(arg0, arg1, arg2, arg3 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchReadMismatch", reflect.TypeOf((*MockDatabaseSeries)(nil).FetchReadMismatch), arg0, arg1, arg2, arg3) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchWideEntry", reflect.TypeOf((*MockDatabaseSeries)(nil).FetchWideEntry), arg0, arg1, arg2) } // ID mocks base method @@ -472,32 +456,17 @@ func (mr *MockQueryableBlockRetrieverMockRecorder) Stream(arg0, arg1, arg2, arg3 return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Stream", reflect.TypeOf((*MockQueryableBlockRetriever)(nil).Stream), arg0, arg1, arg2, arg3, arg4) } -// StreamIndexChecksum mocks base method -func (m *MockQueryableBlockRetriever) StreamIndexChecksum(arg0 context.Context, arg1 ident.ID, arg2 time.Time, arg3 namespace.Context) (block.StreamedChecksum, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "StreamIndexChecksum", arg0, arg1, arg2, arg3) - ret0, _ := ret[0].(block.StreamedChecksum) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// StreamIndexChecksum indicates an expected call of StreamIndexChecksum -func (mr *MockQueryableBlockRetrieverMockRecorder) StreamIndexChecksum(arg0, arg1, arg2, arg3 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StreamIndexChecksum", reflect.TypeOf((*MockQueryableBlockRetriever)(nil).StreamIndexChecksum), arg0, arg1, arg2, arg3) -} - -// StreamReadMismatches mocks base method -func (m *MockQueryableBlockRetriever) StreamReadMismatches(arg0 context.Context, arg1 wide.EntryChecksumMismatchChecker, arg2 ident.ID, arg3 time.Time, arg4 namespace.Context) (wide.StreamedMismatch, error) { +// StreamWideEntry mocks base method +func (m *MockQueryableBlockRetriever) StreamWideEntry(arg0 context.Context, arg1 ident.ID, arg2 time.Time, arg3 namespace.Context) (block.StreamedWideEntry, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "StreamReadMismatches", arg0, arg1, arg2, arg3, arg4) - ret0, _ := ret[0].(wide.StreamedMismatch) + ret := m.ctrl.Call(m, "StreamWideEntry", arg0, arg1, arg2, arg3) + ret0, _ := ret[0].(block.StreamedWideEntry) ret1, _ := ret[1].(error) return ret0, ret1 } -// StreamReadMismatches indicates an expected call of StreamReadMismatches -func (mr *MockQueryableBlockRetrieverMockRecorder) StreamReadMismatches(arg0, arg1, arg2, arg3, arg4 interface{}) *gomock.Call { +// StreamWideEntry indicates an expected call of StreamWideEntry +func (mr *MockQueryableBlockRetrieverMockRecorder) StreamWideEntry(arg0, arg1, arg2, arg3 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StreamReadMismatches", reflect.TypeOf((*MockQueryableBlockRetriever)(nil).StreamReadMismatches), arg0, arg1, arg2, arg3, arg4) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StreamWideEntry", reflect.TypeOf((*MockQueryableBlockRetriever)(nil).StreamWideEntry), arg0, arg1, arg2, arg3) } diff --git a/src/dbnode/storage/series/types.go b/src/dbnode/storage/series/types.go index 439393e8c4..779dcdd858 100644 --- a/src/dbnode/storage/series/types.go +++ b/src/dbnode/storage/series/types.go @@ -26,7 +26,6 @@ import ( "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/storage/block" @@ -87,21 +86,12 @@ type DatabaseSeries interface { nsCtx namespace.Context, ) ([][]xio.BlockReader, error) - // FetchIndexChecksum reads checksums from encoded blocks. - FetchIndexChecksum( + // FetchWideEntry reads wide entries from encoded blocks. + FetchWideEntry( ctx context.Context, blockStart time.Time, nsCtx namespace.Context, - ) (block.StreamedChecksum, error) - - // FetchIndexChecksum reads checksum mismatches from encoded blocks and the - // incoming batchReader. - FetchReadMismatch( - ctx context.Context, - mismatchChecker wide.EntryChecksumMismatchChecker, - blockStart time.Time, - nsCtx namespace.Context, - ) (wide.StreamedMismatch, error) + ) (block.StreamedWideEntry, error) // FetchBlocks returns data blocks given a list of block start times. FetchBlocks( diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 49f8642757..029a8b9049 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -36,7 +36,6 @@ import ( "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/persist/fs" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/storage/block" @@ -399,29 +398,17 @@ func (s *dbShard) Stream( blockStart, onRetrieve, nsCtx) } -// StreamIndexChecksum implements series.QueryableBlockRetriever -func (s *dbShard) StreamIndexChecksum( +// StreamWideEntry implements series.QueryableBlockRetriever +func (s *dbShard) StreamWideEntry( ctx context.Context, id ident.ID, blockStart time.Time, nsCtx namespace.Context, -) (block.StreamedChecksum, error) { - return s.DatabaseBlockRetriever.StreamIndexChecksum(ctx, s.shard, id, +) (block.StreamedWideEntry, error) { + return s.DatabaseBlockRetriever.StreamWideEntry(ctx, s.shard, id, blockStart, nsCtx) } -// StreamIndexChecksum implements series.QueryableBlockRetriever -func (s *dbShard) StreamReadMismatches( - ctx context.Context, - mismatchChecker wide.EntryChecksumMismatchChecker, - id ident.ID, - blockStart time.Time, - nsCtx namespace.Context, -) (wide.StreamedMismatch, error) { - return s.DatabaseBlockRetriever.StreamReadMismatches(ctx, s.shard, - mismatchChecker, id, blockStart, nsCtx) -} - // IsBlockRetrievable implements series.QueryableBlockRetriever func (s *dbShard) IsBlockRetrievable(blockStart time.Time) (bool, error) { return s.hasWarmFlushed(blockStart) @@ -1155,29 +1142,17 @@ func (s *dbShard) ReadEncoded( return reader.ReadEncoded(ctx, start, end, nsCtx) } -func (s *dbShard) FetchIndexChecksum( +func (s *dbShard) FetchWideEntry( ctx context.Context, id ident.ID, blockStart time.Time, nsCtx namespace.Context, -) (block.StreamedChecksum, error) { +) (block.StreamedWideEntry, error) { retriever := s.seriesBlockRetriever opts := s.seriesOpts reader := series.NewReaderUsingRetriever(id, retriever, nil, nil, opts) - return reader.FetchIndexChecksum(ctx, blockStart, nsCtx) -} -func (s *dbShard) FetchReadMismatch( - ctx context.Context, - mismatchChecker wide.EntryChecksumMismatchChecker, - id ident.ID, - blockStart time.Time, - nsCtx namespace.Context, -) (wide.StreamedMismatch, error) { - retriever := s.seriesBlockRetriever - opts := s.seriesOpts - reader := series.NewReaderUsingRetriever(id, retriever, nil, nil, opts) - return reader.FetchReadMismatch(ctx, mismatchChecker, blockStart, nsCtx) + return reader.FetchWideEntry(ctx, blockStart, nsCtx) } // lookupEntryWithLock returns the entry for a given id while holding a read lock or a write lock. diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index 5c5077d801..53c1da1144 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -40,7 +40,6 @@ import ( "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/persist/fs" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/storage/block" @@ -1626,28 +1625,28 @@ func TestShardFetchIndexChecksum(t *testing.T) { retriever := block.NewMockDatabaseBlockRetriever(ctrl) shard.setBlockRetriever(retriever) - checksum := xio.IndexChecksum{ + checksum := xio.WideEntry{ ID: ident.StringID("foo"), MetadataChecksum: 5, } - indexChecksum := block.NewMockStreamedChecksum(ctrl) + wideEntry := block.NewMockStreamedWideEntry(ctrl) retriever.EXPECT(). - StreamIndexChecksum(ctx, shard.shard, ident.NewIDMatcher("foo"), - start, gomock.Any()).Return(indexChecksum, nil).Times(2) + StreamWideEntry(ctx, shard.shard, ident.NewIDMatcher("foo"), + start, gomock.Any()).Return(wideEntry, nil).Times(2) - // First call to RetrieveIndexChecksum is expected to error on retrieval - indexChecksum.EXPECT().RetrieveIndexChecksum(). - Return(xio.IndexChecksum{}, errors.New("err")) - r, err := shard.FetchIndexChecksum(ctx, ident.StringID("foo"), start, namespace.Context{}) + // First call to RetrieveWideEntry is expected to error on retrieval + wideEntry.EXPECT().RetrieveWideEntry(). + Return(xio.WideEntry{}, errors.New("err")) + r, err := shard.FetchWideEntry(ctx, ident.StringID("foo"), start, namespace.Context{}) require.NoError(t, err) - _, err = r.RetrieveIndexChecksum() + _, err = r.RetrieveWideEntry() assert.EqualError(t, err, "err") - indexChecksum.EXPECT().RetrieveIndexChecksum().Return(checksum, nil) - r, err = shard.FetchIndexChecksum(ctx, ident.StringID("foo"), start, namespace.Context{}) + wideEntry.EXPECT().RetrieveWideEntry().Return(checksum, nil) + r, err = shard.FetchWideEntry(ctx, ident.StringID("foo"), start, namespace.Context{}) require.NoError(t, err) - retrieved, err := r.RetrieveIndexChecksum() + retrieved, err := r.RetrieveWideEntry() require.NoError(t, err) assert.Equal(t, checksum, retrieved) @@ -1662,77 +1661,6 @@ func TestShardFetchIndexChecksum(t *testing.T) { require.Nil(t, entry) } -func TestShardFetchReadMismatch(t *testing.T) { - dir, err := ioutil.TempDir("", "testdir") - require.NoError(t, err) - defer os.RemoveAll(dir) - - ctrl := xtest.NewController(t) - defer ctrl.Finish() - - opts := DefaultTestOptions(). - SetSeriesCachePolicy(series.CacheAll) - fsOpts := opts.CommitLogOptions().FilesystemOptions(). - SetFilePathPrefix(dir) - opts = opts. - SetCommitLogOptions(opts.CommitLogOptions(). - SetFilesystemOptions(fsOpts)) - shard := testDatabaseShard(t, opts) - defer shard.Close() - - ctx := context.NewContext() - defer ctx.Close() - - nsCtx := namespace.Context{ID: ident.StringID("foo")} - require.NoError(t, shard.Bootstrap(ctx, nsCtx)) - - ropts := shard.seriesOpts.RetentionOptions() - end := opts.ClockOptions().NowFn()().Truncate(ropts.BlockSize()) - start := end.Add(-2 * ropts.BlockSize()) - shard.markWarmFlushStateSuccess(start) - shard.markWarmFlushStateSuccess(start.Add(ropts.BlockSize())) - - checker := wide.NewMockEntryChecksumMismatchChecker(ctrl) - retriever := block.NewMockDatabaseBlockRetriever(ctrl) - shard.setBlockRetriever(retriever) - - mismatchBatch := wide.ReadMismatch{ - IndexChecksum: xio.IndexChecksum{MetadataChecksum: 1}, - } - - streamedBatch := wide.NewMockStreamedMismatch(ctrl) - retriever.EXPECT(). - StreamReadMismatches(ctx, shard.shard, checker, ident.NewIDMatcher("foo"), - start, gomock.Any()).Return(streamedBatch, nil).Times(2) - - // First call to RetrieveMismatch is expected to error on retrieval - streamedBatch.EXPECT().RetrieveMismatch(). - Return(wide.ReadMismatch{}, errors.New("err")) - r, err := shard.FetchReadMismatch(ctx, checker, - ident.StringID("foo"), start, namespace.Context{}) - require.NoError(t, err) - _, err = r.RetrieveMismatch() - assert.EqualError(t, err, "err") - - streamedBatch.EXPECT().RetrieveMismatch().Return(mismatchBatch, nil) - r, err = shard.StreamReadMismatches(ctx, checker, - ident.StringID("foo"), start, namespace.Context{}) - require.NoError(t, err) - retrieved, err := r.RetrieveMismatch() - require.NoError(t, err) - assert.Equal(t, mismatchBatch, retrieved) - - // Check that nothing has been cached. Should be cached after a second. - time.Sleep(time.Second) - - shard.RLock() - entry, _, err := shard.lookupEntryWithLock(ident.StringID("foo")) - shard.RUnlock() - - require.Equal(t, err, errShardEntryNotFound) - require.Nil(t, entry) -} - func TestShardReadEncodedCachesSeriesWithRecentlyReadPolicy(t *testing.T) { dir, err := ioutil.TempDir("", "testdir") require.NoError(t, err) diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index b11da30f9c..87a3c33efa 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -35,7 +35,6 @@ import ( "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/persist/fs" "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/sharding" "github.com/m3db/m3/src/dbnode/storage/block" @@ -346,10 +345,10 @@ func (mr *MockDatabaseMockRecorder) ReadEncoded(ctx, namespace, id, start, end i } // WideQuery mocks base method -func (m *MockDatabase) WideQuery(ctx context.Context, namespace ident.ID, query index.Query, start time.Time, shards []uint32, iterOpts index.IterationOptions) ([]xio.IndexChecksum, error) { +func (m *MockDatabase) WideQuery(ctx context.Context, namespace ident.ID, query index.Query, start time.Time, shards []uint32, iterOpts index.IterationOptions) ([]xio.WideEntry, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "WideQuery", ctx, namespace, query, start, shards, iterOpts) - ret0, _ := ret[0].([]xio.IndexChecksum) + ret0, _ := ret[0].([]xio.WideEntry) ret1, _ := ret[1].(error) return ret0, ret1 } @@ -360,21 +359,6 @@ func (mr *MockDatabaseMockRecorder) WideQuery(ctx, namespace, query, start, shar return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WideQuery", reflect.TypeOf((*MockDatabase)(nil).WideQuery), ctx, namespace, query, start, shards, iterOpts) } -// ReadMismatches mocks base method -func (m *MockDatabase) ReadMismatches(ctx context.Context, namespace ident.ID, query index.Query, mismatchChecker wide.EntryChecksumMismatchChecker, queryStart time.Time, shards []uint32, iterOpts index.IterationOptions) ([]wide.ReadMismatch, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ReadMismatches", ctx, namespace, query, mismatchChecker, queryStart, shards, iterOpts) - ret0, _ := ret[0].([]wide.ReadMismatch) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// ReadMismatches indicates an expected call of ReadMismatches -func (mr *MockDatabaseMockRecorder) ReadMismatches(ctx, namespace, query, mismatchChecker, queryStart, shards, iterOpts interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ReadMismatches", reflect.TypeOf((*MockDatabase)(nil).ReadMismatches), ctx, namespace, query, mismatchChecker, queryStart, shards, iterOpts) -} - // FetchBlocks mocks base method func (m *MockDatabase) FetchBlocks(ctx context.Context, namespace ident.ID, shard uint32, id ident.ID, starts []time.Time) ([]block.FetchBlockResult, error) { m.ctrl.T.Helper() @@ -786,10 +770,10 @@ func (mr *MockdatabaseMockRecorder) ReadEncoded(ctx, namespace, id, start, end i } // WideQuery mocks base method -func (m *Mockdatabase) WideQuery(ctx context.Context, namespace ident.ID, query index.Query, start time.Time, shards []uint32, iterOpts index.IterationOptions) ([]xio.IndexChecksum, error) { +func (m *Mockdatabase) WideQuery(ctx context.Context, namespace ident.ID, query index.Query, start time.Time, shards []uint32, iterOpts index.IterationOptions) ([]xio.WideEntry, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "WideQuery", ctx, namespace, query, start, shards, iterOpts) - ret0, _ := ret[0].([]xio.IndexChecksum) + ret0, _ := ret[0].([]xio.WideEntry) ret1, _ := ret[1].(error) return ret0, ret1 } @@ -800,21 +784,6 @@ func (mr *MockdatabaseMockRecorder) WideQuery(ctx, namespace, query, start, shar return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WideQuery", reflect.TypeOf((*Mockdatabase)(nil).WideQuery), ctx, namespace, query, start, shards, iterOpts) } -// ReadMismatches mocks base method -func (m *Mockdatabase) ReadMismatches(ctx context.Context, namespace ident.ID, query index.Query, mismatchChecker wide.EntryChecksumMismatchChecker, queryStart time.Time, shards []uint32, iterOpts index.IterationOptions) ([]wide.ReadMismatch, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ReadMismatches", ctx, namespace, query, mismatchChecker, queryStart, shards, iterOpts) - ret0, _ := ret[0].([]wide.ReadMismatch) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// ReadMismatches indicates an expected call of ReadMismatches -func (mr *MockdatabaseMockRecorder) ReadMismatches(ctx, namespace, query, mismatchChecker, queryStart, shards, iterOpts interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ReadMismatches", reflect.TypeOf((*Mockdatabase)(nil).ReadMismatches), ctx, namespace, query, mismatchChecker, queryStart, shards, iterOpts) -} - // FetchBlocks mocks base method func (m *Mockdatabase) FetchBlocks(ctx context.Context, namespace ident.ID, shard uint32, id ident.ID, starts []time.Time) ([]block.FetchBlockResult, error) { m.ctrl.T.Helper() @@ -1499,34 +1468,19 @@ func (mr *MockdatabaseNamespaceMockRecorder) ReadEncoded(ctx, id, start, end int return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ReadEncoded", reflect.TypeOf((*MockdatabaseNamespace)(nil).ReadEncoded), ctx, id, start, end) } -// FetchIndexChecksum mocks base method -func (m *MockdatabaseNamespace) FetchIndexChecksum(ctx context.Context, id ident.ID, blockStart time.Time) (block.StreamedChecksum, error) { +// FetchWideEntry mocks base method +func (m *MockdatabaseNamespace) FetchWideEntry(ctx context.Context, id ident.ID, blockStart time.Time) (block.StreamedWideEntry, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "FetchIndexChecksum", ctx, id, blockStart) - ret0, _ := ret[0].(block.StreamedChecksum) + ret := m.ctrl.Call(m, "FetchWideEntry", ctx, id, blockStart) + ret0, _ := ret[0].(block.StreamedWideEntry) ret1, _ := ret[1].(error) return ret0, ret1 } -// FetchIndexChecksum indicates an expected call of FetchIndexChecksum -func (mr *MockdatabaseNamespaceMockRecorder) FetchIndexChecksum(ctx, id, blockStart interface{}) *gomock.Call { +// FetchWideEntry indicates an expected call of FetchWideEntry +func (mr *MockdatabaseNamespaceMockRecorder) FetchWideEntry(ctx, id, blockStart interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchIndexChecksum", reflect.TypeOf((*MockdatabaseNamespace)(nil).FetchIndexChecksum), ctx, id, blockStart) -} - -// FetchReadMismatch mocks base method -func (m *MockdatabaseNamespace) FetchReadMismatch(ctx context.Context, mismatchChecker wide.EntryChecksumMismatchChecker, id ident.ID, blockStart time.Time) (wide.StreamedMismatch, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "FetchReadMismatch", ctx, mismatchChecker, id, blockStart) - ret0, _ := ret[0].(wide.StreamedMismatch) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// FetchReadMismatch indicates an expected call of FetchReadMismatch -func (mr *MockdatabaseNamespaceMockRecorder) FetchReadMismatch(ctx, mismatchChecker, id, blockStart interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchReadMismatch", reflect.TypeOf((*MockdatabaseNamespace)(nil).FetchReadMismatch), ctx, mismatchChecker, id, blockStart) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchWideEntry", reflect.TypeOf((*MockdatabaseNamespace)(nil).FetchWideEntry), ctx, id, blockStart) } // FetchBlocks mocks base method @@ -2037,34 +1991,19 @@ func (mr *MockdatabaseShardMockRecorder) ReadEncoded(ctx, id, start, end, nsCtx return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ReadEncoded", reflect.TypeOf((*MockdatabaseShard)(nil).ReadEncoded), ctx, id, start, end, nsCtx) } -// FetchIndexChecksum mocks base method -func (m *MockdatabaseShard) FetchIndexChecksum(ctx context.Context, id ident.ID, blockStart time.Time, nsCtx namespace.Context) (block.StreamedChecksum, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "FetchIndexChecksum", ctx, id, blockStart, nsCtx) - ret0, _ := ret[0].(block.StreamedChecksum) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// FetchIndexChecksum indicates an expected call of FetchIndexChecksum -func (mr *MockdatabaseShardMockRecorder) FetchIndexChecksum(ctx, id, blockStart, nsCtx interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchIndexChecksum", reflect.TypeOf((*MockdatabaseShard)(nil).FetchIndexChecksum), ctx, id, blockStart, nsCtx) -} - -// FetchReadMismatch mocks base method -func (m *MockdatabaseShard) FetchReadMismatch(ctx context.Context, mismatchChecker wide.EntryChecksumMismatchChecker, id ident.ID, blockStart time.Time, nsCtx namespace.Context) (wide.StreamedMismatch, error) { +// FetchWideEntry mocks base method +func (m *MockdatabaseShard) FetchWideEntry(ctx context.Context, id ident.ID, blockStart time.Time, nsCtx namespace.Context) (block.StreamedWideEntry, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "FetchReadMismatch", ctx, mismatchChecker, id, blockStart, nsCtx) - ret0, _ := ret[0].(wide.StreamedMismatch) + ret := m.ctrl.Call(m, "FetchWideEntry", ctx, id, blockStart, nsCtx) + ret0, _ := ret[0].(block.StreamedWideEntry) ret1, _ := ret[1].(error) return ret0, ret1 } -// FetchReadMismatch indicates an expected call of FetchReadMismatch -func (mr *MockdatabaseShardMockRecorder) FetchReadMismatch(ctx, mismatchChecker, id, blockStart, nsCtx interface{}) *gomock.Call { +// FetchWideEntry indicates an expected call of FetchWideEntry +func (mr *MockdatabaseShardMockRecorder) FetchWideEntry(ctx, id, blockStart, nsCtx interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchReadMismatch", reflect.TypeOf((*MockdatabaseShard)(nil).FetchReadMismatch), ctx, mismatchChecker, id, blockStart, nsCtx) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchWideEntry", reflect.TypeOf((*MockdatabaseShard)(nil).FetchWideEntry), ctx, id, blockStart, nsCtx) } // FetchBlocks mocks base method diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 185ac9897e..3e28e7f31d 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -31,7 +31,6 @@ import ( "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/persist/fs" "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" - "github.com/m3db/m3/src/dbnode/persist/fs/wide" "github.com/m3db/m3/src/dbnode/runtime" "github.com/m3db/m3/src/dbnode/sharding" "github.com/m3db/m3/src/dbnode/storage/block" @@ -184,19 +183,7 @@ type Database interface { start time.Time, shards []uint32, iterOpts index.IterationOptions, - ) ([]xio.IndexChecksum, error) // FIXME: change when exact type known. - - // ReadMismatches performs a wide blockwise query that applies a received - // index checksum block batch. - ReadMismatches( - ctx context.Context, - namespace ident.ID, - query index.Query, - mismatchChecker wide.EntryChecksumMismatchChecker, - queryStart time.Time, - shards []uint32, - iterOpts index.IterationOptions, - ) ([]wide.ReadMismatch, error) // TODO: update this type when reader hooked up + ) ([]xio.WideEntry, error) // FIXME: change when exact type known. // FetchBlocks retrieves data blocks for a given id and a list of block // start times. @@ -381,22 +368,13 @@ type databaseNamespace interface { start, end time.Time, ) ([][]xio.BlockReader, error) - // FetchIndexChecksum retrieves the index checksum for an ID for the + // FetchWideEntry retrieves the wide entry for an ID for the // block at time start. - FetchIndexChecksum( - ctx context.Context, - id ident.ID, - blockStart time.Time, - ) (block.StreamedChecksum, error) - - // FetchReadMismatch retrieves the read mismatches for an ID for the - // block at time start, with the given batchReader. - FetchReadMismatch( + FetchWideEntry( ctx context.Context, - mismatchChecker wide.EntryChecksumMismatchChecker, id ident.ID, blockStart time.Time, - ) (wide.StreamedMismatch, error) + ) (block.StreamedWideEntry, error) // FetchBlocks retrieves data blocks for a given id and a list of block // start times. @@ -554,23 +532,14 @@ type databaseShard interface { nsCtx namespace.Context, ) ([][]xio.BlockReader, error) - // FetchIndexChecksum retrieves the index checksum for an ID. - FetchIndexChecksum( - ctx context.Context, - id ident.ID, - blockStart time.Time, - nsCtx namespace.Context, - ) (block.StreamedChecksum, error) - - // FetchReadMismatch retrieves the read mismatches for an ID for the - // block at time start, with the given batchReader. - FetchReadMismatch( + // FetchWideEntry retrieves wide entry for an ID for the + // block at time start. + FetchWideEntry( ctx context.Context, - mismatchChecker wide.EntryChecksumMismatchChecker, id ident.ID, blockStart time.Time, nsCtx namespace.Context, - ) (wide.StreamedMismatch, error) + ) (block.StreamedWideEntry, error) // FetchBlocks retrieves data blocks for a given id and a list of block // start times. diff --git a/src/dbnode/tracepoint/tracepoint.go b/src/dbnode/tracepoint/tracepoint.go index 8f8cd66b16..69eefcfe97 100644 --- a/src/dbnode/tracepoint/tracepoint.go +++ b/src/dbnode/tracepoint/tracepoint.go @@ -73,8 +73,8 @@ const ( // DBWriteBatch is the operation name for the db WriteBatch path. DBWriteBatch = "storage.db.WriteBatch" - // DBIndexChecksum is the operation name for the tchannelthrift IndexChecksum path. - DBIndexChecksum = "storage.db.IndexChecksum" + // DBWideEntry is the operation name for the tchannelthrift WideEntry path. + DBWideEntry = "storage.db.WideEntry" // DBFetchMismatch is the operation name for the tchannelthrift DBFetchMismatch path. DBFetchMismatch = "storage.db.FetchMismatch" diff --git a/src/dbnode/x/xio/index_checksum.go b/src/dbnode/x/xio/wide_entry.go similarity index 72% rename from src/dbnode/x/xio/index_checksum.go rename to src/dbnode/x/xio/wide_entry.go index eeedfb31e8..e4724f4026 100644 --- a/src/dbnode/x/xio/index_checksum.go +++ b/src/dbnode/x/xio/wide_entry.go @@ -25,24 +25,47 @@ import ( "github.com/m3db/m3/src/x/ident" ) -// IndexChecksum is an entry from the index file which can be passed to +// WideEntry is an entry from the index file which can be passed to // SeekUsingIndexEntry to seek to the data for that entry. -type IndexChecksum struct { +type WideEntry struct { + finalized bool + + Shard uint32 ID ident.ID Size int64 Offset int64 DataChecksum int64 EncodedTags checked.Bytes MetadataChecksum int64 + Data checked.Bytes +} + +// Empty returns whether the wide entry is empty and not found. +func (c *WideEntry) Empty() bool { + return *c == WideEntry{} } -// Finalize finalizes the index checksum. -func (c *IndexChecksum) Finalize() { +// Finalize finalizes the wide entry. +func (c *WideEntry) Finalize() { + if c.Empty() || c.finalized { + return + } + + c.finalized = true if c.EncodedTags != nil { c.EncodedTags.DecRef() + c.EncodedTags.Finalize() + c.EncodedTags = nil } - if c.ID != nil && c.ID.Bytes() != nil { + if c.ID != nil { c.ID.Finalize() + c.ID = nil + } + + if c.Data != nil { + c.Data.DecRef() + c.Data.Finalize() + c.Data = nil } } diff --git a/src/x/checked/debug.go b/src/x/checked/debug.go index a5f49e63d2..d19ae2476c 100644 --- a/src/x/checked/debug.go +++ b/src/x/checked/debug.go @@ -23,6 +23,7 @@ package checked import ( "bytes" "fmt" + "os" "runtime" "sync" "time" @@ -314,4 +315,8 @@ func tracebackEvent(c *RefCount, ref int, e debuggerEvent) { func init() { leaks.m = make(map[string]uint64) + + if os.Getenv("DEBUG_ENABLE_TRACEBACKS") == "true" { + EnableTracebacks() + } } From 799d9ef946cad9697c21d7b8778717d9a986abe8 Mon Sep 17 00:00:00 2001 From: Linas Medziunas Date: Wed, 11 Nov 2020 16:39:55 +0200 Subject: [PATCH 06/12] Rename Aggregator to TileAggregator --- src/aggregator/aggregator/aggregator_mock.go | 2 +- src/dbnode/server/options.go | 2 +- src/dbnode/server/server.go | 6 +-- src/dbnode/storage/options.go | 16 +++--- src/dbnode/storage/storage_mock.go | 54 ++++++++++---------- src/dbnode/storage/types.go | 16 +++--- 6 files changed, 48 insertions(+), 48 deletions(-) diff --git a/src/aggregator/aggregator/aggregator_mock.go b/src/aggregator/aggregator/aggregator_mock.go index 5107239a52..d32238c8b4 100644 --- a/src/aggregator/aggregator/aggregator_mock.go +++ b/src/aggregator/aggregator/aggregator_mock.go @@ -1,5 +1,5 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/m3db/m3/src/aggregator/aggregator (interfaces: Aggregator,ElectionManager,FlushTimesManager,PlacementManager) +// Source: github.com/m3db/m3/src/aggregator/aggregator (interfaces: TileAggregator,ElectionManager,FlushTimesManager,PlacementManager) // Copyright (c) 2020 Uber Technologies, Inc. // diff --git a/src/dbnode/server/options.go b/src/dbnode/server/options.go index af175fc330..ba082f20bc 100644 --- a/src/dbnode/server/options.go +++ b/src/dbnode/server/options.go @@ -32,5 +32,5 @@ type StorageOptions struct { TChanNodeServerFn node.NewTChanNodeServerFn BackgroundProcessFns []storage.NewBackgroundProcessFn NamespaceHooks storage.NamespaceHooks - NewAggregatorFn storage.NewAggregatorFn + NewTileAggregatorFn storage.NewTileAggregatorFn } diff --git a/src/dbnode/server/server.go b/src/dbnode/server/server.go index 95bf62154f..0bdefc31f4 100644 --- a/src/dbnode/server/server.go +++ b/src/dbnode/server/server.go @@ -866,9 +866,9 @@ func Run(runOpts RunOptions) { opts = opts.SetNamespaceHooks(runOpts.StorageOptions.NamespaceHooks) } - if runOpts.StorageOptions.NewAggregatorFn != nil { - aggregator := runOpts.StorageOptions.NewAggregatorFn(iopts) - opts = opts.SetAggregator(aggregator) + if runOpts.StorageOptions.NewTileAggregatorFn != nil { + aggregator := runOpts.StorageOptions.NewTileAggregatorFn(iopts) + opts = opts.SetTileAggregator(aggregator) } // Set bootstrap options - We need to create a topology map provider from the diff --git a/src/dbnode/storage/options.go b/src/dbnode/storage/options.go index 6611206240..3f01aad1cb 100644 --- a/src/dbnode/storage/options.go +++ b/src/dbnode/storage/options.go @@ -174,7 +174,7 @@ type options struct { wideBatchSize int newBackgroundProcessFns []NewBackgroundProcessFn namespaceHooks NamespaceHooks - aggregator Aggregator + tileAggregator TileAggregator } // NewOptions creates a new set of storage options with defaults @@ -250,7 +250,7 @@ func newOptions(poolOpts pool.ObjectPoolOptions) Options { mediatorTickInterval: defaultMediatorTickInterval, wideBatchSize: defaultWideBatchSize, namespaceHooks: &noopNamespaceHooks{}, - aggregator: &noopAggregator{}, + tileAggregator: &noopTileAggregator{}, } return o.SetEncodingM3TSZPooled() } @@ -865,15 +865,15 @@ func (o *options) NamespaceHooks() NamespaceHooks { return o.namespaceHooks } -func (o *options) SetAggregator(value Aggregator) Options { +func (o *options) SetTileAggregator(value TileAggregator) Options { opts := *o - opts.aggregator = value + opts.tileAggregator = value return &opts } -func (o *options) Aggregator() Aggregator { - return o.aggregator +func (o *options) TileAggregator() TileAggregator { + return o.tileAggregator } type noOpColdFlush struct{} @@ -888,9 +888,9 @@ func (h *noopNamespaceHooks) OnCreatedNamespace(Namespace, GetNamespaceFn) error return nil } -type noopAggregator struct{} +type noopTileAggregator struct{} -func (a *noopAggregator) AggregateTiles( +func (a *noopTileAggregator) AggregateTiles( opts AggregateTilesOptions, ns Namespace, shardID uint32, diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index f624370643..360304a9db 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -4992,32 +4992,32 @@ func (mr *MockOptionsMockRecorder) NamespaceHooks() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NamespaceHooks", reflect.TypeOf((*MockOptions)(nil).NamespaceHooks)) } -// SetAggregator mocks base method -func (m *MockOptions) SetAggregator(aggregator Aggregator) Options { +// SetTileAggregator mocks base method +func (m *MockOptions) SetTileAggregator(aggregator TileAggregator) Options { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "SetAggregator", aggregator) + ret := m.ctrl.Call(m, "SetTileAggregator", aggregator) ret0, _ := ret[0].(Options) return ret0 } -// SetAggregator indicates an expected call of SetAggregator -func (mr *MockOptionsMockRecorder) SetAggregator(aggregator interface{}) *gomock.Call { +// SetTileAggregator indicates an expected call of SetTileAggregator +func (mr *MockOptionsMockRecorder) SetTileAggregator(aggregator interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetAggregator", reflect.TypeOf((*MockOptions)(nil).SetAggregator), aggregator) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetTileAggregator", reflect.TypeOf((*MockOptions)(nil).SetTileAggregator), aggregator) } -// Aggregator mocks base method -func (m *MockOptions) Aggregator() Aggregator { +// TileAggregator mocks base method +func (m *MockOptions) TileAggregator() TileAggregator { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Aggregator") - ret0, _ := ret[0].(Aggregator) + ret := m.ctrl.Call(m, "TileAggregator") + ret0, _ := ret[0].(TileAggregator) return ret0 } -// Aggregator indicates an expected call of Aggregator -func (mr *MockOptionsMockRecorder) Aggregator() *gomock.Call { +// TileAggregator indicates an expected call of TileAggregator +func (mr *MockOptionsMockRecorder) TileAggregator() *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Aggregator", reflect.TypeOf((*MockOptions)(nil).Aggregator)) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TileAggregator", reflect.TypeOf((*MockOptions)(nil).TileAggregator)) } // MockMemoryTracker is a mock of MemoryTracker interface @@ -5107,31 +5107,31 @@ func (mr *MockMemoryTrackerMockRecorder) WaitForDec() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WaitForDec", reflect.TypeOf((*MockMemoryTracker)(nil).WaitForDec)) } -// MockAggregator is a mock of Aggregator interface -type MockAggregator struct { +// MockTileAggregator is a mock of TileAggregator interface +type MockTileAggregator struct { ctrl *gomock.Controller - recorder *MockAggregatorMockRecorder + recorder *MockTileAggregatorMockRecorder } -// MockAggregatorMockRecorder is the mock recorder for MockAggregator -type MockAggregatorMockRecorder struct { - mock *MockAggregator +// MockTileAggregatorMockRecorder is the mock recorder for MockTileAggregator +type MockTileAggregatorMockRecorder struct { + mock *MockTileAggregator } -// NewMockAggregator creates a new mock instance -func NewMockAggregator(ctrl *gomock.Controller) *MockAggregator { - mock := &MockAggregator{ctrl: ctrl} - mock.recorder = &MockAggregatorMockRecorder{mock} +// NewMockTileAggregator creates a new mock instance +func NewMockTileAggregator(ctrl *gomock.Controller) *MockTileAggregator { + mock := &MockTileAggregator{ctrl: ctrl} + mock.recorder = &MockTileAggregatorMockRecorder{mock} return mock } // EXPECT returns an object that allows the caller to indicate expected use -func (m *MockAggregator) EXPECT() *MockAggregatorMockRecorder { +func (m *MockTileAggregator) EXPECT() *MockTileAggregatorMockRecorder { return m.recorder } // AggregateTiles mocks base method -func (m *MockAggregator) AggregateTiles(opts AggregateTilesOptions, ns Namespace, shardID uint32, readers []fs.DataFileSetReader, writer fs.StreamingWriter) (int64, error) { +func (m *MockTileAggregator) AggregateTiles(opts AggregateTilesOptions, ns Namespace, shardID uint32, readers []fs.DataFileSetReader, writer fs.StreamingWriter) (int64, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "AggregateTiles", opts, ns, shardID, readers, writer) ret0, _ := ret[0].(int64) @@ -5140,9 +5140,9 @@ func (m *MockAggregator) AggregateTiles(opts AggregateTilesOptions, ns Namespace } // AggregateTiles indicates an expected call of AggregateTiles -func (mr *MockAggregatorMockRecorder) AggregateTiles(opts, ns, shardID, readers, writer interface{}) *gomock.Call { +func (mr *MockTileAggregatorMockRecorder) AggregateTiles(opts, ns, shardID, readers, writer interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AggregateTiles", reflect.TypeOf((*MockAggregator)(nil).AggregateTiles), opts, ns, shardID, readers, writer) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AggregateTiles", reflect.TypeOf((*MockTileAggregator)(nil).AggregateTiles), opts, ns, shardID, readers, writer) } // MockNamespaceHooks is a mock of NamespaceHooks interface diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 9e1c73d4bb..3e9bc7ae98 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -1324,11 +1324,11 @@ type Options interface { // NamespaceHooks returns the NamespaceHooks. NamespaceHooks() NamespaceHooks - // SetAggregator sets the Aggregator. - SetAggregator(aggregator Aggregator) Options + // SetTileAggregator sets the TileAggregator. + SetTileAggregator(aggregator TileAggregator) Options - // Aggregator returns the Aggregator. - Aggregator() Aggregator + // TileAggregator returns the TileAggregator. + TileAggregator() TileAggregator } // MemoryTracker tracks memory. @@ -1400,8 +1400,8 @@ type AggregateTilesOptions struct { InsOptions instrument.Options } -// Aggregator is the interface for AggregateTiles. -type Aggregator interface { +// TileAggregator is the interface for AggregateTiles. +type TileAggregator interface { // AggregateTiles does tile aggregation. AggregateTiles( opts AggregateTilesOptions, @@ -1412,8 +1412,8 @@ type Aggregator interface { ) (int64, error) } -// NewAggregatorFn creates a new Aggregator. -type NewAggregatorFn func(iOpts instrument.Options) Aggregator +// NewTileAggregatorFn creates a new TileAggregator. +type NewTileAggregatorFn func(iOpts instrument.Options) TileAggregator // NamespaceHooks allows dynamic plugging into the namespace lifecycle. type NamespaceHooks interface { From 26fe4a984d363a6ba5f772d6845763531a942204 Mon Sep 17 00:00:00 2001 From: Linas Medziunas Date: Wed, 11 Nov 2020 16:43:17 +0200 Subject: [PATCH 07/12] mock gen --- src/aggregator/aggregator/aggregator_mock.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/aggregator/aggregator/aggregator_mock.go b/src/aggregator/aggregator/aggregator_mock.go index d32238c8b4..5107239a52 100644 --- a/src/aggregator/aggregator/aggregator_mock.go +++ b/src/aggregator/aggregator/aggregator_mock.go @@ -1,5 +1,5 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/m3db/m3/src/aggregator/aggregator (interfaces: TileAggregator,ElectionManager,FlushTimesManager,PlacementManager) +// Source: github.com/m3db/m3/src/aggregator/aggregator (interfaces: Aggregator,ElectionManager,FlushTimesManager,PlacementManager) // Copyright (c) 2020 Uber Technologies, Inc. // From e01d4d4fd27ce0153847e79305e8ba3aca3710bd Mon Sep 17 00:00:00 2001 From: Linas Medziunas Date: Wed, 11 Nov 2020 17:09:40 +0200 Subject: [PATCH 08/12] Fix --- src/dbnode/storage/shard.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index c6a53c978e..1c13e0f43b 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -323,7 +323,7 @@ func newDatabaseShard( coldWritesEnabled: namespaceMetadata.Options().ColdWritesEnabled(), logger: opts.InstrumentOptions().Logger(), metrics: newDatabaseShardMetrics(shard, scope), - tileAggregator: opts.Aggregator(), + tileAggregator: opts.TileAggregator(), } s.insertQueue = newDatabaseShardInsertQueue(s.insertSeriesBatch, s.nowFn, scope, opts.InstrumentOptions().Logger()) From d8cc5fda2dff4641a1fcdb6095eaa56a4ad804bc Mon Sep 17 00:00:00 2001 From: Linas Medziunas Date: Wed, 11 Nov 2020 17:18:01 +0200 Subject: [PATCH 09/12] Formatting --- src/dbnode/storage/shard.go | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 1c13e0f43b..0a9da191ce 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -313,17 +313,17 @@ func newDatabaseShard( newFSMergeWithMemFn: newFSMergeWithMem, filesetsFn: fs.DataFiles, filesetPathsBeforeFn: fs.DataFileSetsBefore, - deleteFilesFn: fs.DeleteFiles, - snapshotFilesFn: fs.SnapshotFiles, - sleepFn: time.Sleep, - identifierPool: opts.IdentifierPool(), - contextPool: opts.ContextPool(), - flushState: newShardFlushState(), - tickWg: &sync.WaitGroup{}, - coldWritesEnabled: namespaceMetadata.Options().ColdWritesEnabled(), - logger: opts.InstrumentOptions().Logger(), - metrics: newDatabaseShardMetrics(shard, scope), - tileAggregator: opts.TileAggregator(), + deleteFilesFn: fs.DeleteFiles, + snapshotFilesFn: fs.SnapshotFiles, + sleepFn: time.Sleep, + identifierPool: opts.IdentifierPool(), + contextPool: opts.ContextPool(), + flushState: newShardFlushState(), + tickWg: &sync.WaitGroup{}, + coldWritesEnabled: namespaceMetadata.Options().ColdWritesEnabled(), + logger: opts.InstrumentOptions().Logger(), + metrics: newDatabaseShardMetrics(shard, scope), + tileAggregator: opts.TileAggregator(), } s.insertQueue = newDatabaseShardInsertQueue(s.insertSeriesBatch, s.nowFn, scope, opts.InstrumentOptions().Logger()) From 00e3304356e982397b857f49e45e04e9b605b0d0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Linas=20Med=C5=BEi=C5=ABnas?= Date: Wed, 11 Nov 2020 17:30:12 +0200 Subject: [PATCH 10/12] [dbnode] Introduce Aggregator type (#2840) --- src/dbnode/server/options.go | 1 + src/dbnode/server/server.go | 5 +++ src/dbnode/storage/options.go | 25 +++++++++++ src/dbnode/storage/storage_mock.go | 66 ++++++++++++++++++++++++++++++ src/dbnode/storage/types.go | 21 ++++++++++ 5 files changed, 118 insertions(+) diff --git a/src/dbnode/server/options.go b/src/dbnode/server/options.go index fa2cc58c2a..ba082f20bc 100644 --- a/src/dbnode/server/options.go +++ b/src/dbnode/server/options.go @@ -32,4 +32,5 @@ type StorageOptions struct { TChanNodeServerFn node.NewTChanNodeServerFn BackgroundProcessFns []storage.NewBackgroundProcessFn NamespaceHooks storage.NamespaceHooks + NewTileAggregatorFn storage.NewTileAggregatorFn } diff --git a/src/dbnode/server/server.go b/src/dbnode/server/server.go index 31be18cfb0..0ea674252d 100644 --- a/src/dbnode/server/server.go +++ b/src/dbnode/server/server.go @@ -870,6 +870,11 @@ func Run(runOpts RunOptions) { opts = opts.SetNamespaceHooks(runOpts.StorageOptions.NamespaceHooks) } + if runOpts.StorageOptions.NewTileAggregatorFn != nil { + aggregator := runOpts.StorageOptions.NewTileAggregatorFn(iopts) + opts = opts.SetTileAggregator(aggregator) + } + // Set bootstrap options - We need to create a topology map provider from the // same topology that will be passed to the cluster so that when we make // bootstrapping decisions they are in sync with the clustered database diff --git a/src/dbnode/storage/options.go b/src/dbnode/storage/options.go index f5c79e9890..2eda82f7b4 100644 --- a/src/dbnode/storage/options.go +++ b/src/dbnode/storage/options.go @@ -177,6 +177,7 @@ type options struct { wideBatchSize int newBackgroundProcessFns []NewBackgroundProcessFn namespaceHooks NamespaceHooks + tileAggregator TileAggregator } // NewOptions creates a new set of storage options with defaults @@ -252,6 +253,7 @@ func newOptions(poolOpts pool.ObjectPoolOptions) Options { mediatorTickInterval: defaultMediatorTickInterval, wideBatchSize: defaultWideBatchSize, namespaceHooks: &noopNamespaceHooks{}, + tileAggregator: &noopTileAggregator{}, } return o.SetEncodingM3TSZPooled() } @@ -891,6 +893,17 @@ func (o *options) NamespaceHooks() NamespaceHooks { return o.namespaceHooks } +func (o *options) SetTileAggregator(value TileAggregator) Options { + opts := *o + opts.tileAggregator = value + + return &opts +} + +func (o *options) TileAggregator() TileAggregator { + return o.tileAggregator +} + type noOpColdFlush struct{} func (n *noOpColdFlush) ColdFlushNamespace(Namespace) (OnColdFlushNamespace, error) { @@ -902,3 +915,15 @@ type noopNamespaceHooks struct{} func (h *noopNamespaceHooks) OnCreatedNamespace(Namespace, GetNamespaceFn) error { return nil } + +type noopTileAggregator struct{} + +func (a *noopTileAggregator) AggregateTiles( + opts AggregateTilesOptions, + ns Namespace, + shardID uint32, + readers []fs.DataFileSetReader, + writer fs.StreamingWriter, +) (int64, error) { + return 0, nil +} diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index 87a3c33efa..386e0beb13 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -4959,6 +4959,34 @@ func (mr *MockOptionsMockRecorder) NamespaceHooks() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NamespaceHooks", reflect.TypeOf((*MockOptions)(nil).NamespaceHooks)) } +// SetTileAggregator mocks base method +func (m *MockOptions) SetTileAggregator(aggregator TileAggregator) Options { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "SetTileAggregator", aggregator) + ret0, _ := ret[0].(Options) + return ret0 +} + +// SetTileAggregator indicates an expected call of SetTileAggregator +func (mr *MockOptionsMockRecorder) SetTileAggregator(aggregator interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetTileAggregator", reflect.TypeOf((*MockOptions)(nil).SetTileAggregator), aggregator) +} + +// TileAggregator mocks base method +func (m *MockOptions) TileAggregator() TileAggregator { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "TileAggregator") + ret0, _ := ret[0].(TileAggregator) + return ret0 +} + +// TileAggregator indicates an expected call of TileAggregator +func (mr *MockOptionsMockRecorder) TileAggregator() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TileAggregator", reflect.TypeOf((*MockOptions)(nil).TileAggregator)) +} + // MockMemoryTracker is a mock of MemoryTracker interface type MockMemoryTracker struct { ctrl *gomock.Controller @@ -5046,6 +5074,44 @@ func (mr *MockMemoryTrackerMockRecorder) WaitForDec() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WaitForDec", reflect.TypeOf((*MockMemoryTracker)(nil).WaitForDec)) } +// MockTileAggregator is a mock of TileAggregator interface +type MockTileAggregator struct { + ctrl *gomock.Controller + recorder *MockTileAggregatorMockRecorder +} + +// MockTileAggregatorMockRecorder is the mock recorder for MockTileAggregator +type MockTileAggregatorMockRecorder struct { + mock *MockTileAggregator +} + +// NewMockTileAggregator creates a new mock instance +func NewMockTileAggregator(ctrl *gomock.Controller) *MockTileAggregator { + mock := &MockTileAggregator{ctrl: ctrl} + mock.recorder = &MockTileAggregatorMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use +func (m *MockTileAggregator) EXPECT() *MockTileAggregatorMockRecorder { + return m.recorder +} + +// AggregateTiles mocks base method +func (m *MockTileAggregator) AggregateTiles(opts AggregateTilesOptions, ns Namespace, shardID uint32, readers []fs.DataFileSetReader, writer fs.StreamingWriter) (int64, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "AggregateTiles", opts, ns, shardID, readers, writer) + ret0, _ := ret[0].(int64) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// AggregateTiles indicates an expected call of AggregateTiles +func (mr *MockTileAggregatorMockRecorder) AggregateTiles(opts, ns, shardID, readers, writer interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AggregateTiles", reflect.TypeOf((*MockTileAggregator)(nil).AggregateTiles), opts, ns, shardID, readers, writer) +} + // MockNamespaceHooks is a mock of NamespaceHooks interface type MockNamespaceHooks struct { ctrl *gomock.Controller diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 3e28e7f31d..b9a521e785 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -1298,6 +1298,12 @@ type Options interface { // NamespaceHooks returns the NamespaceHooks. NamespaceHooks() NamespaceHooks + + // SetTileAggregator sets the TileAggregator. + SetTileAggregator(aggregator TileAggregator) Options + + // TileAggregator returns the TileAggregator. + TileAggregator() TileAggregator } // MemoryTracker tracks memory. @@ -1369,6 +1375,21 @@ type AggregateTilesOptions struct { InsOptions instrument.Options } +// TileAggregator is the interface for AggregateTiles. +type TileAggregator interface { + // AggregateTiles does tile aggregation. + AggregateTiles( + opts AggregateTilesOptions, + ns Namespace, + shardID uint32, + readers []fs.DataFileSetReader, + writer fs.StreamingWriter, + ) (int64, error) +} + +// NewTileAggregatorFn creates a new TileAggregator. +type NewTileAggregatorFn func(iOpts instrument.Options) TileAggregator + // NamespaceHooks allows dynamic plugging into the namespace lifecycle. type NamespaceHooks interface { // OnCreatedNamespace gets invoked after each namespace is created. From 3b473289a1aba265de7cf600583554d2be338bc4 Mon Sep 17 00:00:00 2001 From: Linas Medziunas Date: Thu, 12 Nov 2020 10:24:46 +0200 Subject: [PATCH 11/12] Address PR feedback --- src/dbnode/storage/shard.go | 5 ++++- src/dbnode/storage/shard_test.go | 35 +++++++++++++++++++------------- 2 files changed, 25 insertions(+), 15 deletions(-) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 0a9da191ce..db2fabe13b 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -2676,7 +2676,9 @@ func (s *dbShard) AggregateTiles( openBlockReaders := make([]fs.DataFileSetReader, 0, len(blockReaders)) defer func() { for _, reader := range openBlockReaders { - _ = reader.Close() + if err := reader.Close(); err != nil { + s.logger.Error("error closing DataFileSetReader", zap.Error(err)) + } } }() @@ -2737,6 +2739,7 @@ func (s *dbShard) AggregateTiles( processedTileCount, err := s.tileAggregator.AggregateTiles( opts, targetNs, s.ID(), openBlockReaders, writer) if err != nil { + // NB: cannot return on the error here, must finish writing. multiErr = multiErr.Add(err) } diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index b402fc978a..2f89ad1a47 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -270,8 +270,8 @@ func TestShardBootstrapWithFlushVersionNoCleanUp(t *testing.T) { defer ctrl.Finish() var ( - opts = DefaultTestOptions() - fsOpts = opts.CommitLogOptions().FilesystemOptions().SetFilePathPrefix(dir) + opts = DefaultTestOptions() + fsOpts = opts.CommitLogOptions().FilesystemOptions().SetFilePathPrefix(dir) newClOpts = opts.CommitLogOptions().SetFilesystemOptions(fsOpts) ) opts = opts. @@ -325,9 +325,9 @@ func TestShardBootstrapWithCacheShardIndices(t *testing.T) { defer ctrl.Finish() var ( - opts = DefaultTestOptions() - fsOpts = opts.CommitLogOptions().FilesystemOptions().SetFilePathPrefix(dir) - newClOpts = opts.CommitLogOptions().SetFilesystemOptions(fsOpts) + opts = DefaultTestOptions() + fsOpts = opts.CommitLogOptions().FilesystemOptions().SetFilePathPrefix(dir) + newClOpts = opts.CommitLogOptions().SetFilesystemOptions(fsOpts) mockRetriever = block.NewMockDatabaseBlockRetriever(ctrl) ) opts = opts.SetCommitLogOptions(newClOpts) @@ -1849,23 +1849,30 @@ func TestShardAggregateTiles(t *testing.T) { targetBlockSize = 2 * time.Hour start = time.Now().Truncate(targetBlockSize) opts = AggregateTilesOptions{Start: start, End: start.Add(targetBlockSize), Step: 10 * time.Minute} - err error + + firstSourceBlockEntries = 3 + secondSourceBlockEntries = 2 + maxSourceBlockEntries = 3 + + expectedProcessedTileCount = int64(4) + + err error ) aggregator := NewMockTileAggregator(ctrl) testOpts := DefaultTestOptions().SetTileAggregator(aggregator) sourceShard := testDatabaseShard(t, testOpts) - defer sourceShard.Close() + defer assert.NoError(t, sourceShard.Close()) sourceNsID := sourceShard.namespace.ID() reader0, volume0 := getMockReader(ctrl, t, sourceShard, start, true) - reader0.EXPECT().Entries().Return(3) + reader0.EXPECT().Entries().Return(firstSourceBlockEntries) secondSourceBlockStart := start.Add(sourceBlockSize) reader1, volume1 := getMockReader(ctrl, t, sourceShard, secondSourceBlockStart, true) - reader1.EXPECT().Entries().Return(2) + reader1.EXPECT().Entries().Return(secondSourceBlockEntries) thirdSourceBlockStart := secondSourceBlockStart.Add(sourceBlockSize) reader2, volume2 := getMockReader(ctrl, t, sourceShard, thirdSourceBlockStart, false) @@ -1878,7 +1885,7 @@ func TestShardAggregateTiles(t *testing.T) { } targetShard := testDatabaseShardWithIndexFn(t, testOpts, nil, true) - defer targetShard.Close() // nolint:errcheck + defer assert.NoError(t, targetShard.Close()) writer := fs.NewMockStreamingWriter(ctrl) gomock.InOrder( @@ -1888,7 +1895,7 @@ func TestShardAggregateTiles(t *testing.T) { BlockStart: opts.Start, BlockSize: targetBlockSize, VolumeIndex: 1, - PlannedRecordsCount: 3, + PlannedRecordsCount: uint(maxSourceBlockEntries), }), writer.EXPECT().Close(), ) @@ -1896,12 +1903,12 @@ func TestShardAggregateTiles(t *testing.T) { targetNs := NewMockNamespace(ctrl) aggregator.EXPECT(). AggregateTiles(opts, targetNs, sourceShard.ID(), gomock.Len(2), writer). - Return(int64(4), nil) + Return(expectedProcessedTileCount, nil) processedTileCount, err := targetShard.AggregateTiles( sourceNsID, targetNs, sourceShard.ID(), blockReaders, writer, sourceBlockVolumes, opts) require.NoError(t, err) - assert.Equal(t, int64(4), processedTileCount) + assert.Equal(t, expectedProcessedTileCount, processedTileCount) } func TestShardAggregateTilesVerifySliceLengths(t *testing.T) { @@ -1914,7 +1921,7 @@ func TestShardAggregateTilesVerifySliceLengths(t *testing.T) { ) targetShard := testDatabaseShardWithIndexFn(t, DefaultTestOptions(), nil, true) - defer targetShard.Close() + defer assert.NoError(t, targetShard.Close()) var blockReaders []fs.DataFileSetReader sourceBlockVolumes := []shardBlockVolume{{start, 0}} From 9488bc5c92b5aafc28995e624f42c5d6bf657f2c Mon Sep 17 00:00:00 2001 From: Linas Medziunas Date: Thu, 12 Nov 2020 15:54:50 +0200 Subject: [PATCH 12/12] Change error msg --- src/dbnode/storage/shard.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index db2fabe13b..061df21615 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -2677,7 +2677,7 @@ func (s *dbShard) AggregateTiles( defer func() { for _, reader := range openBlockReaders { if err := reader.Close(); err != nil { - s.logger.Error("error closing DataFileSetReader", zap.Error(err)) + s.logger.Error("could not close DataFileSetReader", zap.Error(err)) } } }()